From 0a46532005b5be94c85d264de7cb8a21a2a3dee0 Mon Sep 17 00:00:00 2001 From: liyang830 Date: Mon, 14 Mar 2022 18:25:28 +0800 Subject: [PATCH 0001/2047] add check copier drop partition success --- programs/copier/ClusterCopier.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 4d491a06795..b1cafd4ad6d 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -598,6 +598,8 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t ss << "ALTER TABLE " << getQuotedTable(original_table) << ((partition_name == "'all'") ? " DROP PARTITION ID " : " DROP PARTITION ") << partition_name; UInt64 num_shards_drop_partition = executeQueryOnCluster(task_table.cluster_push, ss.str(), task_cluster->settings_push, ClusterExecutionMode::ON_EACH_SHARD); + if (num_shards_drop_partition != task_table.cluster_push->getShardCount()) + return TaskStatus::Error; LOG_INFO(log, "Drop partition {} in original table {} have been executed successfully on {} shards of {}", partition_name, getQuotedTable(original_table), num_shards_drop_partition, task_table.cluster_push->getShardCount()); From 0ad911c413dd9c37f3db61e1af7e613df8af40fa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Oct 2022 11:24:33 +0000 Subject: [PATCH 0002/2047] Add more checks into ThreadStatus ctor. --- src/Common/ThreadStatus.cpp | 4 ++++ src/Processors/Transforms/buildPushingToViewsChain.cpp | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index b62a7af6c71..1e60fb96916 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -92,6 +92,8 @@ std::vector ThreadGroupStatus ThreadStatus::ThreadStatus() : thread_id{getThreadId()} { + chassert(!current_thread); + last_rusage = std::make_unique(); memory_tracker.setDescription("(for thread)"); @@ -145,6 +147,8 @@ ThreadStatus::ThreadStatus() ThreadStatus::~ThreadStatus() { + chassert(current_thread == this); + memory_tracker.adjustWithUntrackedMemory(untracked_memory); if (thread_group) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index ea088c45471..848345784dd 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -273,7 +273,7 @@ Chain buildPushingToViewsChain( /// and switch back to the original thread_status. auto * original_thread = current_thread; SCOPE_EXIT({ current_thread = original_thread; }); - + current_thread = nullptr; std::unique_ptr view_thread_status_ptr = std::make_unique(); /// Disable query profiler for this ThreadStatus since the running (main query) thread should already have one /// If we didn't disable it, then we could end up with N + 1 (N = number of dependencies) profilers which means From 0a80d451d21f1e8cc1314142ab9bf0ceb06d686a Mon Sep 17 00:00:00 2001 From: Eridanus Date: Mon, 3 Oct 2022 00:09:22 +0800 Subject: [PATCH 0003/2047] Rewrite countDistinctIf with count_distinct_implementation configuration. --- src/Core/Settings.h | 3 ++- src/Core/SettingsChangesHistory.h | 1 + src/Interpreters/TreeRewriter.cpp | 9 +++++++++ .../02456_optimize_rewrite_count_distinct_if.reference | 6 ++++++ .../02456_optimize_rewrite_count_distinct_if.sql | 8 ++++++++ 5 files changed, 26 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.reference create mode 100644 tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 26b6fd56ade..cb49182c83e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -29,7 +29,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) /** List of settings: type, name, default value, description, flags * - * This looks rather unconvenient. It is done that way to avoid repeating settings in different places. + * This looks rather inconvenient. It is done that way to avoid repeating settings in different places. * Note: as an alternative, we could implement settings to be completely dynamic in form of map: String -> Field, * but we are not going to do it, because settings is used everywhere as static struct fields. * @@ -480,6 +480,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ + M(Bool, optimize_rewrite_count_distinct_if, false, "Rewrite countDistinctIf with count_distinct_implementation configuration", 0) \ M(Bool, convert_query_to_cnf, false, "Convert SELECT query to CNF", 0) \ M(Bool, optimize_or_like_chain, false, "Optimize multiple OR LIKE into multiMatchAny. This optimization should not be enabled by default, because it defies index analysis in some cases.", 0) \ M(Bool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b78b812da86..419e731b81c 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -78,6 +78,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"22.10", {{"optimize_rewrite_count_distinct_if", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 2f5bfd00938..e938aa00d0f 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -103,6 +103,9 @@ using CustomizeCountDistinctVisitor = InDepthNodeVisitor>, true>; +char countdistinctif[] = "countdistinctif"; +using CustomizeCountDistinctIfVisitor = InDepthNodeVisitor>, true>; + char in[] = "in"; using CustomizeInVisitor = InDepthNodeVisitor>, true>; @@ -1424,6 +1427,12 @@ void TreeRewriter::normalize( CustomizeIfDistinctVisitor::Data data_distinct_if{"DistinctIf"}; CustomizeIfDistinctVisitor(data_distinct_if).visit(query); + if (settings.optimize_rewrite_count_distinct_if) + { + CustomizeCountDistinctIfVisitor::Data data_count_distinct_if{settings.count_distinct_implementation.toString() + "If"}; + CustomizeCountDistinctIfVisitor(data_count_distinct_if).visit(query); + } + ExistsExpressionVisitor::Data exists; ExistsExpressionVisitor(exists).visit(query); diff --git a/tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.reference b/tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.reference new file mode 100644 index 00000000000..85aab5282d3 --- /dev/null +++ b/tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.reference @@ -0,0 +1,6 @@ +2 +SELECT countDistinctIf(number % 10, (number % 5) = 2) +FROM numbers_mt(100000000) +2 +SELECT uniqExactIf(number % 10, (number % 5) = 2) +FROM numbers_mt(100000000) diff --git a/tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.sql b/tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.sql new file mode 100644 index 00000000000..0b7ab21e035 --- /dev/null +++ b/tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.sql @@ -0,0 +1,8 @@ +SET optimize_rewrite_count_distinct_if = FALSE; +SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(100000000); +EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(100000000); + +SET optimize_rewrite_count_distinct_if = TRUE; +SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(100000000); +EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(100000000); + From 2081408c15e330ea737bc99a0afe7d96346c13d3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 16 Jan 2023 15:10:31 +0100 Subject: [PATCH 0004/2047] Revert "Revert "Merge pull request #38953 from ClickHouse/add-allocation-ptr-to-trace-log" --- .../AggregateFunctionFlameGraph.cpp | 647 ++++++++++++++++++ .../registerAggregateFunctions.cpp | 2 + src/Common/AllocationTrace.h | 16 + src/Common/Allocator.h | 22 +- src/Common/AllocatorWithMemoryTracking.h | 11 +- src/Common/CurrentMemoryTracker.cpp | 36 +- src/Common/CurrentMemoryTracker.h | 11 +- src/Common/FiberStack.h | 6 +- src/Common/MemoryTracker.cpp | 112 ++- src/Common/MemoryTracker.h | 10 +- src/Common/MemoryTrackerBlockerInThread.h | 1 + src/Common/TraceSender.cpp | 2 + src/Common/TraceSender.h | 3 +- src/Common/clickhouse_malloc.cpp | 34 +- src/Common/memory.h | 20 +- src/Common/new_delete.cpp | 88 ++- src/Interpreters/TraceCollector.cpp | 5 +- src/Interpreters/TraceLog.cpp | 2 + src/Interpreters/TraceLog.h | 4 +- src/Storages/MergeTree/MergeList.cpp | 4 + 20 files changed, 936 insertions(+), 100 deletions(-) create mode 100644 src/AggregateFunctions/AggregateFunctionFlameGraph.cpp create mode 100644 src/Common/AllocationTrace.h diff --git a/src/AggregateFunctions/AggregateFunctionFlameGraph.cpp b/src/AggregateFunctions/AggregateFunctionFlameGraph.cpp new file mode 100644 index 00000000000..e25dfead466 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionFlameGraph.cpp @@ -0,0 +1,647 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int FUNCTION_NOT_ALLOWED; + extern const int NOT_IMPLEMENTED; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +struct AggregateFunctionFlameGraphTree +{ + struct ListNode; + + struct TreeNode + { + TreeNode * parent = nullptr; + ListNode * children = nullptr; + UInt64 ptr = 0; + size_t allocated = 0; + }; + + struct ListNode + { + ListNode * next = nullptr; + TreeNode * child = nullptr; + }; + + TreeNode root; + + static ListNode * createChild(TreeNode * parent, UInt64 ptr, Arena * arena) + { + + ListNode * list_node = reinterpret_cast(arena->alloc(sizeof(ListNode))); + TreeNode * tree_node = reinterpret_cast(arena->alloc(sizeof(TreeNode))); + + list_node->child = tree_node; + list_node->next = nullptr; + + tree_node->parent =parent; + tree_node->children = nullptr; + tree_node->ptr = ptr; + tree_node->allocated = 0; + + return list_node; + } + + TreeNode * find(const UInt64 * stack, size_t stack_size, Arena * arena) + { + TreeNode * node = &root; + for (size_t i = 0; i < stack_size; ++i) + { + UInt64 ptr = stack[i]; + if (ptr == 0) + break; + + if (!node->children) + { + node->children = createChild(node, ptr, arena); + node = node->children->child; + } + else + { + ListNode * list = node->children; + while (list->child->ptr != ptr && list->next) + list = list->next; + + if (list->child->ptr != ptr) + { + list->next = createChild(node, ptr, arena); + list = list->next; + } + + node = list->child; + } + } + + return node; + } + + static void append(DB::PaddedPODArray & values, DB::PaddedPODArray & offsets, std::vector & frame) + { + UInt64 prev = offsets.empty() ? 0 : offsets.back(); + offsets.push_back(prev + frame.size()); + for (UInt64 val : frame) + values.push_back(val); + } + + struct Trace + { + using Frames = std::vector; + + Frames frames; + + /// The total number of bytes allocated for traces with the same prefix. + size_t allocated_total = 0; + /// This counter is relevant in case we want to filter some traces with small amount of bytes. + /// It shows the total number of bytes for *filtered* traces with the same prefix. + /// This is the value which is used in flamegraph. + size_t allocated_self = 0; + }; + + using Traces = std::vector; + + Traces dump(size_t max_depth, size_t min_bytes) const + { + Traces traces; + Trace::Frames frames; + std::vector allocated_total; + std::vector allocated_self; + std::vector nodes; + + nodes.push_back(root.children); + allocated_total.push_back(root.allocated); + allocated_self.push_back(root.allocated); + + while (!nodes.empty()) + { + if (nodes.back() == nullptr) + { + traces.push_back({frames, allocated_total.back(), allocated_self.back()}); + + nodes.pop_back(); + allocated_total.pop_back(); + allocated_self.pop_back(); + + /// We don't have root's frame so framers are empty in the end. + if (!frames.empty()) + frames.pop_back(); + + continue; + } + + TreeNode * current = nodes.back()->child; + nodes.back() = nodes.back()->next; + + bool enough_bytes = current->allocated >= min_bytes; + bool enough_depth = max_depth == 0 || nodes.size() < max_depth; + + if (enough_bytes) + { + frames.push_back(current->ptr); + allocated_self.back() -= current->allocated; + + if (enough_depth) + { + allocated_total.push_back(current->allocated); + allocated_self.push_back(current->allocated); + nodes.push_back(current->children); + } + else + { + traces.push_back({frames, current->allocated, current->allocated}); + frames.pop_back(); + } + } + } + + return traces; + } +}; + +static void insertData(DB::PaddedPODArray & chars, DB::PaddedPODArray & offsets, const char * pos, size_t length) +{ + const size_t old_size = chars.size(); + const size_t new_size = old_size + length + 1; + + chars.resize(new_size); + if (length) + memcpy(chars.data() + old_size, pos, length); + chars[old_size + length] = 0; + offsets.push_back(new_size); +} + +/// Split str by line feed and write as separate row to ColumnString. +static void fillColumn(DB::PaddedPODArray & chars, DB::PaddedPODArray & offsets, const std::string & str) +{ + size_t start = 0; + size_t end = 0; + size_t size = str.size(); + + while (end < size) + { + if (str[end] == '\n') + { + insertData(chars, offsets, str.data() + start, end - start); + start = end + 1; + } + + ++end; + } + + if (start < end) + insertData(chars, offsets, str.data() + start, end - start); +} + +void dumpFlameGraph( + const AggregateFunctionFlameGraphTree::Traces & traces, + DB::PaddedPODArray & chars, + DB::PaddedPODArray & offsets) +{ + DB::WriteBufferFromOwnString out; + + std::unordered_map mapping; + +#if defined(__ELF__) && !defined(OS_FREEBSD) + auto symbol_index_ptr = DB::SymbolIndex::instance(); + const DB::SymbolIndex & symbol_index = *symbol_index_ptr; +#endif + + for (const auto & trace : traces) + { + if (trace.allocated_self == 0) + continue; + + for (size_t i = 0; i < trace.frames.size(); ++i) + { + if (i) + out << ";"; + + const void * ptr = reinterpret_cast(trace.frames[i]); + +#if defined(__ELF__) && !defined(OS_FREEBSD) + if (const auto * symbol = symbol_index.findSymbol(ptr)) + writeString(demangle(symbol->name), out); + else + DB::writePointerHex(ptr, out); +#else + DB::writePointerHex(ptr, out); +#endif + } + + out << ' ' << trace.allocated_self << "\n"; + } + + fillColumn(chars, offsets, out.str()); +} + +struct AggregateFunctionFlameGraphData +{ + struct Entry + { + AggregateFunctionFlameGraphTree::TreeNode * trace; + UInt64 size; + Entry * next = nullptr; + }; + + struct Pair + { + Entry * allocation = nullptr; + Entry * deallocation = nullptr; + }; + + using Entries = HashMap; + + AggregateFunctionFlameGraphTree tree; + Entries entries; + Entry * free_list = nullptr; + + Entry * alloc(Arena * arena) + { + if (free_list) + { + auto * res = free_list; + free_list = free_list->next; + return res; + } + + return reinterpret_cast(arena->alloc(sizeof(Entry))); + } + + void release(Entry * entry) + { + entry->next = free_list; + free_list = entry; + } + + static void track(Entry * allocation) + { + auto * node = allocation->trace; + while (node) + { + node->allocated += allocation->size; + node = node->parent; + } + } + + static void untrack(Entry * allocation) + { + auto * node = allocation->trace; + while (node) + { + node->allocated -= allocation->size; + node = node->parent; + } + } + + static Entry * tryFindMatchAndRemove(Entry *& list, UInt64 size) + { + if (!list) + return nullptr; + + if (list->size == size) + { + Entry * entry = list; + list = list->next; + return entry; + } + else + { + Entry * parent = list; + while (parent->next && parent->next->size != size) + parent = parent->next; + + if (parent->next && parent->next->size == size) + { + Entry * entry = parent->next; + parent->next = entry->next; + return entry; + } + + return nullptr; + } + } + + void add(UInt64 ptr, Int64 size, const UInt64 * stack, size_t stack_size, Arena * arena) + { + /// In case if argument is nullptr, only track allocations. + if (ptr == 0) + { + if (size > 0) + { + auto * node = tree.find(stack, stack_size, arena); + Entry entry{.trace = node, .size = UInt64(size)}; + track(&entry); + } + + return; + } + + auto & place = entries[ptr]; + if (size > 0) + { + if (auto * deallocation = tryFindMatchAndRemove(place.deallocation, size)) + { + release(deallocation); + } + else + { + auto * node = tree.find(stack, stack_size, arena); + + auto * allocation = alloc(arena); + allocation->size = UInt64(size); + allocation->trace = node; + + track(allocation); + + allocation->next = place.allocation; + place.allocation = allocation; + } + } + else if (size < 0) + { + UInt64 abs_size = -size; + if (auto * allocation = tryFindMatchAndRemove(place.allocation, abs_size)) + { + untrack(allocation); + release(allocation); + } + else + { + auto * deallocation = alloc(arena); + deallocation->size = abs_size; + + deallocation->next = place.deallocation; + place.deallocation = deallocation; + } + } + } + + void merge(const AggregateFunctionFlameGraphTree & other_tree, Arena * arena) + { + AggregateFunctionFlameGraphTree::Trace::Frames frames; + std::vector nodes; + + nodes.push_back(other_tree.root.children); + + while (!nodes.empty()) + { + if (nodes.back() == nullptr) + { + nodes.pop_back(); + + /// We don't have root's frame so framers are empty in the end. + if (!frames.empty()) + frames.pop_back(); + + continue; + } + + AggregateFunctionFlameGraphTree::TreeNode * current = nodes.back()->child; + nodes.back() = nodes.back()->next; + + frames.push_back(current->ptr); + + if (current->children) + nodes.push_back(current->children); + else + { + if (current->allocated) + add(0, current->allocated, frames.data(), frames.size(), arena); + + frames.pop_back(); + } + } + } + + void merge(const AggregateFunctionFlameGraphData & other, Arena * arena) + { + AggregateFunctionFlameGraphTree::Trace::Frames frames; + for (const auto & entry : other.entries) + { + for (auto * allocation = entry.value.second.allocation; allocation; allocation = allocation->next) + { + frames.clear(); + const auto * node = allocation->trace; + while (node->ptr) + { + frames.push_back(node->ptr); + node = node->parent; + } + + std::reverse(frames.begin(), frames.end()); + add(entry.value.first, allocation->size, frames.data(), frames.size(), arena); + untrack(allocation); + } + + for (auto * deallocation = entry.value.second.deallocation; deallocation; deallocation = deallocation->next) + { + add(entry.value.first, -Int64(deallocation->size), nullptr, 0, arena); + } + } + + merge(other.tree, arena); + } + + void dumpFlameGraph( + DB::PaddedPODArray & chars, + DB::PaddedPODArray & offsets, + size_t max_depth, size_t min_bytes) const + { + DB::dumpFlameGraph(tree.dump(max_depth, min_bytes), chars, offsets); + } +}; + +/// Aggregate function which builds a flamegraph using the list of stacktraces. +/// The output is an array of strings which can be used by flamegraph.pl util. +/// See https://github.com/brendangregg/FlameGraph +/// +/// Syntax: flameGraph(traces, [size = 1], [ptr = 0]) +/// - trace : Array(UInt64), a stacktrace +/// - size : Int64, an allocation size (for memory profiling) +/// - ptr : UInt64, an allocation address +/// In case if ptr != 0, a flameGraph will map allocations (size > 0) and deallocations (size < 0) with the same size and ptr. +/// Only allocations which were not freed are shown. Not mapped deallocations are ignored. +/// +/// Usage: +/// +/// * Build a flamegraph based on CPU query profiler +/// set query_profiler_cpu_time_period_ns=10000000; +/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +/// clickhouse client --allow_introspection_functions=1 +/// -q "select arrayJoin(flameGraph(arrayReverse(trace))) from system.trace_log where trace_type = 'CPU' and query_id = 'xxx'" +/// | ~/dev/FlameGraph/flamegraph.pl > flame_cpu.svg +/// +/// * Build a flamegraph based on memory query profiler, showing all allocations +/// set memory_profiler_sample_probability=1, max_untracked_memory=1; +/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +/// clickhouse client --allow_introspection_functions=1 +/// -q "select arrayJoin(flameGraph(trace, size)) from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx'" +/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem.svg +/// +/// * Build a flamegraph based on memory query profiler, showing allocations which were not deallocated in query context +/// set memory_profiler_sample_probability=1, max_untracked_memory=1, use_uncompressed_cache=1, merge_tree_max_rows_to_use_cache=100000000000, merge_tree_max_bytes_to_use_cache=1000000000000; +/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +/// clickhouse client --allow_introspection_functions=1 +/// -q "select arrayJoin(flameGraph(trace, size, ptr)) from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx'" +/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_untracked.svg +/// +/// * Build a flamegraph based on memory query profiler, showing active allocations at the fixed point of time +/// set memory_profiler_sample_probability=1, max_untracked_memory=1; +/// SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10; +/// 1. Memory usage per second +/// select event_time, m, formatReadableSize(max(s) as m) from (select event_time, sum(size) over (order by event_time) as s from system.trace_log where query_id = 'xxx' and trace_type = 'MemorySample') group by event_time order by event_time; +/// 2. Find a time point with maximal memory usage +/// select argMax(event_time, s), max(s) from (select event_time, sum(size) over (order by event_time) as s from system.trace_log where query_id = 'xxx' and trace_type = 'MemorySample'); +/// 3. Fix active allocations at fixed point of time +/// clickhouse client --allow_introspection_functions=1 +/// -q "select arrayJoin(flameGraph(trace, size, ptr)) from (select * from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx' and event_time <= 'yyy' order by event_time)" +/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_time_point_pos.svg +/// 4. Find deallocations at fixed point of time +/// clickhouse client --allow_introspection_functions=1 +/// -q "select arrayJoin(flameGraph(trace, -size, ptr)) from (select * from system.trace_log where trace_type = 'MemorySample' and query_id = 'xxx' and event_time > 'yyy' order by event_time desc)" +/// | ~/dev/FlameGraph/flamegraph.pl --countname=bytes --color=mem > flame_mem_time_point_neg.svg +class AggregateFunctionFlameGraph final : public IAggregateFunctionDataHelper +{ +public: + explicit AggregateFunctionFlameGraph(const DataTypes & argument_types_) + : IAggregateFunctionDataHelper(argument_types_, {}, createResultType()) + {} + + String getName() const override { return "flameGraph"; } + + static DataTypePtr createResultType() + { + return std::make_shared(std::make_shared()); + } + + bool allocatesMemoryInArena() const override { return true; } + + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + const auto * trace = typeid_cast(columns[0]); + + const auto & trace_offsets = trace->getOffsets(); + const auto & trace_values = typeid_cast(&trace->getData())->getData(); + UInt64 prev_offset = 0; + if (row_num) + prev_offset = trace_offsets[row_num - 1]; + UInt64 trace_size = trace_offsets[row_num] - prev_offset; + + Int64 allocated = 1; + if (argument_types.size() >= 2) + { + const auto & sizes = typeid_cast(columns[1])->getData(); + allocated = sizes[row_num]; + } + + UInt64 ptr = 0; + if (argument_types.size() >= 3) + { + const auto & ptrs = typeid_cast(columns[2])->getData(); + ptr = ptrs[row_num]; + } + + this->data(place).add(ptr, allocated, trace_values.data() + prev_offset, trace_size, arena); + } + + void addManyDefaults( + AggregateDataPtr __restrict /*place*/, + const IColumn ** /*columns*/, + size_t /*length*/, + Arena * /*arena*/) const override + { + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override + { + this->data(place).merge(this->data(rhs), arena); + } + + void serialize(ConstAggregateDataPtr __restrict, WriteBuffer &, std::optional /* version */) const override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Serialization for function flameGraph is not implemented."); + } + + void deserialize(AggregateDataPtr __restrict, ReadBuffer &, std::optional /* version */, Arena *) const override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Deserialization for function flameGraph is not implemented."); + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + auto & array = assert_cast(to); + auto & str = assert_cast(array.getData()); + + this->data(place).dumpFlameGraph(str.getChars(), str.getOffsets(), 0, 0); + + array.getOffsets().push_back(str.size()); + } +}; + +static void check(const std::string & name, const DataTypes & argument_types, const Array & params) +{ + assertNoParameters(name, params); + + if (argument_types.empty() || argument_types.size() > 3) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Aggregate function {} requires 1 to 3 arguments : trace, [size = 1], [ptr = 0]", + name); + + auto ptr_type = std::make_shared(); + auto trace_type = std::make_shared(ptr_type); + auto size_type = std::make_shared(); + + if (!argument_types[0]->equals(*trace_type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument (trace) for function {} must be Array(UInt64), but it has type {}", + name, argument_types[0]->getName()); + + if (argument_types.size() >= 2 && !argument_types[1]->equals(*size_type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Second argument (size) for function {} must be Int64, but it has type {}", + name, argument_types[1]->getName()); + + if (argument_types.size() >= 3 && !argument_types[2]->equals(*ptr_type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Third argument (ptr) for function {} must be UInt64, but it has type {}", + name, argument_types[2]->getName()); +} + +AggregateFunctionPtr createAggregateFunctionFlameGraph(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings * settings) +{ + if (!settings->allow_introspection_functions) + throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, + "Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0"); + + check(name, argument_types, params); + return std::make_shared(argument_types); +} + +void registerAggregateFunctionFlameGraph(AggregateFunctionFactory & factory) +{ + AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = true }; + + factory.registerFunction("flameGraph", { createAggregateFunctionFlameGraph, properties }); +} + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 1fe759c122a..355c3c507e0 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -74,6 +74,7 @@ void registerAggregateFunctionExponentialMovingAverage(AggregateFunctionFactory void registerAggregateFunctionSparkbar(AggregateFunctionFactory &); void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory &); void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory &); +void registerAggregateFunctionFlameGraph(AggregateFunctionFactory &); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -160,6 +161,7 @@ void registerAggregateFunctions() registerAggregateFunctionExponentialMovingAverage(factory); registerAggregateFunctionSparkbar(factory); registerAggregateFunctionAnalysisOfVariance(factory); + registerAggregateFunctionFlameGraph(factory); registerWindowFunctions(factory); } diff --git a/src/Common/AllocationTrace.h b/src/Common/AllocationTrace.h new file mode 100644 index 00000000000..332808c8015 --- /dev/null +++ b/src/Common/AllocationTrace.h @@ -0,0 +1,16 @@ +#pragma once +#include + +/// This is a structure which is returned by MemoryTracker. +/// Methods onAlloc/onFree should be called after actual memory allocation if it succeed. +/// For now, it will only collect allocation trace with sample_probability. +struct AllocationTrace +{ + AllocationTrace() = default; + explicit AllocationTrace(double sample_probability_); + + void onAlloc(void * ptr, size_t size) const; + void onFree(void * ptr, size_t size) const; + + double sample_probability = 0; +}; diff --git a/src/Common/Allocator.h b/src/Common/Allocator.h index c348eaea006..8c4f2ef1690 100644 --- a/src/Common/Allocator.h +++ b/src/Common/Allocator.h @@ -92,8 +92,10 @@ public: void * alloc(size_t size, size_t alignment = 0) { checkSize(size); - CurrentMemoryTracker::alloc(size); - return allocNoTrack(size, alignment); + auto trace = CurrentMemoryTracker::alloc(size); + void * ptr = allocNoTrack(size, alignment); + trace.onAlloc(ptr, size); + return ptr; } /// Free memory range. @@ -103,7 +105,8 @@ public: { checkSize(size); freeNoTrack(buf, size); - CurrentMemoryTracker::free(size); + auto trace = CurrentMemoryTracker::free(size); + trace.onFree(buf, size); } catch (...) { @@ -129,13 +132,16 @@ public: && alignment <= MALLOC_MIN_ALIGNMENT) { /// Resize malloc'd memory region with no special alignment requirement. - CurrentMemoryTracker::realloc(old_size, new_size); + auto trace = CurrentMemoryTracker::realloc(old_size, new_size); + trace.onFree(buf, old_size); void * new_buf = ::realloc(buf, new_size); if (nullptr == new_buf) DB::throwFromErrno(fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); buf = new_buf; + trace.onAlloc(buf, new_size); + if constexpr (clear_memory) if (new_size > old_size) memset(reinterpret_cast(buf) + old_size, 0, new_size - old_size); @@ -143,7 +149,8 @@ public: else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD) { /// Resize mmap'd memory region. - CurrentMemoryTracker::realloc(old_size, new_size); + auto trace = CurrentMemoryTracker::realloc(old_size, new_size); + trace.onFree(buf, old_size); // On apple and freebsd self-implemented mremap used (common/mremap.h) buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE, @@ -152,14 +159,17 @@ public: DB::throwFromErrno(fmt::format("Allocator: Cannot mremap memory chunk from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_MREMAP); + trace.onAlloc(buf, new_size); /// No need for zero-fill, because mmap guarantees it. } else if (new_size < MMAP_THRESHOLD) { /// Small allocs that requires a copy. Assume there's enough memory in system. Call CurrentMemoryTracker once. - CurrentMemoryTracker::realloc(old_size, new_size); + auto trace = CurrentMemoryTracker::realloc(old_size, new_size); + trace.onFree(buf, old_size); void * new_buf = allocNoTrack(new_size, alignment); + trace.onAlloc(new_buf, new_size); memcpy(new_buf, buf, std::min(old_size, new_size)); freeNoTrack(buf, old_size); buf = new_buf; diff --git a/src/Common/AllocatorWithMemoryTracking.h b/src/Common/AllocatorWithMemoryTracking.h index 815c326ed62..b43870e05b2 100644 --- a/src/Common/AllocatorWithMemoryTracking.h +++ b/src/Common/AllocatorWithMemoryTracking.h @@ -30,21 +30,24 @@ struct AllocatorWithMemoryTracking throw std::bad_alloc(); size_t bytes = n * sizeof(T); - CurrentMemoryTracker::alloc(bytes); + auto trace = CurrentMemoryTracker::alloc(bytes); T * p = static_cast(malloc(bytes)); if (!p) throw std::bad_alloc(); + trace.onAlloc(p, bytes); + return p; } void deallocate(T * p, size_t n) noexcept { - free(p); - size_t bytes = n * sizeof(T); - CurrentMemoryTracker::free(bytes); + + free(p); + auto trace = CurrentMemoryTracker::free(bytes); + trace.onFree(p, bytes); } }; diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index 720df07efb9..0147a095185 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -37,7 +37,7 @@ MemoryTracker * getMemoryTracker() using DB::current_thread; -void CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) +AllocationTrace CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) { #ifdef MEMORY_TRACKER_DEBUG_CHECKS if (unlikely(memory_tracker_always_throw_logical_error_on_allocation)) @@ -55,8 +55,9 @@ void CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) if (will_be > current_thread->untracked_memory_limit) { - memory_tracker->allocImpl(will_be, throw_if_memory_exceeded); + auto res = memory_tracker->allocImpl(will_be, throw_if_memory_exceeded); current_thread->untracked_memory = 0; + return res; } else { @@ -68,36 +69,40 @@ void CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded) /// total_memory_tracker only, ignore untracked_memory else { - memory_tracker->allocImpl(size, throw_if_memory_exceeded); + return memory_tracker->allocImpl(size, throw_if_memory_exceeded); } + + return AllocationTrace(memory_tracker->getSampleProbability()); } + + return AllocationTrace(0); } void CurrentMemoryTracker::check() { if (auto * memory_tracker = getMemoryTracker()) - memory_tracker->allocImpl(0, true); + std::ignore = memory_tracker->allocImpl(0, true); } -void CurrentMemoryTracker::alloc(Int64 size) +AllocationTrace CurrentMemoryTracker::alloc(Int64 size) { bool throw_if_memory_exceeded = true; - allocImpl(size, throw_if_memory_exceeded); + return allocImpl(size, throw_if_memory_exceeded); } -void CurrentMemoryTracker::allocNoThrow(Int64 size) +AllocationTrace CurrentMemoryTracker::allocNoThrow(Int64 size) { bool throw_if_memory_exceeded = false; - allocImpl(size, throw_if_memory_exceeded); + return allocImpl(size, throw_if_memory_exceeded); } -void CurrentMemoryTracker::realloc(Int64 old_size, Int64 new_size) +AllocationTrace CurrentMemoryTracker::realloc(Int64 old_size, Int64 new_size) { Int64 addition = new_size - old_size; - addition > 0 ? alloc(addition) : free(-addition); + return addition > 0 ? alloc(addition) : free(-addition); } -void CurrentMemoryTracker::free(Int64 size) +AllocationTrace CurrentMemoryTracker::free(Int64 size) { if (auto * memory_tracker = getMemoryTracker()) { @@ -106,15 +111,20 @@ void CurrentMemoryTracker::free(Int64 size) current_thread->untracked_memory -= size; if (current_thread->untracked_memory < -current_thread->untracked_memory_limit) { - memory_tracker->free(-current_thread->untracked_memory); + Int64 untracked_memory = current_thread->untracked_memory; current_thread->untracked_memory = 0; + return memory_tracker->free(-untracked_memory); } } /// total_memory_tracker only, ignore untracked_memory else { - memory_tracker->free(size); + return memory_tracker->free(size); } + + return AllocationTrace(memory_tracker->getSampleProbability()); } + + return AllocationTrace(0); } diff --git a/src/Common/CurrentMemoryTracker.h b/src/Common/CurrentMemoryTracker.h index e125e4cbe4a..ba46f458e4a 100644 --- a/src/Common/CurrentMemoryTracker.h +++ b/src/Common/CurrentMemoryTracker.h @@ -1,19 +1,20 @@ #pragma once #include +#include /// Convenience methods, that use current thread's memory_tracker if it is available. struct CurrentMemoryTracker { /// Call the following functions before calling of corresponding operations with memory allocators. - static void alloc(Int64 size); - static void allocNoThrow(Int64 size); - static void realloc(Int64 old_size, Int64 new_size); + [[nodiscard]] static AllocationTrace alloc(Int64 size); + [[nodiscard]] static AllocationTrace allocNoThrow(Int64 size); + [[nodiscard]] static AllocationTrace realloc(Int64 old_size, Int64 new_size); /// This function should be called after memory deallocation. - static void free(Int64 size); + [[nodiscard]] static AllocationTrace free(Int64 size); static void check(); private: - static void allocImpl(Int64 size, bool throw_if_memory_exceeded); + [[nodiscard]] static AllocationTrace allocImpl(Int64 size, bool throw_if_memory_exceeded); }; diff --git a/src/Common/FiberStack.h b/src/Common/FiberStack.h index c55608311d0..91bb632d807 100644 --- a/src/Common/FiberStack.h +++ b/src/Common/FiberStack.h @@ -57,7 +57,8 @@ public: } /// Do not count guard page in memory usage. - CurrentMemoryTracker::alloc(num_pages * page_size); + auto trace = CurrentMemoryTracker::alloc(num_pages * page_size); + trace.onAlloc(vp, num_pages * page_size); boost::context::stack_context sctx; sctx.size = num_bytes; @@ -77,6 +78,7 @@ public: ::munmap(vp, sctx.size); /// Do not count guard page in memory usage. - CurrentMemoryTracker::free(sctx.size - page_size); + auto trace = CurrentMemoryTracker::free(sctx.size - page_size); + trace.onFree(vp, sctx.size - page_size); } }; diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 27d0adcf24f..d0d0d6b8686 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -1,6 +1,7 @@ #include "MemoryTracker.h" #include +#include #include #include #include @@ -82,6 +83,53 @@ inline std::string_view toDescription(OvercommitResult result) } } +bool shouldTrackAllocation(DB::Float64 probability, void * ptr) +{ + return sipHash64(uintptr_t(ptr)) < std::numeric_limits::max() * probability; +} + +AllocationTrace updateAllocationTrace(AllocationTrace trace, const std::optional & sample_probability) +{ + if (unlikely(sample_probability)) + return AllocationTrace(*sample_probability); + + return trace; +} + +AllocationTrace getAllocationTrace(std::optional & sample_probability) +{ + if (unlikely(sample_probability)) + return AllocationTrace(*sample_probability); + + return AllocationTrace(0); +} + +} + +AllocationTrace::AllocationTrace(double sample_probability_) : sample_probability(sample_probability_) {} + +void AllocationTrace::onAlloc(void * ptr, size_t size) const +{ + if (likely(sample_probability == 0)) + return; + + if (sample_probability < 1 && !shouldTrackAllocation(sample_probability, ptr)) + return; + + MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); + DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = Int64(size), .ptr = ptr}); +} + +void AllocationTrace::onFree(void * ptr, size_t size) const +{ + if (likely(sample_probability == 0)) + return; + + if (sample_probability < 1 && !shouldTrackAllocation(sample_probability, ptr)) + return; + + MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); + DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -Int64(size), .ptr = ptr}); } namespace ProfileEvents @@ -135,7 +183,7 @@ void MemoryTracker::logMemoryUsage(Int64 current) const } -void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker) +AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker) { if (size < 0) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Negative size ({}) is passed to MemoryTracker. It is a bug.", size); @@ -154,9 +202,14 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT /// Since the MemoryTrackerBlockerInThread should respect the level, we should go to the next parent. if (auto * loaded_next = parent.load(std::memory_order_relaxed)) - loaded_next->allocImpl(size, throw_if_memory_exceeded, - level == VariableContext::Process ? this : query_tracker); - return; + { + MemoryTracker * tracker = level == VariableContext::Process ? this : query_tracker; + return updateAllocationTrace( + loaded_next->allocImpl(size, throw_if_memory_exceeded, tracker), + sample_probability); + } + + return getAllocationTrace(sample_probability); } /** Using memory_order_relaxed means that if allocations are done simultaneously, @@ -183,14 +236,6 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT allocation_traced = true; } - std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) - { - MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); - DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size}); - allocation_traced = true; - } - std::bernoulli_distribution fault(fault_probability); if (unlikely(fault_probability > 0.0 && fault(thread_local_rng))) { @@ -309,16 +354,22 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT } if (auto * loaded_next = parent.load(std::memory_order_relaxed)) - loaded_next->allocImpl(size, throw_if_memory_exceeded, - level == VariableContext::Process ? this : query_tracker); + { + MemoryTracker * tracker = level == VariableContext::Process ? this : query_tracker; + return updateAllocationTrace( + loaded_next->allocImpl(size, throw_if_memory_exceeded, tracker), + sample_probability); + } + + return getAllocationTrace(sample_probability); } void MemoryTracker::adjustWithUntrackedMemory(Int64 untracked_memory) { if (untracked_memory > 0) - allocImpl(untracked_memory, /*throw_if_memory_exceeded*/ false); + std::ignore = allocImpl(untracked_memory, /*throw_if_memory_exceeded*/ false); else - free(-untracked_memory); + std::ignore = free(-untracked_memory); } bool MemoryTracker::updatePeak(Int64 will_be, bool log_memory_usage) @@ -337,8 +388,7 @@ bool MemoryTracker::updatePeak(Int64 will_be, bool log_memory_usage) return false; } - -void MemoryTracker::free(Int64 size) +AllocationTrace MemoryTracker::free(Int64 size) { if (MemoryTrackerBlockerInThread::isBlocked(level)) { @@ -353,15 +403,9 @@ void MemoryTracker::free(Int64 size) /// Since the MemoryTrackerBlockerInThread should respect the level, we should go to the next parent. if (auto * loaded_next = parent.load(std::memory_order_relaxed)) - loaded_next->free(size); - return; - } + return updateAllocationTrace(loaded_next->free(size), sample_probability); - std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) - { - MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); - DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size}); + return getAllocationTrace(sample_probability); } Int64 accounted_size = size; @@ -389,12 +433,15 @@ void MemoryTracker::free(Int64 size) if (auto * overcommit_tracker_ptr = overcommit_tracker.load(std::memory_order_relaxed)) overcommit_tracker_ptr->tryContinueQueryExecutionAfterFree(accounted_size); + AllocationTrace res = getAllocationTrace(sample_probability); if (auto * loaded_next = parent.load(std::memory_order_relaxed)) - loaded_next->free(size); + res = updateAllocationTrace(loaded_next->free(size), sample_probability); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) CurrentMetrics::sub(metric_loaded, accounted_size); + + return res; } @@ -478,3 +525,14 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) while ((value == 0 || old_value < value) && !profiler_limit.compare_exchange_weak(old_value, value)) ; } + +double MemoryTracker::getSampleProbability() +{ + if (sample_probability) + return *sample_probability; + + if (auto * loaded_next = parent.load(std::memory_order_relaxed)) + return loaded_next->getSampleProbability(); + + return 0; +} diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index f6113d31423..e1f61b1585a 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -2,9 +2,11 @@ #include #include +#include #include #include #include +#include #if !defined(NDEBUG) #define MEMORY_TRACKER_DEBUG_CHECKS @@ -65,7 +67,7 @@ private: double fault_probability = 0; /// To randomly sample allocations and deallocations in trace_log. - double sample_probability = 0; + std::optional sample_probability; /// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy). /// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker. @@ -90,8 +92,8 @@ private: /// allocImpl(...) and free(...) should not be used directly friend struct CurrentMemoryTracker; - void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); - void free(Int64 size); + [[nodiscard]] AllocationTrace allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); + [[nodiscard]] AllocationTrace free(Int64 size); public: static constexpr auto USAGE_EVENT_NAME = "MemoryTrackerUsage"; @@ -146,6 +148,8 @@ public: sample_probability = value; } + double getSampleProbability(); + void setProfilerStep(Int64 value) { profiler_step = value; diff --git a/src/Common/MemoryTrackerBlockerInThread.h b/src/Common/MemoryTrackerBlockerInThread.h index d3882056f54..73794049007 100644 --- a/src/Common/MemoryTrackerBlockerInThread.h +++ b/src/Common/MemoryTrackerBlockerInThread.h @@ -28,4 +28,5 @@ public: } friend class MemoryTracker; + friend struct AllocationTrace; }; diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index 64d7b2b0eaf..91d07367a82 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -33,6 +33,7 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Ext + sizeof(TraceType) /// trace type + sizeof(UInt64) /// thread_id + sizeof(Int64) /// size + + sizeof(void *) /// ptr + sizeof(ProfileEvents::Event) /// event + sizeof(ProfileEvents::Count); /// increment @@ -74,6 +75,7 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Ext writePODBinary(trace_type, out); writePODBinary(thread_id, out); writePODBinary(extras.size, out); + writePODBinary(UInt64(extras.ptr), out); writePODBinary(extras.event, out); writePODBinary(extras.increment, out); diff --git a/src/Common/TraceSender.h b/src/Common/TraceSender.h index 21b44b651dd..68ba15ee400 100644 --- a/src/Common/TraceSender.h +++ b/src/Common/TraceSender.h @@ -28,8 +28,9 @@ class TraceSender public: struct Extras { - /// size - for memory tracing is the amount of memory allocated; for other trace types it is 0. + /// size, ptr - for memory tracing is the amount of memory allocated; for other trace types it is 0. Int64 size{}; + void * ptr = nullptr; /// Event type and increment for 'ProfileEvent' trace type; for other trace types defaults. ProfileEvents::Event event{ProfileEvents::end()}; ProfileEvents::Count increment{}; diff --git a/src/Common/clickhouse_malloc.cpp b/src/Common/clickhouse_malloc.cpp index 3f69ebdf58d..afdad3c6599 100644 --- a/src/Common/clickhouse_malloc.cpp +++ b/src/Common/clickhouse_malloc.cpp @@ -9,7 +9,11 @@ extern "C" void * clickhouse_malloc(size_t size) { void * res = malloc(size); if (res) - Memory::trackMemory(size); + { + AllocationTrace trace; + size_t actual_size = Memory::trackMemory(size, trace); + trace.onAlloc(res, actual_size); + } return res; } @@ -17,17 +21,29 @@ extern "C" void * clickhouse_calloc(size_t number_of_members, size_t size) { void * res = calloc(number_of_members, size); if (res) - Memory::trackMemory(number_of_members * size); + { + AllocationTrace trace; + size_t actual_size = Memory::trackMemory(number_of_members * size, trace); + trace.onAlloc(res, actual_size); + } return res; } extern "C" void * clickhouse_realloc(void * ptr, size_t size) { if (ptr) - Memory::untrackMemory(ptr); + { + AllocationTrace trace; + size_t actual_size = Memory::untrackMemory(ptr, trace); + trace.onFree(ptr, actual_size); + } void * res = realloc(ptr, size); if (res) - Memory::trackMemory(size); + { + AllocationTrace trace; + size_t actual_size = Memory::trackMemory(size, trace); + trace.onAlloc(res, actual_size); + } return res; } @@ -42,7 +58,9 @@ extern "C" void * clickhouse_reallocarray(void * ptr, size_t number_of_members, extern "C" void clickhouse_free(void * ptr) { - Memory::untrackMemory(ptr); + AllocationTrace trace; + size_t actual_size = Memory::untrackMemory(ptr, trace); + trace.onFree(ptr, actual_size); free(ptr); } @@ -50,6 +68,10 @@ extern "C" int clickhouse_posix_memalign(void ** memptr, size_t alignment, size_ { int res = posix_memalign(memptr, alignment, size); if (res == 0) - Memory::trackMemory(size); + { + AllocationTrace trace; + size_t actual_size = Memory::trackMemory(size, trace); + trace.onAlloc(*memptr, actual_size); + } return res; } diff --git a/src/Common/memory.h b/src/Common/memory.h index 4cb1c535e56..87ccdce070a 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -112,16 +112,19 @@ inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size, TAlign... align template ... TAlign> requires DB::OptionalArgument -inline ALWAYS_INLINE void trackMemory(std::size_t size, TAlign... align) +inline ALWAYS_INLINE size_t trackMemory(std::size_t size, AllocationTrace & trace, TAlign... align) { std::size_t actual_size = getActualAllocationSize(size, align...); - CurrentMemoryTracker::allocNoThrow(actual_size); + trace = CurrentMemoryTracker::allocNoThrow(actual_size); + return actual_size; } template ... TAlign> requires DB::OptionalArgument -inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0, TAlign... align [[maybe_unused]]) noexcept +inline ALWAYS_INLINE size_t untrackMemory(void * ptr [[maybe_unused]], AllocationTrace & trace, std::size_t size [[maybe_unused]] = 0, TAlign... align [[maybe_unused]]) noexcept { + std::size_t actual_size = 0; + try { #if USE_JEMALLOC @@ -130,23 +133,26 @@ inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t if (likely(ptr != nullptr)) { if constexpr (sizeof...(TAlign) == 1) - CurrentMemoryTracker::free(sallocx(ptr, MALLOCX_ALIGN(alignToSizeT(align...)))); + actual_size = sallocx(ptr, MALLOCX_ALIGN(alignToSizeT(align...))); else - CurrentMemoryTracker::free(sallocx(ptr, 0)); + actual_size = sallocx(ptr, 0); } #else if (size) - CurrentMemoryTracker::free(size); + actual_size = size; # if defined(_GNU_SOURCE) /// It's innaccurate resource free for sanitizers. malloc_usable_size() result is greater or equal to allocated size. else - CurrentMemoryTracker::free(malloc_usable_size(ptr)); + actual_size = malloc_usable_size(ptr); # endif #endif + trace = CurrentMemoryTracker::free(actual_size); } catch (...) { } + + return actual_size; } } diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp index 871ab750907..d0170bd820c 100644 --- a/src/Common/new_delete.cpp +++ b/src/Common/new_delete.cpp @@ -50,50 +50,74 @@ static struct InitializeJemallocZoneAllocatorForOSX void * operator new(std::size_t size) { - Memory::trackMemory(size); - return Memory::newImpl(size); + AllocationTrace trace; + std::size_t actual_size = Memory::trackMemory(size, trace); + void * ptr = Memory::newImpl(size); + trace.onAlloc(ptr, actual_size); + return ptr; } void * operator new(std::size_t size, std::align_val_t align) { - Memory::trackMemory(size, align); - return Memory::newImpl(size, align); + AllocationTrace trace; + std::size_t actual_size = Memory::trackMemory(size, trace, align); + void * ptr = Memory::newImpl(size, align); + trace.onAlloc(ptr, actual_size); + return ptr; } void * operator new[](std::size_t size) { - Memory::trackMemory(size); - return Memory::newImpl(size); + AllocationTrace trace; + std::size_t actual_size = Memory::trackMemory(size, trace); + void * ptr = Memory::newImpl(size); + trace.onAlloc(ptr, actual_size); + return ptr; } void * operator new[](std::size_t size, std::align_val_t align) { - Memory::trackMemory(size, align); - return Memory::newImpl(size, align); + AllocationTrace trace; + std::size_t actual_size = Memory::trackMemory(size, trace, align); + void * ptr = Memory::newImpl(size, align); + trace.onAlloc(ptr, actual_size); + return ptr; } void * operator new(std::size_t size, const std::nothrow_t &) noexcept { - Memory::trackMemory(size); - return Memory::newNoExept(size); + AllocationTrace trace; + std::size_t actual_size = Memory::trackMemory(size, trace); + void * ptr = Memory::newNoExept(size); + trace.onAlloc(ptr, actual_size); + return ptr; } void * operator new[](std::size_t size, const std::nothrow_t &) noexcept { - Memory::trackMemory(size); - return Memory::newNoExept(size); + AllocationTrace trace; + std::size_t actual_size = Memory::trackMemory(size, trace); + void * ptr = Memory::newNoExept(size); + trace.onAlloc(ptr, actual_size); + return ptr; } void * operator new(std::size_t size, std::align_val_t align, const std::nothrow_t &) noexcept { - Memory::trackMemory(size, align); - return Memory::newNoExept(size, align); + AllocationTrace trace; + std::size_t actual_size = Memory::trackMemory(size, trace, align); + void * ptr = Memory::newNoExept(size, align); + trace.onAlloc(ptr, actual_size); + return ptr; } void * operator new[](std::size_t size, std::align_val_t align, const std::nothrow_t &) noexcept { - Memory::trackMemory(size, align); - return Memory::newNoExept(size, align); + AllocationTrace trace; + std::size_t actual_size = Memory::trackMemory(size, trace, align); + void * ptr = Memory::newNoExept(size, align); + trace.onAlloc(ptr, actual_size); + return ptr; } /// delete @@ -109,48 +133,64 @@ void * operator new[](std::size_t size, std::align_val_t align, const std::nothr void operator delete(void * ptr) noexcept { - Memory::untrackMemory(ptr); + AllocationTrace trace; + std::size_t actual_size = Memory::untrackMemory(ptr, trace); + trace.onFree(ptr, actual_size); Memory::deleteImpl(ptr); } void operator delete(void * ptr, std::align_val_t align) noexcept { - Memory::untrackMemory(ptr, 0, align); + AllocationTrace trace; + std::size_t actual_size = Memory::untrackMemory(ptr, trace, 0, align); + trace.onFree(ptr, actual_size); Memory::deleteImpl(ptr); } void operator delete[](void * ptr) noexcept { - Memory::untrackMemory(ptr); + AllocationTrace trace; + std::size_t actual_size = Memory::untrackMemory(ptr, trace); + trace.onFree(ptr, actual_size); Memory::deleteImpl(ptr); } void operator delete[](void * ptr, std::align_val_t align) noexcept { - Memory::untrackMemory(ptr, 0, align); + AllocationTrace trace; + std::size_t actual_size = Memory::untrackMemory(ptr, trace, 0, align); + trace.onFree(ptr, actual_size); Memory::deleteImpl(ptr); } void operator delete(void * ptr, std::size_t size) noexcept { - Memory::untrackMemory(ptr, size); + AllocationTrace trace; + std::size_t actual_size = Memory::untrackMemory(ptr, trace, size); + trace.onFree(ptr, actual_size); Memory::deleteSized(ptr, size); } void operator delete(void * ptr, std::size_t size, std::align_val_t align) noexcept { - Memory::untrackMemory(ptr, size, align); + AllocationTrace trace; + std::size_t actual_size = Memory::untrackMemory(ptr, trace, size, align); + trace.onFree(ptr, actual_size); Memory::deleteSized(ptr, size, align); } void operator delete[](void * ptr, std::size_t size) noexcept { - Memory::untrackMemory(ptr, size); + AllocationTrace trace; + std::size_t actual_size = Memory::untrackMemory(ptr, trace, size); + trace.onFree(ptr, actual_size); Memory::deleteSized(ptr, size); } void operator delete[](void * ptr, std::size_t size, std::align_val_t align) noexcept { - Memory::untrackMemory(ptr, size, align); + AllocationTrace trace; + std::size_t actual_size = Memory::untrackMemory(ptr, trace, size, align); + trace.onFree(ptr, actual_size); Memory::deleteSized(ptr, size, align); } diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 49588d490f5..d1d12b6d7a6 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -104,6 +104,9 @@ void TraceCollector::run() Int64 size; readPODBinary(size, in); + UInt64 ptr; + readPODBinary(ptr, in); + ProfileEvents::Event event; readPODBinary(event, in); @@ -119,7 +122,7 @@ void TraceCollector::run() UInt64 time = static_cast(ts.tv_sec * 1000000000LL + ts.tv_nsec); UInt64 time_in_microseconds = static_cast((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); - TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size, event, increment}; + TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size, ptr, event, increment}; trace_log->add(element); } } diff --git a/src/Interpreters/TraceLog.cpp b/src/Interpreters/TraceLog.cpp index 0408ebe504b..cd5f965a679 100644 --- a/src/Interpreters/TraceLog.cpp +++ b/src/Interpreters/TraceLog.cpp @@ -38,6 +38,7 @@ NamesAndTypesList TraceLogElement::getNamesAndTypes() {"query_id", std::make_shared()}, {"trace", std::make_shared(std::make_shared())}, {"size", std::make_shared()}, + {"ptr", std::make_shared()}, {"event", std::make_shared(std::make_shared())}, {"increment", std::make_shared()}, }; @@ -57,6 +58,7 @@ void TraceLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(query_id.data(), query_id.size()); columns[i++]->insert(trace); columns[i++]->insert(size); + columns[i++]->insert(ptr); String event_name; if (event != ProfileEvents::end()) diff --git a/src/Interpreters/TraceLog.h b/src/Interpreters/TraceLog.h index c481f033a72..71aec0b50c4 100644 --- a/src/Interpreters/TraceLog.h +++ b/src/Interpreters/TraceLog.h @@ -27,8 +27,10 @@ struct TraceLogElement UInt64 thread_id{}; String query_id{}; Array trace{}; - /// Allocation size in bytes for TraceType::Memory. + /// Allocation size in bytes for TraceType::Memory and TraceType::MemorySample. Int64 size{}; + /// Allocation ptr for TraceType::MemorySample. + UInt64 ptr{}; /// ProfileEvent for TraceType::ProfileEvent. ProfileEvents::Event event{ProfileEvents::end()}; /// Increment of profile event for TraceType::ProfileEvent. diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 76d69cc6b7d..a833da7064f 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -88,6 +88,10 @@ MergeListElement::MergeListElement( /// thread_group::memory_tracker, but MemoryTrackerThreadSwitcher will reset parent). memory_tracker.setProfilerStep(settings.memory_profiler_step); memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); + /// Specify sample probability also for current thread to track more deallocations. + if (auto * thread_memory_tracker = DB::CurrentThread::getMemoryTracker()) + thread_memory_tracker->setSampleProbability(settings.memory_profiler_sample_probability); + memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator); if (settings.memory_tracker_fault_probability > 0.0) memory_tracker.setFaultProbability(settings.memory_tracker_fault_probability); From 6f737f07411f1a65e8a27da8c74db9e282e961d0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 16 Jan 2023 15:48:44 +0000 Subject: [PATCH 0005/2047] Inline part of AllocationTrace. --- src/Common/AllocationTrace.h | 22 ++++++++++++++++++++-- src/Common/MemoryTracker.cpp | 10 ++-------- 2 files changed, 22 insertions(+), 10 deletions(-) diff --git a/src/Common/AllocationTrace.h b/src/Common/AllocationTrace.h index 332808c8015..97564596e8c 100644 --- a/src/Common/AllocationTrace.h +++ b/src/Common/AllocationTrace.h @@ -1,5 +1,6 @@ #pragma once #include +#include /// This is a structure which is returned by MemoryTracker. /// Methods onAlloc/onFree should be called after actual memory allocation if it succeed. @@ -9,8 +10,25 @@ struct AllocationTrace AllocationTrace() = default; explicit AllocationTrace(double sample_probability_); - void onAlloc(void * ptr, size_t size) const; - void onFree(void * ptr, size_t size) const; + ALWAYS_INLINE void onAlloc(void * ptr, size_t size) const + { + if (likely(sample_probability == 0)) + return; + onAllocImpl(ptr, size); + } + + ALWAYS_INLINE void onFree(void * ptr, size_t size) const + { + if (likely(sample_probability == 0)) + return; + + onFreeImpl(ptr, size); + } + +private: double sample_probability = 0; + + void onAllocImpl(void * ptr, size_t size) const; + void onFreeImpl(void * ptr, size_t size) const; }; diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index d0d0d6b8686..c62cb636dc5 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -108,11 +108,8 @@ AllocationTrace getAllocationTrace(std::optional & sample_probability) AllocationTrace::AllocationTrace(double sample_probability_) : sample_probability(sample_probability_) {} -void AllocationTrace::onAlloc(void * ptr, size_t size) const +void AllocationTrace::onAllocImpl(void * ptr, size_t size) const { - if (likely(sample_probability == 0)) - return; - if (sample_probability < 1 && !shouldTrackAllocation(sample_probability, ptr)) return; @@ -120,11 +117,8 @@ void AllocationTrace::onAlloc(void * ptr, size_t size) const DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = Int64(size), .ptr = ptr}); } -void AllocationTrace::onFree(void * ptr, size_t size) const +void AllocationTrace::onFreeImpl(void * ptr, size_t size) const { - if (likely(sample_probability == 0)) - return; - if (sample_probability < 1 && !shouldTrackAllocation(sample_probability, ptr)) return; From 692e08628c92ab726c54e0e80c93cea2cb7d177e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 17 Jan 2023 14:44:42 +0000 Subject: [PATCH 0006/2047] Better inlining --- src/Common/AllocationTrace.h | 2 +- src/Common/MemoryTracker.cpp | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Common/AllocationTrace.h b/src/Common/AllocationTrace.h index 97564596e8c..0b91f442d63 100644 --- a/src/Common/AllocationTrace.h +++ b/src/Common/AllocationTrace.h @@ -8,7 +8,7 @@ struct AllocationTrace { AllocationTrace() = default; - explicit AllocationTrace(double sample_probability_); + explicit AllocationTrace(double sample_probability_) : sample_probability(sample_probability_) {} ALWAYS_INLINE void onAlloc(void * ptr, size_t size) const { diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index c62cb636dc5..1862c61a222 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -106,8 +106,6 @@ AllocationTrace getAllocationTrace(std::optional & sample_probability) } -AllocationTrace::AllocationTrace(double sample_probability_) : sample_probability(sample_probability_) {} - void AllocationTrace::onAllocImpl(void * ptr, size_t size) const { if (sample_probability < 1 && !shouldTrackAllocation(sample_probability, ptr)) From 03adbfdef773f43b192acd580f227cf1f7d9815e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 18 Jan 2023 13:28:57 +0000 Subject: [PATCH 0007/2047] Simplify code a bit. --- src/Common/AllocationTrace.h | 4 +-- src/Common/MemoryTracker.cpp | 57 ++++++++++++++---------------------- src/Common/MemoryTracker.h | 6 ++-- 3 files changed, 27 insertions(+), 40 deletions(-) diff --git a/src/Common/AllocationTrace.h b/src/Common/AllocationTrace.h index 0b91f442d63..3b0766f8f4d 100644 --- a/src/Common/AllocationTrace.h +++ b/src/Common/AllocationTrace.h @@ -12,7 +12,7 @@ struct AllocationTrace ALWAYS_INLINE void onAlloc(void * ptr, size_t size) const { - if (likely(sample_probability == 0)) + if (likely(sample_probability < 0)) return; onAllocImpl(ptr, size); @@ -20,7 +20,7 @@ struct AllocationTrace ALWAYS_INLINE void onFree(void * ptr, size_t size) const { - if (likely(sample_probability == 0)) + if (likely(sample_probability < 0)) return; onFreeImpl(ptr, size); diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 1862c61a222..22a6381cc67 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -88,22 +88,6 @@ bool shouldTrackAllocation(DB::Float64 probability, void * ptr) return sipHash64(uintptr_t(ptr)) < std::numeric_limits::max() * probability; } -AllocationTrace updateAllocationTrace(AllocationTrace trace, const std::optional & sample_probability) -{ - if (unlikely(sample_probability)) - return AllocationTrace(*sample_probability); - - return trace; -} - -AllocationTrace getAllocationTrace(std::optional & sample_probability) -{ - if (unlikely(sample_probability)) - return AllocationTrace(*sample_probability); - - return AllocationTrace(0); -} - } void AllocationTrace::onAllocImpl(void * ptr, size_t size) const @@ -175,11 +159,14 @@ void MemoryTracker::logMemoryUsage(Int64 current) const } -AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker) +AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker, double _sample_probability) { if (size < 0) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Negative size ({}) is passed to MemoryTracker. It is a bug.", size); + if (_sample_probability < 0) + _sample_probability = sample_probability; + if (MemoryTrackerBlockerInThread::isBlocked(level)) { if (level == VariableContext::Global) @@ -196,12 +183,10 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed if (auto * loaded_next = parent.load(std::memory_order_relaxed)) { MemoryTracker * tracker = level == VariableContext::Process ? this : query_tracker; - return updateAllocationTrace( - loaded_next->allocImpl(size, throw_if_memory_exceeded, tracker), - sample_probability); + return loaded_next->allocImpl(size, throw_if_memory_exceeded, tracker, _sample_probability); } - return getAllocationTrace(sample_probability); + return AllocationTrace(_sample_probability); } /** Using memory_order_relaxed means that if allocations are done simultaneously, @@ -348,12 +333,10 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed if (auto * loaded_next = parent.load(std::memory_order_relaxed)) { MemoryTracker * tracker = level == VariableContext::Process ? this : query_tracker; - return updateAllocationTrace( - loaded_next->allocImpl(size, throw_if_memory_exceeded, tracker), - sample_probability); + return loaded_next->allocImpl(size, throw_if_memory_exceeded, tracker, _sample_probability); } - return getAllocationTrace(sample_probability); + return AllocationTrace(_sample_probability); } void MemoryTracker::adjustWithUntrackedMemory(Int64 untracked_memory) @@ -380,8 +363,11 @@ bool MemoryTracker::updatePeak(Int64 will_be, bool log_memory_usage) return false; } -AllocationTrace MemoryTracker::free(Int64 size) +AllocationTrace MemoryTracker::free(Int64 size, double _sample_probability) { + if (_sample_probability < 0) + _sample_probability = sample_probability; + if (MemoryTrackerBlockerInThread::isBlocked(level)) { if (level == VariableContext::Global) @@ -395,9 +381,9 @@ AllocationTrace MemoryTracker::free(Int64 size) /// Since the MemoryTrackerBlockerInThread should respect the level, we should go to the next parent. if (auto * loaded_next = parent.load(std::memory_order_relaxed)) - return updateAllocationTrace(loaded_next->free(size), sample_probability); + return loaded_next->free(size, _sample_probability); - return getAllocationTrace(sample_probability); + return AllocationTrace(_sample_probability); } Int64 accounted_size = size; @@ -425,15 +411,16 @@ AllocationTrace MemoryTracker::free(Int64 size) if (auto * overcommit_tracker_ptr = overcommit_tracker.load(std::memory_order_relaxed)) overcommit_tracker_ptr->tryContinueQueryExecutionAfterFree(accounted_size); - AllocationTrace res = getAllocationTrace(sample_probability); - if (auto * loaded_next = parent.load(std::memory_order_relaxed)) - res = updateAllocationTrace(loaded_next->free(size), sample_probability); - + /// free should never throw, we can update metric early. auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) CurrentMetrics::sub(metric_loaded, accounted_size); - return res; + AllocationTrace res(_sample_probability); + if (auto * loaded_next = parent.load(std::memory_order_relaxed)) + return loaded_next->free(size, _sample_probability); + + return AllocationTrace(_sample_probability); } @@ -520,8 +507,8 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) double MemoryTracker::getSampleProbability() { - if (sample_probability) - return *sample_probability; + if (sample_probability >= 0) + return sample_probability; if (auto * loaded_next = parent.load(std::memory_order_relaxed)) return loaded_next->getSampleProbability(); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index e1f61b1585a..766546ae127 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -67,7 +67,7 @@ private: double fault_probability = 0; /// To randomly sample allocations and deallocations in trace_log. - std::optional sample_probability; + double sample_probability = -1; /// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy). /// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker. @@ -92,8 +92,8 @@ private: /// allocImpl(...) and free(...) should not be used directly friend struct CurrentMemoryTracker; - [[nodiscard]] AllocationTrace allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); - [[nodiscard]] AllocationTrace free(Int64 size); + [[nodiscard]] AllocationTrace allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr, double _sample_probability = -1.0); + [[nodiscard]] AllocationTrace free(Int64 size, double _sample_probability = -1.0); public: static constexpr auto USAGE_EVENT_NAME = "MemoryTrackerUsage"; From 4ad9898217bcc7d6f9eb9fb65a0b4e2f07d485d6 Mon Sep 17 00:00:00 2001 From: Eridanus <45489268+Eridanus117@users.noreply.github.com> Date: Sat, 28 Jan 2023 04:19:16 +0000 Subject: [PATCH 0008/2047] fix version --- src/Core/SettingsChangesHistory.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 63452e2d171..338021c965f 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,12 +85,12 @@ static std::map sett {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, + {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}, + {"optimize_rewrite_count_distinct_if", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.10", {{"optimize_rewrite_count_distinct_if", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, From e4032e7eecbb28fb7d66dda8ea4e8ff3aaf1d9bf Mon Sep 17 00:00:00 2001 From: Eridanus <45489268+Eridanus117@users.noreply.github.com> Date: Sat, 28 Jan 2023 09:43:56 +0000 Subject: [PATCH 0009/2047] fix test --- ...=> 02541_optimize_rewrite_count_distinct_if.reference} | 4 ++-- ...f.sql => 02541_optimize_rewrite_count_distinct_if.sql} | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) rename tests/queries/0_stateless/{02456_optimize_rewrite_count_distinct_if.reference => 02541_optimize_rewrite_count_distinct_if.reference} (66%) rename tests/queries/0_stateless/{02456_optimize_rewrite_count_distinct_if.sql => 02541_optimize_rewrite_count_distinct_if.sql} (78%) diff --git a/tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.reference b/tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.reference similarity index 66% rename from tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.reference rename to tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.reference index 85aab5282d3..6108ea25324 100644 --- a/tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.reference +++ b/tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.reference @@ -1,6 +1,6 @@ 2 SELECT countDistinctIf(number % 10, (number % 5) = 2) -FROM numbers_mt(100000000) +FROM numbers_mt(1000000) 2 SELECT uniqExactIf(number % 10, (number % 5) = 2) -FROM numbers_mt(100000000) +FROM numbers_mt(1000000) diff --git a/tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.sql b/tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.sql similarity index 78% rename from tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.sql rename to tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.sql index 0b7ab21e035..ceda9a16f48 100644 --- a/tests/queries/0_stateless/02456_optimize_rewrite_count_distinct_if.sql +++ b/tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.sql @@ -1,8 +1,8 @@ SET optimize_rewrite_count_distinct_if = FALSE; -SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(100000000); -EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(100000000); +SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(1000000); +EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(1000000); SET optimize_rewrite_count_distinct_if = TRUE; -SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(100000000); -EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(100000000); +SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(1000000); +EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(1000000); From 7118dae80b3913b4b04a8439e760df9a5136202b Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 5 Feb 2023 11:32:11 +0000 Subject: [PATCH 0010/2047] rewrite for Analyzer --- src/Analyzer/Passes/CustomizeFunctionsPass.cpp | 7 +++++++ src/Core/Settings.h | 2 +- src/Interpreters/TreeRewriter.cpp | 2 +- ...ptimize_rewrite_count_distinct_if.reference | 6 ------ ...2541_optimize_rewrite_count_distinct_if.sql | 8 -------- ...ith_count_distinct_implementation.reference | 12 ++++++++++++ ...t_if_with_count_distinct_implementation.sql | 18 ++++++++++++++++++ 7 files changed, 39 insertions(+), 16 deletions(-) delete mode 100644 tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.reference delete mode 100644 tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.sql create mode 100644 tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.reference create mode 100644 tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.sql diff --git a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp index eef31afdc50..983312728be 100644 --- a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp +++ b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp @@ -54,6 +54,13 @@ public: function_name_lowercase = Poco::toLower(function_name); } + if (settings.rewrite_count_distinct_if_with_count_distinct_implementation && function_name_lowercase == "countdistinctif") + { + resolveAggregateOrWindowFunctionNode(*function_node, count_distinct_implementation_function_name + "If"); + function_name = function_node->getFunctionName(); + function_name_lowercase = Poco::toLower(function_name); + } + /// Replace aggregateFunctionIfDistinct into aggregateFunctionDistinctIf to make execution more optimal if (function_name_lowercase.ends_with("ifdistinct")) { diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e6045172d03..7b38e01b153 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -484,7 +484,7 @@ class IColumn; M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ - M(Bool, optimize_rewrite_count_distinct_if, false, "Rewrite countDistinctIf with count_distinct_implementation configuration", 0) \ + M(Bool, rewrite_count_distinct_if_with_count_distinct_implementation, false, "Rewrite countDistinctIf with count_distinct_implementation configuration", 0) \ M(Bool, convert_query_to_cnf, false, "Convert SELECT query to CNF", 0) \ M(Bool, optimize_or_like_chain, false, "Optimize multiple OR LIKE into multiMatchAny. This optimization should not be enabled by default, because it defies index analysis in some cases.", 0) \ M(Bool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \ diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index ad4f0edf102..e695e8d6bb7 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1413,7 +1413,7 @@ void TreeRewriter::normalize( CustomizeIfDistinctVisitor::Data data_distinct_if{"DistinctIf"}; CustomizeIfDistinctVisitor(data_distinct_if).visit(query); - if (settings.optimize_rewrite_count_distinct_if) + if (settings.rewrite_count_distinct_if_with_count_distinct_implementation) { CustomizeCountDistinctIfVisitor::Data data_count_distinct_if{settings.count_distinct_implementation.toString() + "If"}; CustomizeCountDistinctIfVisitor(data_count_distinct_if).visit(query); diff --git a/tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.reference b/tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.reference deleted file mode 100644 index 6108ea25324..00000000000 --- a/tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.reference +++ /dev/null @@ -1,6 +0,0 @@ -2 -SELECT countDistinctIf(number % 10, (number % 5) = 2) -FROM numbers_mt(1000000) -2 -SELECT uniqExactIf(number % 10, (number % 5) = 2) -FROM numbers_mt(1000000) diff --git a/tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.sql b/tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.sql deleted file mode 100644 index ceda9a16f48..00000000000 --- a/tests/queries/0_stateless/02541_optimize_rewrite_count_distinct_if.sql +++ /dev/null @@ -1,8 +0,0 @@ -SET optimize_rewrite_count_distinct_if = FALSE; -SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(1000000); -EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(1000000); - -SET optimize_rewrite_count_distinct_if = TRUE; -SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(1000000); -EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers_mt(1000000); - diff --git a/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.reference b/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.reference new file mode 100644 index 00000000000..f187b72b8ea --- /dev/null +++ b/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.reference @@ -0,0 +1,12 @@ +2 +SELECT countDistinctIf(number % 10, (number % 5) = 2) +FROM numbers(1000) +2 +SELECT uniqExactIf(number % 10, (number % 5) = 2) +FROM numbers(1000) +2 +SELECT countDistinctIf(number % 10, (number % 5) = 2) +FROM numbers(1000) +2 +SELECT uniqExactIf(number % 10, (number % 5) = 2) +FROM numbers(1000) diff --git a/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.sql b/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.sql new file mode 100644 index 00000000000..a06a22db55e --- /dev/null +++ b/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.sql @@ -0,0 +1,18 @@ +-- Tags: no-parallel +SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); +EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); + +-- disable by default +SET rewrite_count_distinct_if_with_count_distinct_implementation = 1; +SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); +EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); + +SET allow_experimental_analyzer = 1; + +SET rewrite_count_distinct_if_with_count_distinct_implementation = 0; +SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); +EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); + +SET rewrite_count_distinct_if_with_count_distinct_implementation = 1; +SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); +EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); From be99925ab51ecbb862e178a3fc14aa54db219aef Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 5 Feb 2023 11:45:15 +0000 Subject: [PATCH 0011/2047] fix --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 338021c965f..37330b4034f 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,6 +80,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"23.2", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, From 9c92c16d4fe3fdd4c0dfcfe2a8f6a60cf14494b0 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 5 Feb 2023 11:50:40 +0000 Subject: [PATCH 0012/2047] fix --- src/Core/SettingsChangesHistory.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 37330b4034f..8de3ce19eb4 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -86,8 +86,7 @@ static std::map sett {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}, - {"optimize_rewrite_count_distinct_if", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, + {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, From de822cf05f903dd36af679e219c18b07dacb47dc Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 14 Feb 2023 03:21:08 +0000 Subject: [PATCH 0013/2047] fix conflict --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 1a69756394e..5c38a9b0aec 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -82,7 +82,7 @@ static std::map sett { {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, + {"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, From 548d79c2e80bb23f246c63fc7e33d0c01eb6b944 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 2 Mar 2023 12:31:09 +0000 Subject: [PATCH 0014/2047] Remove perf test duplicate_order_by_and_distinct.xml --- tests/performance/duplicate_order_by_and_distinct.xml | 8 -------- 1 file changed, 8 deletions(-) delete mode 100644 tests/performance/duplicate_order_by_and_distinct.xml diff --git a/tests/performance/duplicate_order_by_and_distinct.xml b/tests/performance/duplicate_order_by_and_distinct.xml deleted file mode 100644 index e36bc470512..00000000000 --- a/tests/performance/duplicate_order_by_and_distinct.xml +++ /dev/null @@ -1,8 +0,0 @@ - - 1 - - - SELECT * FROM (SELECT CounterID, EventDate FROM hits_10m_single ORDER BY CounterID DESC) ORDER BY EventDate, CounterID FORMAT Null - SELECT DISTINCT * FROM (SELECT DISTINCT CounterID, EventDate FROM hits_10m_single) FORMAT Null - SELECT DISTINCT * FROM (SELECT DISTINCT CounterID, EventDate FROM hits_10m_single ORDER BY CounterID DESC) ORDER BY toStartOfWeek(EventDate) FORMAT Null - From 4f27391a72d3c87b351c5dbe13bd3c1c8aaeba08 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 31 Mar 2023 13:37:41 +0300 Subject: [PATCH 0015/2047] Revert "Revert "Randomize JIT settings in tests"" --- tests/clickhouse-test | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a355c2f8e73..fa88bc19efd 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -558,6 +558,9 @@ class SettingsRandomizer: "enable_memory_bound_merging_of_aggregation_results": lambda: random.randint( 0, 1 ), + "min_count_to_compile_expression": lambda: random.choice([0, 3]), + "min_count_to_compile_aggregate_expression": lambda: random.choice([0, 3]), + "min_count_to_compile_sort_description": lambda: random.choice([0, 3]), } @staticmethod From b3854b09dc93858c74f7f76c6ae4cefa0ecb0626 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 5 Apr 2023 14:33:34 +0200 Subject: [PATCH 0016/2047] Fix outdated cache configuration in tests s3_storage_policy_by_default.xml --- tests/config/config.d/s3_storage_policy_by_default.xml | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/config/config.d/s3_storage_policy_by_default.xml b/tests/config/config.d/s3_storage_policy_by_default.xml index 9685512a12a..1a23e99b759 100644 --- a/tests/config/config.d/s3_storage_policy_by_default.xml +++ b/tests/config/config.d/s3_storage_policy_by_default.xml @@ -6,14 +6,18 @@ http://localhost:11111/test/test/ clickhouse clickhouse - 1 - 22548578304 + + cache + 1Gi + /var/lib/clickhouse/s3_cache/ + s3 + -
s3
+
cached_s3
From 8bd6fa471de9333878fafb8f9229cae8204faf37 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 5 Apr 2023 17:36:20 +0200 Subject: [PATCH 0017/2047] Remove too noisy logging --- src/Interpreters/Cache/LRUFileCachePriority.cpp | 6 ------ src/Interpreters/Cache/LRUFileCachePriority.h | 2 -- 2 files changed, 8 deletions(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index c20379e1fc1..ccb086ec2c7 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -34,8 +34,6 @@ IFileCachePriority::WriteIterator LRUFileCachePriority::add(const Key & key, siz CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size); CurrentMetrics::add(CurrentMetrics::FilesystemCacheElements); - LOG_TEST(log, "Added entry into LRU queue, key: {}, offset: {}", key.toString(), offset); - return std::make_shared(this, iter); } @@ -54,8 +52,6 @@ void LRUFileCachePriority::removeAll(std::lock_guard &) CurrentMetrics::sub(CurrentMetrics::FilesystemCacheSize, cache_size); CurrentMetrics::sub(CurrentMetrics::FilesystemCacheElements, queue.size()); - LOG_TEST(log, "Removed all entries from LRU queue"); - queue.clear(); cache_size = 0; } @@ -88,8 +84,6 @@ void LRUFileCachePriority::LRUFileCacheIterator::removeAndGetNext(std::lock_guar CurrentMetrics::sub(CurrentMetrics::FilesystemCacheSize, queue_iter->size); CurrentMetrics::sub(CurrentMetrics::FilesystemCacheElements); - LOG_TEST(cache_priority->log, "Removed entry from LRU queue, key: {}, offset: {}", queue_iter->key.toString(), queue_iter->offset); - queue_iter = cache_priority->queue.erase(queue_iter); } diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 2345d3c47db..bc4b6fd88e0 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -2,7 +2,6 @@ #include #include -#include namespace DB { @@ -33,7 +32,6 @@ public: private: LRUQueue queue; - Poco::Logger * log = &Poco::Logger::get("LRUFileCachePriority"); }; class LRUFileCachePriority::LRUFileCacheIterator : public IFileCachePriority::IIterator From 2e663485517e3aa8d814c4b3fb6732ec30e2e189 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Apr 2023 15:40:32 +0200 Subject: [PATCH 0018/2047] Less logging --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 31 ++----------------- .../0_stateless/02344_show_caches.reference | 3 +- 2 files changed, 4 insertions(+), 30 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 72346787cfb..c191d8ac043 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -247,7 +247,6 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil } else { - LOG_TEST(log, "Bypassing cache because `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` option is used"); read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; return getRemoteFSReadBuffer(*file_segment, read_type); } @@ -481,13 +480,6 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext() [[maybe_unused]] const auto & range = file_segment->range(); chassert(file_offset_of_buffer_end > range.right); - LOG_TEST( - log, - "Removing file segment: {}, downloader: {}, state: {}", - file_segment->range().toString(), - file_segment->getDownloader(), - file_segment->state()); - /// Do not hold pointer to file segment if it is not needed anymore /// so can become releasable and can be evicted from cache. file_segment->completeWithoutState(); @@ -501,7 +493,6 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext() if (read_type == ReadType::CACHED) (*current_file_segment_it)->incrementHitsCount(); - LOG_TEST(log, "New segment: {}", (*current_file_segment_it)->range().toString()); return true; } @@ -532,7 +523,6 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) /// downloaded because it intersects with the range he needs. /// But then first downloader fails and second must continue. In this case we need to /// download from offset a'' < a', but return buffer from offset a'. - LOG_TEST(log, "Bytes to predownload: {}, caller_id: {}", bytes_to_predownload, FileSegment::getCallerId()); chassert(implementation_buffer->getFileOffsetOfBufferEnd() == file_segment->getCurrentWriteOffset()); size_t current_offset = file_segment->getCurrentWriteOffset(); @@ -637,8 +627,6 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) bytes_to_predownload = 0; file_segment->completeWithState(FileSegment::State::PARTIALLY_DOWNLOADED_NO_CONTINUATION); - LOG_TEST(log, "Bypassing cache because for {}", file_segment->getInfoForLog()); - read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; swap(*implementation_buffer); @@ -692,8 +680,6 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() auto current_write_offset = file_segment->getCurrentWriteOffset(); bool cached_part_is_finished = current_write_offset == file_offset_of_buffer_end; - LOG_TEST(log, "Current write offset: {}, file offset of buffer end: {}", current_write_offset, file_offset_of_buffer_end); - if (cached_part_is_finished) { /// TODO: makes sense to reuse local file reader if we return here with CACHED read type again? @@ -837,14 +823,6 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() auto & file_segment = *current_file_segment_it; auto current_read_range = file_segment->range(); - LOG_TEST( - log, - "Current count: {}, position: {}, read range: {}, file segment: {}", - implementation_buffer->count(), - implementation_buffer->getPosition(), - implementation_buffer->getRemainingReadRange().toString(), - file_segment->getInfoForLog()); - chassert(current_read_range.left <= file_offset_of_buffer_end); chassert(current_read_range.right >= file_offset_of_buffer_end); @@ -901,12 +879,6 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() size = implementation_buffer->buffer().size(); - LOG_TEST( - log, - "Read {} bytes, read type {}, position: {}, offset: {}, remaining read range: {}", - size, toString(read_type), implementation_buffer->getPosition(), - implementation_buffer->getFileOffsetOfBufferEnd(), implementation_buffer->getRemainingReadRange().toString()); - if (read_type == ReadType::CACHED) { ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); @@ -988,12 +960,13 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() LOG_TEST( log, - "Key: {}. Returning with {} bytes, buffer position: {} (offset: {}, predownloaded: {}), " + "Key: {}. Returning with {} bytes (actually read: {}), buffer position: {} (offset: {}, predownloaded: {}), " "buffer available: {}, current range: {}, current offset: {}, file segment state: {}, " "current write offset: {}, read_type: {}, reading until position: {}, started with offset: {}, " "remaining ranges: {}", getHexUIntLowercase(cache_key), working_buffer.size(), + size, getPosition(), offset(), needed_to_predownload, diff --git a/tests/queries/0_stateless/02344_show_caches.reference b/tests/queries/0_stateless/02344_show_caches.reference index 2ee4f902ba1..e0afce2ad40 100644 --- a/tests/queries/0_stateless/02344_show_caches.reference +++ b/tests/queries/0_stateless/02344_show_caches.reference @@ -1,9 +1,10 @@ cached_azure s3_cache_2 -s3_cache s3_cache_3 +s3_cache s3_cache_multi s3_cache_4 +cached_s3 s3_cache_5 s3_cache_small_segment_size local_cache From 8b98701edcacaa6c3527f340526a6f36f7d2f106 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Apr 2023 22:04:43 +0200 Subject: [PATCH 0019/2047] Remove unfixably flaky test --- tests/config/config.d/text_log.xml | 2 +- .../0_stateless/02344_show_caches.reference | 15 --------------- tests/queries/0_stateless/02344_show_caches.sql | 2 -- 3 files changed, 1 insertion(+), 18 deletions(-) delete mode 100644 tests/queries/0_stateless/02344_show_caches.reference delete mode 100644 tests/queries/0_stateless/02344_show_caches.sql diff --git a/tests/config/config.d/text_log.xml b/tests/config/config.d/text_log.xml index dce4942d952..d3608c5439d 100644 --- a/tests/config/config.d/text_log.xml +++ b/tests/config/config.d/text_log.xml @@ -2,6 +2,6 @@ system text_log
- 7500 + 4000
diff --git a/tests/queries/0_stateless/02344_show_caches.reference b/tests/queries/0_stateless/02344_show_caches.reference deleted file mode 100644 index e0afce2ad40..00000000000 --- a/tests/queries/0_stateless/02344_show_caches.reference +++ /dev/null @@ -1,15 +0,0 @@ -cached_azure -s3_cache_2 -s3_cache_3 -s3_cache -s3_cache_multi -s3_cache_4 -cached_s3 -s3_cache_5 -s3_cache_small_segment_size -local_cache -s3_cache_6 -s3_cache_small -local_cache_2 -local_cache_3 -s3_cache_multi_2 diff --git a/tests/queries/0_stateless/02344_show_caches.sql b/tests/queries/0_stateless/02344_show_caches.sql deleted file mode 100644 index 56f00b89051..00000000000 --- a/tests/queries/0_stateless/02344_show_caches.sql +++ /dev/null @@ -1,2 +0,0 @@ --- Tags: no-fasttest, no-replicated-database, no-cpu-aarch64 -SHOW FILESYSTEM CACHES; From b5c8c7fcc995b3206e9176903b3c3362483addff Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 7 Apr 2023 12:41:15 +0200 Subject: [PATCH 0020/2047] Better way to decide cache is readonly --- src/Interpreters/Cache/FileSegment.cpp | 1 - src/Storages/MergeTree/MergeTreeSequentialSource.cpp | 1 + src/Storages/MergeTree/MutateTask.cpp | 1 + 3 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index bd4554c6532..ff929c3ac47 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -228,7 +228,6 @@ void FileSegment::assertIsDownloaderUnlocked(const std::string & operation, std: { auto caller = getCallerId(); auto current_downloader = getDownloaderUnlocked(segment_lock); - LOG_TEST(log, "Downloader id: {}, caller id: {}", current_downloader, caller); if (caller != current_downloader) { diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 4539e0b36c5..196bac75722 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -134,6 +134,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( ReadSettings read_settings; if (read_with_direct_io) read_settings.direct_io_threshold = 1; + read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; MergeTreeReaderSettings reader_settings = { diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 9f7a12745c6..78319a92821 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1677,6 +1677,7 @@ bool MutateTask::prepare() context_for_reading->setSetting("max_threads", 1); context_for_reading->setSetting("allow_asynchronous_read_from_io_pool_for_merge_tree", false); context_for_reading->setSetting("max_streams_for_merge_tree_reading", Field(0)); + context_for_reading->setSetting("read_from_filesystem_cache_if_exists_otherwise_bypass_cache", 1); MutationHelpers::splitMutationCommands(ctx->source_part, ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames); From eae5df8d0030e54bceae74ad64481f948f438862 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 7 Apr 2023 17:58:41 +0200 Subject: [PATCH 0021/2047] Less logs --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 2 -- src/Interpreters/Cache/FileSegment.cpp | 4 ++-- .../00002_log_and_exception_messages_formatting.reference | 2 +- .../02241_filesystem_cache_on_write_operations.reference | 8 ++++---- 4 files changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index c191d8ac043..1b00af071bf 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -909,8 +909,6 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert( std::next(current_file_segment_it) == file_segments_holder->file_segments.end() || file_segment->getCurrentWriteOffset() == implementation_buffer->getFileOffsetOfBufferEnd()); - - LOG_TEST(log, "Successfully written {} bytes", size); } else { diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index ff929c3ac47..d2fedf68ca5 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -93,7 +93,7 @@ FileSegment::State FileSegment::state() const void FileSegment::setDownloadState(State state) { - LOG_TEST(log, "Updated state from {} to {}", stateToString(download_state), stateToString(state)); + // LOG_TEST(log, "Updated state from {} to {}", stateToString(download_state), stateToString(state)); download_state = state; } @@ -220,7 +220,7 @@ void FileSegment::resetDownloader() void FileSegment::resetDownloaderUnlocked(std::unique_lock & /* segment_lock */) { - LOG_TEST(log, "Resetting downloader from {}", downloader_id); + // LOG_TEST(log, "Resetting downloader from {}", downloader_id); downloader_id.clear(); } diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference index fddfbd49de3..b44f787df52 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -10,6 +10,6 @@ noisy Info messages 0.05 noisy Warning messages 0.01 noisy Error messages 0.02 no Fatal messages 0 -number of too noisy messages 3 +number of too noisy messages 5 number of noisy messages 10 incorrect patterns 15 diff --git a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference index bbca9bbbfee..5e55e4c3c86 100644 --- a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference +++ b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference @@ -24,8 +24,8 @@ state: DOWNLOADED 7 7 21 -31 -38 +28 +35 5010500 18816 Using storage policy: local_cache @@ -54,7 +54,7 @@ state: DOWNLOADED 7 7 21 -31 -38 +28 +35 5010500 18816 From 1d0a9f4551473b9b0cc2d764fd5f12ef09a579b6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 Apr 2023 12:58:51 +0200 Subject: [PATCH 0022/2047] Less logging... --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 61 ++++++++++--------- ...nd_exception_messages_formatting.reference | 2 +- 2 files changed, 32 insertions(+), 31 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 1b00af071bf..4ae65ecd859 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -331,7 +331,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegmentPtr & fil /// ^ /// file_offset_of_buffer_end - LOG_TEST(log, "Predownload. File segment info: {}", file_segment->getInfoForLog()); + // LOG_TEST(log, "Predownload. File segment info: {}", file_segment->getInfoForLog()); chassert(file_offset_of_buffer_end > file_segment->getCurrentWriteOffset()); bytes_to_predownload = file_offset_of_buffer_end - file_segment->getCurrentWriteOffset(); chassert(bytes_to_predownload < file_segment->range().size()); @@ -387,12 +387,12 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se chassert(file_segment->range() == range); chassert(file_offset_of_buffer_end >= range.left && file_offset_of_buffer_end <= range.right); - LOG_TEST( - log, - "Current file segment: {}, read type: {}, current file offset: {}", - range.toString(), - toString(read_type), - file_offset_of_buffer_end); + // LOG_TEST( + // log, + // "Current file segment: {}, read type: {}, current file offset: {}", + // range.toString(), + // toString(read_type), + // file_offset_of_buffer_end); read_buffer_for_file_segment->setReadUntilPosition(range.right + 1); /// [..., range.right] @@ -469,7 +469,7 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegmentPtr & file_se bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext() { - LOG_TEST(log, "Completed segment: {}", (*current_file_segment_it)->range().toString()); + // LOG_TEST(log, "Completed segment: {}", (*current_file_segment_it)->range().toString()); if (enable_logging) appendFilesystemCacheLog((*current_file_segment_it)->range(), read_type); @@ -588,7 +588,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegmentPtr & file_segment) bool continue_predownload = file_segment->reserve(current_predownload_size); if (continue_predownload) { - LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size); + // LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size); chassert(file_segment->getCurrentWriteOffset() == static_cast(implementation_buffer->getPosition())); @@ -817,6 +817,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } chassert(!internal_buffer.empty()); + chassert(internal_buffer.size() == settings.remote_fs_buffer_size); swap(*implementation_buffer); @@ -956,27 +957,27 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert(!file_segment->isDownloader()); - LOG_TEST( - log, - "Key: {}. Returning with {} bytes (actually read: {}), buffer position: {} (offset: {}, predownloaded: {}), " - "buffer available: {}, current range: {}, current offset: {}, file segment state: {}, " - "current write offset: {}, read_type: {}, reading until position: {}, started with offset: {}, " - "remaining ranges: {}", - getHexUIntLowercase(cache_key), - working_buffer.size(), - size, - getPosition(), - offset(), - needed_to_predownload, - available(), - current_read_range.toString(), - file_offset_of_buffer_end, - FileSegment::stateToString(file_segment->state()), - file_segment->getCurrentWriteOffset(), - toString(read_type), - read_until_position, - first_offset, - file_segments_holder->toString()); + // LOG_TEST( + // log, + // "Key: {}. Returning with {} bytes (actually read: {}), buffer position: {} (offset: {}, predownloaded: {}), " + // "buffer available: {}, current range: {}, current offset: {}, file segment state: {}, " + // "current write offset: {}, read_type: {}, reading until position: {}, started with offset: {}, " + // "remaining ranges: {}", + // getHexUIntLowercase(cache_key), + // working_buffer.size(), + // size, + // getPosition(), + // offset(), + // needed_to_predownload, + // available(), + // current_read_range.toString(), + // file_offset_of_buffer_end, + // FileSegment::stateToString(file_segment->state()), + // file_segment->getCurrentWriteOffset(), + // toString(read_type), + // read_until_position, + // first_offset, + // file_segments_holder->toString()); if (size == 0 && file_offset_of_buffer_end < read_until_position) { diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference index b44f787df52..fddfbd49de3 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -10,6 +10,6 @@ noisy Info messages 0.05 noisy Warning messages 0.01 noisy Error messages 0.02 no Fatal messages 0 -number of too noisy messages 5 +number of too noisy messages 3 number of noisy messages 10 incorrect patterns 15 From 1358b1b510b8d1ec54f1694d1e9291792f7b4b14 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 Apr 2023 18:39:30 +0200 Subject: [PATCH 0023/2047] Fix --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 4ae65ecd859..257a53235c2 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -817,7 +817,6 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() } chassert(!internal_buffer.empty()); - chassert(internal_buffer.size() == settings.remote_fs_buffer_size); swap(*implementation_buffer); From deedc6b91a0c0deb005af10903852b3d9990ca04 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 8 Apr 2023 21:20:08 +0200 Subject: [PATCH 0024/2047] Less noisy logging, fix test --- src/Interpreters/Cache/FileSegment.cpp | 2 +- tests/queries/0_stateless/01603_read_with_backoff_bug.sql | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index d2fedf68ca5..ea619155d4c 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -524,7 +524,7 @@ void FileSegment::completePartAndResetDownloaderUnlocked(std::unique_lock Date: Tue, 11 Apr 2023 17:35:47 +0000 Subject: [PATCH 0025/2047] ReadFromMergeTree: update sort description after applying prewhere info --- .../QueryPlan/ReadFromMergeTree.cpp | 66 +++++++++++-------- 1 file changed, 39 insertions(+), 27 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 291499ff412..64a3a4c74ae 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -133,6 +133,35 @@ static bool checkAllPartsOnRemoteFS(const RangesInDataParts & parts) return true; } +/// build sort description for output stream +static void updateSortDescriptionForOutputStream( + DataStream & output_stream, const Names & sorting_key_columns, const int sort_direction, InputOrderInfoPtr input_order_info) +{ + SortDescription sort_description; + const Block & header = output_stream.header; + for (const auto & column_name : sorting_key_columns) + { + if (std::find_if(header.begin(), header.end(), [&](ColumnWithTypeAndName const & col) { return col.name == column_name; }) + == header.end()) + break; + sort_description.emplace_back(column_name, sort_direction); + } + if (!sort_description.empty()) + { + if (input_order_info) + { + output_stream.sort_scope = DataStream::SortScope::Stream; + const size_t used_prefix_of_sorting_key_size = input_order_info->used_prefix_of_sorting_key_size; + if (sort_description.size() > used_prefix_of_sorting_key_size) + sort_description.resize(used_prefix_of_sorting_key_size); + } + else + output_stream.sort_scope = DataStream::SortScope::Chunk; + } + + output_stream.sort_description = std::move(sort_description); +} + void ReadFromMergeTree::AnalysisResult::checkLimits(const Settings & settings, const SelectQueryInfo & query_info_) const { @@ -244,33 +273,11 @@ ReadFromMergeTree::ReadFromMergeTree( /// Add explicit description. setStepDescription(data.getStorageID().getFullNameNotQuoted()); - { /// build sort description for output stream - SortDescription sort_description; - const Names & sorting_key_columns = storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(); - const Block & header = output_stream->header; - const int sort_direction = getSortDirection(); - for (const auto & column_name : sorting_key_columns) - { - if (std::find_if(header.begin(), header.end(), [&](ColumnWithTypeAndName const & col) { return col.name == column_name; }) - == header.end()) - break; - sort_description.emplace_back(column_name, sort_direction); - } - if (!sort_description.empty()) - { - if (query_info.getInputOrderInfo()) - { - output_stream->sort_scope = DataStream::SortScope::Stream; - const size_t used_prefix_of_sorting_key_size = query_info.getInputOrderInfo()->used_prefix_of_sorting_key_size; - if (sort_description.size() > used_prefix_of_sorting_key_size) - sort_description.resize(used_prefix_of_sorting_key_size); - } - else - output_stream->sort_scope = DataStream::SortScope::Chunk; - } - - output_stream->sort_description = std::move(sort_description); - } + updateSortDescriptionForOutputStream( + *output_stream, + storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), + getSortDirection(), + query_info.getInputOrderInfo()); } @@ -1425,6 +1432,11 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info prewhere_info_value, data.getPartitionValueType(), virt_column_names)}; + updateSortDescriptionForOutputStream( + *output_stream, + storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), + getSortDirection(), + query_info.getInputOrderInfo()); } bool ReadFromMergeTree::requestOutputEachPartitionThroughSeparatePort() From 163b5a97978a1cbfbecc6fc5fdfe873b1af58655 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Fri, 14 Apr 2023 13:10:09 +0200 Subject: [PATCH 0026/2047] Set codec none in test_temporary_data_in_cache --- tests/integration/test_temporary_data_in_cache/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_temporary_data_in_cache/test.py b/tests/integration/test_temporary_data_in_cache/test.py index 0e8c7305405..581ba0e001c 100644 --- a/tests/integration/test_temporary_data_in_cache/test.py +++ b/tests/integration/test_temporary_data_in_cache/test.py @@ -37,7 +37,7 @@ def test_cache_evicted_by_temporary_data(start_cluster): assert free_space_initial > 8 * 1024 * 1024 q( - "CREATE TABLE t1 (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS storage_policy = 'tiny_local_cache'" + "CREATE TABLE t1 (x UInt64 CODEC(NONE)) ENGINE = MergeTree ORDER BY x SETTINGS storage_policy = 'tiny_local_cache'" ) q("INSERT INTO t1 SELECT number FROM numbers(1024 * 1024)") From 84bd0fa4ce1e938fa296150cfb7a6ad176f602ad Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 14 Apr 2023 16:03:29 +0200 Subject: [PATCH 0027/2047] Update 02241_filesystem_cache_on_write_operations.reference --- ...system_cache_on_write_operations.reference | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference index 0131ad2ec4f..88b0805104b 100644 --- a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference +++ b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference @@ -19,13 +19,13 @@ file_segment_range_begin: 0 file_segment_range_end: 1659 size: 1660 state: DOWNLOADED -7 -7 -7 -7 -21 -28 -35 +8 +8 +8 +8 +24 +32 +40 5010500 18816 Using storage policy: local_cache @@ -49,12 +49,12 @@ file_segment_range_begin: 0 file_segment_range_end: 1659 size: 1660 state: DOWNLOADED -7 -7 -7 -7 -21 -28 -35 +8 +8 +8 +8 +24 +32 +40 5010500 18816 From b4123145f54162d1e2f0c48d3e81cc7f611c8a7e Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Wed, 19 Apr 2023 21:15:24 +0300 Subject: [PATCH 0028/2047] wip add zookeeper add settings --- src/CMakeLists.txt | 1 + src/Storages/S3Queue/ReadBufferFromS3.cpp | 175 +++++++ src/Storages/S3Queue/ReadBufferFromS3.h | 86 ++++ src/Storages/S3Queue/S3QueueSettings.cpp | 41 ++ src/Storages/S3Queue/S3QueueSettings.h | 28 ++ src/Storages/S3Queue/S3QueueSource.cpp | 132 +++++ src/Storages/S3Queue/S3QueueSource.h | 55 ++ src/Storages/S3Queue/StorageS3Queue.cpp | 578 ++++++++++++++++++++++ src/Storages/S3Queue/StorageS3Queue.h | 152 ++++++ src/Storages/StorageS3.h | 1 + src/Storages/registerStorages.cpp | 2 + 11 files changed, 1251 insertions(+) create mode 100644 src/Storages/S3Queue/ReadBufferFromS3.cpp create mode 100644 src/Storages/S3Queue/ReadBufferFromS3.h create mode 100644 src/Storages/S3Queue/S3QueueSettings.cpp create mode 100644 src/Storages/S3Queue/S3QueueSettings.h create mode 100644 src/Storages/S3Queue/S3QueueSource.cpp create mode 100644 src/Storages/S3Queue/S3QueueSource.h create mode 100644 src/Storages/S3Queue/StorageS3Queue.cpp create mode 100644 src/Storages/S3Queue/StorageS3Queue.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index f6727bad0e8..319a77727d7 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -250,6 +250,7 @@ add_object_library(clickhouse_storages_distributed Storages/Distributed) add_object_library(clickhouse_storages_mergetree Storages/MergeTree) add_object_library(clickhouse_storages_liveview Storages/LiveView) add_object_library(clickhouse_storages_windowview Storages/WindowView) +add_object_library(clickhouse_storages_s3queue Storages/S3Queue) add_object_library(clickhouse_client Client) add_object_library(clickhouse_bridge BridgeHelper) add_object_library(clickhouse_server Server) diff --git a/src/Storages/S3Queue/ReadBufferFromS3.cpp b/src/Storages/S3Queue/ReadBufferFromS3.cpp new file mode 100644 index 00000000000..c91c710bc9b --- /dev/null +++ b/src/Storages/S3Queue/ReadBufferFromS3.cpp @@ -0,0 +1,175 @@ +//#include +//#include +//#include +// +//#include +// +//#include +//#include +// +//namespace DB +//{ +//namespace ErrorCodes +//{ +// extern const int CANNOT_READ_ALL_DATA; +//} +// +//ReadBufferFromS3::ReadBufferFromS3( +// StorageS3Queue & storage_, +// size_t max_batch_size, +// size_t poll_timeout_, +// ContextPtr context_, +// size_t stream_number_, +// size_t max_streams_number_) +// : ReadBuffer(nullptr, 0) +// , log(&Poco::Logger::get("ReadBufferFromS3 " + toString(stream_number_))) +// , storage(storage_) +// , batch_size(max_batch_size) +// , poll_timeout(poll_timeout_) +// , context(context_) +// , stream_number(stream_number_) +// , max_streams_number(max_streams_number_) +//{ +// current = records.begin(); +// allowed = false; +//} +// +//bool ReadBufferFromS3::poll() +//{ +// if (hasMorePolledRecords()) +// { +// allowed = true; +// return true; +// } +// +// auto new_records = pollBatch(batch_size); +// if (new_records.empty()) +// { +// buffer_status = BufferStatus::NO_RECORD_RETURNED; +// LOG_TRACE(log, "No new records to read"); +// return false; +// } +// else +// { +// records = std::move(new_records); +// current = records.begin(); +// +// LOG_TRACE(log, "Polled batch of {} records. ", records.size()); +// +// buffer_status = BufferStatus::POLLED_OK; +// allowed = true; +// return true; +// } +//} +// +//ReadBufferFromS3::Records ReadBufferFromS3::pollBatch(size_t batch_size_) +//{ +// Records new_records; +// new_records.reserve(batch_size_); +// +// readNewRecords(new_records, batch_size); +// if (new_records.size() == batch_size_ || stream_out) +// return new_records; +// +// Stopwatch watch; +// while (watch.elapsedMilliseconds() < poll_timeout && new_records.size() != batch_size_) +// { +// readNewRecords(new_records, batch_size); +// /// All ifstrem reach end, no need to wait for timeout, +// /// since file status can not be updated during a streamToViews +// if (stream_out) +// break; +// } +// +// return new_records; +//} +// +//void ReadBufferFromS3::readNewRecords(ReadBufferFromS3::Records & /*new_records*/, size_t /*batch_size_*/) +//{ +//// size_t need_records_size = batch_size_ - new_records.size(); +//// size_t read_records_size = 0; +//// +//// auto & file_infos = storage.getFileInfos(); +//// +//// size_t files_per_stream = file_infos.file_names.size() / max_streams_number; +//// size_t start = stream_number * files_per_stream; +//// size_t end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; +//// +//// for (size_t i = start; i < end; ++i) +//// { +//// const auto & file_name = file_infos.file_names[i]; +//// +//// auto & file_ctx = StorageFileLog::findInMap(file_infos.context_by_name, file_name); +//// if (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE) +//// continue; +//// +//// auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); +//// +//// if (!file_ctx.reader) +//// throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} is not initialized", file_meta.file_name); +//// +//// auto & reader = file_ctx.reader.value(); +//// StorageFileLog::assertStreamGood(reader); +//// +//// Record record; +//// while (read_records_size < need_records_size) +//// { +//// /// Need to get offset before reading record from stream +//// auto offset = reader.tellg(); +//// if (static_cast(offset) >= file_meta.last_open_end) +//// break; +//// record.offset = offset; +//// StorageFileLog::assertStreamGood(reader); +//// +//// record.file_name = file_name; +//// +//// +//// std::getline(reader, record.data); +//// StorageFileLog::assertStreamGood(reader); +//// +//// new_records.emplace_back(record); +//// ++read_records_size; +//// } +//// +//// UInt64 current_position = reader.tellg(); +//// StorageFileLog::assertStreamGood(reader); +//// +//// file_meta.last_writen_position = current_position; +//// +//// /// stream reach to end +//// if (current_position == file_meta.last_open_end) +//// { +//// file_ctx.status = StorageFileLog::FileStatus::NO_CHANGE; +//// } +//// +//// /// All ifstream reach end +//// if (i == end - 1 && (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE)) +//// { +//// stream_out = true; +//// } +//// +//// if (read_records_size == need_records_size) +//// { +//// break; +//// } +//// } +//} +// +//bool ReadBufferFromS3::nextImpl() +//{ +// if (!allowed || !hasMorePolledRecords()) +// return false; +// +// auto * new_position = const_cast(current->data.data()); +// BufferBase::set(new_position, current->data.size(), 0); +// allowed = false; +// +// current_file = current->file_name; +// current_offset = current->offset; +// +// ++current; +// +// return true; +//} +// +//} diff --git a/src/Storages/S3Queue/ReadBufferFromS3.h b/src/Storages/S3Queue/ReadBufferFromS3.h new file mode 100644 index 00000000000..3cf1d7711da --- /dev/null +++ b/src/Storages/S3Queue/ReadBufferFromS3.h @@ -0,0 +1,86 @@ +//#pragma once +// +//#include +//#include +//#include +// +//#include +//#include +// +//namespace DB +//{ +//class ReadBufferFromS3 : public ReadBuffer +//{ +//public: +// ReadBufferFromS3( +// StorageS3Queue & storage_, +// size_t max_batch_size, +// size_t poll_timeout_, +// ContextPtr context_, +// size_t stream_number_, +// size_t max_streams_number_); +// +// ~ReadBufferFromS3() override = default; +// +// auto pollTimeout() const { return poll_timeout; } +// +// bool hasMorePolledRecords() const { return current != records.end(); } +// +// bool poll(); +// +// bool noRecords() { return buffer_status == BufferStatus::NO_RECORD_RETURNED; } +// +// auto getFileName() const { return current_file; } +// auto getOffset() const { return current_offset; } +// +//private: +// enum class BufferStatus +// { +// INIT, +// NO_RECORD_RETURNED, +// POLLED_OK, +// }; +// +// BufferStatus buffer_status = BufferStatus::INIT; +// +// Poco::Logger * log; +// +// StorageS3Queue & storage; +// +// bool stream_out = false; +// +// size_t batch_size; +// size_t poll_timeout; +// +// ContextPtr context; +// +// size_t stream_number; +// size_t max_streams_number; +// +// bool allowed = true; +// +// using RecordData = std::string; +// struct Record +// { +// RecordData data; +// std::string file_name; +// /// Offset is the start of a row, which is needed for virtual columns. +// UInt64 offset; +// }; +// using Records = std::vector; +// +// Records records; +// Records::const_iterator current; +// +// String current_file; +// UInt64 current_offset = 0; +// +// using TaskThread = BackgroundSchedulePool::TaskHolder; +// +// Records pollBatch(size_t batch_size_); +// +// void readNewRecords(Records & new_records, size_t batch_size_); +// +// bool nextImpl() override; +//}; +//} diff --git a/src/Storages/S3Queue/S3QueueSettings.cpp b/src/Storages/S3Queue/S3QueueSettings.cpp new file mode 100644 index 00000000000..72bf5fb6742 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueSettings.cpp @@ -0,0 +1,41 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(S3QueueSettingsTraits, LIST_OF_S3QUEUE_SETTINGS) + +void S3QueueSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for storage " + storage_def.engine->name); + throw; + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } +} + +} diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h new file mode 100644 index 00000000000..8494ae47a90 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTStorage; + + +#define S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ + M(String, mode, "unordered", "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", 0) \ + M(String, keeper_path, "/", "Zookeeper node path", 0) \ + +#define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ + S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ + FORMAT_FACTORY_SETTINGS(M, ALIAS) + +DECLARE_SETTINGS_TRAITS(S3QueueSettingsTraits, LIST_OF_S3QUEUE_SETTINGS) + + +struct S3QueueSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; + +} diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp new file mode 100644 index 00000000000..8e0d970729b --- /dev/null +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -0,0 +1,132 @@ +//#include +//#include +//#include +//#include +//#include +//#include +//#include +// +//namespace DB +//{ +//static constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; +// +//S3QueueSource::S3QueueSource( +// StorageS3Queue & storage_, +// const StorageSnapshotPtr & storage_snapshot_, +// const ContextPtr & context_, +// const Names & columns, +// size_t max_block_size_, +// size_t poll_time_out_, +// size_t stream_number_, +// size_t max_streams_number_) +// : ISource(storage_snapshot_->getSampleBlockForColumns(columns)) +// , storage(storage_) +// , storage_snapshot(storage_snapshot_) +// , context(context_) +// , column_names(columns) +// , max_block_size(max_block_size_) +// , poll_time_out(poll_time_out_) +// , stream_number(stream_number_) +// , max_streams_number(max_streams_number_) +// , non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized()) +// , virtual_header(storage_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames())) +//{ +// buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); +//} +// +//S3QueueSource::~S3QueueSource() +//{ +// try +// { +// if (!finished) +// onFinish(); +// } +// catch (...) +// { +// tryLogCurrentException(__PRETTY_FUNCTION__); +// } +//} +// +//void S3QueueSource::onFinish() +//{ +// finished = true; +//} +// +//Chunk S3QueueSource::generate() +//{ +// /// Store metas of last written chunk into disk +// // storage.storeMetas(start, end); +// +// if (!buffer || buffer->noRecords()) +// { +// /// There is no onFinish for ISource, we call it +// /// when no records return to close files +// onFinish(); +// return {}; +// } +// +// MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); +// +// auto input_format +// = FormatFactory::instance().getInputFormat(storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); +// +// StreamingFormatExecutor executor(non_virtual_header, input_format); +// +// size_t total_rows = 0; +// size_t failed_poll_attempts = 0; +// +// Stopwatch watch; +// while (true) +// { +// size_t new_rows = 0; +// if (buffer->poll()) +// new_rows = executor.execute(); +// +// if (new_rows) +// { +// auto file_name = buffer->getFileName(); +// auto offset = buffer->getOffset(); +// for (size_t i = 0; i < new_rows; ++i) +// { +// virtual_columns[0]->insert(file_name); +// virtual_columns[1]->insert(offset); +// } +// total_rows = total_rows + new_rows; +// } +// else /// poll succeed, but parse failed +// { +// ++failed_poll_attempts; +// } +// +// if (!buffer->hasMorePolledRecords() +// && ((total_rows >= max_block_size) || watch.elapsedMilliseconds() > poll_time_out +// || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS)) +// { +// break; +// } +// } +// +// if (total_rows == 0) +// { +// onFinish(); +// return {}; +// } +// +// auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); +// auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); +// +// for (const auto & column : virtual_block.getColumnsWithTypeAndName()) +// result_block.insert(column); +// +// auto converting_dag = ActionsDAG::makeConvertingActions( +// result_block.cloneEmpty().getColumnsWithTypeAndName(), +// getPort().getHeader().getColumnsWithTypeAndName(), +// ActionsDAG::MatchColumnsMode::Name); +// +// auto converting_actions = std::make_shared(std::move(converting_dag)); +// converting_actions->execute(result_block); +// +// return Chunk(result_block.getColumns(), result_block.rows()); +//} +// +//} diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h new file mode 100644 index 00000000000..4714e23f748 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -0,0 +1,55 @@ +//#pragma once +// +//#include +//#include +//#include +// +//namespace Poco +//{ +//class Logger; +//} +//namespace DB +//{ +//class S3QueueSource : public ISource +//{ +//public: +// S3QueueSource( +// StorageS3Queue & storage_, +// const StorageSnapshotPtr & storage_snapshot_, +// const ContextPtr & context_, +// const Names & columns, +// size_t max_block_size_, +// size_t poll_time_out_, +// size_t stream_number_, +// size_t max_streams_number_); +// +// String getName() const override { return "S3Queue"; } +// +// bool noRecords() { return !buffer || buffer->noRecords(); } +// +// void onFinish(); +// +// virtual ~S3QueueSource() override; +// +//protected: +// Chunk generate() override; +// +//private: +// StorageS3Queue & storage; +// StorageSnapshotPtr storage_snapshot; +// ContextPtr context; +// Names column_names; +// UInt64 max_block_size; +// +// size_t poll_time_out; +// +// size_t stream_number; +// size_t max_streams_number; +// +// std::unique_ptr buffer; +// +// Block non_virtual_header; +// Block virtual_header; +//}; +// +//} diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp new file mode 100644 index 00000000000..ae162b90906 --- /dev/null +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -0,0 +1,578 @@ +#include "config.h" +#include +#include "IO/ParallelReadBuffer.h" +#include "IO/IOThreadPool.h" +#include "Parsers/ASTCreateQuery.h" + +#if USE_AWS_S3 + +#include +#include + +#include + +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +#include +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include + +#include + +#include + +#include + +#include +#include +#include + +#include +#include +#include +#include + +namespace fs = std::filesystem; + + +namespace ProfileEvents +{ +extern const Event S3DeleteObjects; +extern const Event S3ListObjects; +} + +namespace DB +{ + +static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + +static const std::unordered_set required_configuration_keys = { + "url", +}; +static const std::unordered_set optional_configuration_keys = { + "format", + "compression", + "compression_method", + "structure", + "access_key_id", + "secret_access_key", + "filename", + "use_environment_credentials", + "max_single_read_retries", + "min_upload_part_size", + "upload_part_size_multiply_factor", + "upload_part_size_multiply_parts_count_threshold", + "max_single_part_upload_size", + "max_connections", +}; + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_TEXT; + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int S3_ERROR; + extern const int UNEXPECTED_EXPRESSION; + extern const int DATABASE_ACCESS_DENIED; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int NOT_IMPLEMENTED; + extern const int CANNOT_COMPILE_REGEXP; + extern const int FILE_DOESNT_EXIST; + extern const int QUERY_NOT_ALLOWED; + extern const int NO_ZOOKEEPER; + extern const int REPLICA_ALREADY_EXISTS; +} + +class IOutputFormat; +using OutputFormatPtr = std::shared_ptr; + +const String StorageS3Queue::default_zookeeper_name = "default"; + + +StorageS3Queue::StorageS3Queue( + const String & zookeeper_path_, + const String & mode_, + const StorageS3::Configuration & configuration_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_, + std::optional format_settings_, + bool distributed_processing_, + ASTPtr partition_by_) + : IStorage(table_id_) + , WithContext(context_) + , s3_configuration{configuration_} + , keys({s3_configuration.url.key}) + , format_name(configuration_.format) + , compression_method(configuration_.compression_method) + , name(s3_configuration.url.storage_name) + , distributed_processing(distributed_processing_) + , format_settings(format_settings_) + , partition_by(partition_by_) + , is_key_with_globs(s3_configuration.url.key.find_first_of("*?{") != std::string::npos) + , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) + , mode(mode_) + , zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_)) + , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ true, log)) +{ + FormatFactory::instance().checkFormatName(format_name); + context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); + + StorageInMemoryMetadata storage_metadata; + + StorageS3::updateConfiguration(context_, s3_configuration); + if (columns_.empty()) + { + auto columns = StorageS3::getTableStructureFromDataImpl( + format_name, + s3_configuration, + compression_method, + is_key_with_globs, + format_settings, + context_); + + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); + + auto default_virtuals = NamesAndTypesList{ + {"_path", std::make_shared(std::make_shared())}, + {"_file", std::make_shared(std::make_shared())}}; + + auto columns = storage_metadata.getSampleBlock().getNamesAndTypesList(); + virtual_columns = getVirtualsForStorage(columns, default_virtuals); + for (const auto & column : virtual_columns) + virtual_block.insert({column.type->createColumn(), column.type, column.name}); + + auto thread = context_->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); + setZooKeeper(); + auto metadata_snapshot = getInMemoryMetadataPtr(); + createTableIfNotExists(metadata_snapshot); + task = std::make_shared(std::move(thread)); +} + + +bool StorageS3Queue::supportsSubcolumns() const +{ + return FormatFactory::instance().checkIfFormatSupportsSubcolumns(format_name); +} + +bool StorageS3Queue::supportsSubsetOfColumns() const +{ + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name); +} + +Pipe StorageS3Queue::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t num_streams) +{ +// if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) +// throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, +// "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + + if (mv_attached) + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageRabbitMQ with attached materialized views"); + + auto query_s3_configuration = StorageS3::copyAndUpdateConfiguration(local_context, s3_configuration); + + bool has_wildcards = + query_s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos + || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; + + if (partition_by && has_wildcards) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); + + Pipes pipes; + + std::unordered_set column_names_set(column_names.begin(), column_names.end()); + std::vector requested_virtual_columns; + + for (const auto & virtual_column : getVirtuals()) + { + if (column_names_set.contains(virtual_column.name)) + requested_virtual_columns.push_back(virtual_column); + } + + std::shared_ptr iterator_wrapper = StorageS3::createFileIterator( + query_s3_configuration, + keys, + is_key_with_globs, + distributed_processing, + local_context, + query_info.query, + virtual_block); + + ColumnsDescription columns_description; + Block block_for_format; + if (supportsSubsetOfColumns()) + { + auto fetch_columns = column_names; + const auto & virtuals = getVirtuals(); + std::erase_if( + fetch_columns, + [&](const String & col) + { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); + + if (fetch_columns.empty()) + fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); + + columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); + block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); + } + else + { + columns_description = storage_snapshot->metadata->getColumns(); + block_for_format = storage_snapshot->metadata->getSampleBlock(); + } + + const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; + // const size_t max_download_threads = 1; + LOG_WARNING(log, "num_streams"); + + for (size_t i = 0; i < num_streams; ++i) + { + pipes.emplace_back(std::make_shared( + requested_virtual_columns, + format_name, + getName(), + block_for_format, + local_context, + format_settings, + columns_description, + max_block_size, + query_s3_configuration.request_settings, + compression_method, + query_s3_configuration.client, + query_s3_configuration.url.bucket, + query_s3_configuration.url.version_id, + iterator_wrapper, + max_download_threads)); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + LOG_WARNING(log, "unitePipes"); + + narrowPipe(pipe, num_streams); + LOG_WARNING(log, "narrowPipe"); + + return pipe; +} + +NamesAndTypesList StorageS3Queue::getVirtuals() const +{ + return virtual_columns; +} + +Names StorageS3Queue::getVirtualColumnNames() +{ + return {"_path", "_file"}; +} + +bool StorageS3Queue::supportsPartitionBy() const +{ + return true; +} + +void StorageS3Queue::startup() +{ + if (task) + task->holder->activateAndSchedule(); +} + +void StorageS3Queue::shutdown() +{ + shutdown_called = true; + LOG_TRACE(log, "Deactivating background tasks"); + + if (task) + { + task->stream_cancelled = true; + + /// Reader thread may wait for wake up +// wakeUp(); + + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); + /// If no reading call and threadFunc, the log files will never + /// be opened, also just leave the work of close files and + /// store meta to streams. because if we close files in here, + /// may result in data race with unfinishing reading pipeline + } +} + +size_t StorageS3Queue::getTableDependentCount() const +{ + auto table_id = getStorageID(); + // Check if at least one direct dependency is attached + return DatabaseCatalog::instance().getDependentViews(table_id).size(); +} + +bool StorageS3Queue::hasDependencies(const StorageID & table_id) +{ + // Check if all dependencies are attached + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + LOG_TEST(log, "Number of attached views {} for {}", view_ids.size(), table_id.getNameForLogs()); + + if (view_ids.empty()) + return false; + + // Check the dependencies are ready? + for (const auto & view_id : view_ids) + { + auto view = DatabaseCatalog::instance().tryGetTable(view_id, getContext()); + if (!view) + return false; + + // If it materialized view, check it's target table + auto * materialized_view = dynamic_cast(view.get()); + if (materialized_view && !materialized_view->tryGetTargetTable()) + return false; + } + + return true; +} + +void StorageS3Queue::threadFunc() +{ + bool reschedule = true; + try + { + auto table_id = getStorageID(); + + auto dependencies_count = getTableDependentCount(); + + if (dependencies_count) + { + // auto start_time = std::chrono::steady_clock::now(); + + mv_attached.store(true); + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!task->stream_cancelled) + { + if (!hasDependencies(table_id)) + { + /// For this case, we can not wait for watch thread to wake up + reschedule = true; + break; + } + + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + + // if (streamToViews()) + // { + // LOG_TRACE(log, "Stream stalled. Reschedule."); + // break; + // } + + // auto ts = std::chrono::steady_clock::now(); + // auto duration = std::chrono::duration_cast(ts-start_time); + // if (duration.count() > MAX_THREAD_WORK_DURATION_MS) + // { + // LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); + // reschedule = true; + // break; + // } + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + mv_attached.store(false); + + // Wait for attached views + if (reschedule && !shutdown_called) + { + LOG_TRACE(log, "Reschedule S3 Queue thread func."); + /// Reschedule with backoff. + task->holder->scheduleAfter(milliseconds_to_wait); + } +} + + +void StorageS3Queue::setZooKeeper() +{ + std::lock_guard lock(current_zookeeper_mutex); + LOG_WARNING(log, "zookeeper name {}", zookeeper_name); + if (zookeeper_name == default_zookeeper_name) + { + current_zookeeper = getContext()->getZooKeeper(); + } + else + { + current_zookeeper = getContext()->getAuxiliaryZooKeeper(zookeeper_name); + } +} + +zkutil::ZooKeeperPtr StorageS3Queue::tryGetZooKeeper() const +{ + std::lock_guard lock(current_zookeeper_mutex); + return current_zookeeper; +} + +zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const +{ + auto res = tryGetZooKeeper(); + if (!res) + throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); + return res; +} + + +bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot) +{ + auto zookeeper = getZooKeeper(); + zookeeper->createAncestors(zookeeper_path); + + for (size_t i = 0; i < 1000; ++i) + { + if (zookeeper->exists(zookeeper_path + "")) + { + LOG_DEBUG(log, "This table {} is already created, will add new replica", zookeeper_path); + return false; + } + /// We write metadata of table so that the replicas can check table parameters with them. + // String metadata_str = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", + zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), + zkutil::CreateMode::Persistent)); +// ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, +// zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); + if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zookeeper_path); + continue; + } + else if (code != Coordination::Error::ZOK) + { + zkutil::KeeperMultiException::check(code, ops, responses); + } + + return true; + } + + /// Do not use LOGICAL_ERROR code, because it may happen if user has specified wrong zookeeper_path + throw Exception(ErrorCodes::REPLICA_ALREADY_EXISTS, + "Cannot create table, because it is created concurrently every time or because " + "of wrong zookeeper_path or because of logical error"); +} + +void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) +{ + factory.registerStorage(name, [](const StorageFactory::Arguments & args) + { + auto & engine_args = args.engine_args; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + + auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); + // Use format settings from global server context + settings from + // the SETTINGS clause of the create query. Settings from current + // session and user are ignored. + auto s3queue_settings = std::make_unique(); + std::optional format_settings; + + if (args.storage_def->settings) + { + s3queue_settings->loadFromQuery(*args.storage_def); + + } + format_settings = getFormatSettings(args.getContext()); + + ASTPtr partition_by; + if (args.storage_def->partition_by) + partition_by = args.storage_def->partition_by->clone(); + + String keeper_path = s3queue_settings->keeper_path; + String mode = s3queue_settings->mode; + + return std::make_shared( + keeper_path, + mode, + configuration, + args.table_id, + args.columns, + args.constraints, + args.comment, + args.getContext(), + format_settings, + /* distributed_processing_ */false, + partition_by); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +void registerStorageS3Queue(StorageFactory & factory) +{ + return registerStorageS3QueueImpl("S3Queue", factory); +} + +} + + +#endif diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h new file mode 100644 index 00000000000..9c1390c24a8 --- /dev/null +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -0,0 +1,152 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include + +#include +#include + +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace Aws::S3 +{ +class Client; +} + +namespace DB +{ + + + +class StorageS3Queue : public IStorage, WithContext +{ +public: + using Configuration = typename StorageS3::Configuration; + StorageS3Queue( + const String & zookeper_path_, + const String & mode_, + const Configuration & configuration_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_, + std::optional format_settings_, + bool distributed_processing_ = false, + ASTPtr partition_by_ = nullptr); + + String getName() const override { return "S3Queue"; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); + } + + void truncate(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*local_context*/, TableExclusiveLockHolder &) override { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); + } + + NamesAndTypesList getVirtuals() const override; + + bool supportsPartitionBy() const override; + + static ColumnsDescription getTableStructureFromData( + Configuration & configuration, const std::optional & format_settings, ContextPtr ctx) + { + return StorageS3::getTableStructureFromData(configuration, format_settings, ctx); + } + + const auto & getFormatName() const { return format_name; } + +private: + + Configuration s3_configuration; + std::vector keys; + NamesAndTypesList virtual_columns; + Block virtual_block; + uint64_t milliseconds_to_wait = 10000; + + String format_name; + String compression_method; + String name; + const bool distributed_processing; + std::optional format_settings; + ASTPtr partition_by; + bool is_key_with_globs = false; + + bool supportsSubcolumns() const override; + + void threadFunc(); + size_t getTableDependentCount() const; + std::atomic mv_attached = false; + bool hasDependencies(const StorageID & table_id); + std::atomic shutdown_called{false}; + Poco::Logger * log; + + + void startup() override; + void shutdown() override; + + struct TaskContext + { + BackgroundSchedulePool::TaskHolder holder; + std::atomic stream_cancelled {false}; + explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) + { + } + }; + std::shared_ptr task; + + bool supportsSubsetOfColumns() const override; + static Names getVirtualColumnNames(); + + const String mode; + + static const String default_zookeeper_name; + const String zookeeper_name; + const String zookeeper_path; + const String replica_name; + const String replica_path; + + zkutil::ZooKeeperPtr current_zookeeper; + mutable std::mutex current_zookeeper_mutex; + + void setZooKeeper(); + zkutil::ZooKeeperPtr tryGetZooKeeper() const; + zkutil::ZooKeeperPtr getZooKeeper() const; + bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); + // Return default or custom zookeeper name for table + const String & getZooKeeperName() const { return zookeeper_name; } + const String & getZooKeeperPath() const { return zookeeper_path; } + +}; + +} + +#endif diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index f4d915e9c55..131649ff5fd 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -304,6 +304,7 @@ protected: private: friend class StorageS3Cluster; friend class TableFunctionS3Cluster; + friend class StorageS3Queue; Configuration s3_configuration; std::vector keys; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index e690189c365..61c415a3f9d 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -35,6 +35,7 @@ void registerStorageCOS(StorageFactory & factory); void registerStorageOSS(StorageFactory & factory); void registerStorageHudi(StorageFactory & factory); void registerStorageDeltaLake(StorageFactory & factory); +void registerStorageS3Queue(StorageFactory & factory); #if USE_AVRO void registerStorageIceberg(StorageFactory & factory); #endif @@ -127,6 +128,7 @@ void registerStorages() registerStorageOSS(factory); registerStorageHudi(factory); registerStorageDeltaLake(factory); + registerStorageS3Queue(factory); #if USE_AVRO registerStorageIceberg(factory); From 125e5c50b891038740a2a75e2570a201999c09f3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 25 Apr 2023 21:30:03 +0000 Subject: [PATCH 0029/2047] allow to flush async insert queue --- programs/server/Server.cpp | 11 ++- src/Access/Common/AccessType.h | 1 + src/Core/Settings.h | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 71 +++++++++++++++---- src/Interpreters/AsynchronousInsertQueue.h | 11 ++- src/Interpreters/InterpreterSystemQuery.cpp | 17 +++++ src/Parsers/ASTSystemQuery.h | 1 + .../02726_async_insert_flush_queue.reference | 5 ++ .../02726_async_insert_flush_queue.sql | 28 ++++++++ 9 files changed, 128 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/02726_async_insert_flush_queue.reference create mode 100644 tests/queries/0_stateless/02726_async_insert_flush_queue.sql diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8c0d50bae55..cd08de126c9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1461,16 +1461,21 @@ try /// Load global settings from default_profile and system_profile. global_context->setDefaultProfiles(config()); - const Settings & settings = global_context->getSettingsRef(); /// Initialize background executors after we load default_profile config. /// This is needed to load proper values of background_pool_size etc. global_context->initializeBackgroundExecutorsIfNeeded(); - if (settings.async_insert_threads) + size_t async_insert_threads = config().getUInt("async_insert_threads", 16); + bool async_insert_queue_flush_on_shutdown = config().getBool("async_insert_queue_flush_on_shutdown", false); + + if (async_insert_threads) + { global_context->setAsynchronousInsertQueue(std::make_shared( global_context, - settings.async_insert_threads)); + async_insert_threads, + async_insert_queue_flush_on_shutdown)); + } size_t mark_cache_size = server_settings.mark_cache_size; String mark_cache_policy = server_settings.mark_cache_policy; diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 57fa75dc67b..ae7e7ab5bf0 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -182,6 +182,7 @@ enum class AccessType M(SYSTEM_SYNC_FILE_CACHE, "SYNC FILE CACHE", GLOBAL, SYSTEM) \ M(SYSTEM_FLUSH_DISTRIBUTED, "FLUSH DISTRIBUTED", TABLE, SYSTEM_FLUSH) \ M(SYSTEM_FLUSH_LOGS, "FLUSH LOGS", GLOBAL, SYSTEM_FLUSH) \ + M(SYSTEM_FLUSH_ASYNC_INSERT_QUEUE, "FLUSH ASYNC INSERT QUEUE", GLOBAL, SYSTEM_FLUSH) \ M(SYSTEM_FLUSH, "", GROUP, SYSTEM) \ M(SYSTEM_THREAD_FUZZER, "SYSTEM START THREAD FUZZER, SYSTEM STOP THREAD FUZZER, START THREAD FUZZER, STOP THREAD FUZZER", GLOBAL, SYSTEM) \ M(SYSTEM_UNFREEZE, "SYSTEM UNFREEZE", GLOBAL, SYSTEM) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 101f6f1f934..96dbe26f820 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -646,7 +646,6 @@ class IColumn; M(UInt64, merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \ M(UInt64, remote_read_min_bytes_for_seek, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes required for remote read (url, s3) to do seek, instead of read with ignore.", 0) \ \ - M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ M(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background. Makes sense only for inserts via HTTP protocol. If wait_for_async_insert is false, INSERT query is processed almost instantly, otherwise client will wait until data will be flushed to table", 0) \ M(Bool, wait_for_async_insert, true, "If true wait for processing of asynchronous insertion", 0) \ M(Seconds, wait_for_async_insert_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "Timeout for waiting for processing asynchronous insertion", 0) \ @@ -783,6 +782,7 @@ class IColumn; MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_distributed_schedule_pool_size, 16) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_remote_read_network_bandwidth_for_server, 0) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, max_remote_write_network_bandwidth_for_server, 0) \ + MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, async_insert_threads, 16) \ /* ---- */ \ MAKE_OBSOLETE(M, DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic) \ MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \ diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index b8de0246ae2..0a817995eb4 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -128,9 +128,10 @@ void AsynchronousInsertQueue::InsertData::Entry::finish(std::exception_ptr excep } } -AsynchronousInsertQueue::AsynchronousInsertQueue(ContextPtr context_, size_t pool_size_) +AsynchronousInsertQueue::AsynchronousInsertQueue(ContextPtr context_, size_t pool_size_, bool flush_on_shutdown_) : WithContext(context_) , pool_size(pool_size_) + , flush_on_shutdown(flush_on_shutdown_) , queue_shards(pool_size) , pool(CurrentMetrics::AsynchronousInsertThreads, CurrentMetrics::AsynchronousInsertThreadsActive, pool_size) { @@ -143,8 +144,6 @@ AsynchronousInsertQueue::AsynchronousInsertQueue(ContextPtr context_, size_t poo AsynchronousInsertQueue::~AsynchronousInsertQueue() { - /// TODO: add a setting for graceful shutdown. - LOG_TRACE(log, "Shutting down the asynchronous insertion queue"); shutdown = true; @@ -156,17 +155,18 @@ AsynchronousInsertQueue::~AsynchronousInsertQueue() assert(dump_by_first_update_threads[i].joinable()); dump_by_first_update_threads[i].join(); + if (flush_on_shutdown) + { + for (auto & [_, elem] : shard.queue) + scheduleDataProcessingJob(elem.key, std::move(elem.data), getContext()); + } + else { - std::lock_guard lock(shard.mutex); for (auto & [_, elem] : shard.queue) - { for (const auto & entry : elem.data->entries) - { entry->finish(std::make_exception_ptr(Exception( ErrorCodes::TIMEOUT_EXCEEDED, "Wait for async insert timeout exceeded)"))); - } - } } } @@ -210,7 +210,9 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) /// to avoid buffering of huge amount of data in memory. auto read_buf = getReadBufferFromASTInsertQuery(query); - LimitReadBuffer limit_buf(*read_buf, settings.async_insert_max_data_size, /* trow_exception */ false, /* exact_limit */ {}); + LimitReadBuffer limit_buf( + *read_buf, settings.async_insert_max_data_size, + /*throw_exception=*/ false, /*exact_limit=*/ {}); WriteBufferFromString write_buf(bytes); copyData(limit_buf, write_buf); @@ -262,18 +264,19 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) assert(data); data->size_in_bytes += entry_data_size; - ++data->query_number; data->entries.emplace_back(entry); insert_future = entry->getFuture(); LOG_TRACE(log, "Have {} pending inserts with total {} bytes of data for query '{}'", data->entries.size(), data->size_in_bytes, key.query_str); + bool has_enough_bytes = data->size_in_bytes >= key.settings.async_insert_max_data_size; + bool has_enough_queries = data->entries.size() >= key.settings.async_insert_max_query_number && key.settings.async_insert_deduplicate; + /// Here we check whether we hit the limit on maximum data size in the buffer. /// And use setting from query context. /// It works, because queries with the same set of settings are already grouped together. - if (data->size_in_bytes >= key.settings.async_insert_max_data_size - || (data->query_number >= key.settings.async_insert_max_query_number && key.settings.async_insert_deduplicate)) + if (!flush_stopped && (has_enough_bytes || has_enough_queries)) { data_to_process = std::move(data); shard.iterators.erase(it); @@ -297,6 +300,47 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) }; } +void AsynchronousInsertQueue::flushAll() +{ + std::lock_guard flush_lock(flush_mutex); + + LOG_DEBUG(log, "Requested to flush asynchronous insert queue"); + + flush_stopped = true; + std::vector queues_to_flush(pool_size); + + for (size_t i = 0; i < pool_size; ++i) + { + std::lock_guard lock(queue_shards[i].mutex); + queues_to_flush[i] = std::move(queue_shards[i].queue); + queue_shards[i].iterators.clear(); + } + + size_t total_queries = 0; + size_t total_bytes = 0; + size_t total_entries = 0; + + for (auto & queue : queues_to_flush) + { + total_queries += queue.size(); + for (auto & [_, entry] : queue) + { + total_bytes += entry.data->size_in_bytes; + total_entries += entry.data->entries.size(); + scheduleDataProcessingJob(entry.key, std::move(entry.data), getContext()); + } + } + + LOG_DEBUG(log, + "Will wait for finishing of {} flushing jobs (about {} inserts, {} bytes, {} distinct queries)", + pool.active(), total_entries, total_bytes, total_queries); + + pool.wait(); + + LOG_DEBUG(log, "Finished flushing of asynchronous insert queue"); + flush_stopped = false; +} + void AsynchronousInsertQueue::processBatchDeadlines(size_t shard_num) { auto & shard = queue_shards[shard_num]; @@ -322,6 +366,9 @@ void AsynchronousInsertQueue::processBatchDeadlines(size_t shard_num) if (shutdown) return; + if (flush_stopped) + continue; + const auto now = std::chrono::steady_clock::now(); while (true) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index 23a2860364d..97294d70ead 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -16,7 +16,7 @@ class AsynchronousInsertQueue : public WithContext public: using Milliseconds = std::chrono::milliseconds; - AsynchronousInsertQueue(ContextPtr context_, size_t pool_size_); + AsynchronousInsertQueue(ContextPtr context_, size_t pool_size_, bool flush_on_shutdown_); ~AsynchronousInsertQueue(); struct PushResult @@ -37,6 +37,7 @@ public: std::unique_ptr insert_data_buffer; }; + void flushAll(); PushResult push(ASTPtr query, ContextPtr query_context); size_t getPoolSize() const { return pool_size; } @@ -82,9 +83,7 @@ private: using EntryPtr = std::shared_ptr; std::list entries; - size_t size_in_bytes = 0; - size_t query_number = 0; }; using InsertDataPtr = std::unique_ptr; @@ -112,6 +111,8 @@ private: }; const size_t pool_size; + const bool flush_on_shutdown; + std::vector queue_shards; /// Logic and events behind queue are as follows: @@ -123,6 +124,10 @@ private: /// (async_insert_max_data_size setting). If so, then again we dump the data. std::atomic shutdown{false}; + std::atomic flush_stopped{false}; + + /// A mutex that prevents concurrent forced flushes of queue. + mutable std::mutex flush_mutex; /// Dump the data only inside this pool. ThreadPool pool; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 36cb57c3678..f73429913b3 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include #include @@ -564,6 +565,17 @@ BlockIO InterpreterSystemQuery::execute() ); break; } + case Type::FLUSH_ASYNC_INSERT_QUEUE: + { + getContext()->checkAccess(AccessType::SYSTEM_FLUSH_ASYNC_INSERT_QUEUE); + auto * queue = getContext()->getAsynchronousInsertQueue(); + if (!queue) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot flush asynchronous insert queue because it is not initialized"); + + queue->flushAll(); + break; + } case Type::STOP_LISTEN_QUERIES: case Type::START_LISTEN_QUERIES: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is not supported yet", query.type); @@ -1156,6 +1168,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_FLUSH_LOGS); break; } + case Type::FLUSH_ASYNC_INSERT_QUEUE: + { + required_access.emplace_back(AccessType::SYSTEM_FLUSH_ASYNC_INSERT_QUEUE); + break; + } case Type::RESTART_DISK: { required_access.emplace_back(AccessType::SYSTEM_RESTART_DISK); diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index dfe2389edb7..9e2dca8bb23 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -72,6 +72,7 @@ public: START_REPLICATION_QUEUES, FLUSH_LOGS, FLUSH_DISTRIBUTED, + FLUSH_ASYNC_INSERT_QUEUE, STOP_DISTRIBUTED_SENDS, START_DISTRIBUTED_SENDS, START_THREAD_FUZZER, diff --git a/tests/queries/0_stateless/02726_async_insert_flush_queue.reference b/tests/queries/0_stateless/02726_async_insert_flush_queue.reference new file mode 100644 index 00000000000..b94888d227e --- /dev/null +++ b/tests/queries/0_stateless/02726_async_insert_flush_queue.reference @@ -0,0 +1,5 @@ +JSONEachRow 3 +Values 2 +0 +0 +9 diff --git a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql new file mode 100644 index 00000000000..33f40eef14e --- /dev/null +++ b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql @@ -0,0 +1,28 @@ +DROP TABLE IF EXISTS t_async_inserts_flush; + +CREATE TABLE t_async_inserts_flush (a UInt64) ENGINE = Memory; + +SET async_insert = 1; +SET wait_for_async_insert = 0; +SET async_insert_busy_timeout_ms = 1000000; + +INSERT INTO t_async_inserts_flush VALUES (1) (2); +INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": 10} {"a": 20}; +INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": "str"} +INSERT INTO t_async_inserts_flush FORMAT JSONEachRow {"a": 100} {"a": 200} +INSERT INTO t_async_inserts_flush VALUES (3) (4) (5); + +SELECT sleep(1) FORMAT Null; + +SELECT format, length(entries.query_id) FROM system.asynchronous_inserts +WHERE database = currentDatabase() AND table = 't_async_inserts_flush' +ORDER BY format; + +SELECT count() FROM t_async_inserts_flush; + +SYSTEM FLUSH ASYNC INSERT QUEUE; + +SELECT count() FROM system.asynchronous_inserts; +SELECT count() FROM t_async_inserts_flush; + +DROP TABLE t_async_inserts_flush; From 2b7a403752b523e5384cfe42d76907a3c097386a Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 27 Apr 2023 14:55:23 +0300 Subject: [PATCH 0030/2047] wip: select and streaming to views --- src/Storages/S3Queue/ReadBufferFromS3.cpp | 175 ------- src/Storages/S3Queue/ReadBufferFromS3.h | 86 --- src/Storages/S3Queue/S3QueueSource.cpp | 608 +++++++++++++++++----- src/Storages/S3Queue/S3QueueSource.h | 201 +++++-- src/Storages/S3Queue/StorageS3Queue.cpp | 277 ++++++++-- src/Storages/S3Queue/StorageS3Queue.h | 15 +- src/Storages/StorageS3.h | 2 + 7 files changed, 871 insertions(+), 493 deletions(-) delete mode 100644 src/Storages/S3Queue/ReadBufferFromS3.cpp delete mode 100644 src/Storages/S3Queue/ReadBufferFromS3.h diff --git a/src/Storages/S3Queue/ReadBufferFromS3.cpp b/src/Storages/S3Queue/ReadBufferFromS3.cpp deleted file mode 100644 index c91c710bc9b..00000000000 --- a/src/Storages/S3Queue/ReadBufferFromS3.cpp +++ /dev/null @@ -1,175 +0,0 @@ -//#include -//#include -//#include -// -//#include -// -//#include -//#include -// -//namespace DB -//{ -//namespace ErrorCodes -//{ -// extern const int CANNOT_READ_ALL_DATA; -//} -// -//ReadBufferFromS3::ReadBufferFromS3( -// StorageS3Queue & storage_, -// size_t max_batch_size, -// size_t poll_timeout_, -// ContextPtr context_, -// size_t stream_number_, -// size_t max_streams_number_) -// : ReadBuffer(nullptr, 0) -// , log(&Poco::Logger::get("ReadBufferFromS3 " + toString(stream_number_))) -// , storage(storage_) -// , batch_size(max_batch_size) -// , poll_timeout(poll_timeout_) -// , context(context_) -// , stream_number(stream_number_) -// , max_streams_number(max_streams_number_) -//{ -// current = records.begin(); -// allowed = false; -//} -// -//bool ReadBufferFromS3::poll() -//{ -// if (hasMorePolledRecords()) -// { -// allowed = true; -// return true; -// } -// -// auto new_records = pollBatch(batch_size); -// if (new_records.empty()) -// { -// buffer_status = BufferStatus::NO_RECORD_RETURNED; -// LOG_TRACE(log, "No new records to read"); -// return false; -// } -// else -// { -// records = std::move(new_records); -// current = records.begin(); -// -// LOG_TRACE(log, "Polled batch of {} records. ", records.size()); -// -// buffer_status = BufferStatus::POLLED_OK; -// allowed = true; -// return true; -// } -//} -// -//ReadBufferFromS3::Records ReadBufferFromS3::pollBatch(size_t batch_size_) -//{ -// Records new_records; -// new_records.reserve(batch_size_); -// -// readNewRecords(new_records, batch_size); -// if (new_records.size() == batch_size_ || stream_out) -// return new_records; -// -// Stopwatch watch; -// while (watch.elapsedMilliseconds() < poll_timeout && new_records.size() != batch_size_) -// { -// readNewRecords(new_records, batch_size); -// /// All ifstrem reach end, no need to wait for timeout, -// /// since file status can not be updated during a streamToViews -// if (stream_out) -// break; -// } -// -// return new_records; -//} -// -//void ReadBufferFromS3::readNewRecords(ReadBufferFromS3::Records & /*new_records*/, size_t /*batch_size_*/) -//{ -//// size_t need_records_size = batch_size_ - new_records.size(); -//// size_t read_records_size = 0; -//// -//// auto & file_infos = storage.getFileInfos(); -//// -//// size_t files_per_stream = file_infos.file_names.size() / max_streams_number; -//// size_t start = stream_number * files_per_stream; -//// size_t end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; -//// -//// for (size_t i = start; i < end; ++i) -//// { -//// const auto & file_name = file_infos.file_names[i]; -//// -//// auto & file_ctx = StorageFileLog::findInMap(file_infos.context_by_name, file_name); -//// if (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE) -//// continue; -//// -//// auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); -//// -//// if (!file_ctx.reader) -//// throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} is not initialized", file_meta.file_name); -//// -//// auto & reader = file_ctx.reader.value(); -//// StorageFileLog::assertStreamGood(reader); -//// -//// Record record; -//// while (read_records_size < need_records_size) -//// { -//// /// Need to get offset before reading record from stream -//// auto offset = reader.tellg(); -//// if (static_cast(offset) >= file_meta.last_open_end) -//// break; -//// record.offset = offset; -//// StorageFileLog::assertStreamGood(reader); -//// -//// record.file_name = file_name; -//// -//// -//// std::getline(reader, record.data); -//// StorageFileLog::assertStreamGood(reader); -//// -//// new_records.emplace_back(record); -//// ++read_records_size; -//// } -//// -//// UInt64 current_position = reader.tellg(); -//// StorageFileLog::assertStreamGood(reader); -//// -//// file_meta.last_writen_position = current_position; -//// -//// /// stream reach to end -//// if (current_position == file_meta.last_open_end) -//// { -//// file_ctx.status = StorageFileLog::FileStatus::NO_CHANGE; -//// } -//// -//// /// All ifstream reach end -//// if (i == end - 1 && (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE)) -//// { -//// stream_out = true; -//// } -//// -//// if (read_records_size == need_records_size) -//// { -//// break; -//// } -//// } -//} -// -//bool ReadBufferFromS3::nextImpl() -//{ -// if (!allowed || !hasMorePolledRecords()) -// return false; -// -// auto * new_position = const_cast(current->data.data()); -// BufferBase::set(new_position, current->data.size(), 0); -// allowed = false; -// -// current_file = current->file_name; -// current_offset = current->offset; -// -// ++current; -// -// return true; -//} -// -//} diff --git a/src/Storages/S3Queue/ReadBufferFromS3.h b/src/Storages/S3Queue/ReadBufferFromS3.h deleted file mode 100644 index 3cf1d7711da..00000000000 --- a/src/Storages/S3Queue/ReadBufferFromS3.h +++ /dev/null @@ -1,86 +0,0 @@ -//#pragma once -// -//#include -//#include -//#include -// -//#include -//#include -// -//namespace DB -//{ -//class ReadBufferFromS3 : public ReadBuffer -//{ -//public: -// ReadBufferFromS3( -// StorageS3Queue & storage_, -// size_t max_batch_size, -// size_t poll_timeout_, -// ContextPtr context_, -// size_t stream_number_, -// size_t max_streams_number_); -// -// ~ReadBufferFromS3() override = default; -// -// auto pollTimeout() const { return poll_timeout; } -// -// bool hasMorePolledRecords() const { return current != records.end(); } -// -// bool poll(); -// -// bool noRecords() { return buffer_status == BufferStatus::NO_RECORD_RETURNED; } -// -// auto getFileName() const { return current_file; } -// auto getOffset() const { return current_offset; } -// -//private: -// enum class BufferStatus -// { -// INIT, -// NO_RECORD_RETURNED, -// POLLED_OK, -// }; -// -// BufferStatus buffer_status = BufferStatus::INIT; -// -// Poco::Logger * log; -// -// StorageS3Queue & storage; -// -// bool stream_out = false; -// -// size_t batch_size; -// size_t poll_timeout; -// -// ContextPtr context; -// -// size_t stream_number; -// size_t max_streams_number; -// -// bool allowed = true; -// -// using RecordData = std::string; -// struct Record -// { -// RecordData data; -// std::string file_name; -// /// Offset is the start of a row, which is needed for virtual columns. -// UInt64 offset; -// }; -// using Records = std::vector; -// -// Records records; -// Records::const_iterator current; -// -// String current_file; -// UInt64 current_offset = 0; -// -// using TaskThread = BackgroundSchedulePool::TaskHolder; -// -// Records pollBatch(size_t batch_size_); -// -// void readNewRecords(Records & new_records, size_t batch_size_); -// -// bool nextImpl() override; -//}; -//} diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 8e0d970729b..9272d4f4c02 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -1,132 +1,476 @@ -//#include -//#include -//#include -//#include -//#include -//#include -//#include -// -//namespace DB -//{ -//static constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; -// -//S3QueueSource::S3QueueSource( -// StorageS3Queue & storage_, -// const StorageSnapshotPtr & storage_snapshot_, -// const ContextPtr & context_, -// const Names & columns, -// size_t max_block_size_, -// size_t poll_time_out_, -// size_t stream_number_, -// size_t max_streams_number_) -// : ISource(storage_snapshot_->getSampleBlockForColumns(columns)) -// , storage(storage_) -// , storage_snapshot(storage_snapshot_) -// , context(context_) -// , column_names(columns) -// , max_block_size(max_block_size_) -// , poll_time_out(poll_time_out_) -// , stream_number(stream_number_) -// , max_streams_number(max_streams_number_) -// , non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized()) -// , virtual_header(storage_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames())) -//{ -// buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); -//} -// -//S3QueueSource::~S3QueueSource() -//{ -// try -// { -// if (!finished) -// onFinish(); -// } -// catch (...) -// { -// tryLogCurrentException(__PRETTY_FUNCTION__); -// } -//} -// -//void S3QueueSource::onFinish() -//{ -// finished = true; -//} -// -//Chunk S3QueueSource::generate() -//{ -// /// Store metas of last written chunk into disk -// // storage.storeMetas(start, end); -// -// if (!buffer || buffer->noRecords()) -// { -// /// There is no onFinish for ISource, we call it -// /// when no records return to close files -// onFinish(); -// return {}; -// } -// -// MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); -// -// auto input_format -// = FormatFactory::instance().getInputFormat(storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); -// -// StreamingFormatExecutor executor(non_virtual_header, input_format); -// -// size_t total_rows = 0; -// size_t failed_poll_attempts = 0; -// -// Stopwatch watch; -// while (true) -// { -// size_t new_rows = 0; -// if (buffer->poll()) -// new_rows = executor.execute(); -// -// if (new_rows) -// { -// auto file_name = buffer->getFileName(); -// auto offset = buffer->getOffset(); -// for (size_t i = 0; i < new_rows; ++i) -// { -// virtual_columns[0]->insert(file_name); -// virtual_columns[1]->insert(offset); -// } -// total_rows = total_rows + new_rows; -// } -// else /// poll succeed, but parse failed -// { -// ++failed_poll_attempts; -// } -// -// if (!buffer->hasMorePolledRecords() -// && ((total_rows >= max_block_size) || watch.elapsedMilliseconds() > poll_time_out -// || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS)) -// { -// break; -// } -// } -// -// if (total_rows == 0) -// { -// onFinish(); -// return {}; -// } -// -// auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); -// auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); -// -// for (const auto & column : virtual_block.getColumnsWithTypeAndName()) -// result_block.insert(column); -// -// auto converting_dag = ActionsDAG::makeConvertingActions( -// result_block.cloneEmpty().getColumnsWithTypeAndName(), -// getPort().getHeader().getColumnsWithTypeAndName(), -// ActionsDAG::MatchColumnsMode::Name); -// -// auto converting_actions = std::make_shared(std::move(converting_dag)); -// converting_actions->execute(result_block); -// -// return Chunk(result_block.getColumns(), result_block.rows()); -//} -// -//} +#include "config.h" +#include +#include "IO/ParallelReadBuffer.h" +#include "IO/IOThreadPool.h" +#include "Parsers/ASTCreateQuery.h" +#include +#include + +#if USE_AWS_S3 + +#include + +#include + +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include + +#include + +#include + +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include + +namespace fs = std::filesystem; + + +namespace CurrentMetrics +{ +extern const Metric StorageS3Threads; +extern const Metric StorageS3ThreadsActive; +} + +namespace ProfileEvents +{ +extern const Event S3DeleteObjects; +extern const Event S3ListObjects; +} + +namespace DB +{ + +static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + +static const std::unordered_set required_configuration_keys = { + "url", +}; +static const std::unordered_set optional_configuration_keys = { + "format", + "compression", + "compression_method", + "structure", + "access_key_id", + "secret_access_key", + "filename", + "use_environment_credentials", + "max_single_read_retries", + "min_upload_part_size", + "upload_part_size_multiply_factor", + "upload_part_size_multiply_parts_count_threshold", + "max_single_part_upload_size", + "max_connections", + "expiration_window_seconds", + "no_sign_request" +}; + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_TEXT; + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int S3_ERROR; + extern const int UNEXPECTED_EXPRESSION; + extern const int DATABASE_ACCESS_DENIED; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int NOT_IMPLEMENTED; + extern const int CANNOT_COMPILE_REGEXP; + extern const int FILE_DOESNT_EXIST; +} + +class IOutputFormat; +using OutputFormatPtr = std::shared_ptr; + + + +StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( + const S3::Client & client_, + const S3::URI & globbed_uri_, + ASTPtr query, + const Block & virtual_header, + ContextPtr context, + StorageS3QueueSource::KeysWithInfo * read_keys_, + const S3Settings::RequestSettings & request_settings_) + : bucket(globbed_uri_.bucket), glob_iterator(std::make_unique(client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) +{ + while (true) { + KeyWithInfo val = glob_iterator->next(); + if (val.key.empty()) { + break; + } + keys_buf.push_back(val); + } +} + +Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, std::unordered_set & exclude_keys) { + for (KeyWithInfo val: keys_buf) { + if (exclude_keys.find(bucket + '/' + val.key) != exclude_keys.end()) { + LOG_INFO(log, "Found in exclude keys {}", val.key); + continue; + } + if (processing_keys.size() < max_poll_size) { + processing_keys.push_back(val); + } else { + break; + } + } + + if (mode == "ordered") { + std::sort(processing_keys.begin( ), processing_keys.end( ), [ ]( const KeyWithInfo& lhs, const KeyWithInfo& rhs ) + { + return lhs.key < rhs.key; + }); + } + + Strings keys; + for (auto v: processing_keys) { + keys.push_back(bucket + '/' + v.key); + } + processing_keys.push_back(KeyWithInfo()); + + processing_iterator = processing_keys.begin(); + return keys; +} + + +StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next() { + std::lock_guard lock(mutex); + if (processing_iterator != processing_keys.end()) { + return *processing_iterator++; + } + + return KeyWithInfo(); +} + +size_t StorageS3QueueSource::QueueGlobIterator::getTotalSize() const +{ + return glob_iterator->getTotalSize(); +} + + +Block StorageS3QueueSource::getHeader(Block sample_block, const std::vector & requested_virtual_columns) +{ + for (const auto & virtual_column : requested_virtual_columns) + sample_block.insert({virtual_column.type->createColumn(), virtual_column.type, virtual_column.name}); + + return sample_block; +} + +StorageS3QueueSource::StorageS3QueueSource( + const std::vector & requested_virtual_columns_, + const String & format_, + String name_, + const Block & sample_block_, + ContextPtr context_, + std::optional format_settings_, + const ColumnsDescription & columns_, + UInt64 max_block_size_, + const S3Settings::RequestSettings & request_settings_, + String compression_hint_, + const std::shared_ptr & client_, + const String & bucket_, + const String & version_id_, + std::shared_ptr file_iterator_, + zkutil::ZooKeeperPtr current_zookeeper, + const String & zookeeper_path_, + const size_t download_thread_num_) + : ISource(getHeader(sample_block_, requested_virtual_columns_)) + , WithContext(context_) + , name(std::move(name_)) + , bucket(bucket_) + , version_id(version_id_) + , format(format_) + , columns_desc(columns_) + , max_block_size(max_block_size_) + , request_settings(request_settings_) + , compression_hint(std::move(compression_hint_)) + , client(client_) + , sample_block(sample_block_) + , format_settings(format_settings_) + , requested_virtual_columns(requested_virtual_columns_) + , file_iterator(file_iterator_) + , download_thread_num(download_thread_num_) + , zookeeper(current_zookeeper) + , zookeeper_path(zookeeper_path_) + , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) + , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3QueueReader")) +{ + reader = createReader(); + if (reader) + reader_future = createReaderAsync(); +} + +StorageS3QueueSource::ReaderHolder StorageS3QueueSource::createReader() +{ + auto [current_key, info] = (*file_iterator)(); + if (current_key.empty()) + return {}; + + size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id, request_settings); + + int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); + auto read_buf = wrapReadBufferWithCompressionMethod( + createS3ReadBuffer(current_key, object_size), + chooseCompressionMethod(current_key, compression_hint), + zstd_window_log_max); + + auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size, format_settings); + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); + + if (columns_desc.hasDefaults()) + { + builder.addSimpleTransform( + [&](const Block & header) + { return std::make_shared(header, columns_desc, *input_format, getContext()); }); + } + + auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + auto current_reader = std::make_unique(*pipeline); + + return ReaderHolder{fs::path(bucket) / current_key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; +} + +std::future StorageS3QueueSource::createReaderAsync() +{ + return create_reader_scheduler([this] { return createReader(); }, 0); +} + +std::unique_ptr StorageS3QueueSource::createS3ReadBuffer(const String & key, size_t object_size) +{ + auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); + read_settings.enable_filesystem_cache = false; + + auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; + const bool use_parallel_download = download_buffer_size > 0 && download_thread_num > 1; + const bool object_too_small = object_size < download_thread_num * download_buffer_size; + + if (!use_parallel_download || object_too_small) + { + LOG_TRACE(log, "Downloading object of size {} from S3 in single thread", object_size); + if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) + return createAsyncS3ReadBuffer(key, read_settings, object_size); + + return std::make_unique(client, bucket, key, version_id, request_settings, read_settings); + } + + assert(object_size > 0); + if (download_buffer_size < DBMS_DEFAULT_BUFFER_SIZE) + { + LOG_WARNING(log, "Downloading buffer {} bytes too small, set at least {} bytes", download_buffer_size, DBMS_DEFAULT_BUFFER_SIZE); + download_buffer_size = DBMS_DEFAULT_BUFFER_SIZE; + } + + auto factory = std::make_unique( + client, bucket, key, version_id, download_buffer_size, object_size, request_settings, read_settings); + + LOG_TRACE(log, + "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", + download_thread_num, object_size, download_buffer_size); + + return std::make_unique(std::move(factory), threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelRead"), download_thread_num); +} + +std::unique_ptr StorageS3QueueSource::createAsyncS3ReadBuffer( + const String & key, const ReadSettings & read_settings, size_t object_size) +{ + auto read_buffer_creator = + [this, read_settings] + (const std::string & path, size_t read_until_position) -> std::shared_ptr + { + return std::make_shared( + client, + bucket, + path, + version_id, + request_settings, + read_settings, + /* use_external_buffer */true, + /* offset */0, + read_until_position, + /* restricted_seek */true); + }; + + auto s3_impl = std::make_unique( + std::move(read_buffer_creator), + StoredObjects{StoredObject{key, object_size}}, + read_settings); + + auto & pool_reader = getContext()->getThreadPoolReader(Context::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + auto async_reader = std::make_unique(pool_reader, read_settings, std::move(s3_impl)); + + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); + + return async_reader; +} + +StorageS3QueueSource::~StorageS3QueueSource() +{ + create_reader_pool.wait(); +} + +String StorageS3QueueSource::getName() const +{ + return name; +} + +Chunk StorageS3QueueSource::generate() +{ + while (true) + { + if (isCancelled() || !reader) + { + if (reader) + reader->cancel(); + break; + } + + Chunk chunk; + LOG_WARNING(log, "Try to pull new chunk"); + try { + if (reader->pull(chunk)) + { + LOG_WARNING(log, "Success in pulling!"); + UInt64 num_rows = chunk.getNumRows(); + + const auto & file_path = reader.getPath(); + size_t total_size = file_iterator->getTotalSize(); + if (num_rows && total_size) + { + updateRowsProgressApprox( + *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + } + + for (const auto & virtual_column : requested_virtual_columns) + { + if (virtual_column.name == "_path") + { + chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_path)->convertToFullColumnIfConst()); + } + else if (virtual_column.name == "_file") + { + size_t last_slash_pos = file_path.find_last_of('/'); + auto column = virtual_column.type->createColumnConst(num_rows, file_path.substr(last_slash_pos + 1)); + chunk.addColumn(column->convertToFullColumnIfConst()); + } + } + LOG_WARNING(log, "Set processed: {}", file_path); + setFileProcessed(file_path); + // TODO: Set processed + return chunk; + } + } catch (const Exception & e) { + LOG_ERROR(log, "Exception: {} ", e.displayText()); + const auto & failed_file_path = reader.getPath(); + LOG_WARNING(log, "Set failed: {}", failed_file_path); + setFileFailed(failed_file_path); + } + + + + assert(reader_future.valid()); + reader = reader_future.get(); + + if (!reader) + break; + + /// Even if task is finished the thread may be not freed in pool. + /// So wait until it will be freed before scheduling a new task. + create_reader_pool.wait(); + reader_future = createReaderAsync(); + } + + return {}; +} + +void StorageS3QueueSource::setFileProcessed(const String & file_path) { + std::lock_guard lock(mutex); + String processed_files = zookeeper->get(zookeeper_path + "/processed"); + std::unordered_set processed = parseCollection(processed_files); + + processed.insert(file_path); + Strings set_processed; + set_processed.insert(set_processed.end(), processed.begin(), processed.end()); + + zookeeper->set(zookeeper_path + "/processed", toString(set_processed)); +} + + +void StorageS3QueueSource::setFileFailed(const String & file_path) { + std::lock_guard lock(mutex); + String processed_files = zookeeper->get(zookeeper_path + "/failed"); + std::unordered_set processed = parseCollection(processed_files); + + processed.insert(file_path); + Strings set_failed; + set_failed.insert(set_failed.end(), processed.begin(), processed.end()); + + zookeeper->set(zookeeper_path + "/failed", toString(set_failed)); +} + +std::unordered_set StorageS3QueueSource::parseCollection(String & files) { + ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); + Strings deserialized; + try { + readQuoted(deserialized, rb); + } catch (...) { + deserialized = {}; + } + + std::unordered_set processed(deserialized.begin(), deserialized.end()); + + return processed; +} + + +} + +#endif diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 4714e23f748..c9bc9660a34 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -1,55 +1,146 @@ -//#pragma once -// -//#include -//#include -//#include -// -//namespace Poco -//{ -//class Logger; -//} -//namespace DB -//{ -//class S3QueueSource : public ISource -//{ -//public: -// S3QueueSource( -// StorageS3Queue & storage_, -// const StorageSnapshotPtr & storage_snapshot_, -// const ContextPtr & context_, -// const Names & columns, -// size_t max_block_size_, -// size_t poll_time_out_, -// size_t stream_number_, -// size_t max_streams_number_); -// -// String getName() const override { return "S3Queue"; } -// -// bool noRecords() { return !buffer || buffer->noRecords(); } -// -// void onFinish(); -// -// virtual ~S3QueueSource() override; -// -//protected: -// Chunk generate() override; -// -//private: -// StorageS3Queue & storage; -// StorageSnapshotPtr storage_snapshot; -// ContextPtr context; -// Names column_names; -// UInt64 max_block_size; -// -// size_t poll_time_out; -// -// size_t stream_number; -// size_t max_streams_number; -// -// std::unique_ptr buffer; -// -// Block non_virtual_header; -// Block virtual_header; -//}; -// -//} +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include + +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + + +class StorageS3QueueSource : public ISource, WithContext +{ +public: + using IIterator = StorageS3Source::IIterator; + using DisclosedGlobIterator = StorageS3Source::DisclosedGlobIterator; + using KeysWithInfo = StorageS3Source::KeysWithInfo; + using KeyWithInfo = StorageS3Source::KeyWithInfo; + + class QueueGlobIterator : public IIterator + { + public: + QueueGlobIterator( + const S3::Client & client_, + const S3::URI & globbed_uri_, + ASTPtr query, + const Block & virtual_header, + ContextPtr context, + KeysWithInfo * read_keys_ = nullptr, + const S3Settings::RequestSettings & request_settings_ = {}); + + KeyWithInfo next() override; + size_t getTotalSize() const override; + + Strings setProcessing(String & mode, std::unordered_set & exclude_keys); + private: + size_t max_poll_size = 10; + const String bucket; + KeysWithInfo keys_buf; + KeysWithInfo processing_keys; + mutable std::mutex mutex; + std::unique_ptr glob_iterator; + std::vector::iterator processing_iterator; + + Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSourceIterator"); + }; + + static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); + + StorageS3QueueSource( + const std::vector & requested_virtual_columns_, + const String & format, + String name_, + const Block & sample_block, + ContextPtr context_, + std::optional format_settings_, + const ColumnsDescription & columns_, + UInt64 max_block_size_, + const S3Settings::RequestSettings & request_settings_, + String compression_hint_, + const std::shared_ptr & client_, + const String & bucket, + const String & version_id, + std::shared_ptr file_iterator_, + zkutil::ZooKeeperPtr current_zookeeper, + const String & zookeeper_path_, + size_t download_thread_num); + + ~StorageS3QueueSource() override; + + String getName() const override; + + Chunk generate() override; + + static std::unordered_set parseCollection(String & files); + + +private: + String name; + String bucket; + String version_id; + String format; + ColumnsDescription columns_desc; + UInt64 max_block_size; + S3Settings::RequestSettings request_settings; + String compression_hint; + std::shared_ptr client; + Block sample_block; + std::optional format_settings; + + using ReaderHolder = StorageS3Source::ReaderHolder; + ReaderHolder reader; + + std::vector requested_virtual_columns; + std::shared_ptr file_iterator; + size_t download_thread_num = 1; + + Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSource"); + + zkutil::ZooKeeperPtr zookeeper; + const String zookeeper_path; + + ThreadPool create_reader_pool; + ThreadPoolCallbackRunner create_reader_scheduler; + std::future reader_future; + + UInt64 total_rows_approx_max = 0; + size_t total_rows_count_times = 0; + UInt64 total_rows_approx_accumulated = 0; + + mutable std::mutex mutex; + + + ReaderHolder createReader(); + std::future createReaderAsync(); + + std::unique_ptr createS3ReadBuffer(const String & key, size_t object_size); + std::unique_ptr createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size); + + void setFileProcessed(const String & file_path); + void setFileFailed(const String & file_path); + +}; + +} +#endif diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index ae162b90906..972302300ec 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -3,6 +3,14 @@ #include "IO/ParallelReadBuffer.h" #include "IO/IOThreadPool.h" #include "Parsers/ASTCreateQuery.h" +#include +#include +#include +#include + +#include +#include + #if USE_AWS_S3 @@ -23,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -68,11 +77,16 @@ namespace fs = std::filesystem; +//namespace CurrentMetrics +//{ +//extern const Metric S3QueueBackgroundReads; +//} namespace ProfileEvents { extern const Event S3DeleteObjects; extern const Event S3ListObjects; +extern const Event S3QueueBackgroundReads; } namespace DB @@ -151,6 +165,9 @@ StorageS3Queue::StorageS3Queue( , zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_)) , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ true, log)) { + if (!is_key_with_globs) { + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from key with globs"); + } FormatFactory::instance().checkFormatName(format_name); context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); @@ -212,6 +229,7 @@ Pipe StorageS3Queue::read( size_t max_block_size, size_t num_streams) { + // if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) // throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, // "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); @@ -239,14 +257,9 @@ Pipe StorageS3Queue::read( requested_virtual_columns.push_back(virtual_column); } - std::shared_ptr iterator_wrapper = StorageS3::createFileIterator( - query_s3_configuration, - keys, - is_key_with_globs, - distributed_processing, + std::shared_ptr iterator_wrapper = createFileIterator( local_context, - query_info.query, - virtual_block); + query_info.query); ColumnsDescription columns_description; Block block_for_format; @@ -273,11 +286,12 @@ Pipe StorageS3Queue::read( const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; // const size_t max_download_threads = 1; - LOG_WARNING(log, "num_streams"); + + auto zookeeper = getZooKeeper(); for (size_t i = 0; i < num_streams; ++i) { - pipes.emplace_back(std::make_shared( + pipes.emplace_back(std::make_shared( requested_virtual_columns, format_name, getName(), @@ -292,14 +306,13 @@ Pipe StorageS3Queue::read( query_s3_configuration.url.bucket, query_s3_configuration.url.version_id, iterator_wrapper, + zookeeper, + zookeeper_path, max_download_threads)); } auto pipe = Pipe::unitePipes(std::move(pipes)); - LOG_WARNING(log, "unitePipes"); - narrowPipe(pipe, num_streams); - LOG_WARNING(log, "narrowPipe"); return pipe; } @@ -386,10 +399,10 @@ void StorageS3Queue::threadFunc() auto table_id = getStorageID(); auto dependencies_count = getTableDependentCount(); - + LOG_TRACE(log, "dependencies_count {}", toString(dependencies_count)); if (dependencies_count) { - // auto start_time = std::chrono::steady_clock::now(); + auto start_time = std::chrono::steady_clock::now(); mv_attached.store(true); // Keep streaming as long as there are attached views and streaming is not cancelled @@ -404,20 +417,20 @@ void StorageS3Queue::threadFunc() LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - // if (streamToViews()) - // { - // LOG_TRACE(log, "Stream stalled. Reschedule."); - // break; - // } + if (streamToViews()) + { + LOG_TRACE(log, "Stream stalled. Reschedule."); + break; + } - // auto ts = std::chrono::steady_clock::now(); - // auto duration = std::chrono::duration_cast(ts-start_time); - // if (duration.count() > MAX_THREAD_WORK_DURATION_MS) - // { - // LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); - // reschedule = true; - // break; - // } + auto ts = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(ts-start_time); + if (duration.count() > 600000) + { + LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); + reschedule = true; + break; + } } } } @@ -438,6 +451,125 @@ void StorageS3Queue::threadFunc() } +bool StorageS3Queue::streamToViews() +{ + LOG_TRACE(log, "streamToViews"); + + Stopwatch watch; + + auto table_id = getStorageID(); + auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); + if (!table) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Engine table {} doesn't exist.", table_id.getNameForLogs()); + + // CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaBackgroundReads}; + // ProfileEvents::increment(ProfileEvents::S3QueueBackgroundReads); + + auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); + + // Create an INSERT query for streaming data + auto insert = std::make_shared(); + insert->table_id = table_id; + + size_t block_size = 100; + + auto s3queue_context = Context::createCopy(getContext()); + s3queue_context->makeQueryContext(); + auto query_s3_configuration = StorageS3::copyAndUpdateConfiguration(s3queue_context, s3_configuration); + + // s3queue_context->applySettingsChanges(settings_adjustments); + + // Create a stream for each consumer and join them in a union stream + // Only insert into dependent views and expect that input blocks contain virtual columns + InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); + auto block_io = interpreter.execute(); + auto column_names = block_io.pipeline.getHeader().getNames(); + + // Create a stream for each consumer and join them in a union stream + std::vector requested_virtual_columns; + + for (const auto & virtual_column : getVirtuals()) + { + requested_virtual_columns.push_back(virtual_column); + } + + std::shared_ptr iterator_wrapper = createFileIterator( + s3queue_context, + nullptr); + ColumnsDescription columns_description; + Block block_for_format; + if (supportsSubsetOfColumns()) + { + auto fetch_columns = column_names; + const auto & virtuals = getVirtuals(); + std::erase_if( + fetch_columns, + [&](const String & col) + { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); + + if (fetch_columns.empty()) + fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); + + columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); + block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); + } + else + { + columns_description = storage_snapshot->metadata->getColumns(); + block_for_format = storage_snapshot->metadata->getSampleBlock(); + } + + const size_t max_download_threads = s3queue_context->getSettingsRef().max_download_threads; + + Pipes pipes; + + auto zookeeper = getZooKeeper(); + size_t num_streams = 1; + for (size_t i = 0; i < num_streams; ++i) + { + pipes.emplace_back(std::make_shared( + requested_virtual_columns, + format_name, + getName(), + block_for_format, + s3queue_context, + format_settings, + columns_description, + block_size, + query_s3_configuration.request_settings, + compression_method, + query_s3_configuration.client, + query_s3_configuration.url.bucket, + query_s3_configuration.url.version_id, + iterator_wrapper, + zookeeper, + zookeeper_path, + max_download_threads)); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + + // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. + // It will be cancelled on underlying layer (kafka buffer) + + std::atomic_size_t rows = 0; + { + block_io.pipeline.complete(std::move(pipe)); + + // we need to read all consumers in parallel (sequential read may lead to situation + // when some of consumers are not used, and will break some Kafka consumer invariants) + block_io.pipeline.setNumThreads(num_streams); + + block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); + } + + bool some_stream_is_stalled = false; + return some_stream_is_stalled; +} + + void StorageS3Queue::setZooKeeper() { std::lock_guard lock(current_zookeeper_mutex); @@ -474,28 +606,31 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ for (size_t i = 0; i < 1000; ++i) { + Coordination::Requests ops; + auto table_uuid = getStorageID().uuid; + if (zookeeper->exists(zookeeper_path + "")) { + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "processing" / toString(table_uuid), "{}", + zkutil::CreateMode::Ephemeral)); LOG_DEBUG(log, "This table {} is already created, will add new replica", zookeeper_path); - return false; } - /// We write metadata of table so that the replicas can check table parameters with them. - // String metadata_str = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); - - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "", - zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "", - zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", - zkutil::CreateMode::Ephemeral)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), - zkutil::CreateMode::Persistent)); -// ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, -// zkutil::CreateMode::Persistent)); + else + { + /// We write metadata of table so that the replicas can check table parameters with them. + // String metadata_str = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "[]", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(zookeeper_path) / "processing" / toString(table_uuid), "[]", zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeCreateRequest( + zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); + // ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, + // zkutil::CreateMode::Persistent)); + } Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZNODEEXISTS) @@ -517,6 +652,60 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ "of wrong zookeeper_path or because of logical error"); } + +std::shared_ptr StorageS3Queue::createFileIterator( + ContextPtr local_context, + ASTPtr query, + KeysWithInfo * read_keys) +{ + /// Iterate through disclosed globs and make a source for each file + auto it = std::make_shared( + *s3_configuration.client, s3_configuration.url, query, virtual_block, + local_context, read_keys, s3_configuration.request_settings); + mode = "unordered"; + + std::lock_guard lock{sync_mutex}; + std::unordered_set exclude = getExcludedFiles(); + + auto zookeeper = getZooKeeper(); + auto table_uuid = getStorageID().uuid; + Strings processing = it->setProcessing(mode, exclude); + zookeeper->set(fs::path(zookeeper_path) / "processing" / toString(table_uuid), toString(processing)); + + return it; +} + +std::unordered_set StorageS3Queue::getExcludedFiles() { + auto zookeeper = getZooKeeper(); + std::unordered_set exclude_files; + + String failed = zookeeper->get(zookeeper_path + "/failed"); + std::unordered_set failed_files = StorageS3QueueSource::parseCollection(failed); + + LOG_DEBUG(log, "failed_files {}", failed_files.size()); + String processed = zookeeper->get(zookeeper_path + "/processed"); + std::unordered_set processed_files = StorageS3QueueSource::parseCollection(processed); + LOG_DEBUG(log, "processed_files {}", processed_files.size()); + + exclude_files.merge(failed_files); + exclude_files.merge(processed_files); + + Strings consumer_table_uuids; + zookeeper->tryGetChildren(zookeeper_path + "/processing", consumer_table_uuids); + + for (const auto & uuid : consumer_table_uuids) + { + String processing = zookeeper->get(fs::path(zookeeper_path) / "processing" / toString(uuid)); + std::unordered_set processing_files = StorageS3QueueSource::parseCollection(processing); + LOG_DEBUG(log, "processing {}", processing_files.size()); + exclude_files.merge(processing_files); + } + + return exclude_files; +} + + + void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { factory.registerStorage(name, [](const StorageFactory::Arguments & args) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 9c1390c24a8..24d99960f17 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -126,7 +127,7 @@ private: bool supportsSubsetOfColumns() const override; static Names getVirtualColumnNames(); - const String mode; + String mode; static const String default_zookeeper_name; const String zookeeper_name; @@ -136,6 +137,7 @@ private: zkutil::ZooKeeperPtr current_zookeeper; mutable std::mutex current_zookeeper_mutex; + mutable std::mutex sync_mutex; void setZooKeeper(); zkutil::ZooKeeperPtr tryGetZooKeeper() const; @@ -145,6 +147,17 @@ private: const String & getZooKeeperName() const { return zookeeper_name; } const String & getZooKeeperPath() const { return zookeeper_path; } + using KeysWithInfo = StorageS3QueueSource::KeysWithInfo; + + std::shared_ptr createFileIterator( + ContextPtr local_context, + ASTPtr query, + KeysWithInfo * read_keys = nullptr); + + static std::unordered_set parseCollection(String & files); + std::unordered_set getExcludedFiles(); + + bool streamToViews(); }; } diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 131649ff5fd..80510e84b78 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -145,6 +145,8 @@ public: Chunk generate() override; private: + friend class StorageS3QueueSource; + String name; String bucket; String version_id; From 60d7e7f57445f208e0b501e3b78faf4c5d4abe5e Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Sat, 29 Apr 2023 18:48:37 +0300 Subject: [PATCH 0031/2047] update settings --- src/Core/SettingsEnums.cpp | 4 + src/Core/SettingsEnums.h | 8 + src/Storages/S3Queue/S3QueueSettings.cpp | 4 +- src/Storages/S3Queue/S3QueueSettings.h | 14 +- src/Storages/S3Queue/S3QueueSource.cpp | 291 ++++++++++--------- src/Storages/S3Queue/S3QueueSource.h | 38 +-- src/Storages/S3Queue/StorageS3Queue.cpp | 341 ++++++++++++----------- src/Storages/S3Queue/StorageS3Queue.h | 83 +++--- 8 files changed, 420 insertions(+), 363 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index e0f16ea00db..fdda4ec8bc6 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -202,4 +202,8 @@ IMPLEMENT_SETTING_ENUM(ORCCompression, ErrorCodes::BAD_ARGUMENTS, {"zlib", FormatSettings::ORCCompression::ZLIB}, {"lz4", FormatSettings::ORCCompression::LZ4}}) +IMPLEMENT_SETTING_ENUM(S3QueueMode, ErrorCodes::BAD_ARGUMENTS, + {{"ordered", S3QueueMode::ORDERED}, + {"unordered", S3QueueMode::UNORDERED}}) + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 3ae7bfaa673..b8912e5b9b4 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -218,4 +218,12 @@ enum class ParallelReplicasCustomKeyFilterType : uint8_t DECLARE_SETTING_ENUM(ParallelReplicasCustomKeyFilterType) DECLARE_SETTING_ENUM(LocalFSReadMethod) + +enum class S3QueueMode +{ + ORDERED, + UNORDERED, +}; + +DECLARE_SETTING_ENUM(S3QueueMode) } diff --git a/src/Storages/S3Queue/S3QueueSettings.cpp b/src/Storages/S3Queue/S3QueueSettings.cpp index 72bf5fb6742..b74cf8d39bb 100644 --- a/src/Storages/S3Queue/S3QueueSettings.cpp +++ b/src/Storages/S3Queue/S3QueueSettings.cpp @@ -1,7 +1,7 @@ -#include #include -#include #include +#include +#include #include diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 8494ae47a90..3860a236a9b 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -10,8 +11,17 @@ class ASTStorage; #define S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ - M(String, mode, "unordered", "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", 0) \ - M(String, keeper_path, "/", "Zookeeper node path", 0) \ + M(S3QueueMode, \ + mode, \ + S3QueueMode::ORDERED, \ + "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", \ + 0) \ + M(String, keeper_path, "", "Zookeeper node path", 0) \ + M(String, processed_action, "keep", "Keep, delete or move file after successful loading", 0) \ + M(UInt64, s3queue_max_retries, 0, "Retry loading up to specified number of times", 0) \ + M(UInt64, s3queue_polling_min_timeout, 1000, "Minimal timeout before next polling", 0) \ + M(UInt64, s3queue_polling_max_timeout, 10000, "Maximum timeout before next polling", 0) \ + M(UInt64, s3queue_polling_backoff, 0, "Retry loading up to specified number of times", 0) #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 9272d4f4c02..92056c79d8f 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -1,72 +1,72 @@ -#include "config.h" -#include -#include "IO/ParallelReadBuffer.h" -#include "IO/IOThreadPool.h" -#include "Parsers/ASTCreateQuery.h" -#include #include +#include +#include +#include "IO/IOThreadPool.h" +#include "IO/ParallelReadBuffer.h" +#include "Parsers/ASTCreateQuery.h" +#include "config.h" #if USE_AWS_S3 -#include +# include -#include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include -#include +# include +# include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include -#include +# include -#include +# include -#include +# include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include -#include +# include namespace fs = std::filesystem; @@ -91,24 +91,23 @@ static const String PARTITION_ID_WILDCARD = "{_partition_id}"; static const std::unordered_set required_configuration_keys = { "url", }; -static const std::unordered_set optional_configuration_keys = { - "format", - "compression", - "compression_method", - "structure", - "access_key_id", - "secret_access_key", - "filename", - "use_environment_credentials", - "max_single_read_retries", - "min_upload_part_size", - "upload_part_size_multiply_factor", - "upload_part_size_multiply_parts_count_threshold", - "max_single_part_upload_size", - "max_connections", - "expiration_window_seconds", - "no_sign_request" -}; +static const std::unordered_set optional_configuration_keys + = {"format", + "compression", + "compression_method", + "structure", + "access_key_id", + "secret_access_key", + "filename", + "use_environment_credentials", + "max_single_read_retries", + "min_upload_part_size", + "upload_part_size_multiply_factor", + "upload_part_size_multiply_parts_count_threshold", + "max_single_part_upload_size", + "max_connections", + "expiration_window_seconds", + "no_sign_request"}; namespace ErrorCodes { @@ -128,7 +127,6 @@ class IOutputFormat; using OutputFormatPtr = std::shared_ptr; - StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( const S3::Client & client_, const S3::URI & globbed_uri_, @@ -137,39 +135,51 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( ContextPtr context, StorageS3QueueSource::KeysWithInfo * read_keys_, const S3Settings::RequestSettings & request_settings_) - : bucket(globbed_uri_.bucket), glob_iterator(std::make_unique(client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) + : bucket(globbed_uri_.bucket) + , glob_iterator(std::make_unique( + client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) { - while (true) { + while (true) + { KeyWithInfo val = glob_iterator->next(); - if (val.key.empty()) { + if (val.key.empty()) + { break; } keys_buf.push_back(val); } } -Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, std::unordered_set & exclude_keys) { - for (KeyWithInfo val: keys_buf) { - if (exclude_keys.find(bucket + '/' + val.key) != exclude_keys.end()) { +Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, std::unordered_set & exclude_keys) +{ + for (KeyWithInfo val : keys_buf) + { + if (exclude_keys.find(bucket + '/' + val.key) != exclude_keys.end()) + { LOG_INFO(log, "Found in exclude keys {}", val.key); continue; } - if (processing_keys.size() < max_poll_size) { + if (processing_keys.size() < max_poll_size) + { processing_keys.push_back(val); - } else { + } + else + { break; } } - if (mode == "ordered") { - std::sort(processing_keys.begin( ), processing_keys.end( ), [ ]( const KeyWithInfo& lhs, const KeyWithInfo& rhs ) - { - return lhs.key < rhs.key; - }); + if (mode == "ordered") + { + std::sort( + processing_keys.begin(), + processing_keys.end(), + [](const KeyWithInfo & lhs, const KeyWithInfo & rhs) { return lhs.key < rhs.key; }); } Strings keys; - for (auto v: processing_keys) { + for (auto v : processing_keys) + { keys.push_back(bucket + '/' + v.key); } processing_keys.push_back(KeyWithInfo()); @@ -179,9 +189,11 @@ Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, st } -StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next() { +StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next() +{ std::lock_guard lock(mutex); - if (processing_iterator != processing_keys.end()) { + if (processing_iterator != processing_keys.end()) + { return *processing_iterator++; } @@ -253,14 +265,32 @@ StorageS3QueueSource::ReaderHolder StorageS3QueueSource::createReader() return {}; size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id, request_settings); + auto compression_method = chooseCompressionMethod(current_key, compression_hint); - int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - auto read_buf = wrapReadBufferWithCompressionMethod( - createS3ReadBuffer(current_key, object_size), - chooseCompressionMethod(current_key, compression_hint), - zstd_window_log_max); + InputFormatPtr input_format; + std::unique_ptr owned_read_buf; + + auto read_buf_or_factory = createS3ReadBuffer(current_key, object_size); + if (read_buf_or_factory.buf_factory) + { + input_format = FormatFactory::instance().getInputRandomAccess( + format, + std::move(read_buf_or_factory.buf_factory), + sample_block, + getContext(), + max_block_size, + /* is_remote_fs */ true, + compression_method, + format_settings); + } + else + { + owned_read_buf = wrapReadBufferWithCompressionMethod( + std::move(read_buf_or_factory.buf), compression_method, static_cast(getContext()->getSettingsRef().zstd_window_log_max)); + input_format + = FormatFactory::instance().getInput(format, *owned_read_buf, sample_block, getContext(), max_block_size, format_settings); + } - auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size, format_settings); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -274,7 +304,7 @@ StorageS3QueueSource::ReaderHolder StorageS3QueueSource::createReader() auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); - return ReaderHolder{fs::path(bucket) / current_key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{fs::path(bucket) / current_key, std::move(owned_read_buf), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3QueueSource::createReaderAsync() @@ -282,47 +312,31 @@ std::future StorageS3QueueSource::createRead return create_reader_scheduler([this] { return createReader(); }, 0); } -std::unique_ptr StorageS3QueueSource::createS3ReadBuffer(const String & key, size_t object_size) +StorageS3QueueSource::ReadBufferOrFactory StorageS3QueueSource::createS3ReadBuffer(const String & key, size_t object_size) { auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); read_settings.enable_filesystem_cache = false; - auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; - const bool use_parallel_download = download_buffer_size > 0 && download_thread_num > 1; - const bool object_too_small = object_size < download_thread_num * download_buffer_size; + const bool object_too_small = object_size <= 2 * download_buffer_size; - if (!use_parallel_download || object_too_small) + // Create a read buffer that will prefetch the first ~1 MB of the file. + // When reading lots of tiny files, this prefetching almost doubles the throughput. + // For bigger files, parallel reading is more useful. + if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { - LOG_TRACE(log, "Downloading object of size {} from S3 in single thread", object_size); - if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - return createAsyncS3ReadBuffer(key, read_settings, object_size); - - return std::make_unique(client, bucket, key, version_id, request_settings, read_settings); + LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size); + return {.buf = createAsyncS3ReadBuffer(key, read_settings, object_size)}; } - assert(object_size > 0); - if (download_buffer_size < DBMS_DEFAULT_BUFFER_SIZE) - { - LOG_WARNING(log, "Downloading buffer {} bytes too small, set at least {} bytes", download_buffer_size, DBMS_DEFAULT_BUFFER_SIZE); - download_buffer_size = DBMS_DEFAULT_BUFFER_SIZE; - } - - auto factory = std::make_unique( - client, bucket, key, version_id, download_buffer_size, object_size, request_settings, read_settings); - - LOG_TRACE(log, - "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", - download_thread_num, object_size, download_buffer_size); - - return std::make_unique(std::move(factory), threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelRead"), download_thread_num); + auto factory = std::make_unique(client, bucket, key, version_id, object_size, request_settings, read_settings); + return {.buf_factory = std::move(factory)}; } -std::unique_ptr StorageS3QueueSource::createAsyncS3ReadBuffer( - const String & key, const ReadSettings & read_settings, size_t object_size) +std::unique_ptr +StorageS3QueueSource::createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size) { auto read_buffer_creator = - [this, read_settings] - (const std::string & path, size_t read_until_position) -> std::shared_ptr + [this, read_settings, object_size](const std::string & path, size_t read_until_position) -> std::shared_ptr { return std::make_shared( client, @@ -331,16 +345,15 @@ std::unique_ptr StorageS3QueueSource::createAsyncS3ReadBuffer( version_id, request_settings, read_settings, - /* use_external_buffer */true, - /* offset */0, + /* use_external_buffer */ true, + /* offset */ 0, read_until_position, - /* restricted_seek */true); + /* restricted_seek */ true, + object_size); }; auto s3_impl = std::make_unique( - std::move(read_buffer_creator), - StoredObjects{StoredObject{key, object_size}}, - read_settings); + std::move(read_buffer_creator), StoredObjects{StoredObject{key, object_size}}, read_settings); auto & pool_reader = getContext()->getThreadPoolReader(Context::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); auto async_reader = std::make_unique(pool_reader, read_settings, std::move(s3_impl)); @@ -375,7 +388,8 @@ Chunk StorageS3QueueSource::generate() Chunk chunk; LOG_WARNING(log, "Try to pull new chunk"); - try { + try + { if (reader->pull(chunk)) { LOG_WARNING(log, "Success in pulling!"); @@ -407,7 +421,9 @@ Chunk StorageS3QueueSource::generate() // TODO: Set processed return chunk; } - } catch (const Exception & e) { + } + catch (const Exception & e) + { LOG_ERROR(log, "Exception: {} ", e.displayText()); const auto & failed_file_path = reader.getPath(); LOG_WARNING(log, "Set failed: {}", failed_file_path); @@ -415,7 +431,6 @@ Chunk StorageS3QueueSource::generate() } - assert(reader_future.valid()); reader = reader_future.get(); @@ -431,7 +446,8 @@ Chunk StorageS3QueueSource::generate() return {}; } -void StorageS3QueueSource::setFileProcessed(const String & file_path) { +void StorageS3QueueSource::setFileProcessed(const String & file_path) +{ std::lock_guard lock(mutex); String processed_files = zookeeper->get(zookeeper_path + "/processed"); std::unordered_set processed = parseCollection(processed_files); @@ -444,7 +460,8 @@ void StorageS3QueueSource::setFileProcessed(const String & file_path) { } -void StorageS3QueueSource::setFileFailed(const String & file_path) { +void StorageS3QueueSource::setFileFailed(const String & file_path) +{ std::lock_guard lock(mutex); String processed_files = zookeeper->get(zookeeper_path + "/failed"); std::unordered_set processed = parseCollection(processed_files); @@ -456,12 +473,16 @@ void StorageS3QueueSource::setFileFailed(const String & file_path) { zookeeper->set(zookeeper_path + "/failed", toString(set_failed)); } -std::unordered_set StorageS3QueueSource::parseCollection(String & files) { +std::unordered_set StorageS3QueueSource::parseCollection(String & files) +{ ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); Strings deserialized; - try { + try + { readQuoted(deserialized, rb); - } catch (...) { + } + catch (...) + { deserialized = {}; } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index c9bc9660a34..3abb388cd2d 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -4,25 +4,25 @@ #if USE_AWS_S3 -#include +# include -#include +# include -#include -#include -#include +# include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include namespace DB @@ -36,7 +36,7 @@ public: using DisclosedGlobIterator = StorageS3Source::DisclosedGlobIterator; using KeysWithInfo = StorageS3Source::KeysWithInfo; using KeyWithInfo = StorageS3Source::KeyWithInfo; - + using ReadBufferOrFactory = StorageS3Source::ReadBufferOrFactory; class QueueGlobIterator : public IIterator { public: @@ -53,6 +53,7 @@ public: size_t getTotalSize() const override; Strings setProcessing(String & mode, std::unordered_set & exclude_keys); + private: size_t max_poll_size = 10; const String bucket; @@ -134,12 +135,11 @@ private: ReaderHolder createReader(); std::future createReaderAsync(); - std::unique_ptr createS3ReadBuffer(const String & key, size_t object_size); + ReadBufferOrFactory createS3ReadBuffer(const String & key, size_t object_size); std::unique_ptr createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size); void setFileProcessed(const String & file_path); void setFileFailed(const String & file_path); - }; } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 972302300ec..1cb04350635 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -1,79 +1,79 @@ -#include "config.h" -#include -#include "IO/ParallelReadBuffer.h" -#include "IO/IOThreadPool.h" -#include "Parsers/ASTCreateQuery.h" -#include +#include +#include #include +#include +#include #include #include - -#include -#include +#include +#include "IO/IOThreadPool.h" +#include "IO/ParallelReadBuffer.h" +#include "Parsers/ASTCreateQuery.h" +#include "config.h" #if USE_AWS_S3 -#include -#include +# include +# include -#include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include -#include +# include +# include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include -#include +# include -#include +# include -#include +# include -#include -#include -#include +# include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include namespace fs = std::filesystem; @@ -134,12 +134,8 @@ namespace ErrorCodes class IOutputFormat; using OutputFormatPtr = std::shared_ptr; -const String StorageS3Queue::default_zookeeper_name = "default"; - - StorageS3Queue::StorageS3Queue( - const String & zookeeper_path_, - const String & mode_, + std::unique_ptr s3queue_settings_, const StorageS3::Configuration & configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, @@ -151,6 +147,7 @@ StorageS3Queue::StorageS3Queue( ASTPtr partition_by_) : IStorage(table_id_) , WithContext(context_) + , s3queue_settings(std::move(s3queue_settings_)) , s3_configuration{configuration_} , keys({s3_configuration.url.key}) , format_name(configuration_.format) @@ -161,29 +158,43 @@ StorageS3Queue::StorageS3Queue( , partition_by(partition_by_) , is_key_with_globs(s3_configuration.url.key.find_first_of("*?{") != std::string::npos) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) - , mode(mode_) - , zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_)) - , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ true, log)) { - if (!is_key_with_globs) { + String setting_zookeeper_path = s3queue_settings->keeper_path; + if (setting_zookeeper_path == "") + { + auto table_id = getStorageID(); + auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); + bool is_in_replicated_database = database->getEngineName() == "Replicated"; + if (is_in_replicated_database) + { + LOG_INFO(log, "S3Queue engine keeper_path not specified. Use replicated database zookeeper path"); + String base_zookeeper_path = assert_cast(database.get())->getZooKeeperPath(); + zookeeper_path = zkutil::extractZooKeeperPath( + fs::path(base_zookeeper_path) / toString(table_id.uuid), /* check_starts_with_slash */ true, log); + } + else + { + throw Exception(ErrorCodes::NO_ZOOKEEPER, "S3Queue zookeeper path not specified and table not in replicated database."); + } + } + else + { + zookeeper_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path, /* check_starts_with_slash */ true, log); + } + LOG_INFO(log, "Storage S3Queue zookeeper_path= {} with mode", zookeeper_path); + + if (!is_key_with_globs) + { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from key with globs"); } FormatFactory::instance().checkFormatName(format_name); context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); - StorageInMemoryMetadata storage_metadata; + s3_configuration.update(context_); - StorageS3::updateConfiguration(context_, s3_configuration); if (columns_.empty()) { - auto columns = StorageS3::getTableStructureFromDataImpl( - format_name, - s3_configuration, - compression_method, - is_key_with_globs, - format_settings, - context_); - + auto columns = StorageS3::getTableStructureFromDataImpl(s3_configuration, format_settings, context_); storage_metadata.setColumns(columns); } else @@ -207,6 +218,7 @@ StorageS3Queue::StorageS3Queue( auto metadata_snapshot = getInMemoryMetadataPtr(); createTableIfNotExists(metadata_snapshot); task = std::make_shared(std::move(thread)); + LOG_TRACE(log, "Complete"); } @@ -229,18 +241,16 @@ Pipe StorageS3Queue::read( size_t max_block_size, size_t num_streams) { - -// if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) -// throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, -// "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + // if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) + // throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, + // "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); if (mv_attached) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageRabbitMQ with attached materialized views"); - auto query_s3_configuration = StorageS3::copyAndUpdateConfiguration(local_context, s3_configuration); + auto query_s3_configuration = updateConfigurationAndGetCopy(local_context); - bool has_wildcards = - query_s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos + bool has_wildcards = query_s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; if (partition_by && has_wildcards) @@ -257,9 +267,7 @@ Pipe StorageS3Queue::read( requested_virtual_columns.push_back(virtual_column); } - std::shared_ptr iterator_wrapper = createFileIterator( - local_context, - query_info.query); + std::shared_ptr iterator_wrapper = createFileIterator(local_context, query_info.query); ColumnsDescription columns_description; Block block_for_format; @@ -269,8 +277,10 @@ Pipe StorageS3Queue::read( const auto & virtuals = getVirtuals(); std::erase_if( fetch_columns, - [&](const String & col) - { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); + [&](const String & col) { + return std::any_of( + virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); + }); if (fetch_columns.empty()) fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); @@ -348,7 +358,7 @@ void StorageS3Queue::shutdown() task->stream_cancelled = true; /// Reader thread may wait for wake up -// wakeUp(); + // wakeUp(); LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); @@ -424,7 +434,7 @@ void StorageS3Queue::threadFunc() } auto ts = std::chrono::steady_clock::now(); - auto duration = std::chrono::duration_cast(ts-start_time); + auto duration = std::chrono::duration_cast(ts - start_time); if (duration.count() > 600000) { LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); @@ -475,7 +485,7 @@ bool StorageS3Queue::streamToViews() auto s3queue_context = Context::createCopy(getContext()); s3queue_context->makeQueryContext(); - auto query_s3_configuration = StorageS3::copyAndUpdateConfiguration(s3queue_context, s3_configuration); + auto query_s3_configuration = updateConfigurationAndGetCopy(s3queue_context); // s3queue_context->applySettingsChanges(settings_adjustments); @@ -493,9 +503,7 @@ bool StorageS3Queue::streamToViews() requested_virtual_columns.push_back(virtual_column); } - std::shared_ptr iterator_wrapper = createFileIterator( - s3queue_context, - nullptr); + std::shared_ptr iterator_wrapper = createFileIterator(s3queue_context, nullptr); ColumnsDescription columns_description; Block block_for_format; if (supportsSubsetOfColumns()) @@ -504,8 +512,10 @@ bool StorageS3Queue::streamToViews() const auto & virtuals = getVirtuals(); std::erase_if( fetch_columns, - [&](const String & col) - { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); + [&](const String & col) { + return std::any_of( + virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); + }); if (fetch_columns.empty()) fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); @@ -569,19 +579,16 @@ bool StorageS3Queue::streamToViews() return some_stream_is_stalled; } +StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) +{ + s3_configuration.update(local_context); + return s3_configuration; +} void StorageS3Queue::setZooKeeper() { std::lock_guard lock(current_zookeeper_mutex); - LOG_WARNING(log, "zookeeper name {}", zookeeper_name); - if (zookeeper_name == default_zookeeper_name) - { - current_zookeeper = getContext()->getZooKeeper(); - } - else - { - current_zookeeper = getContext()->getAuxiliaryZooKeeper(zookeeper_name); - } + current_zookeeper = getContext()->getZooKeeper(); } zkutil::ZooKeeperPtr StorageS3Queue::tryGetZooKeeper() const @@ -611,8 +618,8 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ if (zookeeper->exists(zookeeper_path + "")) { - ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "processing" / toString(table_uuid), "{}", - zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(zookeeper_path) / "processing" / toString(table_uuid), "{}", zkutil::CreateMode::Ephemeral)); LOG_DEBUG(log, "This table {} is already created, will add new replica", zookeeper_path); } else @@ -647,35 +654,34 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ } /// Do not use LOGICAL_ERROR code, because it may happen if user has specified wrong zookeeper_path - throw Exception(ErrorCodes::REPLICA_ALREADY_EXISTS, - "Cannot create table, because it is created concurrently every time or because " - "of wrong zookeeper_path or because of logical error"); + throw Exception( + ErrorCodes::REPLICA_ALREADY_EXISTS, + "Cannot create table, because it is created concurrently every time or because " + "of wrong zookeeper_path or because of logical error"); } -std::shared_ptr StorageS3Queue::createFileIterator( - ContextPtr local_context, - ASTPtr query, - KeysWithInfo * read_keys) +std::shared_ptr +StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys) { /// Iterate through disclosed globs and make a source for each file - auto it = std::make_shared( - *s3_configuration.client, s3_configuration.url, query, virtual_block, - local_context, read_keys, s3_configuration.request_settings); - mode = "unordered"; + auto it = std::make_shared( + *s3_configuration.client, s3_configuration.url, query, virtual_block, local_context, read_keys, s3_configuration.request_settings); + String cur_mode = "unordered"; std::lock_guard lock{sync_mutex}; std::unordered_set exclude = getExcludedFiles(); auto zookeeper = getZooKeeper(); auto table_uuid = getStorageID().uuid; - Strings processing = it->setProcessing(mode, exclude); + Strings processing = it->setProcessing(cur_mode, exclude); zookeeper->set(fs::path(zookeeper_path) / "processing" / toString(table_uuid), toString(processing)); return it; } -std::unordered_set StorageS3Queue::getExcludedFiles() { +std::unordered_set StorageS3Queue::getExcludedFiles() +{ auto zookeeper = getZooKeeper(); std::unordered_set exclude_files; @@ -705,55 +711,74 @@ std::unordered_set StorageS3Queue::getExcludedFiles() { } - void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { - factory.registerStorage(name, [](const StorageFactory::Arguments & args) - { - auto & engine_args = args.engine_args; - if (engine_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + factory.registerStorage( + name, + [](const StorageFactory::Arguments & args) + { + auto & engine_args = args.engine_args; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); - auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); - // Use format settings from global server context + settings from - // the SETTINGS clause of the create query. Settings from current - // session and user are ignored. - auto s3queue_settings = std::make_unique(); - std::optional format_settings; + // Use format settings from global server context + settings from + // the SETTINGS clause of the create query. Settings from current + // session and user are ignored. + std::optional format_settings; - if (args.storage_def->settings) - { - s3queue_settings->loadFromQuery(*args.storage_def); + auto s3queue_settings = std::make_unique(); + if (args.storage_def->settings) + { + s3queue_settings->loadFromQuery(*args.storage_def); + FormatFactorySettings user_format_settings; - } - format_settings = getFormatSettings(args.getContext()); + // Apply changed settings from global context, but ignore the + // unknown ones, because we only have the format settings here. + const auto & changes = args.getContext()->getSettingsRef().changes(); + for (const auto & change : changes) + { + if (user_format_settings.has(change.name)) + user_format_settings.set(change.name, change.value); + else + LOG_TRACE(&Poco::Logger::get("StorageS3"), "Remove: {}", change.name); + args.storage_def->settings->changes.removeSetting(change.name); + } - ASTPtr partition_by; - if (args.storage_def->partition_by) - partition_by = args.storage_def->partition_by->clone(); + for (const auto & change : args.storage_def->settings->changes) + { + if (user_format_settings.has(change.name)) + user_format_settings.applyChange(change); + } + format_settings = getFormatSettings(args.getContext(), user_format_settings); + } + else + { + format_settings = getFormatSettings(args.getContext()); + } - String keeper_path = s3queue_settings->keeper_path; - String mode = s3queue_settings->mode; + ASTPtr partition_by; + if (args.storage_def->partition_by) + partition_by = args.storage_def->partition_by->clone(); - return std::make_shared( - keeper_path, - mode, - configuration, - args.table_id, - args.columns, - args.constraints, - args.comment, - args.getContext(), - format_settings, - /* distributed_processing_ */false, - partition_by); - }, - { - .supports_settings = true, - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::S3, - }); + return std::make_shared( + std::move(s3queue_settings), + std::move(configuration), + args.table_id, + args.columns, + args.constraints, + args.comment, + args.getContext(), + format_settings, + /* distributed_processing_ */ false, + partition_by); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); } void registerStorageS3Queue(StorageFactory & factory) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 24d99960f17..1763e882d59 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -4,28 +4,28 @@ #if USE_AWS_S3 -#include +# include -#include -#include +# include +# include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include namespace Aws::S3 { @@ -36,14 +36,12 @@ namespace DB { - class StorageS3Queue : public IStorage, WithContext { public: using Configuration = typename StorageS3::Configuration; StorageS3Queue( - const String & zookeper_path_, - const String & mode_, + std::unique_ptr s3queue_settings_, const Configuration & configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, @@ -65,11 +63,17 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override { + SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override + { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); } - void truncate(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*local_context*/, TableExclusiveLockHolder &) override { + void truncate( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + ContextPtr /*local_context*/, + TableExclusiveLockHolder &) override + { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); } @@ -77,16 +81,10 @@ public: bool supportsPartitionBy() const override; - static ColumnsDescription getTableStructureFromData( - Configuration & configuration, const std::optional & format_settings, ContextPtr ctx) - { - return StorageS3::getTableStructureFromData(configuration, format_settings, ctx); - } - const auto & getFormatName() const { return format_name; } private: - + std::unique_ptr s3queue_settings; Configuration s3_configuration; std::vector keys; NamesAndTypesList virtual_columns; @@ -117,23 +115,15 @@ private: struct TaskContext { BackgroundSchedulePool::TaskHolder holder; - std::atomic stream_cancelled {false}; - explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) - { - } + std::atomic stream_cancelled{false}; + explicit TaskContext(BackgroundSchedulePool::TaskHolder && task_) : holder(std::move(task_)) { } }; std::shared_ptr task; bool supportsSubsetOfColumns() const override; static Names getVirtualColumnNames(); - String mode; - - static const String default_zookeeper_name; - const String zookeeper_name; - const String zookeeper_path; - const String replica_name; - const String replica_path; + String zookeeper_path; zkutil::ZooKeeperPtr current_zookeeper; mutable std::mutex current_zookeeper_mutex; @@ -144,20 +134,19 @@ private: zkutil::ZooKeeperPtr getZooKeeper() const; bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); // Return default or custom zookeeper name for table - const String & getZooKeeperName() const { return zookeeper_name; } const String & getZooKeeperPath() const { return zookeeper_path; } using KeysWithInfo = StorageS3QueueSource::KeysWithInfo; - std::shared_ptr createFileIterator( - ContextPtr local_context, - ASTPtr query, - KeysWithInfo * read_keys = nullptr); + std::shared_ptr + createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys = nullptr); static std::unordered_set parseCollection(String & files); std::unordered_set getExcludedFiles(); bool streamToViews(); + + Configuration updateConfigurationAndGetCopy(ContextPtr local_context); }; } From c9e30d3cf5f5f0ac9f35e2e08df429bacbe4cd25 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Apr 2023 19:04:20 +0200 Subject: [PATCH 0032/2047] Properly check the limit for `sleepEachRow` function. Add a setting `function_sleep_max_microseconds_per_block` --- src/Core/Settings.h | 3 ++- src/Functions/sleep.h | 21 +++++++++++++++---- .../02725_sleep_max_time.reference | 0 .../0_stateless/02725_sleep_max_time.sql | 1 + 4 files changed, 20 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02725_sleep_max_time.reference create mode 100644 tests/queries/0_stateless/02725_sleep_max_time.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7f1fe838b80..5aa054d43b0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -633,7 +633,8 @@ class IColumn; M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \ M(UInt64, offset, 0, "Offset on read rows from the most 'end' result for select query", 0) \ \ - M(UInt64, function_range_max_elements_in_block, 500000000, "Maximum number of values generated by function 'range' per block of data (sum of array sizes for every row in a block, see also 'max_block_size' and 'min_insert_block_size_rows'). It is a safety threshold.", 0) \ + M(UInt64, function_range_max_elements_in_block, 500000000, "Maximum number of values generated by function `range` per block of data (sum of array sizes for every row in a block, see also 'max_block_size' and 'min_insert_block_size_rows'). It is a safety threshold.", 0) \ + M(UInt64, function_sleep_max_microseconds_per_block, 3000, "Maximum number of microseconds the function `sleep` is allowed to sleep for each block. If a user called it with a larger value, it throws an exception. It is a safety threshold.", 0) \ M(ShortCircuitFunctionEvaluation, short_circuit_function_evaluation, ShortCircuitFunctionEvaluation::ENABLE, "Setting for short-circuit function evaluation configuration. Possible values: 'enable' - use short-circuit function evaluation for functions that are suitable for it, 'disable' - disable short-circuit function evaluation, 'force_enable' - use short-circuit function evaluation for all functions.", 0) \ \ M(LocalFSReadMethod, storage_file_read_method, LocalFSReadMethod::mmap, "Method of reading data from storage file, one of: read, pread, mmap.", 0) \ diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index d1960860308..93525c3f310 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -9,7 +9,8 @@ #include #include #include -#include +#include + namespace ProfileEvents { @@ -40,11 +41,17 @@ enum class FunctionSleepVariant template class FunctionSleep : public IFunction { +private: + UInt64 max_microseconds; public: static constexpr auto name = variant == FunctionSleepVariant::PerBlock ? "sleep" : "sleepEachRow"; - static FunctionPtr create(ContextPtr) + static FunctionPtr create(ContextPtr context) + { + return std::make_shared>(context->getSettingsRef().function_sleep_max_microseconds_per_block); + } + + FunctionSleep(UInt64 max_microseconds_) : max_microseconds(max_microseconds_) { - return std::make_shared>(); } /// Get the name of the function. @@ -105,13 +112,19 @@ public: if (size > 0) { /// When sleeping, the query cannot be cancelled. For ability to cancel query, we limit sleep time. - if (seconds > 3.0) /// The choice is arbitrary + if (seconds * 1e6 > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is 3 seconds. Requested: {}", toString(seconds)); if (!dry_run) { UInt64 count = (variant == FunctionSleepVariant::PerBlock ? 1 : size); UInt64 microseconds = static_cast(seconds * count * 1e6); + + if (microseconds > max_microseconds) + throw Exception(ErrorCodes::TOO_SLOW, + "The maximum sleep time is 3 seconds. Requested: {} microseconds per block (of size {})", + microseconds, size); + sleepForMicroseconds(microseconds); ProfileEvents::increment(ProfileEvents::SleepFunctionCalls, count); ProfileEvents::increment(ProfileEvents::SleepFunctionMicroseconds, microseconds); diff --git a/tests/queries/0_stateless/02725_sleep_max_time.reference b/tests/queries/0_stateless/02725_sleep_max_time.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02725_sleep_max_time.sql b/tests/queries/0_stateless/02725_sleep_max_time.sql new file mode 100644 index 00000000000..b8378aee17e --- /dev/null +++ b/tests/queries/0_stateless/02725_sleep_max_time.sql @@ -0,0 +1 @@ +SELECT * FROM system.numbers WHERE sleepEachRow(0.05) LIMIT 10; -- { serverError TOO_SLOW } From 3de0c319c2d6b6206196ece48b228f72f3a9aecd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Apr 2023 19:08:52 +0200 Subject: [PATCH 0033/2047] Add compatibility --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 + src/Functions/sleep.h | 4 ++-- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5aa054d43b0..2ab4fe9b32a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -634,7 +634,7 @@ class IColumn; M(UInt64, offset, 0, "Offset on read rows from the most 'end' result for select query", 0) \ \ M(UInt64, function_range_max_elements_in_block, 500000000, "Maximum number of values generated by function `range` per block of data (sum of array sizes for every row in a block, see also 'max_block_size' and 'min_insert_block_size_rows'). It is a safety threshold.", 0) \ - M(UInt64, function_sleep_max_microseconds_per_block, 3000, "Maximum number of microseconds the function `sleep` is allowed to sleep for each block. If a user called it with a larger value, it throws an exception. It is a safety threshold.", 0) \ + M(UInt64, function_sleep_max_microseconds_per_block, 3000000, "Maximum number of microseconds the function `sleep` is allowed to sleep for each block. If a user called it with a larger value, it throws an exception. It is a safety threshold.", 0) \ M(ShortCircuitFunctionEvaluation, short_circuit_function_evaluation, ShortCircuitFunctionEvaluation::ENABLE, "Setting for short-circuit function evaluation configuration. Possible values: 'enable' - use short-circuit function evaluation for functions that are suitable for it, 'disable' - disable short-circuit function evaluation, 'force_enable' - use short-circuit function evaluation for all functions.", 0) \ \ M(LocalFSReadMethod, storage_file_read_method, LocalFSReadMethod::mmap, "Method of reading data from storage file, one of: read, pread, mmap.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 266d14f645b..33010dc6b3b 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,6 +80,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"23.5", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximim sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}}}, {"23.4", {{"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index 93525c3f310..db4f0e7dd3e 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -112,7 +112,7 @@ public: if (size > 0) { /// When sleeping, the query cannot be cancelled. For ability to cancel query, we limit sleep time. - if (seconds * 1e6 > max_microseconds) + if (max_microseconds && seconds * 1e6 > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is 3 seconds. Requested: {}", toString(seconds)); if (!dry_run) @@ -120,7 +120,7 @@ public: UInt64 count = (variant == FunctionSleepVariant::PerBlock ? 1 : size); UInt64 microseconds = static_cast(seconds * count * 1e6); - if (microseconds > max_microseconds) + if (max_microseconds && microseconds > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is 3 seconds. Requested: {} microseconds per block (of size {})", microseconds, size); From 582cf2ca8427c572a83c0bc249275c22fae6de5c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Apr 2023 19:48:33 +0200 Subject: [PATCH 0034/2047] Update tests --- src/Functions/sleep.h | 6 +++--- tests/queries/0_stateless/00956_sensitive_data_masking.sh | 1 + tests/queries/0_stateless/01107_atomic_db_detach_attach.sh | 4 ++-- tests/queries/0_stateless/01114_database_atomic.sh | 6 +++--- .../queries/0_stateless/01192_rename_database_zookeeper.sh | 4 ++-- tests/queries/0_stateless/01238_http_memory_tracking.sh | 2 +- tests/queries/0_stateless/01246_buffer_flush.sql | 2 ++ tests/queries/0_stateless/01338_long_select_and_alter.sh | 2 +- .../0_stateless/01338_long_select_and_alter_zookeeper.sh | 2 +- .../01532_execute_merges_on_single_replica_long.sql | 1 + .../01715_background_checker_blather_zookeeper_long.sql | 1 + .../01737_clickhouse_server_wait_server_pool_long.sh | 2 +- 12 files changed, 19 insertions(+), 14 deletions(-) diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index db4f0e7dd3e..fba8293e5ff 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -113,7 +113,7 @@ public: { /// When sleeping, the query cannot be cancelled. For ability to cancel query, we limit sleep time. if (max_microseconds && seconds * 1e6 > max_microseconds) - throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is 3 seconds. Requested: {}", toString(seconds)); + throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is {} microseconds. Requested: {}", max_microseconds, seconds); if (!dry_run) { @@ -122,8 +122,8 @@ public: if (max_microseconds && microseconds > max_microseconds) throw Exception(ErrorCodes::TOO_SLOW, - "The maximum sleep time is 3 seconds. Requested: {} microseconds per block (of size {})", - microseconds, size); + "The maximum sleep time is {} microseconds. Requested: {} microseconds per block (of size {})", + max_microseconds, microseconds, size); sleepForMicroseconds(microseconds); ProfileEvents::increment(ProfileEvents::SleepFunctionCalls, count); diff --git a/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/tests/queries/0_stateless/00956_sensitive_data_masking.sh index ccd9bbcf10e..a31a71ce381 100755 --- a/tests/queries/0_stateless/00956_sensitive_data_masking.sh +++ b/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -65,6 +65,7 @@ echo 5 # run in background rm -f "$tmp_file2" >/dev/null 2>&1 bash -c "$CLICKHOUSE_CLIENT \ + --function_sleep_max_microseconds_per_block 60 \ --query=\"select sleepEachRow(1) from numbers(10) where ignore('find_me_TOPSECRET=TOPSECRET')=0 and ignore('fwerkh_that_magic_string_make_me_unique') = 0 FORMAT Null\" \ --log_queries=1 --ignore-error --multiquery |& grep -v '^(query: ' > $tmp_file2" & diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index e4dad56bc29..e2a23258584 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107" $CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & sleep 1 $CLICKHOUSE_CLIENT -q "DETACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 @@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" --database_atomic_wait_for_dr $CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & sleep 1 $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 && sleep 1 && echo "dropped" wait diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 4a3d35e48b7..634b19a7624 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -49,8 +49,8 @@ $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW $CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" -$CLICKHOUSE_CLIENT -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s (1.5s * 30 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s (1.5s * 30 rows) sleep 1 # SELECT and INSERT should start before the following RENAMEs $CLICKHOUSE_CLIENT -nm -q " @@ -74,7 +74,7 @@ INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5); SELECT count() FROM test_01114_1.mt " # result: 5 -$CLICKHOUSE_CLIENT -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s (3s * 5 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s (3s * 5 rows) sleep 1 $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped" diff --git a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh index dec1276111a..ac516e83c84 100755 --- a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh +++ b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh @@ -20,7 +20,7 @@ $CLICKHOUSE_CLIENT -q "SELECT engine, splitByChar('/', data_path)[-2], uuid, spl # 3. check RENAME don't wait for INSERT $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01192.mt (n UInt64) ENGINE=MergeTree ORDER BY n" -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01192.mt SELECT number + sleepEachRow(1.5) FROM numbers(10)" && echo "inserted" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 15 -q "INSERT INTO test_01192.mt SELECT number + sleepEachRow(1.5) FROM numbers(10)" && echo "inserted" & sleep 1 $CLICKHOUSE_CLIENT -q "RENAME DATABASE test_01192 TO default" 2>&1| grep -F "already exists" > /dev/null && echo "ok" @@ -60,7 +60,7 @@ $CLICKHOUSE_CLIENT -q "SELECT database, name, status, origin FROM system.diction $CLICKHOUSE_CLIENT -q "SELECT dictGet('test_01192_atomic.dict', '_part', toUInt64(1))" # 8. check RENAME don't wait for INSERT -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01192_atomic.mt SELECT number + sleepEachRow(1) + 10 FROM numbers(10)" && echo "inserted" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 -q "INSERT INTO test_01192_atomic.mt SELECT number + sleepEachRow(1) + 10 FROM numbers(10)" && echo "inserted" & sleep 1 $CLICKHOUSE_CLIENT --check_table_dependencies=0 -q "RENAME DATABASE test_01192 TO test_01192_renamed" 2>&1| grep -F "not supported" > /dev/null && echo "ok" diff --git a/tests/queries/0_stateless/01238_http_memory_tracking.sh b/tests/queries/0_stateless/01238_http_memory_tracking.sh index 9b0fe875416..eb42159ce15 100755 --- a/tests/queries/0_stateless/01238_http_memory_tracking.sh +++ b/tests/queries/0_stateless/01238_http_memory_tracking.sh @@ -10,7 +10,7 @@ set -o pipefail # This is needed to keep at least one running query for user for the time of test. # (1k http queries takes ~1 second, let's run for 5x more to avoid flaps) -${CLICKHOUSE_CLIENT} --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)' & +${CLICKHOUSE_CLIENT} --function_sleep_max_microseconds_per_block 5 --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)' & # ignore "yes: standard output: Broken pipe" yes 'SELECT 1' 2>/dev/null | { diff --git a/tests/queries/0_stateless/01246_buffer_flush.sql b/tests/queries/0_stateless/01246_buffer_flush.sql index ac507d94b69..36bcaae383f 100644 --- a/tests/queries/0_stateless/01246_buffer_flush.sql +++ b/tests/queries/0_stateless/01246_buffer_flush.sql @@ -1,5 +1,7 @@ -- Tags: no-fasttest +SET function_sleep_max_microseconds_per_block = 4000000; + drop table if exists data_01256; drop table if exists buffer_01256; diff --git a/tests/queries/0_stateless/01338_long_select_and_alter.sh b/tests/queries/0_stateless/01338_long_select_and_alter.sh index 2e3080e9cfc..04a10cfe55e 100755 --- a/tests/queries/0_stateless/01338_long_select_and_alter.sh +++ b/tests/queries/0_stateless/01338_long_select_and_alter.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENG $CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" -$CLICKHOUSE_CLIENT --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & # to be sure that select took all required locks sleep 2 diff --git a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh index 12bc3b09472..829352110f6 100755 --- a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh +++ b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENG $CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" -$CLICKHOUSE_CLIENT --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & # to be sure that select took all required locks sleep 2 diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql index f217b6094b2..d39ffdc4049 100644 --- a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql +++ b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql @@ -44,6 +44,7 @@ SYSTEM STOP REPLICATION QUEUES execute_on_single_replica_r2; OPTIMIZE TABLE execute_on_single_replica_r1 FINAL SETTINGS replication_alter_partitions_sync=0; /* if we will check immediately we can find the log entry unchecked */ +SET function_sleep_max_microseconds_per_block = 4000000; SELECT * FROM numbers(4) where sleepEachRow(1); SELECT '****************************'; diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql index 87e1a039488..32481be1bcd 100644 --- a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql +++ b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql @@ -18,6 +18,7 @@ DETACH TABLE i20203_1; ATTACH TABLE i20203_2; -- sleep 10 seconds +SET function_sleep_max_microseconds_per_block = 10000000; SELECT number from numbers(10) where sleepEachRow(1) Format Null; SELECT num_tries < 50 diff --git a/tests/queries/0_stateless/01737_clickhouse_server_wait_server_pool_long.sh b/tests/queries/0_stateless/01737_clickhouse_server_wait_server_pool_long.sh index d83656e0e8c..adab3906e5b 100755 --- a/tests/queries/0_stateless/01737_clickhouse_server_wait_server_pool_long.sh +++ b/tests/queries/0_stateless/01737_clickhouse_server_wait_server_pool_long.sh @@ -54,7 +54,7 @@ if ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null fi query_id="$CLICKHOUSE_DATABASE-$SECONDS" -$CLICKHOUSE_CLIENT_BINARY --query_id "$query_id" --host 127.1 --port "$server_port" --format Null -q 'select sleepEachRow(1) from numbers(10)' 2>/dev/null & +$CLICKHOUSE_CLIENT_BINARY --query_id "$query_id" --host 127.1 --port "$server_port" --format Null --function_sleep_max_microseconds_per_block 0 -q 'select sleepEachRow(1) from numbers(10)' 2>/dev/null & client_pid=$! # wait until the query will appear in processlist (max 10 second) From 95caa02cbc053f672ffa83a6dbe1a96259ea4d25 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Apr 2023 23:28:08 +0200 Subject: [PATCH 0035/2047] Update test --- ...02494_zero_copy_and_projection_and_mutation_work_together.sql | 1 + .../02572_query_views_log_background_thread.reference | 1 + .../0_stateless/02572_query_views_log_background_thread.sql | 1 + 3 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql index 7a51d86dd30..b6ab9b7d0c3 100644 --- a/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql +++ b/tests/queries/0_stateless/02494_zero_copy_and_projection_and_mutation_work_together.sql @@ -70,6 +70,7 @@ SYSTEM SYNC REPLICA wikistat2; -- it doesn't make test flaky, rarely we will not delete the parts because of cleanup thread was slow. -- Such condition will lead to successful queries. +SET function_sleep_max_microseconds_per_block = 5000000; SELECT 0 FROM numbers(5) WHERE sleepEachRow(1) = 1; select sum(hits), count() from wikistat1 GROUP BY project, subproject, path settings allow_experimental_projection_optimization = 1, force_optimize_projection = 1; diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference index eeba62c5dc8..22dfaf93781 100644 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.reference +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.reference @@ -4,6 +4,7 @@ insert into buffer_02572 values (1); select * from data_02572; select * from copy_02572; -- we cannot use OPTIMIZE, this will attach query context, so let's wait +SET function_sleep_max_microseconds_per_block = 6000000; select sleepEachRow(1) from numbers(3*2) format Null; select * from data_02572; 1 diff --git a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql b/tests/queries/0_stateless/02572_query_views_log_background_thread.sql index dc229412b13..939c189c5fe 100644 --- a/tests/queries/0_stateless/02572_query_views_log_background_thread.sql +++ b/tests/queries/0_stateless/02572_query_views_log_background_thread.sql @@ -22,6 +22,7 @@ insert into buffer_02572 values (1); select * from data_02572; select * from copy_02572; -- we cannot use OPTIMIZE, this will attach query context, so let's wait +SET function_sleep_max_microseconds_per_block = 6000000; select sleepEachRow(1) from numbers(3*2) format Null; select * from data_02572; select * from copy_02572; From 748a21b791f5846a4f9f1d49d38fc077c7f9d3d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Apr 2023 01:44:03 +0200 Subject: [PATCH 0036/2047] Fix typo --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 33010dc6b3b..e0d23d139f3 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,7 +80,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"23.5", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximim sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, + {"23.5", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}}}, {"23.4", {{"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, From dd5e95e2e36fa4c781e7a12cef2264dee67c5c97 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 1 May 2023 08:23:33 +0300 Subject: [PATCH 0037/2047] add ordered mode --- src/Core/SettingsEnums.cpp | 3 + src/Core/SettingsEnums.h | 9 ++ src/Storages/S3Queue/S3QueueSettings.h | 11 +- src/Storages/S3Queue/S3QueueSource.cpp | 49 ++++-- src/Storages/S3Queue/S3QueueSource.h | 7 +- src/Storages/S3Queue/StorageS3Queue.cpp | 190 ++++++++++-------------- src/Storages/S3Queue/StorageS3Queue.h | 11 +- 7 files changed, 153 insertions(+), 127 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index fdda4ec8bc6..303a05ce2e0 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -206,4 +206,7 @@ IMPLEMENT_SETTING_ENUM(S3QueueMode, ErrorCodes::BAD_ARGUMENTS, {{"ordered", S3QueueMode::ORDERED}, {"unordered", S3QueueMode::UNORDERED}}) +IMPLEMENT_SETTING_ENUM(S3QueueAction, ErrorCodes::BAD_ARGUMENTS, + {{"keep", S3QueueAction::KEEP}, + {"delete", S3QueueAction::DELETE}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index b8912e5b9b4..8ac727170cf 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -226,4 +226,13 @@ enum class S3QueueMode }; DECLARE_SETTING_ENUM(S3QueueMode) + +enum class S3QueueAction +{ + KEEP, + DELETE, +}; + +DECLARE_SETTING_ENUM(S3QueueAction) + } diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 3860a236a9b..d556a768a15 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -15,13 +15,14 @@ class ASTStorage; mode, \ S3QueueMode::ORDERED, \ "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", \ - 0) \ - M(String, keeper_path, "", "Zookeeper node path", 0) \ + 0) \ + M(S3QueueAction, after_processing, S3QueueAction::KEEP , "Delete, keep or move file in S3 after processing", 0) \ + M(String, keeper_path, "", "Zookeeper node path", 0) \ M(String, processed_action, "keep", "Keep, delete or move file after successful loading", 0) \ M(UInt64, s3queue_max_retries, 0, "Retry loading up to specified number of times", 0) \ - M(UInt64, s3queue_polling_min_timeout, 1000, "Minimal timeout before next polling", 0) \ - M(UInt64, s3queue_polling_max_timeout, 10000, "Maximum timeout before next polling", 0) \ - M(UInt64, s3queue_polling_backoff, 0, "Retry loading up to specified number of times", 0) + M(UInt64, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ + M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ + M(UInt64, s3queue_polling_backoff_ms, 0, "Retry loading up to specified number of times", 0) #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 92056c79d8f..80478d2f326 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -150,15 +150,19 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( } } -Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, std::unordered_set & exclude_keys) +Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file) { for (KeyWithInfo val : keys_buf) { - if (exclude_keys.find(bucket + '/' + val.key) != exclude_keys.end()) + auto full_path = bucket + '/' + val.key; + if (exclude_keys.find(full_path) != exclude_keys.end()) { LOG_INFO(log, "Found in exclude keys {}", val.key); continue; } + if (engine_mode == S3QueueMode::ORDERED && full_path.compare(max_file) <= 0) { + continue; + } if (processing_keys.size() < max_poll_size) { processing_keys.push_back(val); @@ -169,7 +173,7 @@ Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, st } } - if (mode == "ordered") + if (engine_mode == S3QueueMode::ORDERED) { std::sort( processing_keys.begin(), @@ -229,6 +233,8 @@ StorageS3QueueSource::StorageS3QueueSource( const String & bucket_, const String & version_id_, std::shared_ptr file_iterator_, + const S3QueueMode & mode_, + const S3QueueAction & action_, zkutil::ZooKeeperPtr current_zookeeper, const String & zookeeper_path_, const size_t download_thread_num_) @@ -247,6 +253,8 @@ StorageS3QueueSource::StorageS3QueueSource( , format_settings(format_settings_) , requested_virtual_columns(requested_virtual_columns_) , file_iterator(file_iterator_) + , mode(mode_) + , action(action_) , download_thread_num(download_thread_num_) , zookeeper(current_zookeeper) , zookeeper_path(zookeeper_path_) @@ -418,7 +426,7 @@ Chunk StorageS3QueueSource::generate() } LOG_WARNING(log, "Set processed: {}", file_path); setFileProcessed(file_path); - // TODO: Set processed + applyActionAfterProcessing(file_path); return chunk; } } @@ -449,14 +457,18 @@ Chunk StorageS3QueueSource::generate() void StorageS3QueueSource::setFileProcessed(const String & file_path) { std::lock_guard lock(mutex); - String processed_files = zookeeper->get(zookeeper_path + "/processed"); - std::unordered_set processed = parseCollection(processed_files); + if (mode == S3QueueMode::UNORDERED) { + String processed_files = zookeeper->get(zookeeper_path + "/processed"); + std::unordered_set processed = parseCollection(processed_files); - processed.insert(file_path); - Strings set_processed; - set_processed.insert(set_processed.end(), processed.begin(), processed.end()); + processed.insert(file_path); + Strings set_processed; + set_processed.insert(set_processed.end(), processed.begin(), processed.end()); - zookeeper->set(zookeeper_path + "/processed", toString(set_processed)); + zookeeper->set(zookeeper_path + "/processed", toString(set_processed)); + } else { + zookeeper->set(zookeeper_path + "/processed", file_path); + } } @@ -473,6 +485,23 @@ void StorageS3QueueSource::setFileFailed(const String & file_path) zookeeper->set(zookeeper_path + "/failed", toString(set_failed)); } + +void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) +{ + LOG_WARNING(log, "Delete {} Bucke {}", file_path, bucket); + S3::DeleteObjectRequest request; + request.SetBucket(bucket); + request.SetKey(file_path); + auto outcome = client->DeleteObject(request); + if (!outcome.IsSuccess() && !S3::isNotFoundError(outcome.GetError().GetErrorType())) + { + const auto & err = outcome.GetError(); + LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); + } else { + LOG_TRACE(log, "Object with path {} was removed from S3", file_path); + } +} + std::unordered_set StorageS3QueueSource::parseCollection(String & files) { ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 3abb388cd2d..b5160588bab 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -52,7 +52,7 @@ public: KeyWithInfo next() override; size_t getTotalSize() const override; - Strings setProcessing(String & mode, std::unordered_set & exclude_keys); + Strings setProcessing(S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file = ""); private: size_t max_poll_size = 10; @@ -83,6 +83,8 @@ public: const String & bucket, const String & version_id, std::shared_ptr file_iterator_, + const S3QueueMode & mode_, + const S3QueueAction & action_, zkutil::ZooKeeperPtr current_zookeeper, const String & zookeeper_path_, size_t download_thread_num); @@ -114,6 +116,8 @@ private: std::vector requested_virtual_columns; std::shared_ptr file_iterator; + const S3QueueMode mode; + const S3QueueAction action; size_t download_thread_num = 1; Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSource"); @@ -140,6 +144,7 @@ private: void setFileProcessed(const String & file_path); void setFileFailed(const String & file_path); + void applyActionAfterProcessing(const String & file_path); }; } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 1cb04350635..9febd6b2c9f 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -86,33 +86,13 @@ namespace ProfileEvents { extern const Event S3DeleteObjects; extern const Event S3ListObjects; -extern const Event S3QueueBackgroundReads; } namespace DB { static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - -static const std::unordered_set required_configuration_keys = { - "url", -}; -static const std::unordered_set optional_configuration_keys = { - "format", - "compression", - "compression_method", - "structure", - "access_key_id", - "secret_access_key", - "filename", - "use_environment_credentials", - "max_single_read_retries", - "min_upload_part_size", - "upload_part_size_multiply_factor", - "upload_part_size_multiply_parts_count_threshold", - "max_single_part_upload_size", - "max_connections", -}; +static const auto MAX_THREAD_WORK_DURATION_MS = 60000; namespace ErrorCodes { @@ -131,8 +111,6 @@ namespace ErrorCodes extern const int REPLICA_ALREADY_EXISTS; } -class IOutputFormat; -using OutputFormatPtr = std::shared_ptr; StorageS3Queue::StorageS3Queue( std::unique_ptr s3queue_settings_, @@ -150,6 +128,9 @@ StorageS3Queue::StorageS3Queue( , s3queue_settings(std::move(s3queue_settings_)) , s3_configuration{configuration_} , keys({s3_configuration.url.key}) + , mode(s3queue_settings->mode) + , after_processing(s3queue_settings->after_processing) + , milliseconds_to_wait(s3queue_settings->s3queue_polling_min_timeout_ms) , format_name(configuration_.format) , compression_method(configuration_.compression_method) , name(s3_configuration.url.storage_name) @@ -159,7 +140,15 @@ StorageS3Queue::StorageS3Queue( , is_key_with_globs(s3_configuration.url.key.find_first_of("*?{") != std::string::npos) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { + LOG_INFO(log, "Init engine"); + + if (!is_key_with_globs) + { + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); + } + String setting_zookeeper_path = s3queue_settings->keeper_path; + LOG_INFO(log, "Setttings zookeeper_path={}", setting_zookeeper_path); if (setting_zookeeper_path == "") { auto table_id = getStorageID(); @@ -181,12 +170,8 @@ StorageS3Queue::StorageS3Queue( { zookeeper_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path, /* check_starts_with_slash */ true, log); } - LOG_INFO(log, "Storage S3Queue zookeeper_path= {} with mode", zookeeper_path); + LOG_INFO(log, "Set zookeeper_path={}", zookeeper_path); - if (!is_key_with_globs) - { - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from key with globs"); - } FormatFactory::instance().checkFormatName(format_name); context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); StorageInMemoryMetadata storage_metadata; @@ -213,12 +198,12 @@ StorageS3Queue::StorageS3Queue( for (const auto & column : virtual_columns) virtual_block.insert({column.type->createColumn(), column.type, column.name}); - auto thread = context_->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); setZooKeeper(); auto metadata_snapshot = getInMemoryMetadataPtr(); createTableIfNotExists(metadata_snapshot); - task = std::make_shared(std::move(thread)); - LOG_TRACE(log, "Complete"); + + auto poll_thread = context_->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); + task = std::make_shared(std::move(poll_thread)); } @@ -241,21 +226,15 @@ Pipe StorageS3Queue::read( size_t max_block_size, size_t num_streams) { - // if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) - // throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, - // "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, + "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); if (mv_attached) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageRabbitMQ with attached materialized views"); auto query_s3_configuration = updateConfigurationAndGetCopy(local_context); - bool has_wildcards = query_s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos - || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; - - if (partition_by && has_wildcards) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); - Pipes pipes; std::unordered_set column_names_set(column_names.begin(), column_names.end()); @@ -295,9 +274,6 @@ Pipe StorageS3Queue::read( } const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; - // const size_t max_download_threads = 1; - - auto zookeeper = getZooKeeper(); for (size_t i = 0; i < num_streams; ++i) { @@ -316,6 +292,8 @@ Pipe StorageS3Queue::read( query_s3_configuration.url.bucket, query_s3_configuration.url.version_id, iterator_wrapper, + mode, + after_processing, zookeeper, zookeeper_path, max_download_threads)); @@ -332,11 +310,6 @@ NamesAndTypesList StorageS3Queue::getVirtuals() const return virtual_columns; } -Names StorageS3Queue::getVirtualColumnNames() -{ - return {"_path", "_file"}; -} - bool StorageS3Queue::supportsPartitionBy() const { return true; @@ -351,21 +324,10 @@ void StorageS3Queue::startup() void StorageS3Queue::shutdown() { shutdown_called = true; - LOG_TRACE(log, "Deactivating background tasks"); - if (task) { task->stream_cancelled = true; - - /// Reader thread may wait for wake up - // wakeUp(); - - LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); - /// If no reading call and threadFunc, the log files will never - /// be opened, also just leave the work of close files and - /// store meta to streams. because if we close files in here, - /// may result in data race with unfinishing reading pipeline } } @@ -409,7 +371,6 @@ void StorageS3Queue::threadFunc() auto table_id = getStorageID(); auto dependencies_count = getTableDependentCount(); - LOG_TRACE(log, "dependencies_count {}", toString(dependencies_count)); if (dependencies_count) { auto start_time = std::chrono::steady_clock::now(); @@ -426,21 +387,18 @@ void StorageS3Queue::threadFunc() } LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - - if (streamToViews()) - { - LOG_TRACE(log, "Stream stalled. Reschedule."); - break; - } + streamToViews(); auto ts = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(ts - start_time); - if (duration.count() > 600000) + if (duration.count() > MAX_THREAD_WORK_DURATION_MS) { LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); reschedule = true; break; } + + milliseconds_to_wait = s3queue_settings->s3queue_polling_min_timeout_ms; } } } @@ -451,30 +409,24 @@ void StorageS3Queue::threadFunc() mv_attached.store(false); - // Wait for attached views if (reschedule && !shutdown_called) { LOG_TRACE(log, "Reschedule S3 Queue thread func."); /// Reschedule with backoff. + if (milliseconds_to_wait < s3queue_settings->s3queue_polling_max_timeout_ms) + milliseconds_to_wait += s3queue_settings->s3queue_polling_backoff_ms; task->holder->scheduleAfter(milliseconds_to_wait); } } -bool StorageS3Queue::streamToViews() +void StorageS3Queue::streamToViews() { - LOG_TRACE(log, "streamToViews"); - - Stopwatch watch; - auto table_id = getStorageID(); auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); if (!table) throw Exception(ErrorCodes::LOGICAL_ERROR, "Engine table {} doesn't exist.", table_id.getNameForLogs()); - // CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaBackgroundReads}; - // ProfileEvents::increment(ProfileEvents::S3QueueBackgroundReads); - auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); // Create an INSERT query for streaming data @@ -487,8 +439,6 @@ bool StorageS3Queue::streamToViews() s3queue_context->makeQueryContext(); auto query_s3_configuration = updateConfigurationAndGetCopy(s3queue_context); - // s3queue_context->applySettingsChanges(settings_adjustments); - // Create a stream for each consumer and join them in a union stream // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); @@ -552,6 +502,8 @@ bool StorageS3Queue::streamToViews() query_s3_configuration.url.bucket, query_s3_configuration.url.version_id, iterator_wrapper, + mode, + after_processing, zookeeper, zookeeper_path, max_download_threads)); @@ -559,24 +511,15 @@ bool StorageS3Queue::streamToViews() auto pipe = Pipe::unitePipes(std::move(pipes)); - // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. - // It will be cancelled on underlying layer (kafka buffer) - std::atomic_size_t rows = 0; { block_io.pipeline.complete(std::move(pipe)); - - // we need to read all consumers in parallel (sequential read may lead to situation - // when some of consumers are not used, and will break some Kafka consumer invariants) block_io.pipeline.setNumThreads(num_streams); block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); } - - bool some_stream_is_stalled = false; - return some_stream_is_stalled; } StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) @@ -620,23 +563,18 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ { ops.emplace_back(zkutil::makeCreateRequest( fs::path(zookeeper_path) / "processing" / toString(table_uuid), "{}", zkutil::CreateMode::Ephemeral)); - LOG_DEBUG(log, "This table {} is already created, will add new replica", zookeeper_path); } else { - /// We write metadata of table so that the replicas can check table parameters with them. - // String metadata_str = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); + String deafult_processed = mode == S3QueueMode::ORDERED ? "" : "[]"; ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "[]", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", deafult_processed, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( fs::path(zookeeper_path) / "processing" / toString(table_uuid), "[]", zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); - // ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, - // zkutil::CreateMode::Persistent)); } Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); @@ -653,7 +591,6 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ return true; } - /// Do not use LOGICAL_ERROR code, because it may happen if user has specified wrong zookeeper_path throw Exception( ErrorCodes::REPLICA_ALREADY_EXISTS, "Cannot create table, because it is created concurrently every time or because " @@ -667,45 +604,82 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW /// Iterate through disclosed globs and make a source for each file auto it = std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_block, local_context, read_keys, s3_configuration.request_settings); - String cur_mode = "unordered"; std::lock_guard lock{sync_mutex}; std::unordered_set exclude = getExcludedFiles(); + Strings processing; + if (mode == S3QueueMode::UNORDERED) { + processing = it->setProcessing(mode, exclude); + } else { + String max_processed_file = getMaxProcessedFile(); + processing = it->setProcessing(mode, exclude, max_processed_file); + } + auto zookeeper = getZooKeeper(); auto table_uuid = getStorageID().uuid; - Strings processing = it->setProcessing(cur_mode, exclude); zookeeper->set(fs::path(zookeeper_path) / "processing" / toString(table_uuid), toString(processing)); return it; } -std::unordered_set StorageS3Queue::getExcludedFiles() -{ +std::unordered_set StorageS3Queue::getFailedFiles() { auto zookeeper = getZooKeeper(); - std::unordered_set exclude_files; String failed = zookeeper->get(zookeeper_path + "/failed"); std::unordered_set failed_files = StorageS3QueueSource::parseCollection(failed); - LOG_DEBUG(log, "failed_files {}", failed_files.size()); + return failed_files; +} + +std::unordered_set StorageS3Queue::getProcessedFiles() { + auto zookeeper = getZooKeeper(); + String processed = zookeeper->get(zookeeper_path + "/processed"); std::unordered_set processed_files = StorageS3QueueSource::parseCollection(processed); - LOG_DEBUG(log, "processed_files {}", processed_files.size()); - exclude_files.merge(failed_files); - exclude_files.merge(processed_files); + return processed_files; +} + +String StorageS3Queue::getMaxProcessedFile() { + auto zookeeper = getZooKeeper(); + + String processed = zookeeper->get(zookeeper_path + "/processed"); + return processed; +} + +std::unordered_set StorageS3Queue::getProcessingFiles() { + auto zookeeper = getZooKeeper(); Strings consumer_table_uuids; zookeeper->tryGetChildren(zookeeper_path + "/processing", consumer_table_uuids); - + std::unordered_set processing_files; for (const auto & uuid : consumer_table_uuids) { String processing = zookeeper->get(fs::path(zookeeper_path) / "processing" / toString(uuid)); - std::unordered_set processing_files = StorageS3QueueSource::parseCollection(processing); - LOG_DEBUG(log, "processing {}", processing_files.size()); - exclude_files.merge(processing_files); + std::unordered_set cur_processing_files = StorageS3QueueSource::parseCollection(processing); + processing_files.merge(cur_processing_files); } + return processing_files; +} + +std::unordered_set StorageS3Queue::getExcludedFiles() +{ + std::unordered_set exclude_files; + + std::unordered_set failed_files = getFailedFiles(); + LOG_DEBUG(log, "failed_files {}", failed_files.size()); + exclude_files.merge(failed_files); + + if (mode != S3QueueMode::ORDERED) { + std::unordered_set processed_files = getProcessedFiles(); + LOG_DEBUG(log, "processed_files {}", processed_files.size()); + exclude_files.merge(processed_files); + } + + std::unordered_set processing_files = getProcessingFiles(); + LOG_DEBUG(log, "processing {}", processing_files.size()); + exclude_files.merge(processing_files); return exclude_files; } diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 1763e882d59..257cb23f303 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -89,6 +89,8 @@ private: std::vector keys; NamesAndTypesList virtual_columns; Block virtual_block; + S3QueueMode mode; + S3QueueAction after_processing; uint64_t milliseconds_to_wait = 10000; String format_name; @@ -121,8 +123,6 @@ private: std::shared_ptr task; bool supportsSubsetOfColumns() const override; - static Names getVirtualColumnNames(); - String zookeeper_path; zkutil::ZooKeeperPtr current_zookeeper; @@ -142,9 +142,14 @@ private: createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys = nullptr); static std::unordered_set parseCollection(String & files); + + std::unordered_set getFailedFiles(); + std::unordered_set getProcessedFiles(); + String getMaxProcessedFile(); + std::unordered_set getProcessingFiles(); std::unordered_set getExcludedFiles(); - bool streamToViews(); + void streamToViews(); Configuration updateConfigurationAndGetCopy(ContextPtr local_context); }; From 624d846feae7bcc73b0d5683d52d02c187fb7ea3 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 1 May 2023 15:50:20 +0300 Subject: [PATCH 0038/2047] add template for tests --- src/Storages/S3Queue/S3QueueSource.cpp | 2 +- .../test_storage_s3_queue/__init__.py | 0 .../configs/defaultS3.xml | 11 ++ .../configs/named_collections.xml | 43 ++++++ .../test_storage_s3_queue/configs/users.xml | 7 + .../integration/test_storage_s3_queue/test.py | 123 ++++++++++++++++++ 6 files changed, 185 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_storage_s3_queue/__init__.py create mode 100644 tests/integration/test_storage_s3_queue/configs/defaultS3.xml create mode 100644 tests/integration/test_storage_s3_queue/configs/named_collections.xml create mode 100644 tests/integration/test_storage_s3_queue/configs/users.xml create mode 100644 tests/integration/test_storage_s3_queue/test.py diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 80478d2f326..e117ec52e0e 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -259,7 +259,7 @@ StorageS3QueueSource::StorageS3QueueSource( , zookeeper(current_zookeeper) , zookeeper_path(zookeeper_path_) , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) - , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3QueueReader")) + , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3QReader")) { reader = createReader(); if (reader) diff --git a/tests/integration/test_storage_s3_queue/__init__.py b/tests/integration/test_storage_s3_queue/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_s3_queue/configs/defaultS3.xml b/tests/integration/test_storage_s3_queue/configs/defaultS3.xml new file mode 100644 index 00000000000..7dac6d9fbb5 --- /dev/null +++ b/tests/integration/test_storage_s3_queue/configs/defaultS3.xml @@ -0,0 +1,11 @@ + + + + http://resolver:8080 +
Authorization: Bearer TOKEN
+
+ + http://resolver:8080/root-with-auth/restricteddirectory/ + +
+
diff --git a/tests/integration/test_storage_s3_queue/configs/named_collections.xml b/tests/integration/test_storage_s3_queue/configs/named_collections.xml new file mode 100644 index 00000000000..64674e2a3e3 --- /dev/null +++ b/tests/integration/test_storage_s3_queue/configs/named_collections.xml @@ -0,0 +1,43 @@ + + + + http://minio1:9001/root/test_table + minio + minio123 + + + http://minio1:9001/root/test_parquet + minio + minio123 + + + http://minio1:9001/root/test_parquet_gz + minio + minio123 + + + http://minio1:9001/root/test_orc + minio + minio123 + + + http://minio1:9001/root/test_native + minio + minio123 + + + http://minio1:9001/root/test.arrow + minio + minio123 + + + http://minio1:9001/root/test.parquet + minio + minio123 + + + http://minio1:9001/root/test_cache4.jsonl + true + + + diff --git a/tests/integration/test_storage_s3_queue/configs/users.xml b/tests/integration/test_storage_s3_queue/configs/users.xml new file mode 100644 index 00000000000..2cef0a6de3c --- /dev/null +++ b/tests/integration/test_storage_s3_queue/configs/users.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py new file mode 100644 index 00000000000..92f7955c9a4 --- /dev/null +++ b/tests/integration/test_storage_s3_queue/test.py @@ -0,0 +1,123 @@ +import gzip +import json +import logging +import os +import io +import random +import threading +import time + +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.network import PartitionManager +from helpers.mock_servers import start_mock_servers +from helpers.test_tools import exec_query_with_retry +from helpers.s3_tools import prepare_s3_bucket + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "instance", + user_configs=["configs/users.xml"], + with_minio=True, + with_zookeeper=True, +) + + +MINIO_INTERNAL_PORT = 9001 + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +def put_s3_file_content(started_cluster, bucket, filename, data): + buf = io.BytesIO(data) + started_cluster.minio_client.put_object(bucket, filename, buf, len(data)) + + +# Returns content of given S3 file as string. +def get_s3_file_content(started_cluster, bucket, filename, decode=True): + # type: (ClickHouseCluster, str, str, bool) -> str + + data = started_cluster.minio_client.get_object(bucket, filename) + data_str = b"" + for chunk in data.stream(): + data_str += chunk + if decode: + return data_str.decode() + return data_str + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "instance", + user_configs=["configs/users.xml"], + with_minio=True, + with_zookeeper=True, + main_configs=[ + "configs/defaultS3.xml", + "configs/named_collections.xml" + ], + ) + + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + prepare_s3_bucket(cluster) + # logging.info("S3 bucket created") + # run_s3_mocks(cluster) + + yield cluster + finally: + cluster.shutdown() + + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + + +def test_get_file(started_cluster): + auth = "'minio','minio123'," + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + values = [ + [12549, 2463, 19893], + [64021, 38652, 66703], + [81611, 39650, 83516], + [11079, 59507, 61546], + [51764, 69952, 6876], + [41165, 90293, 29095], + [40167, 78432, 48309], + [81629, 81327, 11855], + [55852, 21643, 98507], + [6738, 54643, 41155], + ] + values_csv = ( + "\n".join((",".join(map(str, row)) for row in values)) + "\n" + ).encode() + filename = f"test.csv" + put_s3_file_content(started_cluster, bucket, filename, values_csv) + + instance.query( + f"create table test ({table_format}) engine=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') SETTINGS mode = 'unordered', keeper_path = '/clickhouse/testing'" + ) + + get_query = f"SELECT * FROM test" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == values + + get_query = f"SELECT * FROM test" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] \ No newline at end of file From f21c838f111aa4715727f5a275a5b8ba2e36e2ca Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Tue, 2 May 2023 17:27:53 +0300 Subject: [PATCH 0039/2047] add some tests --- .../integration/test_storage_s3_queue/test.py | 303 +++++++++++++++--- 1 file changed, 265 insertions(+), 38 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 92f7955c9a4..8cdf7e8dda0 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -1,30 +1,26 @@ -import gzip -import json +import io import logging import os -import io import random -import threading import time -import helpers.client import pytest from helpers.cluster import ClickHouseCluster, ClickHouseInstance -from helpers.network import PartitionManager -from helpers.mock_servers import start_mock_servers -from helpers.test_tools import exec_query_with_retry from helpers.s3_tools import prepare_s3_bucket -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "instance", - user_configs=["configs/users.xml"], - with_minio=True, - with_zookeeper=True, -) + +@pytest.fixture(autouse=True) +def s3_queue_setup_teardown(started_cluster): + instance = started_cluster.instances["instance"] + instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") + # logging.debug("kafka is available - running test") + yield # run test MINIO_INTERNAL_PORT = 9001 +AVAILABLE_MODES = ["ordered", "unordered"] +AUTH = "'minio','minio123'," + SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -34,6 +30,24 @@ def put_s3_file_content(started_cluster, bucket, filename, data): started_cluster.minio_client.put_object(bucket, filename, buf, len(data)) +def generate_random_files( + count, prefix, cluster, bucket, column_num=3, row_num=10, start_ind=0 +): + total_values = [] + for i in range(start_ind, start_ind + count): + print(i) + rand_values = [ + [random.randint(0, 50) for _ in range(column_num)] for _ in range(row_num) + ] + total_values += rand_values + values_csv = ( + "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" + ).encode() + filename = f"{prefix}/test_{i}.csv" + put_s3_file_content(cluster, bucket, filename, values_csv) + return total_values + + # Returns content of given S3 file as string. def get_s3_file_content(started_cluster, bucket, filename, decode=True): # type: (ClickHouseCluster, str, str, bool) -> str @@ -56,10 +70,7 @@ def started_cluster(): user_configs=["configs/users.xml"], with_minio=True, with_zookeeper=True, - main_configs=[ - "configs/defaultS3.xml", - "configs/named_collections.xml" - ], + main_configs=["configs/defaultS3.xml", "configs/named_collections.xml"], ) logging.info("Starting cluster...") @@ -67,9 +78,6 @@ def started_cluster(): logging.info("Cluster started") prepare_s3_bucket(cluster) - # logging.info("S3 bucket created") - # run_s3_mocks(cluster) - yield cluster finally: cluster.shutdown() @@ -85,7 +93,8 @@ def run_query(instance, query, stdin=None, settings=None): return result -def test_get_file(started_cluster): +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_direct_select_file(started_cluster, mode): auth = "'minio','minio123'," bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] @@ -94,30 +103,248 @@ def test_get_file(started_cluster): [12549, 2463, 19893], [64021, 38652, 66703], [81611, 39650, 83516], - [11079, 59507, 61546], - [51764, 69952, 6876], - [41165, 90293, 29095], - [40167, 78432, 48309], - [81629, 81327, 11855], - [55852, 21643, 98507], - [6738, 54643, 41155], ] values_csv = ( - "\n".join((",".join(map(str, row)) for row in values)) + "\n" + "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() filename = f"test.csv" put_s3_file_content(started_cluster, bucket, filename, values_csv) + instance.query("drop table if exists test.s3_queue") + instance.query("drop table if exists test.s3_queue_2") + instance.query("drop table if exists test.s3_queue_3") instance.query( - f"create table test ({table_format}) engine=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') SETTINGS mode = 'unordered', keeper_path = '/clickhouse/testing'" + f""" + CREATE TABLE test.s3_queue ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_{mode}' + """ ) - get_query = f"SELECT * FROM test" + get_query = f"SELECT * FROM test.s3_queue" assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == values + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == values - get_query = f"SELECT * FROM test" + instance.query( + f""" + CREATE TABLE test.s3_queue_2 ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_{mode}' + """ + ) + + get_query = f"SELECT * FROM test.s3_queue" assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == [] \ No newline at end of file + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] + # New table with same zookeeper path + get_query = f"SELECT * FROM test.s3_queue_2" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] + # New table with different zookeeper path + instance.query( + f""" + CREATE TABLE test.s3_queue_3 ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') + SETTINGS + mode = '{mode}', + keeper_path='/clickhouse/select_{mode}_2' + """ + ) + get_query = f"SELECT * FROM test.s3_queue_3" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == values + + values = [ + [1, 1, 1], + ] + values_csv = ( + "\n".join((",".join(map(str, row)) for row in values)) + "\n" + ).encode() + filename = f"t.csv" + put_s3_file_content(started_cluster, bucket, filename, values_csv) + + get_query = f"SELECT * FROM test.s3_queue_3" + if mode == "unordered": + assert [ + list(map(int, l.split())) + for l in run_query(instance, get_query).splitlines() + ] == values + elif mode == "ordered": + assert [ + list(map(int, l.split())) + for l in run_query(instance, get_query).splitlines() + ] == [] + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_direct_select_multiple_files(started_cluster, mode): + prefix = f"multiple_files_{mode}" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + instance.query("drop table if exists test.s3_queue") + instance.query( + f""" + CREATE TABLE test.s3_queue ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_multiple_{mode}' + """ + ) + + for i in range(10): + print(i) + rand_values = [[random.randint(0, 50) for _ in range(3)] for _ in range(10)] + + values_csv = ( + "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" + ).encode() + filename = f"{prefix}/test_{i}.csv" + put_s3_file_content(started_cluster, bucket, filename, values_csv) + + get_query = f"SELECT * FROM test.s3_queue" + assert [ + list(map(int, l.split())) + for l in run_query(instance, get_query).splitlines() + ] == rand_values + + total_values = generate_random_files( + 5, prefix, started_cluster, bucket, start_ind=10 + ) + get_query = f"SELECT * FROM test.s3_queue" + assert { + tuple(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + } == set([tuple(i) for i in total_values]) + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_streaming_to_view_(started_cluster, mode): + prefix = f"streaming_files_{mode}" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + total_values = generate_random_files(10, prefix, started_cluster, bucket) + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue_persistent; + DROP TABLE IF EXISTS test.s3_queue; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv; + + CREATE TABLE test.s3_queue_persistent ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_multiple_{mode}'; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS + SELECT + * + FROM test.s3_queue; + """ + ) + expected_values = set([tuple(i) for i in total_values]) + for i in range(10): + get_query = f"SELECT * FROM test.persistent_s3_queue_mv" + + selected_values = { + tuple(map(int, l.split())) + for l in run_query(instance, get_query).splitlines() + } + if selected_values != expected_values: + time.sleep(1) + else: + break + + assert selected_values == expected_values + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_streaming_to_many_views(started_cluster, mode): + prefix = f"streaming_files_{mode}" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + retry_cnt = 10 + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue_persistent; + DROP TABLE IF EXISTS test.s3_queue_persistent_2; + DROP TABLE IF EXISTS test.s3_queue_persistent_3; + DROP TABLE IF EXISTS test.s3_queue; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv_2; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv_3; + + + CREATE TABLE test.s3_queue_persistent ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue_persistent_2 ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue_persistent_3 ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_multiple_{mode}'; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS + SELECT + * + FROM test.s3_queue; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_2 TO test.s3_queue_persistent_2 AS + SELECT + * + FROM test.s3_queue; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_3 TO test.s3_queue_persistent_3 AS + SELECT + * + FROM test.s3_queue; + """ + ) + total_values = generate_random_files(10, prefix, started_cluster, bucket) + expected_values = set([tuple(i) for i in total_values]) + + for i in range(retry_cnt): + retry = False + for get_query in [ + f"SELECT * FROM test.s3_queue_persistent", + f"SELECT * FROM test.s3_queue_persistent_2", + f"SELECT * FROM test.s3_queue_persistent_3", + ]: + selected_values = { + tuple(map(int, l.split())) + for l in run_query(instance, get_query).splitlines() + } + if i == retry_cnt - 1: + assert selected_values == expected_values + if selected_values != expected_values: + retry = True + break + if retry: + time.sleep(1) + else: + break From 559e8fa6e5e1ed2f58d2acb738a8dc26f1bdee0a Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Tue, 2 May 2023 18:10:57 +0300 Subject: [PATCH 0040/2047] fix codestyle --- src/Storages/S3Queue/S3QueueSource.cpp | 28 +++++--------- src/Storages/S3Queue/StorageS3Queue.cpp | 50 +++++++++++++++++-------- src/Storages/S3Queue/StorageS3Queue.h | 10 +---- 3 files changed, 46 insertions(+), 42 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index e117ec52e0e..e77680c0d36 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -109,20 +109,6 @@ static const std::unordered_set optional_configuration_keys "expiration_window_seconds", "no_sign_request"}; -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_TEXT; - extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int S3_ERROR; - extern const int UNEXPECTED_EXPRESSION; - extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - extern const int NOT_IMPLEMENTED; - extern const int CANNOT_COMPILE_REGEXP; - extern const int FILE_DOESNT_EXIST; -} - class IOutputFormat; using OutputFormatPtr = std::shared_ptr; @@ -160,7 +146,8 @@ Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(S3QueueMode & eng LOG_INFO(log, "Found in exclude keys {}", val.key); continue; } - if (engine_mode == S3QueueMode::ORDERED && full_path.compare(max_file) <= 0) { + if (engine_mode == S3QueueMode::ORDERED && full_path.compare(max_file) <= 0) + { continue; } if (processing_keys.size() < max_poll_size) @@ -457,7 +444,8 @@ Chunk StorageS3QueueSource::generate() void StorageS3QueueSource::setFileProcessed(const String & file_path) { std::lock_guard lock(mutex); - if (mode == S3QueueMode::UNORDERED) { + if (mode == S3QueueMode::UNORDERED) + { String processed_files = zookeeper->get(zookeeper_path + "/processed"); std::unordered_set processed = parseCollection(processed_files); @@ -466,7 +454,9 @@ void StorageS3QueueSource::setFileProcessed(const String & file_path) set_processed.insert(set_processed.end(), processed.begin(), processed.end()); zookeeper->set(zookeeper_path + "/processed", toString(set_processed)); - } else { + } + else + { zookeeper->set(zookeeper_path + "/processed", file_path); } } @@ -497,7 +487,9 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) { const auto & err = outcome.GetError(); LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); - } else { + } + else + { LOG_TRACE(log, "Object with path {} was removed from S3", file_path); } } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 9febd6b2c9f..ac9300ac4f4 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -96,16 +96,10 @@ static const auto MAX_THREAD_WORK_DURATION_MS = 60000; namespace ErrorCodes { - extern const int CANNOT_PARSE_TEXT; + extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int S3_ERROR; - extern const int UNEXPECTED_EXPRESSION; - extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; extern const int NOT_IMPLEMENTED; - extern const int CANNOT_COMPILE_REGEXP; - extern const int FILE_DOESNT_EXIST; extern const int QUERY_NOT_ALLOWED; extern const int NO_ZOOKEEPER; extern const int REPLICA_ALREADY_EXISTS; @@ -256,7 +250,8 @@ Pipe StorageS3Queue::read( const auto & virtuals = getVirtuals(); std::erase_if( fetch_columns, - [&](const String & col) { + [&](const String & col) + { return std::any_of( virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); }); @@ -305,6 +300,20 @@ Pipe StorageS3Queue::read( return pipe; } +SinkToStoragePtr StorageS3Queue::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); +} + +void StorageS3Queue::truncate( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + ContextPtr /*local_context*/, + TableExclusiveLockHolder &) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); +} + NamesAndTypesList StorageS3Queue::getVirtuals() const { return virtual_columns; @@ -462,7 +471,8 @@ void StorageS3Queue::streamToViews() const auto & virtuals = getVirtuals(); std::erase_if( fetch_columns, - [&](const String & col) { + [&](const String & col) + { return std::any_of( virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); }); @@ -609,9 +619,12 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW std::unordered_set exclude = getExcludedFiles(); Strings processing; - if (mode == S3QueueMode::UNORDERED) { + if (mode == S3QueueMode::UNORDERED) + { processing = it->setProcessing(mode, exclude); - } else { + } + else + { String max_processed_file = getMaxProcessedFile(); processing = it->setProcessing(mode, exclude, max_processed_file); } @@ -623,7 +636,8 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW return it; } -std::unordered_set StorageS3Queue::getFailedFiles() { +std::unordered_set StorageS3Queue::getFailedFiles() +{ auto zookeeper = getZooKeeper(); String failed = zookeeper->get(zookeeper_path + "/failed"); @@ -632,7 +646,8 @@ std::unordered_set StorageS3Queue::getFailedFiles() { return failed_files; } -std::unordered_set StorageS3Queue::getProcessedFiles() { +std::unordered_set StorageS3Queue::getProcessedFiles() +{ auto zookeeper = getZooKeeper(); String processed = zookeeper->get(zookeeper_path + "/processed"); @@ -641,14 +656,16 @@ std::unordered_set StorageS3Queue::getProcessedFiles() { return processed_files; } -String StorageS3Queue::getMaxProcessedFile() { +String StorageS3Queue::getMaxProcessedFile() +{ auto zookeeper = getZooKeeper(); String processed = zookeeper->get(zookeeper_path + "/processed"); return processed; } -std::unordered_set StorageS3Queue::getProcessingFiles() { +std::unordered_set StorageS3Queue::getProcessingFiles() +{ auto zookeeper = getZooKeeper(); Strings consumer_table_uuids; @@ -671,7 +688,8 @@ std::unordered_set StorageS3Queue::getExcludedFiles() LOG_DEBUG(log, "failed_files {}", failed_files.size()); exclude_files.merge(failed_files); - if (mode != S3QueueMode::ORDERED) { + if (mode != S3QueueMode::ORDERED) + { std::unordered_set processed_files = getProcessedFiles(); LOG_DEBUG(log, "processed_files {}", processed_files.size()); exclude_files.merge(processed_files); diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 257cb23f303..4ca6ba73271 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -63,19 +63,13 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); - } + SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override; void truncate( const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*local_context*/, - TableExclusiveLockHolder &) override - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); - } + TableExclusiveLockHolder &) override; NamesAndTypesList getVirtuals() const override; From 7073d377eb5d5e564f8b3a24fdc2814fd56766d9 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Tue, 2 May 2023 18:18:49 +0300 Subject: [PATCH 0041/2047] fix typo --- src/Storages/S3Queue/StorageS3Queue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index ac9300ac4f4..6d2bec7a0d3 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -142,7 +142,7 @@ StorageS3Queue::StorageS3Queue( } String setting_zookeeper_path = s3queue_settings->keeper_path; - LOG_INFO(log, "Setttings zookeeper_path={}", setting_zookeeper_path); + LOG_INFO(log, "Settings zookeeper_path={}", setting_zookeeper_path); if (setting_zookeeper_path == "") { auto table_id = getStorageID(); From 7498ed23b1d88a8cf539690813877a689158dd4e Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 4 May 2023 10:04:04 +0300 Subject: [PATCH 0042/2047] check meta, reformat code, add tests --- src/Storages/S3Queue/S3QueueHolder.cpp | 186 +++++++++ src/Storages/S3Queue/S3QueueHolder.h | 49 +++ src/Storages/S3Queue/S3QueueSettings.h | 11 +- src/Storages/S3Queue/S3QueueSource.cpp | 139 ++----- src/Storages/S3Queue/S3QueueSource.h | 38 +- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 115 ++++++ src/Storages/S3Queue/S3QueueTableMetadata.h | 45 +++ src/Storages/S3Queue/StorageS3Queue.cpp | 365 ++++++++---------- src/Storages/S3Queue/StorageS3Queue.h | 12 +- .../integration/test_storage_s3_queue/test.py | 113 +++++- 10 files changed, 725 insertions(+), 348 deletions(-) create mode 100644 src/Storages/S3Queue/S3QueueHolder.cpp create mode 100644 src/Storages/S3Queue/S3QueueHolder.h create mode 100644 src/Storages/S3Queue/S3QueueTableMetadata.cpp create mode 100644 src/Storages/S3Queue/S3QueueTableMetadata.h diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp new file mode 100644 index 00000000000..119cbe8e6e4 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -0,0 +1,186 @@ +#include "config.h" + +#if USE_AWS_S3 +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int NO_ZOOKEEPER; +} + +S3QueueHolder::S3QueueHolder(const String & zookeeper_path_, const S3QueueMode & mode_, const UUID & table_uuid_, ContextPtr context_) + : WithContext(context_) + , zookeeper_path(zookeeper_path_) + , zookeeper_failed_path(zookeeper_path_ + "/failed") + , zookeeper_processing_path(zookeeper_path_ + "/processing") + , zookeeper_processed_path(zookeeper_path_ + "/processed") + , mode(mode_) + , table_uuid(table_uuid_) + , log(&Poco::Logger::get("S3QueueHolder")) +{ + current_zookeeper = getContext()->getZooKeeper(); +} + + +zkutil::ZooKeeperPtr S3QueueHolder::tryGetZooKeeper() const +{ + std::lock_guard lock(current_zookeeper_mutex); + return current_zookeeper; +} + +zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const +{ + auto res = tryGetZooKeeper(); + if (!res) + throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); + return res; +} + + +void S3QueueHolder::setFileProcessed(const String & file_path) +{ + auto zookeeper = getZooKeeper(); + std::lock_guard lock(mutex); + + switch (mode) + { + case S3QueueMode::UNORDERED: { + String processed_files = zookeeper->get(zookeeper_processed_path); + S3FilesCollection processed = parseCollection(processed_files); + processed.insert(file_path); + Strings set_processed; + set_processed.insert(set_processed.end(), processed.begin(), processed.end()); + zookeeper->set(zookeeper_processed_path, toString(set_processed)); + break; + } + case S3QueueMode::ORDERED: { + zookeeper->set(zookeeper_processed_path, file_path); + } + } +} + + +void S3QueueHolder::setFileFailed(const String & file_path) +{ + auto zookeeper = getZooKeeper(); + + std::lock_guard lock(mutex); + String failed_files = zookeeper->get(zookeeper_failed_path); + S3FilesCollection failed = parseCollection(failed_files); + + failed.insert(file_path); + Strings set_failed; + set_failed.insert(set_failed.end(), failed.begin(), failed.end()); + + zookeeper->set(zookeeper_failed_path, toString(set_failed)); +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) +{ + ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); + Strings deserialized; + try + { + readQuoted(deserialized, rb); + } + catch (...) + { + deserialized = {}; + } + + std::unordered_set processed(deserialized.begin(), deserialized.end()); + + return processed; +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::getExcludedFiles() +{ + std::unordered_set exclude_files; + + std::unordered_set failed_files = getFailedFiles(); + LOG_DEBUG(log, "failed_files {}", failed_files.size()); + exclude_files.merge(failed_files); + + if (mode != S3QueueMode::ORDERED) + { + std::unordered_set processed_files = getProcessedFiles(); + LOG_DEBUG(log, "processed_files {}", processed_files.size()); + exclude_files.merge(processed_files); + } + + std::unordered_set processing_files = getProcessingFiles(); + LOG_DEBUG(log, "processing {}", processing_files.size()); + exclude_files.merge(processing_files); + + return exclude_files; +} + +String S3QueueHolder::getMaxProcessedFile() +{ + if (mode != S3QueueMode::ORDERED) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getMaxProcessedFile not implemented for unordered mode"); + + auto zookeeper = getZooKeeper(); + std::lock_guard lock(mutex); + String processed = zookeeper->get(zookeeper_path + "/processed"); + return processed; +} + +void S3QueueHolder::setFilesProcessing(Strings & file_paths) +{ + auto zookeeper = getZooKeeper(); + + std::lock_guard lock(mutex); + String node_data; + if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) + { + S3FilesCollection processing_files = parseCollection(node_data); + for (auto x : processing_files) + { + if (!std::count(file_paths.begin(), file_paths.end(), x)) + { + file_paths.push_back(x); + } + } + } + zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() +{ + auto zookeeper = getZooKeeper(); + std::lock_guard lock(mutex); + + String failed = zookeeper->get(zookeeper_failed_path); + return parseCollection(failed); +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessedFiles() +{ + auto zookeeper = getZooKeeper(); + + std::lock_guard lock(mutex); + String processed = zookeeper->get(zookeeper_processed_path); + return parseCollection(processed); +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() +{ + auto zookeeper = getZooKeeper(); + + std::lock_guard lock(mutex); + String processing = zookeeper->get(fs::path(zookeeper_processing_path)); + return parseCollection(processing); +} + +} + +#endif diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h new file mode 100644 index 00000000000..686b5fc1ddf --- /dev/null +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -0,0 +1,49 @@ +#pragma once + +#if USE_AWS_S3 + +#include +#include +#include +#include + +namespace DB +{ +class S3QueueHolder : public WithContext +{ +public: + using S3FilesCollection = std::unordered_set; + S3QueueHolder(const String & zookeeper_path_, const S3QueueMode & mode_, const UUID & table_uuid_, ContextPtr context_); + + void setFileProcessed(const String & file_path); + void setFileFailed(const String & file_path); + void setFilesProcessing(Strings & file_paths); + static S3FilesCollection parseCollection(String & files); + + S3FilesCollection getExcludedFiles(); + String getMaxProcessedFile(); + S3FilesCollection getFailedFiles(); + S3FilesCollection getProcessedFiles(); + S3FilesCollection getProcessingFiles(); + +private: + zkutil::ZooKeeperPtr current_zookeeper; + mutable std::mutex current_zookeeper_mutex; + mutable std::mutex mutex; + const String zookeeper_path; + const String zookeeper_failed_path; + const String zookeeper_processing_path; + const String zookeeper_processed_path; + const S3QueueMode mode; + const UUID table_uuid; + Poco::Logger * log; + + zkutil::ZooKeeperPtr tryGetZooKeeper() const; + zkutil::ZooKeeperPtr getZooKeeper() const; +}; + + +} + + +#endif diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index d556a768a15..a2798a42263 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -15,14 +15,15 @@ class ASTStorage; mode, \ S3QueueMode::ORDERED, \ "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", \ - 0) \ - M(S3QueueAction, after_processing, S3QueueAction::KEEP , "Delete, keep or move file in S3 after processing", 0) \ - M(String, keeper_path, "", "Zookeeper node path", 0) \ - M(String, processed_action, "keep", "Keep, delete or move file after successful loading", 0) \ + 0) \ + M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete, keep or move file in S3 after processing", 0) \ + M(String, keeper_path, "", "Zookeeper node path", 0) \ M(UInt64, s3queue_max_retries, 0, "Retry loading up to specified number of times", 0) \ M(UInt64, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ - M(UInt64, s3queue_polling_backoff_ms, 0, "Retry loading up to specified number of times", 0) + M(UInt64, s3queue_polling_backoff_ms, 0, "Retry loading up to specified number of times", 0) \ + M(UInt64, s3queue_max_set_size, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ + M(UInt64, s3queue_max_set_age_s, 0, "Maximum number of seconds to store processed files in ZooKeeper node (forever default)", 0) #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index e77680c0d36..c2ee5154d75 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -8,65 +8,51 @@ #if USE_AWS_S3 -# include +#include -# include +#include -# include -# include +#include +#include -# include -# include +#include -# include -# include +#include +#include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include +#include -# include -# include -# include +#include +#include +#include -# include -# include +#include -# include -# include +#include -# include -# include -# include -# include +#include +#include +#include -# include +#include -# include +#include -# include +#include +#include +#include -# include -# include -# include -# include -# include - -# include -# include -# include -# include - -# include +#include +#include +#include +#include namespace fs = std::filesystem; @@ -220,7 +206,7 @@ StorageS3QueueSource::StorageS3QueueSource( const String & bucket_, const String & version_id_, std::shared_ptr file_iterator_, - const S3QueueMode & mode_, + std::shared_ptr queue_holder_, const S3QueueAction & action_, zkutil::ZooKeeperPtr current_zookeeper, const String & zookeeper_path_, @@ -238,9 +224,9 @@ StorageS3QueueSource::StorageS3QueueSource( , client(client_) , sample_block(sample_block_) , format_settings(format_settings_) + , queue_holder(queue_holder_) , requested_virtual_columns(requested_virtual_columns_) , file_iterator(file_iterator_) - , mode(mode_) , action(action_) , download_thread_num(download_thread_num_) , zookeeper(current_zookeeper) @@ -412,7 +398,7 @@ Chunk StorageS3QueueSource::generate() } } LOG_WARNING(log, "Set processed: {}", file_path); - setFileProcessed(file_path); + queue_holder->setFileProcessed(file_path); applyActionAfterProcessing(file_path); return chunk; } @@ -422,7 +408,7 @@ Chunk StorageS3QueueSource::generate() LOG_ERROR(log, "Exception: {} ", e.displayText()); const auto & failed_file_path = reader.getPath(); LOG_WARNING(log, "Set failed: {}", failed_file_path); - setFileFailed(failed_file_path); + queue_holder->setFileFailed(failed_file_path); } @@ -441,41 +427,6 @@ Chunk StorageS3QueueSource::generate() return {}; } -void StorageS3QueueSource::setFileProcessed(const String & file_path) -{ - std::lock_guard lock(mutex); - if (mode == S3QueueMode::UNORDERED) - { - String processed_files = zookeeper->get(zookeeper_path + "/processed"); - std::unordered_set processed = parseCollection(processed_files); - - processed.insert(file_path); - Strings set_processed; - set_processed.insert(set_processed.end(), processed.begin(), processed.end()); - - zookeeper->set(zookeeper_path + "/processed", toString(set_processed)); - } - else - { - zookeeper->set(zookeeper_path + "/processed", file_path); - } -} - - -void StorageS3QueueSource::setFileFailed(const String & file_path) -{ - std::lock_guard lock(mutex); - String processed_files = zookeeper->get(zookeeper_path + "/failed"); - std::unordered_set processed = parseCollection(processed_files); - - processed.insert(file_path); - Strings set_failed; - set_failed.insert(set_failed.end(), processed.begin(), processed.end()); - - zookeeper->set(zookeeper_path + "/failed", toString(set_failed)); -} - - void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) { LOG_WARNING(log, "Delete {} Bucke {}", file_path, bucket); @@ -494,24 +445,6 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) } } -std::unordered_set StorageS3QueueSource::parseCollection(String & files) -{ - ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); - Strings deserialized; - try - { - readQuoted(deserialized, rb); - } - catch (...) - { - deserialized = {}; - } - - std::unordered_set processed(deserialized.begin(), deserialized.end()); - - return processed; -} - } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index b5160588bab..0c36499d516 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -1,28 +1,28 @@ #pragma once - #include "config.h" #if USE_AWS_S3 -# include +#include -# include +#include -# include -# include -# include +#include +#include +#include +#include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -83,7 +83,7 @@ public: const String & bucket, const String & version_id, std::shared_ptr file_iterator_, - const S3QueueMode & mode_, + std::shared_ptr queue_holder_, const S3QueueAction & action_, zkutil::ZooKeeperPtr current_zookeeper, const String & zookeeper_path_, @@ -111,12 +111,12 @@ private: Block sample_block; std::optional format_settings; + std::shared_ptr queue_holder; using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; std::vector requested_virtual_columns; std::shared_ptr file_iterator; - const S3QueueMode mode; const S3QueueAction action; size_t download_thread_num = 1; diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp new file mode 100644 index 00000000000..40d29e26b68 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -0,0 +1,115 @@ +#include + +#if USE_AWS_S3 + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int METADATA_MISMATCH; +} + +S3QueueTableMetadata::S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings) +{ + format_name = configuration.format; + after_processing = engine_settings.after_processing.toString(); + mode = engine_settings.mode.toString(); + s3queue_max_set_size = engine_settings.s3queue_max_set_size; + s3queue_max_set_age_s = engine_settings.s3queue_max_set_age_s; +} + +void S3QueueTableMetadata::write(WriteBuffer & out) const +{ + out << "metadata format version: 1\n" + << "after processing: " << after_processing << "\n" + << "mode: " << mode << "\n" + << "s3queue_max_set_size: " << s3queue_max_set_size << "\n" + << "s3queue_max_set_age_s: " << s3queue_max_set_age_s << "\n" + << "format name: " << format_name << "\n"; +} + +String S3QueueTableMetadata::toString() const +{ + WriteBufferFromOwnString out; + write(out); + return out.str(); +} + +void S3QueueTableMetadata::read(ReadBuffer & in) +{ + in >> "metadata format version: 1\n"; + in >> "after processing: " >> after_processing >> "\n"; + in >> "mode: " >> mode >> "\n"; + in >> "s3queue_max_set_size: " >> s3queue_max_set_size >> "\n"; + in >> "s3queue_max_set_age_s: " >> s3queue_max_set_age_s >> "\n"; + in >> "format name: " >> format_name >> "\n"; +} + +S3QueueTableMetadata S3QueueTableMetadata::parse(const String & s) +{ + S3QueueTableMetadata metadata; + ReadBufferFromString buf(s); + metadata.read(buf); + return metadata; +} + + +void S3QueueTableMetadata::checkImmutableFieldsEquals(const S3QueueTableMetadata & from_zk) const +{ + if (after_processing != from_zk.after_processing) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs " + "in action after processing. Stored in ZooKeeper: {}, local: {}", + DB::toString(from_zk.after_processing), + DB::toString(after_processing)); + + if (mode != from_zk.mode) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in engine mode. " + "Stored in ZooKeeper: {}, local: {}", + DB::toString(from_zk.after_processing), + DB::toString(after_processing)); + + if (s3queue_max_set_size != from_zk.s3queue_max_set_size) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in max set size. " + "Stored in ZooKeeper: {}, local: {}", + from_zk.s3queue_max_set_size, + s3queue_max_set_size); + + if (s3queue_max_set_age_s != from_zk.s3queue_max_set_age_s) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in max set age. " + "Stored in ZooKeeper: {}, local: {}", + from_zk.s3queue_max_set_age_s, + s3queue_max_set_age_s); + + if (format_name != from_zk.format_name) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in format name. " + "Stored in ZooKeeper: {}, local: {}", + from_zk.format_name, + format_name); +} + +void S3QueueTableMetadata::checkEquals(const S3QueueTableMetadata & from_zk) const +{ + checkImmutableFieldsEquals(from_zk); +} + +} + +#endif diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h new file mode 100644 index 00000000000..85df226a02f --- /dev/null +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -0,0 +1,45 @@ +#pragma once + +#if USE_AWS_S3 + +#include +#include +#include + +namespace DB +{ + +class MergeTreeData; +class WriteBuffer; +class ReadBuffer; + +/** The basic parameters of S3Queue table engine for saving in ZooKeeper. + * Lets you verify that they match local ones. + */ +struct S3QueueTableMetadata +{ + String format_name; + String after_processing; + String mode; + UInt64 s3queue_max_set_size; + UInt64 s3queue_max_set_age_s; + + S3QueueTableMetadata() = default; + S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings); + + void read(ReadBuffer & in); + static S3QueueTableMetadata parse(const String & s); + + void write(WriteBuffer & out) const; + String toString() const; + + void checkEquals(const S3QueueTableMetadata & from_zk) const; + +private: + void checkImmutableFieldsEquals(const S3QueueTableMetadata & from_zk) const; +}; + + +} + +#endif diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 6d2bec7a0d3..7c0348347d5 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -1,3 +1,8 @@ +#include "config.h" + + +#if USE_AWS_S3 + #include #include #include @@ -6,74 +11,70 @@ #include #include #include +#include +#include #include "IO/IOThreadPool.h" #include "IO/ParallelReadBuffer.h" #include "Parsers/ASTCreateQuery.h" -#include "config.h" + +#include + +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include -#if USE_AWS_S3 +#include +#include +#include -# include -# include +#include +#include -# include +#include +#include -# include -# include +#include +#include +#include +#include -# include -# include +#include -# include -# include +#include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include -# include -# include -# include - -# include -# include - -# include -# include - -# include -# include -# include -# include - -# include - -# include - -# include - -# include -# include -# include - -# include -# include -# include -# include +#include +#include +#include +#include namespace fs = std::filesystem; @@ -103,6 +104,7 @@ namespace ErrorCodes extern const int QUERY_NOT_ALLOWED; extern const int NO_ZOOKEEPER; extern const int REPLICA_ALREADY_EXISTS; + extern const int INCOMPATIBLE_COLUMNS; } @@ -183,6 +185,18 @@ StorageS3Queue::StorageS3Queue( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); + setZooKeeper(); + auto metadata_snapshot = getInMemoryMetadataPtr(); + const bool is_first_replica = createTableIfNotExists(metadata_snapshot); + + if (!is_first_replica) + { + checkTableStructure(zookeeper_path, metadata_snapshot); + } + + auto table_uuid = getStorageID().uuid; + queue_holder = std::make_unique(zookeeper_path, mode, table_uuid, getContext()); + auto default_virtuals = NamesAndTypesList{ {"_path", std::make_shared(std::make_shared())}, {"_file", std::make_shared(std::make_shared())}}; @@ -192,10 +206,6 @@ StorageS3Queue::StorageS3Queue( for (const auto & column : virtual_columns) virtual_block.insert({column.type->createColumn(), column.type, column.name}); - setZooKeeper(); - auto metadata_snapshot = getInMemoryMetadataPtr(); - createTableIfNotExists(metadata_snapshot); - auto poll_thread = context_->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); task = std::make_shared(std::move(poll_thread)); } @@ -218,11 +228,11 @@ Pipe StorageS3Queue::read( ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - size_t num_streams) + size_t /* num_streams */) { if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, - "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + throw Exception( + ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); if (mv_attached) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageRabbitMQ with attached materialized views"); @@ -270,34 +280,27 @@ Pipe StorageS3Queue::read( const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; auto zookeeper = getZooKeeper(); - for (size_t i = 0; i < num_streams; ++i) - { - pipes.emplace_back(std::make_shared( - requested_virtual_columns, - format_name, - getName(), - block_for_format, - local_context, - format_settings, - columns_description, - max_block_size, - query_s3_configuration.request_settings, - compression_method, - query_s3_configuration.client, - query_s3_configuration.url.bucket, - query_s3_configuration.url.version_id, - iterator_wrapper, - mode, - after_processing, - zookeeper, - zookeeper_path, - max_download_threads)); - } - auto pipe = Pipe::unitePipes(std::move(pipes)); - narrowPipe(pipe, num_streams); - - return pipe; + return Pipe(std::make_shared( + requested_virtual_columns, + format_name, + getName(), + block_for_format, + local_context, + format_settings, + columns_description, + max_block_size, + query_s3_configuration.request_settings, + compression_method, + query_s3_configuration.client, + query_s3_configuration.url.bucket, + query_s3_configuration.url.version_id, + iterator_wrapper, + queue_holder, + after_processing, + zookeeper, + zookeeper_path, + max_download_threads)); } SinkToStoragePtr StorageS3Queue::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) @@ -306,10 +309,7 @@ SinkToStoragePtr StorageS3Queue::write(const ASTPtr & /*query*/, const StorageMe } void StorageS3Queue::truncate( - const ASTPtr & /*query*/, - const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr /*local_context*/, - TableExclusiveLockHolder &) + const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*local_context*/, TableExclusiveLockHolder &) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); } @@ -494,37 +494,32 @@ void StorageS3Queue::streamToViews() Pipes pipes; auto zookeeper = getZooKeeper(); - size_t num_streams = 1; - for (size_t i = 0; i < num_streams; ++i) - { - pipes.emplace_back(std::make_shared( - requested_virtual_columns, - format_name, - getName(), - block_for_format, - s3queue_context, - format_settings, - columns_description, - block_size, - query_s3_configuration.request_settings, - compression_method, - query_s3_configuration.client, - query_s3_configuration.url.bucket, - query_s3_configuration.url.version_id, - iterator_wrapper, - mode, - after_processing, - zookeeper, - zookeeper_path, - max_download_threads)); - } + auto pipe = Pipe(std::make_shared( + requested_virtual_columns, + format_name, + getName(), + block_for_format, + s3queue_context, + format_settings, + columns_description, + block_size, + query_s3_configuration.request_settings, + compression_method, + query_s3_configuration.client, + query_s3_configuration.url.bucket, + query_s3_configuration.url.version_id, + iterator_wrapper, + queue_holder, + after_processing, + zookeeper, + zookeeper_path, + max_download_threads)); - auto pipe = Pipe::unitePipes(std::move(pipes)); std::atomic_size_t rows = 0; { block_io.pipeline.complete(std::move(pipe)); - block_io.pipeline.setNumThreads(num_streams); + block_io.pipeline.setNumThreads(1); block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); CompletedPipelineExecutor executor(block_io.pipeline); @@ -569,23 +564,29 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ Coordination::Requests ops; auto table_uuid = getStorageID().uuid; - if (zookeeper->exists(zookeeper_path + "")) + if (zookeeper->exists(zookeeper_path + "/metadata")) { - ops.emplace_back(zkutil::makeCreateRequest( - fs::path(zookeeper_path) / "processing" / toString(table_uuid), "{}", zkutil::CreateMode::Ephemeral)); + LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zookeeper_path); + return false; } else { - String deafult_processed = mode == S3QueueMode::ORDERED ? "" : "[]"; + String metadata_str = S3QueueTableMetadata(s3_configuration, *s3queue_settings).toString(); + String default_processed = mode == S3QueueMode::ORDERED ? "" : "[]"; + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", deafult_processed, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", default_processed, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( fs::path(zookeeper_path) / "processing" / toString(table_uuid), "[]", zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, zkutil::CreateMode::Persistent)); } + Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZNODEEXISTS) @@ -608,101 +609,61 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ } +/** Verify that list of columns and table settings match those specified in ZK (/metadata). + * If not, throw an exception. + */ +void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot) +{ + auto zookeeper = getZooKeeper(); + + S3QueueTableMetadata old_metadata(s3_configuration, *s3queue_settings); + + Coordination::Stat metadata_stat; + String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata", &metadata_stat); + auto metadata_from_zk = S3QueueTableMetadata::parse(metadata_str); + old_metadata.checkEquals(metadata_from_zk); + + Coordination::Stat columns_stat; + auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns", &columns_stat)); + + const ColumnsDescription & old_columns = metadata_snapshot->getColumns(); + if (columns_from_zk != old_columns) + { + throw Exception( + ErrorCodes::INCOMPATIBLE_COLUMNS, + "Table columns structure in ZooKeeper is different from local table structure. Local columns:\n" + "{}\nZookeeper columns:\n{}", + old_columns.toString(), + columns_from_zk.toString()); + } +} + + std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys) { /// Iterate through disclosed globs and make a source for each file + std::lock_guard lock{sync_mutex}; + auto it = std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_block, local_context, read_keys, s3_configuration.request_settings); - std::lock_guard lock{sync_mutex}; - std::unordered_set exclude = getExcludedFiles(); + S3QueueHolder::S3FilesCollection exclude = queue_holder->getExcludedFiles(); - Strings processing; + Strings processing_files; if (mode == S3QueueMode::UNORDERED) { - processing = it->setProcessing(mode, exclude); + processing_files = it->setProcessing(mode, exclude); } else { - String max_processed_file = getMaxProcessedFile(); - processing = it->setProcessing(mode, exclude, max_processed_file); + String max_processed_file = queue_holder->getMaxProcessedFile(); + processing_files = it->setProcessing(mode, exclude, max_processed_file); } - - auto zookeeper = getZooKeeper(); - auto table_uuid = getStorageID().uuid; - zookeeper->set(fs::path(zookeeper_path) / "processing" / toString(table_uuid), toString(processing)); - + queue_holder->setFilesProcessing(processing_files); return it; } -std::unordered_set StorageS3Queue::getFailedFiles() -{ - auto zookeeper = getZooKeeper(); - - String failed = zookeeper->get(zookeeper_path + "/failed"); - std::unordered_set failed_files = StorageS3QueueSource::parseCollection(failed); - - return failed_files; -} - -std::unordered_set StorageS3Queue::getProcessedFiles() -{ - auto zookeeper = getZooKeeper(); - - String processed = zookeeper->get(zookeeper_path + "/processed"); - std::unordered_set processed_files = StorageS3QueueSource::parseCollection(processed); - - return processed_files; -} - -String StorageS3Queue::getMaxProcessedFile() -{ - auto zookeeper = getZooKeeper(); - - String processed = zookeeper->get(zookeeper_path + "/processed"); - return processed; -} - -std::unordered_set StorageS3Queue::getProcessingFiles() -{ - auto zookeeper = getZooKeeper(); - - Strings consumer_table_uuids; - zookeeper->tryGetChildren(zookeeper_path + "/processing", consumer_table_uuids); - std::unordered_set processing_files; - for (const auto & uuid : consumer_table_uuids) - { - String processing = zookeeper->get(fs::path(zookeeper_path) / "processing" / toString(uuid)); - std::unordered_set cur_processing_files = StorageS3QueueSource::parseCollection(processing); - processing_files.merge(cur_processing_files); - } - return processing_files; -} - -std::unordered_set StorageS3Queue::getExcludedFiles() -{ - std::unordered_set exclude_files; - - std::unordered_set failed_files = getFailedFiles(); - LOG_DEBUG(log, "failed_files {}", failed_files.size()); - exclude_files.merge(failed_files); - - if (mode != S3QueueMode::ORDERED) - { - std::unordered_set processed_files = getProcessedFiles(); - LOG_DEBUG(log, "processed_files {}", processed_files.size()); - exclude_files.merge(processed_files); - } - - std::unordered_set processing_files = getProcessingFiles(); - LOG_DEBUG(log, "processing {}", processing_files.size()); - exclude_files.merge(processing_files); - - return exclude_files; -} - - void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { factory.registerStorage( diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 4ca6ba73271..ac098cdda99 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -13,6 +13,7 @@ # include # include # include +# include # include # include @@ -79,6 +80,7 @@ public: private: std::unique_ptr s3queue_settings; + std::shared_ptr queue_holder; Configuration s3_configuration; std::vector keys; NamesAndTypesList virtual_columns; @@ -127,6 +129,7 @@ private: zkutil::ZooKeeperPtr tryGetZooKeeper() const; zkutil::ZooKeeperPtr getZooKeeper() const; bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); + void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); // Return default or custom zookeeper name for table const String & getZooKeeperPath() const { return zookeeper_path; } @@ -135,16 +138,7 @@ private: std::shared_ptr createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys = nullptr); - static std::unordered_set parseCollection(String & files); - - std::unordered_set getFailedFiles(); - std::unordered_set getProcessedFiles(); - String getMaxProcessedFile(); - std::unordered_set getProcessingFiles(); - std::unordered_set getExcludedFiles(); - void streamToViews(); - Configuration updateConfigurationAndGetCopy(ContextPtr local_context); }; diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 8cdf7e8dda0..5d5d43347cb 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -5,6 +5,7 @@ import random import time import pytest +from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.s3_tools import prepare_s3_bucket @@ -13,7 +14,13 @@ from helpers.s3_tools import prepare_s3_bucket def s3_queue_setup_teardown(started_cluster): instance = started_cluster.instances["instance"] instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") - # logging.debug("kafka is available - running test") + + minio = started_cluster.minio_client + objects = list( + minio.list_objects(started_cluster.minio_restricted_bucket, recursive=True) + ) + for obj in objects: + minio.remove_object(started_cluster.minio_restricted_bucket, obj.object_name) yield # run test @@ -109,9 +116,13 @@ def test_direct_select_file(started_cluster, mode): ).encode() filename = f"test.csv" put_s3_file_content(started_cluster, bucket, filename, values_csv) - instance.query("drop table if exists test.s3_queue") - instance.query("drop table if exists test.s3_queue_2") - instance.query("drop table if exists test.s3_queue_3") + instance.query( + """ + DROP TABLE IF EXISTS test.s3_queue; + DROP TABLE IF EXISTS test.s3_queue_2; + DROP TABLE IF EXISTS test.s3_queue_3; + """ + ) instance.query( f""" @@ -201,8 +212,7 @@ def test_direct_select_multiple_files(started_cluster, mode): """ ) - for i in range(10): - print(i) + for i in range(5): rand_values = [[random.randint(0, 50) for _ in range(3)] for _ in range(10)] values_csv = ( @@ -218,7 +228,7 @@ def test_direct_select_multiple_files(started_cluster, mode): ] == rand_values total_values = generate_random_files( - 5, prefix, started_cluster, bucket, start_ind=10 + 4, prefix, started_cluster, bucket, start_ind=5 ) get_query = f"SELECT * FROM test.s3_queue" assert { @@ -248,7 +258,7 @@ def test_streaming_to_view_(started_cluster, mode): ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', - keeper_path = '/clickhouse/select_multiple_{mode}'; + keeper_path = '/clickhouse/view_{mode}'; CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS SELECT @@ -307,7 +317,7 @@ def test_streaming_to_many_views(started_cluster, mode): ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', - keeper_path = '/clickhouse/select_multiple_{mode}'; + keeper_path = '/clickhouse/multiple_view_{mode}'; CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS SELECT @@ -325,7 +335,7 @@ def test_streaming_to_many_views(started_cluster, mode): FROM test.s3_queue; """ ) - total_values = generate_random_files(10, prefix, started_cluster, bucket) + total_values = generate_random_files(5, prefix, started_cluster, bucket) expected_values = set([tuple(i) for i in total_values]) for i in range(retry_cnt): @@ -348,3 +358,86 @@ def test_streaming_to_many_views(started_cluster, mode): time.sleep(1) else: break + + +def test_multiple_tables_meta_mismatch(started_cluster): + prefix = f"test_meta" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'ordered', + keeper_path = '/clickhouse/test_meta'; + """ + ) + # check mode + failed = False + try: + instance.query( + f""" + CREATE TABLE test.s3_queue_copy ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'unordered', + keeper_path = '/clickhouse/test_meta'; + """ + ) + except QueryRuntimeException as e: + assert "Existing table metadata in ZooKeeper differs in engine mode" in str(e) + failed = True + assert failed is True + + # check columns + table_format_copy = table_format + ", column4 UInt32" + try: + instance.query( + f""" + CREATE TABLE test.s3_queue_copy ({table_format_copy}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'ordered', + keeper_path = '/clickhouse/test_meta'; + """ + ) + except QueryRuntimeException as e: + assert ( + "Table columns structure in ZooKeeper is different from local table structure" + in str(e) + ) + failed = True + + assert failed is True + + # check format + try: + instance.query( + f""" + CREATE TABLE test.s3_queue_copy ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'TSV') + SETTINGS + mode = 'ordered', + keeper_path = '/clickhouse/test_meta'; + """ + ) + except QueryRuntimeException as e: + assert "Existing table metadata in ZooKeeper differs in format name" in str(e) + failed = True + assert failed is True + + # create working engine + instance.query( + f""" + CREATE TABLE test.s3_queue_copy ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'ordered', + keeper_path = '/clickhouse/test_meta'; + """ + ) From c24ec8f83f732eb6752ae08f324e925f3aa92cd4 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 4 May 2023 11:58:09 +0300 Subject: [PATCH 0043/2047] fix --- src/Storages/S3Queue/S3QueueSource.cpp | 1 - src/Storages/S3Queue/StorageS3Queue.cpp | 19 ------------------- .../integration/test_storage_s3_queue/test.py | 4 ++-- 3 files changed, 2 insertions(+), 22 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index c2ee5154d75..3da725cee71 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -1,7 +1,6 @@ #include #include #include -#include "IO/IOThreadPool.h" #include "IO/ParallelReadBuffer.h" #include "Parsers/ASTCreateQuery.h" #include "config.h" diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 7c0348347d5..f91ca11b491 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -4,8 +4,6 @@ #if USE_AWS_S3 #include -#include -#include #include #include #include @@ -13,17 +11,13 @@ #include #include #include -#include "IO/IOThreadPool.h" #include "IO/ParallelReadBuffer.h" -#include "Parsers/ASTCreateQuery.h" #include -#include #include #include -#include #include #include @@ -37,39 +31,26 @@ #include #include #include -#include #include -#include #include -#include #include #include #include #include -#include - -#include -#include #include -#include #include #include #include -#include #include #include -#include - -#include #include -#include #include #include diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 5d5d43347cb..b8d0ec3d5da 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -408,8 +408,8 @@ def test_multiple_tables_meta_mismatch(started_cluster): ) except QueryRuntimeException as e: assert ( - "Table columns structure in ZooKeeper is different from local table structure" - in str(e) + "Table columns structure in ZooKeeper is different from local table structure" + in str(e) ) failed = True From 9032e23f87d2f8ed88ad62c79017c2c951d74464 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 5 May 2023 15:43:23 +0200 Subject: [PATCH 0044/2047] Clean up storage_conf.xml, use dynamic disks for tests --- tests/config/config.d/storage_conf.xml | 133 ------------------ ...ilesystem_cache_bypass_cache_threshold.sql | 19 ++- .../02240_filesystem_query_cache.sql | 18 ++- .../02344_describe_cache.reference | 1 - .../0_stateless/02344_describe_cache.sql | 3 - ..._cache_on_write_with_small_segment_size.sh | 17 ++- 6 files changed, 51 insertions(+), 140 deletions(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index dee03307177..98798d5746e 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -10,46 +10,6 @@ clickhouse 20000 - - s3 - s3_disk_2/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_3/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_4/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_5/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - - - s3 - s3_disk_6/ - http://localhost:11111/test/00170_test/ - clickhouse - clickhouse - 20000 - cache @@ -60,54 +20,6 @@ 0 100 - - cache - s3_disk_2 - s3_cache_2/ - 128Mi - 0 - 100Mi - 100 - - - cache - s3_disk_3 - s3_disk_3_cache/ - 128Mi - 22548578304 - 1 - 1 - 0 - 100 - - - cache - s3_disk_4 - s3_cache_4/ - 128Mi - 1 - 1 - 0 - 100 - - - cache - s3_disk_5 - s3_cache_5/ - 128Mi - 0 - 100 - - - cache - s3_disk_6 - s3_cache_6/ - 128Mi - 0 - 1 - 100 - 100 - cache s3_disk_6 @@ -116,16 +28,6 @@ 1 100 - - cache - s3_disk_6 - s3_cache_small_segment_size/ - 128Mi - 10Ki - 0 - 1 - 100 - local_blob_storage @@ -193,34 +95,6 @@ - - -
- s3_cache_2 -
-
-
- - -
- s3_cache_3 -
-
-
- - -
- s3_cache_4 -
-
-
- - -
- s3_cache_6 -
-
-
@@ -256,13 +130,6 @@
- - -
- s3_cache_small_segment_size -
-
-
diff --git a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql index f6671b82291..ae2cd1b8cd1 100644 --- a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql +++ b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql @@ -6,7 +6,24 @@ SYSTEM DROP FILESYSTEM CACHE; SET enable_filesystem_cache_on_write_operations=0; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_6', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; +CREATE TABLE test (key UInt32, value String) +Engine=MergeTree() +ORDER BY key +SETTINGS min_bytes_for_wide_part = 10485760, + compress_marks=false, + compress_primary_key=false, + disk = disk( + type = cache, + max_size = '128Mi', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + enable_bypass_cache_with_threashold = 1, + bypass_cache_threashold = 100, + cache_on_write_operations = 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); + INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; diff --git a/tests/queries/0_stateless/02240_filesystem_query_cache.sql b/tests/queries/0_stateless/02240_filesystem_query_cache.sql index 94eb4bc5ccd..f2664de5c63 100644 --- a/tests/queries/0_stateless/02240_filesystem_query_cache.sql +++ b/tests/queries/0_stateless/02240_filesystem_query_cache.sql @@ -8,7 +8,23 @@ SET skip_download_if_exceeds_query_cache=1; SET filesystem_cache_max_download_size=128; DROP TABLE IF EXISTS test; -CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_4', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false; + +CREATE TABLE test (key UInt32, value String) +Engine=MergeTree() +ORDER BY key +SETTINGS min_bytes_for_wide_part = 10485760, + compress_marks=false, + compress_primary_key=false, + disk = disk( + type = cache, + max_size = '128Mi', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + cache_on_write_operations= 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); + INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 7561b32bae1..4302b05e136 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1,2 +1 @@ 134217728 1048576 104857600 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 0 -134217728 1048576 104857600 0 0 0 0 /var/lib/clickhouse/caches/s3_cache_2/ 0 diff --git a/tests/queries/0_stateless/02344_describe_cache.sql b/tests/queries/0_stateless/02344_describe_cache.sql index a687ad01394..9c5c5c10952 100644 --- a/tests/queries/0_stateless/02344_describe_cache.sql +++ b/tests/queries/0_stateless/02344_describe_cache.sql @@ -1,7 +1,4 @@ -- Tags: no-fasttest, no-parallel SYSTEM DROP FILESYSTEM CACHE 's3_cache'; -SYSTEM DROP FILESYSTEM CACHE 's3_cache_2'; - DESCRIBE FILESYSTEM CACHE 's3_cache'; -DESCRIBE FILESYSTEM CACHE 's3_cache_2'; diff --git a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh index ed66c36b823..2c526d10cc9 100755 --- a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh +++ b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh @@ -13,7 +13,22 @@ function random { ${CLICKHOUSE_CLIENT} --multiline --multiquery -q " drop table if exists ttt; -create table ttt (id Int32, value String) engine=MergeTree() order by tuple() settings storage_policy='s3_cache_small_segment_size', min_bytes_for_wide_part=0; + +CREATE TABLE ttt (id Int32, value String) +Engine=MergeTree() +ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, + disk = disk( + type = cache, + max_size = '128Mi', + max_file_segment_size = '10Ki', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + cache_on_write_operations = 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); + insert into ttt select number, toString(number) from numbers(100000) settings throw_on_error_from_cache_on_write_operations = 1; " From 431b2e94a936f5021b6d834994614d8d03ddafcb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 5 May 2023 19:10:12 +0200 Subject: [PATCH 0045/2047] Update storage_conf.xml --- tests/config/config.d/storage_conf.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 98798d5746e..923240d5a91 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -22,7 +22,7 @@ cache - s3_disk_6 + s3_disk s3_cache_small/ 1000 1 From bc16cc59fff3ccab70fd0ca54ece50e15f99e266 Mon Sep 17 00:00:00 2001 From: Bin Xie Date: Sat, 6 May 2023 11:09:45 +0800 Subject: [PATCH 0046/2047] If a dictionary is created with a complex key, automatically choose the "complex key" layout variant. --- src/Dictionaries/DictionaryFactory.cpp | 23 +++++++++++++++++-- src/Dictionaries/DictionaryFactory.h | 4 +++- src/Dictionaries/FlatDictionary.cpp | 2 +- .../getDictionaryConfigurationFromAST.cpp | 6 +++++ 4 files changed, 31 insertions(+), 4 deletions(-) diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index d091e49d1f0..c3102632167 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -17,13 +17,13 @@ namespace ErrorCodes extern const int UNKNOWN_ELEMENT_IN_CONFIG; } -void DictionaryFactory::registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex) +void DictionaryFactory::registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex, bool has_layout_complex) { auto it = registered_layouts.find(layout_type); if (it != registered_layouts.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "DictionaryFactory: the layout name '{}' is not unique", layout_type); - RegisteredLayout layout { .layout_create_function = create_layout, .is_layout_complex = is_layout_complex }; + RegisteredLayout layout { .layout_create_function = create_layout, .is_layout_complex = is_layout_complex, .has_layout_complex = has_layout_complex }; registered_layouts.emplace(layout_type, std::move(layout)); } @@ -89,6 +89,25 @@ bool DictionaryFactory::isComplex(const std::string & layout_type) const return it->second.is_layout_complex; } +bool DictionaryFactory::convertToComplex(std::string & layout_type) const +{ + auto it = registered_layouts.find(layout_type); + + if (it == registered_layouts.end()) + { + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, + "Unknown dictionary layout type: {}", + layout_type); + } + + if (!it->second.is_layout_complex && it->second.has_layout_complex) + { + layout_type = "complex_key_" + layout_type; + return true; + } + return false; +} + DictionaryFactory & DictionaryFactory::instance() { diff --git a/src/Dictionaries/DictionaryFactory.h b/src/Dictionaries/DictionaryFactory.h index b1dad340f4b..0dc80af62fc 100644 --- a/src/Dictionaries/DictionaryFactory.h +++ b/src/Dictionaries/DictionaryFactory.h @@ -54,14 +54,16 @@ public: bool created_from_ddl)>; bool isComplex(const std::string & layout_type) const; + bool convertToComplex(std::string & layout_type) const; - void registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex); + void registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex, bool has_layout_complex = true); private: struct RegisteredLayout { LayoutCreateFunction layout_create_function; bool is_layout_complex; + bool has_layout_complex; }; using LayoutRegistry = std::unordered_map; diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index d3699a150c4..5bbb5a33fa9 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -683,7 +683,7 @@ void registerDictionaryFlat(DictionaryFactory & factory) return std::make_unique(dict_id, dict_struct, std::move(source_ptr), configuration); }; - factory.registerLayout("flat", create_layout, false); + factory.registerLayout("flat", create_layout, false, false); } diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 0de8b843604..6d3d08a323b 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -609,6 +609,12 @@ getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr conte bool complex = DictionaryFactory::instance().isComplex(dictionary_layout->layout_type); + if (pk_attrs.size() > 1 && !complex + && DictionaryFactory::instance().convertToComplex(dictionary_layout->layout_type)) + { + complex = true; + } + auto all_attr_names_and_types = buildDictionaryAttributesConfiguration( xml_document, structure_element, query.dictionary_attributes_list, pk_attrs); From d3c3d8b8e401d239416f323c69ceb12c67e3c26d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 16:48:40 +0100 Subject: [PATCH 0047/2047] Remove export of dynamic symbols --- CMakeLists.txt | 10 ++++++++-- programs/library-bridge/CMakeLists.txt | 4 ---- programs/odbc-bridge/CMakeLists.txt | 6 ------ 3 files changed, 8 insertions(+), 12 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 263b202049b..3283ca52ca7 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -155,8 +155,14 @@ elseif(GLIBC_COMPATIBILITY) message (${RECONFIGURE_MESSAGE_LEVEL} "Glibc compatibility cannot be enabled in current configuration") endif () -# Make sure the final executable has symbols exported -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") +if (OS_LINUX) + # We should not export dynamic symbols, because: + # - The main clickhouse binary does not use dlopen, + # and whatever is poisoning it by LD_PRELOAD should not link to our symbols. + # - The clickhouse-odbc-bridge and clickhouse-library-bridge binaries + # should not expose their symbols to ODBC drivers and libraries. + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") +endif () if (OS_DARWIN) # The `-all_load` flag forces loading of all symbols from all libraries, diff --git a/programs/library-bridge/CMakeLists.txt b/programs/library-bridge/CMakeLists.txt index 1cacc391ca5..dd0bf67cb64 100644 --- a/programs/library-bridge/CMakeLists.txt +++ b/programs/library-bridge/CMakeLists.txt @@ -13,10 +13,6 @@ set (CLICKHOUSE_LIBRARY_BRIDGE_SOURCES library-bridge.cpp ) -if (OS_LINUX) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") -endif () - clickhouse_add_executable(clickhouse-library-bridge ${CLICKHOUSE_LIBRARY_BRIDGE_SOURCES}) target_link_libraries(clickhouse-library-bridge PRIVATE diff --git a/programs/odbc-bridge/CMakeLists.txt b/programs/odbc-bridge/CMakeLists.txt index 118610e4dcd..56373601b95 100644 --- a/programs/odbc-bridge/CMakeLists.txt +++ b/programs/odbc-bridge/CMakeLists.txt @@ -15,12 +15,6 @@ set (CLICKHOUSE_ODBC_BRIDGE_SOURCES validateODBCConnectionString.cpp ) -if (OS_LINUX) - # clickhouse-odbc-bridge is always a separate binary. - # Reason: it must not export symbols from SSL, mariadb-client, etc. to not break ABI compatibility with ODBC drivers. - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic") -endif () - clickhouse_add_executable(clickhouse-odbc-bridge ${CLICKHOUSE_ODBC_BRIDGE_SOURCES}) target_link_libraries(clickhouse-odbc-bridge PRIVATE From 5a3281bb4912dce6a6125681b0804b97653da763 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 19:30:03 +0100 Subject: [PATCH 0048/2047] Remove unused code --- src/Common/getResource.cpp | 35 ----------------------------------- 1 file changed, 35 deletions(-) diff --git a/src/Common/getResource.cpp b/src/Common/getResource.cpp index fe603fcc550..09777640dd9 100644 --- a/src/Common/getResource.cpp +++ b/src/Common/getResource.cpp @@ -1,6 +1,4 @@ #include "getResource.h" -#include -#include #include #include @@ -14,39 +12,6 @@ std::string_view getResource(std::string_view name) std::replace(name_replaced.begin(), name_replaced.end(), '.', '_'); boost::replace_all(name_replaced, "+", "_PLUS_"); -#if defined USE_MUSL /// If static linking is used, we cannot use dlsym and have to parse ELF symbol table by ourself. return DB::SymbolIndex::instance()->getResource(name_replaced); - -#else - // In most `dlsym(3)` APIs, one passes the symbol name as it appears via - // something like `nm` or `objdump -t`. For example, a symbol `_foo` would be - // looked up with the string `"_foo"`. - // - // Apple's linker is confusingly different. The NOTES on the man page for - // `dlsym(3)` claim that one looks up the symbol with "the name used in C - // source code". In this example, that would mean using the string `"foo"`. - // This apparently applies even in the case where the symbol did not originate - // from C source, such as the embedded binary resource files used here. So - // the symbol name must not have a leading `_` on Apple platforms. It's not - // clear how this applies to other symbols, such as those which _have_ a leading - // underscore in them by design, many leading underscores, etc. -#if defined OS_DARWIN - std::string prefix = "binary_"; -#else - std::string prefix = "_binary_"; -#endif - std::string symbol_name_start = prefix + name_replaced + "_start"; - std::string symbol_name_end = prefix + name_replaced + "_end"; - - const char * sym_start = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_start.c_str())); - const char * sym_end = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_end.c_str())); - - if (sym_start && sym_end) - { - auto resource_size = static_cast(std::distance(sym_start, sym_end)); - return { sym_start, resource_size }; - } - return {}; -#endif } From 8cd9fc4a2d6936343c2be119d8fdd61986cdd77f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 11 Mar 2023 21:20:20 +0100 Subject: [PATCH 0049/2047] Fix build --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 2cd0a011013..c7f31e13287 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -62,7 +62,7 @@ then ninja $NINJA_FLAGS clickhouse-keeper ls -la ./programs/ - ldd ./programs/clickhouse-keeper + ldd ./programs/clickhouse-keeper ||: if [ -n "$MAKE_DEB" ]; then # No quotes because I want it to expand to nothing if empty. From 03845ba9c5fb1ccb03330059369641751f1c9ab1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 00:16:23 +0200 Subject: [PATCH 0050/2047] Fix MSan. --- src/Common/SymbolIndex.cpp | 81 +++++++++++++++++++++++--------------- 1 file changed, 50 insertions(+), 31 deletions(-) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index f1cace5017c..79f97e93a2f 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -9,7 +9,6 @@ #include -//#include #include #include @@ -63,9 +62,11 @@ Otherwise you will get only exported symbols from program headers. #endif #define __msan_unpoison_string(X) // NOLINT +#define __msan_unpoison(X, Y) // NOLINT #if defined(ch_has_feature) # if ch_has_feature(memory_sanitizer) # undef __msan_unpoison_string +# undef __msan_unpoison # include # endif #endif @@ -136,10 +137,12 @@ void collectSymbolsFromProgramHeaders( /* Iterate over all headers of the current shared lib * (first call is for the executable itself) */ + __msan_unpoison(&info->dlpi_phnum, sizeof(info->dlpi_phnum)); for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) { /* Further processing is only needed if the dynamic section is reached */ + __msan_unpoison(&info->dlpi_phdr[header_index], sizeof(info->dlpi_phdr[header_index])); if (info->dlpi_phdr[header_index].p_type != PT_DYNAMIC) continue; @@ -160,44 +163,53 @@ void collectSymbolsFromProgramHeaders( */ size_t sym_cnt = 0; - for (const auto * it = dyn_begin; it->d_tag != DT_NULL; ++it) { - ElfW(Addr) base_address = correct_address(info->dlpi_addr, it->d_un.d_ptr); - - // TODO: this branch leads to invalid address of the hash table. Need further investigation. - // if (it->d_tag == DT_HASH) - // { - // const ElfW(Word) * hash = reinterpret_cast(base_address); - // sym_cnt = hash[1]; - // break; - // } - if (it->d_tag == DT_GNU_HASH) + const auto * it = dyn_begin; + while (true) { - /// This code based on Musl-libc. + __msan_unpoison(it, sizeof(*it)); + if (it->d_tag != DT_NULL) + break; - const uint32_t * buckets = nullptr; - const uint32_t * hashval = nullptr; + ElfW(Addr) base_address = correct_address(info->dlpi_addr, it->d_un.d_ptr); - const ElfW(Word) * hash = reinterpret_cast(base_address); - - buckets = hash + 4 + (hash[2] * sizeof(size_t) / 4); - - for (ElfW(Word) i = 0; i < hash[0]; ++i) - if (buckets[i] > sym_cnt) - sym_cnt = buckets[i]; - - if (sym_cnt) + if (it->d_tag == DT_GNU_HASH) { - sym_cnt -= hash[1]; - hashval = buckets + hash[0] + sym_cnt; - do + /// This code based on Musl-libc. + + const uint32_t * buckets = nullptr; + const uint32_t * hashval = nullptr; + + const ElfW(Word) * hash = reinterpret_cast(base_address); + + __msan_unpoison(&hash[0], sizeof(*hash)); + __msan_unpoison(&hash[1], sizeof(*hash)); + __msan_unpoison(&hash[2], sizeof(*hash)); + + buckets = hash + 4 + (hash[2] * sizeof(size_t) / 4); + + __msan_unpoison(buckets, hash[0] * sizeof(buckets[0])); + + for (ElfW(Word) i = 0; i < hash[0]; ++i) + if (buckets[i] > sym_cnt) + sym_cnt = buckets[i]; + + if (sym_cnt) { - ++sym_cnt; + sym_cnt -= hash[1]; + hashval = buckets + hash[0] + sym_cnt; + __msan_unpoison(&hashval, sizeof(hashval)); + do + { + ++sym_cnt; + } + while (!(*hashval++ & 1)); } - while (!(*hashval++ & 1)); + + break; } - break; + ++it; } } @@ -228,6 +240,8 @@ void collectSymbolsFromProgramHeaders( /* Get the pointer to the first entry of the symbol table */ const ElfW(Sym) * elf_sym = reinterpret_cast(base_address); + __msan_unpoison(elf_sym, sym_cnt * sizeof(*elf_sym)); + /* Iterate over the symbol table */ for (ElfW(Word) sym_index = 0; sym_index < ElfW(Word)(sym_cnt); ++sym_index) { @@ -235,6 +249,7 @@ void collectSymbolsFromProgramHeaders( * This is located at the address of st_name relative to the beginning of the string table. */ const char * sym_name = &strtab[elf_sym[sym_index].st_name]; + __msan_unpoison_string(sym_name); if (!sym_name) continue; @@ -264,13 +279,17 @@ void collectSymbolsFromProgramHeaders( #if !defined USE_MUSL String getBuildIDFromProgramHeaders(dl_phdr_info * info) { + __msan_unpoison(&info->dlpi_phnum, sizeof(info->dlpi_phnum)); for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) { const ElfPhdr & phdr = info->dlpi_phdr[header_index]; + __msan_unpoison(&phdr, sizeof(phdr)); if (phdr.p_type != PT_NOTE) continue; - return Elf::getBuildID(reinterpret_cast(info->dlpi_addr + phdr.p_vaddr), phdr.p_memsz); + std::string_view view(reinterpret_cast(info->dlpi_addr + phdr.p_vaddr), phdr.p_memsz); + __msan_unpoison(view.data(), view.size()); + return Elf::getBuildID(view.data(), view.size()); } return {}; } From 9e513a147b5ca7ca0b75feec5488093f32df77d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 04:22:01 +0200 Subject: [PATCH 0051/2047] Fixup --- src/Common/SymbolIndex.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index 79f97e93a2f..394ae1a0592 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -138,6 +138,7 @@ void collectSymbolsFromProgramHeaders( * (first call is for the executable itself) */ __msan_unpoison(&info->dlpi_phnum, sizeof(info->dlpi_phnum)); + __msan_unpoison(&info->dlpi_phdr, sizeof(info->dlpi_phdr)); for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) { /* Further processing is only needed if the dynamic section is reached @@ -280,6 +281,7 @@ void collectSymbolsFromProgramHeaders( String getBuildIDFromProgramHeaders(dl_phdr_info * info) { __msan_unpoison(&info->dlpi_phnum, sizeof(info->dlpi_phnum)); + __msan_unpoison(&info->dlpi_phdr, sizeof(info->dlpi_phdr)); for (size_t header_index = 0; header_index < info->dlpi_phnum; ++header_index) { const ElfPhdr & phdr = info->dlpi_phdr[header_index]; From 72dd039d1c9cedafea3f6aa6dbf8c47b24ab61c9 Mon Sep 17 00:00:00 2001 From: xbthink Date: Sun, 7 May 2023 16:22:05 +0800 Subject: [PATCH 0052/2047] add comments and functional test --- src/Dictionaries/DictionaryFactory.h | 3 ++ ...ayout_to_complex_by_complex_keys.reference | 4 +++ ...nary_layout_to_complex_by_complex_keys.sql | 29 +++++++++++++++++++ 3 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference create mode 100644 tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql diff --git a/src/Dictionaries/DictionaryFactory.h b/src/Dictionaries/DictionaryFactory.h index 0dc80af62fc..309f14f6b7f 100644 --- a/src/Dictionaries/DictionaryFactory.h +++ b/src/Dictionaries/DictionaryFactory.h @@ -54,6 +54,9 @@ public: bool created_from_ddl)>; bool isComplex(const std::string & layout_type) const; + + /// if the argument `layout_type` is not complex layout and has corresponding complex layout, + /// change `layout_type` to corresponding complex and return true; otherwise do nothing and return false. bool convertToComplex(std::string & layout_type) const; void registerLayout(const std::string & layout_type, LayoutCreateFunction create_layout, bool is_layout_complex, bool has_layout_complex = true); diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference new file mode 100644 index 00000000000..7f38556e7d1 --- /dev/null +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference @@ -0,0 +1,4 @@ +dict_flat_simple Flat +dict_hashed_simple Hashed +dict_hashed_complex ComplexKeyHashed +dict_hashed_simple_auto_convert ComplexKeyHashed diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql new file mode 100644 index 00000000000..98c7c5c05f9 --- /dev/null +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql @@ -0,0 +1,29 @@ +DROP DICTIONARY IF EXISTS dict_flat_simple; +DROP DICTIONARY IF EXISTS dict_hashed_simple; +DROP DICTIONARY IF EXISTS dict_hashed_complex; +DROP DICTIONARY IF EXISTS dict_hashed_simple_auto_convert; +DROP TABLE IF EXISTS dict_data; + +CREATE TABLE dict_data (v0 UInt16, v1 UInt16, v2 UInt16, v3 UInt16, v4 UInt16) engine=Memory() AS SELECT number, number%65535, number%65535, number%6553, number%655355 FROM numbers(100); + +CREATE DICTIONARY dict_flat_simple (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(flat()); +SYSTEM RELOAD DICTIONARY dict_flat_simple; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_flat_simple'; +DROP DICTIONARY dict_flat_simple; + +CREATE DICTIONARY dict_hashed_simple (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple'; +DROP DICTIONARY dict_hashed_simple; + +CREATE DICTIONARY dict_hashed_complex (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0,v1 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(complex_key_hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_complex; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_complex'; +DROP DICTIONARY dict_hashed_complex; + +CREATE DICTIONARY dict_hashed_simple_auto_convert (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0,v1 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple_auto_convert; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_auto_convert'; +DROP DICTIONARY dict_hashed_simple_auto_convert; + +DROP TABLE dict_data; From 7f9b21849c4ac6dc9243426ae885e3e6b7208d6f Mon Sep 17 00:00:00 2001 From: xiebin Date: Sun, 7 May 2023 19:06:06 +0800 Subject: [PATCH 0053/2047] Fixed a lowercase initial letter and removed needless data --- src/Dictionaries/DictionaryFactory.h | 2 +- ...uto_convert_dictionary_layout_to_complex_by_complex_keys.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/DictionaryFactory.h b/src/Dictionaries/DictionaryFactory.h index 309f14f6b7f..35097a5ed24 100644 --- a/src/Dictionaries/DictionaryFactory.h +++ b/src/Dictionaries/DictionaryFactory.h @@ -55,7 +55,7 @@ public: bool isComplex(const std::string & layout_type) const; - /// if the argument `layout_type` is not complex layout and has corresponding complex layout, + /// If the argument `layout_type` is not complex layout and has corresponding complex layout, /// change `layout_type` to corresponding complex and return true; otherwise do nothing and return false. bool convertToComplex(std::string & layout_type) const; diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql index 98c7c5c05f9..0fb06e5acc2 100644 --- a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql @@ -4,7 +4,7 @@ DROP DICTIONARY IF EXISTS dict_hashed_complex; DROP DICTIONARY IF EXISTS dict_hashed_simple_auto_convert; DROP TABLE IF EXISTS dict_data; -CREATE TABLE dict_data (v0 UInt16, v1 UInt16, v2 UInt16, v3 UInt16, v4 UInt16) engine=Memory() AS SELECT number, number%65535, number%65535, number%6553, number%655355 FROM numbers(100); +CREATE TABLE dict_data (v0 UInt16, v1 UInt16, v2 UInt16, v3 UInt16, v4 UInt16) engine=Memory(); CREATE DICTIONARY dict_flat_simple (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(flat()); SYSTEM RELOAD DICTIONARY dict_flat_simple; From 8c0b634a644ac85832658cc6ca863909ef455795 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 7 May 2023 14:38:27 +0200 Subject: [PATCH 0054/2047] Update storage_conf.xml --- tests/config/config.d/storage_conf.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 923240d5a91..aad93a017c2 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -72,7 +72,7 @@ cache - s3_cache_5 + s3_cache s3_cache_multi/ 22548578304 0 From 726222f1ea69018115642156a06c64ec546244d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:33:11 +0200 Subject: [PATCH 0055/2047] Fix tests --- tests/queries/0_stateless/00956_sensitive_data_masking.sh | 2 +- tests/queries/0_stateless/01107_atomic_db_detach_attach.sh | 4 ++-- tests/queries/0_stateless/01114_database_atomic.sh | 6 +++--- .../queries/0_stateless/01192_rename_database_zookeeper.sh | 4 ++-- tests/queries/0_stateless/01238_http_memory_tracking.sh | 2 +- tests/queries/0_stateless/01338_long_select_and_alter.sh | 2 +- .../0_stateless/01338_long_select_and_alter_zookeeper.sh | 2 +- 7 files changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/00956_sensitive_data_masking.sh b/tests/queries/0_stateless/00956_sensitive_data_masking.sh index a31a71ce381..926557e4ba6 100755 --- a/tests/queries/0_stateless/00956_sensitive_data_masking.sh +++ b/tests/queries/0_stateless/00956_sensitive_data_masking.sh @@ -65,7 +65,7 @@ echo 5 # run in background rm -f "$tmp_file2" >/dev/null 2>&1 bash -c "$CLICKHOUSE_CLIENT \ - --function_sleep_max_microseconds_per_block 60 \ + --function_sleep_max_microseconds_per_block 60000000 \ --query=\"select sleepEachRow(1) from numbers(10) where ignore('find_me_TOPSECRET=TOPSECRET')=0 and ignore('fwerkh_that_magic_string_make_me_unique') = 0 FORMAT Null\" \ --log_queries=1 --ignore-error --multiquery |& grep -v '^(query: ' > $tmp_file2" & diff --git a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh index e2a23258584..bcaa70abbb5 100755 --- a/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh +++ b/tests/queries/0_stateless/01107_atomic_db_detach_attach.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01107" $CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01107 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01107.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(3) FROM numbers(5)" & sleep 1 $CLICKHOUSE_CLIENT -q "DETACH TABLE test_01107.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 @@ -23,7 +23,7 @@ $CLICKHOUSE_CLIENT -q "DETACH DATABASE test_01107" --database_atomic_wait_for_dr $CLICKHOUSE_CLIENT -q "ATTACH DATABASE test_01107" $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01107.mt" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01107.mt SELECT number + sleepEachRow(1) FROM numbers(5)" && echo "end" & sleep 1 $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01107" --database_atomic_wait_for_drop_and_detach_synchronously=0 && sleep 1 && echo "dropped" wait diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 634b19a7624..decbe136fc4 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -49,8 +49,8 @@ $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW $CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s (1.5s * 30 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 33s (1.5s * 22 rows per partition), result: 110, 5995 +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s (1.5s * 30 rows) sleep 1 # SELECT and INSERT should start before the following RENAMEs $CLICKHOUSE_CLIENT -nm -q " @@ -74,7 +74,7 @@ INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5); SELECT count() FROM test_01114_1.mt " # result: 5 -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s (3s * 5 rows) +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s (3s * 5 rows) sleep 1 $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped" diff --git a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh index ac516e83c84..6dd7ff3cdc8 100755 --- a/tests/queries/0_stateless/01192_rename_database_zookeeper.sh +++ b/tests/queries/0_stateless/01192_rename_database_zookeeper.sh @@ -20,7 +20,7 @@ $CLICKHOUSE_CLIENT -q "SELECT engine, splitByChar('/', data_path)[-2], uuid, spl # 3. check RENAME don't wait for INSERT $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01192.mt (n UInt64) ENGINE=MergeTree ORDER BY n" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 15 -q "INSERT INTO test_01192.mt SELECT number + sleepEachRow(1.5) FROM numbers(10)" && echo "inserted" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 15000000 -q "INSERT INTO test_01192.mt SELECT number + sleepEachRow(1.5) FROM numbers(10)" && echo "inserted" & sleep 1 $CLICKHOUSE_CLIENT -q "RENAME DATABASE test_01192 TO default" 2>&1| grep -F "already exists" > /dev/null && echo "ok" @@ -60,7 +60,7 @@ $CLICKHOUSE_CLIENT -q "SELECT database, name, status, origin FROM system.diction $CLICKHOUSE_CLIENT -q "SELECT dictGet('test_01192_atomic.dict', '_part', toUInt64(1))" # 8. check RENAME don't wait for INSERT -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 -q "INSERT INTO test_01192_atomic.mt SELECT number + sleepEachRow(1) + 10 FROM numbers(10)" && echo "inserted" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10000000 -q "INSERT INTO test_01192_atomic.mt SELECT number + sleepEachRow(1) + 10 FROM numbers(10)" && echo "inserted" & sleep 1 $CLICKHOUSE_CLIENT --check_table_dependencies=0 -q "RENAME DATABASE test_01192 TO test_01192_renamed" 2>&1| grep -F "not supported" > /dev/null && echo "ok" diff --git a/tests/queries/0_stateless/01238_http_memory_tracking.sh b/tests/queries/0_stateless/01238_http_memory_tracking.sh index eb42159ce15..26d3dd8acd4 100755 --- a/tests/queries/0_stateless/01238_http_memory_tracking.sh +++ b/tests/queries/0_stateless/01238_http_memory_tracking.sh @@ -10,7 +10,7 @@ set -o pipefail # This is needed to keep at least one running query for user for the time of test. # (1k http queries takes ~1 second, let's run for 5x more to avoid flaps) -${CLICKHOUSE_CLIENT} --function_sleep_max_microseconds_per_block 5 --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)' & +${CLICKHOUSE_CLIENT} --function_sleep_max_microseconds_per_block 5000000 --format Null -n <<<'SELECT sleepEachRow(1) FROM numbers(5)' & # ignore "yes: standard output: Broken pipe" yes 'SELECT 1' 2>/dev/null | { diff --git a/tests/queries/0_stateless/01338_long_select_and_alter.sh b/tests/queries/0_stateless/01338_long_select_and_alter.sh index 04a10cfe55e..fcdfa2dec82 100755 --- a/tests/queries/0_stateless/01338_long_select_and_alter.sh +++ b/tests/queries/0_stateless/01338_long_select_and_alter.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENG $CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10000000 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & # to be sure that select took all required locks sleep 2 diff --git a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh index 829352110f6..50ade3fad45 100755 --- a/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh +++ b/tests/queries/0_stateless/01338_long_select_and_alter_zookeeper.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE alter_mt (key UInt64, value String) ENG $CLICKHOUSE_CLIENT --query "INSERT INTO alter_mt SELECT number, toString(number) FROM numbers(5)" -$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & +$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 10000000 --query "SELECT count(distinct concat(value, '_')) FROM alter_mt WHERE not sleepEachRow(2)" & # to be sure that select took all required locks sleep 2 From e159ee84e918c587f873a27665ca346cb3b4f7db Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:38:30 +0200 Subject: [PATCH 0056/2047] Fix tests --- .../0_stateless/01098_temporary_and_external_tables.sh | 2 +- .../01532_execute_merges_on_single_replica_long.sql | 2 +- tests/queries/0_stateless/02473_optimize_old_parts.sh | 2 +- tests/queries/0_stateless/02530_dictionaries_update_field.sh | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01098_temporary_and_external_tables.sh b/tests/queries/0_stateless/01098_temporary_and_external_tables.sh index 860529a26e5..9ed78fd9f81 100755 --- a/tests/queries/0_stateless/01098_temporary_and_external_tables.sh +++ b/tests/queries/0_stateless/01098_temporary_and_external_tables.sh @@ -25,7 +25,7 @@ echo "SELECT COUNT() FROM $internal_table_name" | ${CLICKHOUSE_CURL} -m 60 -sSgk echo -ne '0\n1\n' | ${CLICKHOUSE_CURL} -m 30 -sSkF 'file=@-' "$url&file_format=CSV&file_types=UInt64&query=SELECT+sum((number+GLOBAL+IN+(SELECT+number+AS+n+FROM+remote('127.0.0.2',+numbers(5))+WHERE+n+GLOBAL+IN+(SELECT+*+FROM+tmp_table)+AND+n+GLOBAL+NOT+IN+(SELECT+*+FROM+file)+))+AS+res),+sum(number*res)+FROM+remote('127.0.0.2',+numbers(10))" -echo -ne '0\n1\n' | ${CLICKHOUSE_CURL} -m 30 -sSkF 'file=@-' "$url&file_format=CSV&file_types=UInt64&query=SELECT+_1%2BsleepEachRow(3)+FROM+file" & +echo -ne '0\n1\n' | ${CLICKHOUSE_CURL} -m 30 -sSkF 'file=@-' "$url&function_sleep_max_microseconds_per_block=0&file_format=CSV&file_types=UInt64&query=SELECT+_1%2BsleepEachRow(3)+FROM+file" & wait ${CLICKHOUSE_CURL} -m 30 -sSk "$url" --data "DROP TEMPORARY TABLE tmp_table" diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql index 4bd5e79d1b3..30beb29251e 100644 --- a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql +++ b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql @@ -44,7 +44,7 @@ SYSTEM STOP REPLICATION QUEUES execute_on_single_replica_r2; OPTIMIZE TABLE execute_on_single_replica_r1 FINAL SETTINGS replication_alter_partitions_sync=0; /* if we will check immediately we can find the log entry unchecked */ -SET function_sleep_max_microseconds_per_block = 4000000; +SET function_sleep_max_microseconds_per_block = 10000000; SELECT * FROM numbers(4) where sleepEachRow(1); SELECT '****************************'; diff --git a/tests/queries/0_stateless/02473_optimize_old_parts.sh b/tests/queries/0_stateless/02473_optimize_old_parts.sh index 0c2dd04d024..b563bc31b39 100755 --- a/tests/queries/0_stateless/02473_optimize_old_parts.sh +++ b/tests/queries/0_stateless/02473_optimize_old_parts.sh @@ -61,7 +61,7 @@ INSERT INTO test_with_merge SELECT 3;" wait_for_number_of_parts 'test_with_merge' 1 100 $CLICKHOUSE_CLIENT -nmq " -SELECT sleepEachRow(1) FROM numbers(9) FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one +SELECT sleepEachRow(1) FROM numbers(9) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one SELECT (now() - modification_time) > 5 FROM system.parts WHERE database = currentDatabase() AND table='test_with_merge' AND active; DROP TABLE test_with_merge;" diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.sh b/tests/queries/0_stateless/02530_dictionaries_update_field.sh index 569466fe606..44000e5d2cd 100755 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.sh +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.sh @@ -53,13 +53,13 @@ for layout in "${layouts[@]}"; do SELECT key, value FROM $dictionary_name ORDER BY key ASC; INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); - SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; + SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM $dictionary_name ORDER BY key ASC; INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); - SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; + SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM $dictionary_name ORDER BY key ASC; -- { echoOff } From fbda7974a5424b79a952fa30b16b7cd3c390bdc8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:39:23 +0200 Subject: [PATCH 0057/2047] Fix tests --- .../queries/0_stateless/02676_optimize_old_parts_replicated.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02676_optimize_old_parts_replicated.sh b/tests/queries/0_stateless/02676_optimize_old_parts_replicated.sh index 2202a349c56..c1f28f9f079 100755 --- a/tests/queries/0_stateless/02676_optimize_old_parts_replicated.sh +++ b/tests/queries/0_stateless/02676_optimize_old_parts_replicated.sh @@ -61,7 +61,7 @@ INSERT INTO test_replicated SELECT 3;" wait_for_number_of_parts 'test_replicated' 1 100 $CLICKHOUSE_CLIENT -nmq " -SELECT sleepEachRow(1) FROM numbers(9) FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one +SELECT sleepEachRow(1) FROM numbers(9) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one SELECT (now() - modification_time) > 5 FROM system.parts WHERE database = currentDatabase() AND table='test_replicated' AND active; DROP TABLE test_replicated;" From 08a9d97de74a27bd28d7cc387d7f5cdba707d6cb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:40:43 +0200 Subject: [PATCH 0058/2047] Fix tests --- tests/queries/0_stateless/02352_rwlock.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02352_rwlock.sh b/tests/queries/0_stateless/02352_rwlock.sh index 7a0b9ef8911..7505a03a382 100755 --- a/tests/queries/0_stateless/02352_rwlock.sh +++ b/tests/queries/0_stateless/02352_rwlock.sh @@ -51,7 +51,7 @@ while :; do insert_query_id="insert-$(random_str 10)" # 20 seconds sleep - $CLICKHOUSE_CLIENT --query_id "$insert_query_id" -q "INSERT INTO ${CLICKHOUSE_DATABASE}_ordinary.data_02352 SELECT sleepEachRow(1) FROM numbers(20) GROUP BY number" & + $CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 20000000 --query_id "$insert_query_id" -q "INSERT INTO ${CLICKHOUSE_DATABASE}_ordinary.data_02352 SELECT sleepEachRow(1) FROM numbers(20) GROUP BY number" & if ! wait_query_by_id_started "$insert_query_id"; then wait continue From 0818092ae8d49f2e7f87fed6c8703374384719fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 19:45:57 +0200 Subject: [PATCH 0059/2047] Enable Sparse columns by default --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 5416b77a97e..27f482d79ba 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -37,7 +37,7 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Float, ratio_of_defaults_for_sparse_serialization, 1.0, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ + M(Float, ratio_of_defaults_for_sparse_serialization, 0.95, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \ From 7ec98205b58ab36eb28b2f46348dfcfe22215a3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 7 May 2023 22:54:14 +0300 Subject: [PATCH 0060/2047] Update MergeTreeSettings.h --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 27f482d79ba..caac86c6706 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -37,7 +37,7 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Float, ratio_of_defaults_for_sparse_serialization, 0.95, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ + M(Float, ratio_of_defaults_for_sparse_serialization, 0.9375f, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \ From f3f6ccd7733aa4946c339b4973210f85243e44d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:28:54 +0200 Subject: [PATCH 0061/2047] Update tests --- .../0_stateless/00443_preferred_block_size_bytes.sh | 6 +++--- ...0484_preferred_max_column_in_block_size_bytes.sql | 8 ++++---- .../00804_test_delta_codec_compression.sql | 12 ++++++------ .../0_stateless/00950_test_double_delta_codec.sql | 2 +- ...00961_checksums_in_system_parts_columns_table.sql | 2 +- .../0_stateless/01055_compact_parts_granularity.sh | 2 +- .../queries/0_stateless/01786_explain_merge_tree.sh | 4 ++-- tests/queries/0_stateless/02263_lazy_mark_load.sh | 2 +- .../0_stateless/02293_selected_rows_and_merges.sh | 8 +++----- .../0_stateless/02361_fsync_profile_events.sh | 7 ++++--- .../02381_compress_marks_and_primary_key.sql | 4 ++-- 11 files changed, 28 insertions(+), 29 deletions(-) diff --git a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh index c184b58bf53..27b9f5c00c7 100755 --- a/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh +++ b/tests/queries/0_stateless/00443_preferred_block_size_bytes.sh @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" -$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE preferred_block_size_bytes (p Date, s String) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=1, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "INSERT INTO preferred_block_size_bytes (s) SELECT '16_bytes_-_-_-_' AS s FROM system.numbers LIMIT 10, 90" $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE preferred_block_size_bytes" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=26 -q "SELECT DISTINCT blockSize(), ignore(p, s) FROM preferred_block_size_bytes" @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS preferred_block_size_bytes" # PREWHERE using empty column $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS pbs" -$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE pbs (p Date, i UInt64, sa Array(String)) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=100, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "INSERT INTO pbs (p, i, sa) SELECT toDate(i % 30) AS p, number AS i, ['a'] AS sa FROM system.numbers LIMIT 1000" $CLICKHOUSE_CLIENT -q "ALTER TABLE pbs ADD COLUMN s UInt8 DEFAULT 0" $CLICKHOUSE_CLIENT --preferred_block_size_bytes=100000 -q "SELECT count() FROM pbs PREWHERE s = 0" @@ -30,7 +30,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE pbs" # Nullable PREWHERE $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" -$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE nullable_prewhere (p Date, f Nullable(UInt64), d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY p SETTINGS index_granularity=8, index_granularity_bytes=0, min_bytes_for_wide_part = 0, min_rows_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "INSERT INTO nullable_prewhere SELECT toDate(0) AS p, if(number % 2 = 0, CAST(number AS Nullable(UInt64)), CAST(NULL AS Nullable(UInt64))) AS f, number as d FROM system.numbers LIMIT 1001" $CLICKHOUSE_CLIENT -q "SELECT sum(d), sum(f), max(d) FROM nullable_prewhere PREWHERE NOT isNull(f)" $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS nullable_prewhere" diff --git a/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql b/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql index 470bca70e06..be4af2221a5 100644 --- a/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql +++ b/tests/queries/0_stateless/00484_preferred_max_column_in_block_size_bytes.sql @@ -1,7 +1,7 @@ -- Tags: no-random-settings drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 8192; set preferred_block_size_bytes = 2000000; @@ -17,19 +17,19 @@ set preferred_max_column_in_block_size_bytes = 4194304; select max(blockSize()), min(blockSize()), any(ignore(*)) from tab_00484; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 47; set preferred_max_column_in_block_size_bytes = 1152; select blockSize(), * from tab_00484 where x = 1 or x > 36 format Null; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s FixedString(128)) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, toFixedString('', 128) from system.numbers limit 10; set preferred_max_column_in_block_size_bytes = 128; select s from tab_00484 where s == '' format Null; drop table if exists tab_00484; -create table tab_00484 (date Date, x UInt64, s String) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0; +create table tab_00484 (date Date, x UInt64, s String) engine = MergeTree PARTITION BY date ORDER BY (date, x) SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; insert into tab_00484 select today(), number, 'abc' from system.numbers limit 81920; set preferred_block_size_bytes = 0; select count(*) from tab_00484 prewhere s != 'abc' format Null; diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index 25988f6474b..01a2f53bf93 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -9,12 +9,12 @@ DROP TABLE IF EXISTS default_codec_synthetic; CREATE TABLE delta_codec_synthetic ( id UInt64 Codec(Delta, ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; CREATE TABLE default_codec_synthetic ( id UInt64 Codec(ZSTD(3)) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO delta_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; INSERT INTO default_codec_synthetic SELECT number FROM system.numbers LIMIT 5000000; @@ -47,12 +47,12 @@ DROP TABLE IF EXISTS default_codec_float; CREATE TABLE delta_codec_float ( id Float64 Codec(Delta, LZ4HC) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; CREATE TABLE default_codec_float ( id Float64 Codec(LZ4HC) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO delta_codec_float SELECT number FROM numbers(1547510400, 500000) WHERE number % 3 == 0 OR number % 5 == 0 OR number % 7 == 0 OR number % 11 == 0; INSERT INTO default_codec_float SELECT * from delta_codec_float; @@ -85,12 +85,12 @@ DROP TABLE IF EXISTS default_codec_string; CREATE TABLE delta_codec_string ( id Float64 Codec(Delta, LZ4) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; CREATE TABLE default_codec_string ( id Float64 Codec(LZ4) -) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key=false; +) ENGINE MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO delta_codec_string SELECT concat(toString(number), toString(number % 100)) FROM numbers(1547510400, 500000); INSERT INTO default_codec_string SELECT * from delta_codec_string; diff --git a/tests/queries/0_stateless/00950_test_double_delta_codec.sql b/tests/queries/0_stateless/00950_test_double_delta_codec.sql index f6199a6e4ec..58cf35b5248 100644 --- a/tests/queries/0_stateless/00950_test_double_delta_codec.sql +++ b/tests/queries/0_stateless/00950_test_double_delta_codec.sql @@ -24,7 +24,7 @@ CREATE TABLE codecTest ( valueI8 Int8 CODEC(DoubleDelta), valueDT DateTime CODEC(DoubleDelta), valueD Date CODEC(DoubleDelta) -) Engine = MergeTree ORDER BY key SETTINGS min_bytes_for_wide_part = 0; +) Engine = MergeTree ORDER BY key SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; -- checking for overflow diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql index 43b7775e816..8df7d728560 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS test_00961; CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) ENGINE = MergeTree PARTITION BY d ORDER BY (a, b) - SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks = 0, compress_primary_key = 0, index_granularity_bytes = '10Mi'; + SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks = 0, compress_primary_key = 0, index_granularity_bytes = '10Mi', ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); diff --git a/tests/queries/0_stateless/01055_compact_parts_granularity.sh b/tests/queries/0_stateless/01055_compact_parts_granularity.sh index f3da33f6ccf..3e5da1e6f90 100755 --- a/tests/queries/0_stateless/01055_compact_parts_granularity.sh +++ b/tests/queries/0_stateless/01055_compact_parts_granularity.sh @@ -11,7 +11,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mt_compact" $CLICKHOUSE_CLIENT -q "CREATE TABLE mt_compact(a Int, s String) ENGINE = MergeTree ORDER BY a SETTINGS min_rows_for_wide_part = 1000, - index_granularity = 14;" + index_granularity = 14, ratio_of_defaults_for_sparse_serialization = 1;" $CLICKHOUSE_CLIENT -q "SYSTEM STOP MERGES mt_compact" diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.sh b/tests/queries/0_stateless/01786_explain_merge_tree.sh index 15f8821d80d..0d4acba338a 100755 --- a/tests/queries/0_stateless/01786_explain_merge_tree.sh +++ b/tests/queries/0_stateless/01786_explain_merge_tree.sh @@ -10,7 +10,7 @@ CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --optimize_move_to_prewhere=1 --convert_qu $CLICKHOUSE_CLIENT -q "drop table if exists test_index" $CLICKHOUSE_CLIENT -q "drop table if exists idx" -$CLICKHOUSE_CLIENT -q "create table test_index (x UInt32, y UInt32, z UInt32, t UInt32, index t_minmax t % 20 TYPE minmax GRANULARITY 2, index t_set t % 19 type set(4) granularity 2) engine = MergeTree order by (x, y) partition by (y, bitAnd(z, 3), intDiv(t, 15)) settings index_granularity = 2, min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "create table test_index (x UInt32, y UInt32, z UInt32, t UInt32, index t_minmax t % 20 TYPE minmax GRANULARITY 2, index t_set t % 19 type set(4) granularity 2) engine = MergeTree order by (x, y) partition by (y, bitAnd(z, 3), intDiv(t, 15)) settings index_granularity = 2, min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "insert into test_index select number, number > 3 ? 3 : number, number = 1 ? 1 : 0, number from numbers(20)" $CLICKHOUSE_CLIENT -q " @@ -35,7 +35,7 @@ $CLICKHOUSE_CLIENT -q " explain actions = 1 select x from test_index where x > 15 order by x desc; " | grep -A 100 "ReadFromMergeTree" -$CLICKHOUSE_CLIENT -q "CREATE TABLE idx (x UInt32, y UInt32, z UInt32) ENGINE = MergeTree ORDER BY (x, x + y) settings min_bytes_for_wide_part = 0" +$CLICKHOUSE_CLIENT -q "CREATE TABLE idx (x UInt32, y UInt32, z UInt32) ENGINE = MergeTree ORDER BY (x, x + y) settings min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT -q "insert into idx select number, number, number from numbers(10)" $CLICKHOUSE_CLIENT -q " diff --git a/tests/queries/0_stateless/02263_lazy_mark_load.sh b/tests/queries/0_stateless/02263_lazy_mark_load.sh index bf37556bfa6..35a1b4a44dd 100755 --- a/tests/queries/0_stateless/02263_lazy_mark_load.sh +++ b/tests/queries/0_stateless/02263_lazy_mark_load.sh @@ -24,7 +24,7 @@ CREATE TABLE lazy_mark_test n9 UInt64 ) ENGINE = MergeTree -ORDER BY n0 SETTINGS min_bytes_for_wide_part = 0; +ORDER BY n0 SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1; EOF ${CLICKHOUSE_CLIENT} -q "SYSTEM STOP MERGES lazy_mark_test" diff --git a/tests/queries/0_stateless/02293_selected_rows_and_merges.sh b/tests/queries/0_stateless/02293_selected_rows_and_merges.sh index 9d1483f5bf7..76c562c9744 100755 --- a/tests/queries/0_stateless/02293_selected_rows_and_merges.sh +++ b/tests/queries/0_stateless/02293_selected_rows_and_merges.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) query_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString(generateUUIDv4()))))") -${CLICKHOUSE_CLIENT} -q "create table tt (x UInt32, y UInt32) engine = MergeTree order by x" +${CLICKHOUSE_CLIENT} -q "create table tt (x UInt32, y UInt32) engine = MergeTree order by x SETTINGS ratio_of_defaults_for_sparse_serialization = 1" ${CLICKHOUSE_CLIENT} -q "insert into tt select number, 0 from numbers(1e6)" ${CLICKHOUSE_CLIENT} -q "insert into tt select number, 1 from numbers(1e6)" @@ -17,13 +17,11 @@ ${CLICKHOUSE_CLIENT} --optimize_throw_if_noop 1 -q "optimize table tt final" "-- # Here SelectRows and SelectBytes should be zero, MergedRows is 2m and MergedUncompressedBytes is 16m ${CLICKHOUSE_CLIENT} -q "system flush logs" -${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'], ProfileEvents['SelecteBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'optimize%' and current_database = currentDatabase()" +${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'], ProfileEvents['SelectedBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'optimize%' and current_database = currentDatabase()" ${CLICKHOUSE_CLIENT} --mutations_sync 1 -q "alter table tt update y = y + 1 where 1" "--query_id=$query_id" ${CLICKHOUSE_CLIENT} -q "system flush logs" # Here for mutation all values are 0, cause mutation is executed async. # It's pretty hard to write a test with total counter. -${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'] > 10, ProfileEvents['SelecteBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'alter%' and current_database = currentDatabase()" - - +${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'] > 10, ProfileEvents['SelectedBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'alter%' and current_database = currentDatabase()" diff --git a/tests/queries/0_stateless/02361_fsync_profile_events.sh b/tests/queries/0_stateless/02361_fsync_profile_events.sh index 5b603133f6c..e150d70b896 100755 --- a/tests/queries/0_stateless/02361_fsync_profile_events.sh +++ b/tests/queries/0_stateless/02361_fsync_profile_events.sh @@ -12,9 +12,10 @@ $CLICKHOUSE_CLIENT -nm -q " create table data_fsync_pe (key Int) engine=MergeTree() order by key settings - min_rows_for_wide_part=2, - fsync_after_insert=1, - fsync_part_directory=1; + min_rows_for_wide_part = 2, + fsync_after_insert = 1, + fsync_part_directory = 1, + ratio_of_defaults_for_sparse_serialization = 1; " ret=1 diff --git a/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql b/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql index 842e22ba87d..2fe0943745d 100644 --- a/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql +++ b/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql @@ -1,12 +1,12 @@ -- Tags: no-upgrade-check, no-random-merge-tree-settings drop table if exists test_02381; -create table test_02381(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) SETTINGS compress_marks=false, compress_primary_key=false; +create table test_02381(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) SETTINGS compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; insert into test_02381 select number, number * 10 from system.numbers limit 1000000; drop table if exists test_02381_compress; create table test_02381_compress(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) - SETTINGS compress_marks=true, compress_primary_key=true, marks_compression_codec='ZSTD(3)', primary_key_compression_codec='ZSTD(3)', marks_compress_block_size=65536, primary_key_compress_block_size=65536; + SETTINGS compress_marks = true, compress_primary_key = true, marks_compression_codec = 'ZSTD(3)', primary_key_compression_codec = 'ZSTD(3)', marks_compress_block_size = 65536, primary_key_compress_block_size = 65536, ratio_of_defaults_for_sparse_serialization = 1; insert into test_02381_compress select number, number * 10 from system.numbers limit 1000000; select * from test_02381_compress where a = 1000 limit 1; From e8f7a84ca6c4e00f6f9ddbf282b109f491244c4c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:37:10 +0200 Subject: [PATCH 0062/2047] Update a few tests --- tests/queries/0_stateless/01375_compact_parts_codecs.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01375_compact_parts_codecs.sql b/tests/queries/0_stateless/01375_compact_parts_codecs.sql index 1dd39e67876..1c89eb09d0b 100644 --- a/tests/queries/0_stateless/01375_compact_parts_codecs.sql +++ b/tests/queries/0_stateless/01375_compact_parts_codecs.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS codecs; CREATE TABLE codecs (id UInt32, val UInt32, s String) ENGINE = MergeTree ORDER BY id - SETTINGS min_rows_for_wide_part = 10000; + SETTINGS min_rows_for_wide_part = 10000, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts @@ -21,7 +21,7 @@ DROP TABLE codecs; CREATE TABLE codecs (id UInt32 CODEC(NONE), val UInt32 CODEC(NONE), s String CODEC(NONE)) ENGINE = MergeTree ORDER BY id - SETTINGS min_rows_for_wide_part = 10000; + SETTINGS min_rows_for_wide_part = 10000, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts @@ -38,7 +38,7 @@ DROP TABLE codecs; CREATE TABLE codecs (id UInt32, val UInt32 CODEC(Delta, ZSTD), s String CODEC(ZSTD)) ENGINE = MergeTree ORDER BY id - SETTINGS min_rows_for_wide_part = 10000; + SETTINGS min_rows_for_wide_part = 10000, ratio_of_defaults_for_sparse_serialization = 1; INSERT INTO codecs SELECT number, number, toString(number) FROM numbers(1000); SELECT sum(data_compressed_bytes), sum(data_uncompressed_bytes) FROM system.parts From 7c03801bf7da6803e47f57ab78478c33a9c9a764 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:54:57 +0200 Subject: [PATCH 0063/2047] Update a test --- tests/queries/0_stateless/02725_parquet_preserve_order.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02725_parquet_preserve_order.sh b/tests/queries/0_stateless/02725_parquet_preserve_order.sh index ea3e4219e35..ac29ef3f361 100755 --- a/tests/queries/0_stateless/02725_parquet_preserve_order.sh +++ b/tests/queries/0_stateless/02725_parquet_preserve_order.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # It'll be read into two blocks. The first block will sleep 2x longer than the second. # So reordering is very likely if the order-preservation doesn't work. -$CLICKHOUSE_LOCAL -q "select number+sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1" +$CLICKHOUSE_LOCAL -q "select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1, function_sleep_max_microseconds_per_block = 6000000" -$CLICKHOUSE_LOCAL -q "explain pipeline select number+sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1, max_threads=2" -$CLICKHOUSE_LOCAL -q "explain pipeline select number+sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=0, parallelize_output_from_storages=1, max_threads=2" +$CLICKHOUSE_LOCAL -q "explain pipeline select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1, max_threads=2" +$CLICKHOUSE_LOCAL -q "explain pipeline select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=0, parallelize_output_from_storages=1, max_threads=2" From a25de5fb4186fbe103f916b07aa8bd89975048b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 00:55:44 +0200 Subject: [PATCH 0064/2047] Update a test --- .../02530_dictionaries_update_field.reference | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.reference b/tests/queries/0_stateless/02530_dictionaries_update_field.reference index 40f2c0ee400..88c910e0313 100644 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.reference +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.reference @@ -4,13 +4,13 @@ flat SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 SecondUpdated @@ -21,13 +21,13 @@ flat/custom SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -38,13 +38,13 @@ hashed SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -55,13 +55,13 @@ hashed/custom SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -72,13 +72,13 @@ complex_key_hashed SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -89,13 +89,13 @@ complex_key_hashed/custom SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated From 63b559df17a07e42768c4425538426e245d829fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 8 May 2023 06:49:41 +0200 Subject: [PATCH 0065/2047] Update a test --- .../02530_dictionaries_update_field.reference | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.reference b/tests/queries/0_stateless/02530_dictionaries_update_field.reference index 40f2c0ee400..88c910e0313 100644 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.reference +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.reference @@ -4,13 +4,13 @@ flat SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 SecondUpdated @@ -21,13 +21,13 @@ flat/custom SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -38,13 +38,13 @@ hashed SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -55,13 +55,13 @@ hashed/custom SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -72,13 +72,13 @@ complex_key_hashed SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -89,13 +89,13 @@ complex_key_hashed/custom SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated From a8d56b2290cf0ada1a3598f295ed45575778afe4 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 8 May 2023 10:26:52 +0300 Subject: [PATCH 0066/2047] fix ordered mode + more tests --- src/Storages/S3Queue/S3QueueHolder.cpp | 206 +++++++++++--- src/Storages/S3Queue/S3QueueHolder.h | 36 ++- src/Storages/S3Queue/S3QueueSettings.h | 3 +- src/Storages/S3Queue/S3QueueSource.cpp | 85 +++--- src/Storages/S3Queue/S3QueueSource.h | 45 ++- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 10 +- src/Storages/S3Queue/S3QueueTableMetadata.h | 7 +- src/Storages/S3Queue/StorageS3Queue.cpp | 108 ++++---- .../integration/test_storage_s3_queue/test.py | 262 +++++++++++++++++- 9 files changed, 593 insertions(+), 169 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 119cbe8e6e4..21b2b7a45ce 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -1,11 +1,15 @@ #include "config.h" #if USE_AWS_S3 -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include + namespace DB { @@ -14,16 +18,99 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int NO_ZOOKEEPER; + extern const int TIMEOUT_EXCEEDED; } -S3QueueHolder::S3QueueHolder(const String & zookeeper_path_, const S3QueueMode & mode_, const UUID & table_uuid_, ContextPtr context_) +S3QueueHolder::ProcessedCollection::ProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) + : max_size(max_size_), max_age(max_age_) +{ +} + +void S3QueueHolder::ProcessedCollection::read(ReadBuffer & in) +{ + files = {}; + in >> "processed_files\n"; + while (!in.eof()) + { + String file_name; + Int64 timestamp; + in >> file_name >> "\n"; + in >> timestamp >> "\n"; + auto pair = std::make_pair(file_name, timestamp); + files.push_back(pair); + } +} + +void S3QueueHolder::ProcessedCollection::write(WriteBuffer & out) const +{ + out << "processed_files\n"; + for (const auto & processed_file : files) + { + out << processed_file.first << "\n"; + out << processed_file.second << "\n"; + } +} + +void S3QueueHolder::ProcessedCollection::parse(const String & s) +{ + ReadBufferFromString buf(s); + read(buf); + // Remove old items + if (max_age > 0) + { + Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + Int64 max_seconds_diff = max_age; + auto new_end = std::remove_if( + files.begin(), + files.end(), + [×tamp, &max_seconds_diff](std::pair processed_file) + { return (timestamp - processed_file.second) > max_seconds_diff; }); + files.erase(new_end, files.end()); + } +} + +String S3QueueHolder::ProcessedCollection::toString() const +{ + WriteBufferFromOwnString out; + write(out); + return out.str(); +} + +void S3QueueHolder::ProcessedCollection::add(const String & file_name) +{ + Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + auto pair = std::make_pair(file_name, timestamp); + files.push_back(pair); + + // Check set size + if (files.size() > max_size) + { + files.erase(files.begin(), files.begin() + (files.size() - max_size)); + } +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::ProcessedCollection::getFileNames() +{ + S3FilesCollection keys = {}; + for (auto & pair : files) + { + keys.insert(pair.first); + } + return keys; +} + + +S3QueueHolder::S3QueueHolder( + const String & zookeeper_path_, const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, UInt64 & max_set_age_s_) : WithContext(context_) + , max_set_size(max_set_size_) + , max_set_age_s(max_set_age_s_) , zookeeper_path(zookeeper_path_) , zookeeper_failed_path(zookeeper_path_ + "/failed") , zookeeper_processing_path(zookeeper_path_ + "/processing") , zookeeper_processed_path(zookeeper_path_ + "/processed") + , zookeeper_lock_path(zookeeper_path_ + "/lock") , mode(mode_) - , table_uuid(table_uuid_) , log(&Poco::Logger::get("S3QueueHolder")) { current_zookeeper = getContext()->getZooKeeper(); @@ -48,31 +135,47 @@ zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const void S3QueueHolder::setFileProcessed(const String & file_path) { auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); + auto lock = AcquireLock(); - switch (mode) + if (mode == S3QueueMode::UNORDERED) { - case S3QueueMode::UNORDERED: { - String processed_files = zookeeper->get(zookeeper_processed_path); - S3FilesCollection processed = parseCollection(processed_files); - processed.insert(file_path); - Strings set_processed; - set_processed.insert(set_processed.end(), processed.begin(), processed.end()); - zookeeper->set(zookeeper_processed_path, toString(set_processed)); - break; - } - case S3QueueMode::ORDERED: { + String processed_files = zookeeper->get(zookeeper_processed_path); + auto processed = ProcessedCollection(max_set_size, max_set_age_s); + processed.parse(processed_files); + processed.add(file_path); + zookeeper->set(zookeeper_processed_path, processed.toString()); + } + else + { + String max_file = getMaxProcessedFile(); + if (max_file.compare(file_path) <= 0) + { zookeeper->set(zookeeper_processed_path, file_path); } } + + String node_data; + Strings file_paths; + if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) + { + S3FilesCollection processing_files = parseCollection(node_data); + for (auto x : processing_files) + { + if (x != file_path) + { + file_paths.push_back(x); + } + } + } + zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); } void S3QueueHolder::setFileFailed(const String & file_path) { auto zookeeper = getZooKeeper(); + auto lock = AcquireLock(); - std::lock_guard lock(mutex); String failed_files = zookeeper->get(zookeeper_failed_path); S3FilesCollection failed = parseCollection(failed_files); @@ -104,20 +207,27 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) S3QueueHolder::S3FilesCollection S3QueueHolder::getExcludedFiles() { std::unordered_set exclude_files; + auto zookeeper = getZooKeeper(); - std::unordered_set failed_files = getFailedFiles(); - LOG_DEBUG(log, "failed_files {}", failed_files.size()); + String failed = zookeeper->get(zookeeper_failed_path); + S3FilesCollection failed_files = parseCollection(failed); exclude_files.merge(failed_files); + String processed = zookeeper->get(zookeeper_processed_path); if (mode != S3QueueMode::ORDERED) { - std::unordered_set processed_files = getProcessedFiles(); - LOG_DEBUG(log, "processed_files {}", processed_files.size()); + auto collection = ProcessedCollection(max_set_size, max_set_age_s); + collection.parse(processed); + S3FilesCollection processed_files = collection.getFileNames(); exclude_files.merge(processed_files); } + else + { + exclude_files.insert(processed); + } - std::unordered_set processing_files = getProcessingFiles(); - LOG_DEBUG(log, "processing {}", processing_files.size()); + String processing = zookeeper->get(fs::path(zookeeper_processing_path)); + S3FilesCollection processing_files = parseCollection(processing); exclude_files.merge(processing_files); return exclude_files; @@ -129,7 +239,6 @@ String S3QueueHolder::getMaxProcessedFile() throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getMaxProcessedFile not implemented for unordered mode"); auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); String processed = zookeeper->get(zookeeper_path + "/processed"); return processed; } @@ -138,7 +247,6 @@ void S3QueueHolder::setFilesProcessing(Strings & file_paths) { auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); String node_data; if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) { @@ -157,8 +265,8 @@ void S3QueueHolder::setFilesProcessing(Strings & file_paths) S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() { auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); + auto lock = AcquireLock(); String failed = zookeeper->get(zookeeper_failed_path); return parseCollection(failed); } @@ -167,20 +275,54 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessedFiles() { auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); + auto lock = AcquireLock(); String processed = zookeeper->get(zookeeper_processed_path); - return parseCollection(processed); + auto collection = ProcessedCollection(max_set_size, max_set_age_s); + collection.parse(processed); + return collection.getFileNames(); } S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() { auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); + auto lock = AcquireLock(); String processing = zookeeper->get(fs::path(zookeeper_processing_path)); return parseCollection(processing); } + +std::shared_ptr S3QueueHolder::AcquireLock() +{ + auto zookeeper = getZooKeeper(); + UInt32 retry_count = 200; + UInt32 sleep_ms = 100; + + UInt32 retries = 0; + while (true) + { + Coordination::Error code = zookeeper->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + { + retries++; + if (retries >= retry_count) + { + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Can't acquire zookeeper lock"); + } + sleepForMilliseconds(sleep_ms); + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception(code, zookeeper_lock_path); + } + else + { + return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zookeeper); + } + } +} + + } #endif diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index 686b5fc1ddf..081e58a9ea2 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -2,10 +2,10 @@ #if USE_AWS_S3 -#include -#include -#include -#include +# include +# include +# include +# include namespace DB { @@ -13,7 +13,10 @@ class S3QueueHolder : public WithContext { public: using S3FilesCollection = std::unordered_set; - S3QueueHolder(const String & zookeeper_path_, const S3QueueMode & mode_, const UUID & table_uuid_, ContextPtr context_); + using ProcessedFiles = std::vector>; + + S3QueueHolder( + const String & zookeeper_path_, const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, UInt64 & max_set_age_s_); void setFileProcessed(const String & file_path); void setFileFailed(const String & file_path); @@ -25,6 +28,28 @@ public: S3FilesCollection getFailedFiles(); S3FilesCollection getProcessedFiles(); S3FilesCollection getProcessingFiles(); + std::shared_ptr AcquireLock(); + + struct ProcessedCollection + { + ProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_); + + void parse(const String & s); + + String toString() const; + + void add(const String & file_name); + S3FilesCollection getFileNames(); + const UInt64 max_size; + const UInt64 max_age; + + void read(ReadBuffer & in); + void write(WriteBuffer & out) const; + ProcessedFiles files; + }; + + const UInt64 max_set_size; + const UInt64 max_set_age_s; private: zkutil::ZooKeeperPtr current_zookeeper; @@ -34,6 +59,7 @@ private: const String zookeeper_failed_path; const String zookeeper_processing_path; const String zookeeper_processed_path; + const String zookeeper_lock_path; const S3QueueMode mode; const UUID table_uuid; Poco::Logger * log; diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index a2798a42263..c2b8e51a1f8 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -23,7 +23,8 @@ class ASTStorage; M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ M(UInt64, s3queue_polling_backoff_ms, 0, "Retry loading up to specified number of times", 0) \ M(UInt64, s3queue_max_set_size, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ - M(UInt64, s3queue_max_set_age_s, 0, "Maximum number of seconds to store processed files in ZooKeeper node (forever default)", 0) + M(UInt64, s3queue_max_set_age_s, 0, "Maximum number of seconds to store processed files in ZooKeeper node (forever default)", 0) \ + M(UInt64, s3queue_polling_size, 50, "Maximum files to fetch from S3 with SELECT", 0) #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 3da725cee71..b934f78c8df 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -7,51 +7,51 @@ #if USE_AWS_S3 -#include +# include -#include +# include -#include -#include +# include +# include -#include +# include -#include -#include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include -#include +# include +# include +# include -#include +# include -#include +# include -#include -#include -#include +# include +# include +# include -#include +# include -#include +# include -#include -#include -#include +# include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include namespace fs = std::filesystem; @@ -104,9 +104,11 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( ASTPtr query, const Block & virtual_header, ContextPtr context, + UInt64 & max_poll_size_, StorageS3QueueSource::KeysWithInfo * read_keys_, const S3Settings::RequestSettings & request_settings_) - : bucket(globbed_uri_.bucket) + : max_poll_size(max_poll_size_) + , bucket(globbed_uri_.bucket) , glob_iterator(std::make_unique( client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) { @@ -121,7 +123,8 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( } } -Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file) +Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( + const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file) { for (KeyWithInfo val : keys_buf) { @@ -131,11 +134,11 @@ Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(S3QueueMode & eng LOG_INFO(log, "Found in exclude keys {}", val.key); continue; } - if (engine_mode == S3QueueMode::ORDERED && full_path.compare(max_file) <= 0) + if ((engine_mode == S3QueueMode::ORDERED) && (full_path.compare(max_file) <= 0)) { continue; } - if (processing_keys.size() < max_poll_size) + if ((processing_keys.size() < max_poll_size) || (engine_mode == S3QueueMode::ORDERED)) { processing_keys.push_back(val); } @@ -150,7 +153,11 @@ Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(S3QueueMode & eng std::sort( processing_keys.begin(), processing_keys.end(), - [](const KeyWithInfo & lhs, const KeyWithInfo & rhs) { return lhs.key < rhs.key; }); + [](const KeyWithInfo & lhs, const KeyWithInfo & rhs) { return lhs.key.compare(rhs.key) < 0; }); + if (processing_keys.size() > max_poll_size) + { + processing_keys.erase(processing_keys.begin() + max_poll_size, processing_keys.end()); + } } Strings keys; @@ -207,8 +214,6 @@ StorageS3QueueSource::StorageS3QueueSource( std::shared_ptr file_iterator_, std::shared_ptr queue_holder_, const S3QueueAction & action_, - zkutil::ZooKeeperPtr current_zookeeper, - const String & zookeeper_path_, const size_t download_thread_num_) : ISource(getHeader(sample_block_, requested_virtual_columns_)) , WithContext(context_) @@ -228,8 +233,6 @@ StorageS3QueueSource::StorageS3QueueSource( , file_iterator(file_iterator_) , action(action_) , download_thread_num(download_thread_num_) - , zookeeper(current_zookeeper) - , zookeeper_path(zookeeper_path_) , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3QReader")) { diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 0c36499d516..2891a5946af 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -3,26 +3,26 @@ #if USE_AWS_S3 -#include +# include -#include +# include -#include -#include -#include -#include +# include +# include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include namespace DB @@ -46,16 +46,18 @@ public: ASTPtr query, const Block & virtual_header, ContextPtr context, + UInt64 & max_poll_size_, KeysWithInfo * read_keys_ = nullptr, const S3Settings::RequestSettings & request_settings_ = {}); KeyWithInfo next() override; size_t getTotalSize() const override; - Strings setProcessing(S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file = ""); + Strings + filterProcessingFiles(const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file = ""); private: - size_t max_poll_size = 10; + UInt64 max_poll_size; const String bucket; KeysWithInfo keys_buf; KeysWithInfo processing_keys; @@ -85,8 +87,6 @@ public: std::shared_ptr file_iterator_, std::shared_ptr queue_holder_, const S3QueueAction & action_, - zkutil::ZooKeeperPtr current_zookeeper, - const String & zookeeper_path_, size_t download_thread_num); ~StorageS3QueueSource() override; @@ -122,9 +122,6 @@ private: Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSource"); - zkutil::ZooKeeperPtr zookeeper; - const String zookeeper_path; - ThreadPool create_reader_pool; ThreadPoolCallbackRunner create_reader_scheduler; std::future reader_future; diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 40d29e26b68..340890f75de 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -2,11 +2,11 @@ #if USE_AWS_S3 -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include namespace DB diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 85df226a02f..6e87528db37 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -2,14 +2,13 @@ #if USE_AWS_S3 -#include -#include -#include +# include +# include +# include namespace DB { -class MergeTreeData; class WriteBuffer; class ReadBuffer; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index f91ca11b491..a5dd5b421c8 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -3,59 +3,59 @@ #if USE_AWS_S3 -#include -#include -#include -#include -#include -#include -#include -#include -#include "IO/ParallelReadBuffer.h" +# include +# include +# include +# include +# include +# include +# include +# include +# include "IO/ParallelReadBuffer.h" -#include +# include -#include +# include -#include +# include -#include -#include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include +# include +# include -#include +# include -#include -#include -#include +# include +# include +# include -#include +# include -#include +# include -#include +# include -#include -#include -#include -#include +# include +# include +# include +# include namespace fs = std::filesystem; @@ -175,8 +175,8 @@ StorageS3Queue::StorageS3Queue( checkTableStructure(zookeeper_path, metadata_snapshot); } - auto table_uuid = getStorageID().uuid; - queue_holder = std::make_unique(zookeeper_path, mode, table_uuid, getContext()); + queue_holder = std::make_unique( + zookeeper_path, mode, getContext(), s3queue_settings->s3queue_max_set_size.value, s3queue_settings->s3queue_max_set_age_s.value); auto default_virtuals = NamesAndTypesList{ {"_path", std::make_shared(std::make_shared())}, @@ -216,7 +216,7 @@ Pipe StorageS3Queue::read( ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); if (mv_attached) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageRabbitMQ with attached materialized views"); + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageS3Queue with attached materialized views"); auto query_s3_configuration = updateConfigurationAndGetCopy(local_context); @@ -279,8 +279,6 @@ Pipe StorageS3Queue::read( iterator_wrapper, queue_holder, after_processing, - zookeeper, - zookeeper_path, max_download_threads)); } @@ -492,8 +490,6 @@ void StorageS3Queue::streamToViews() iterator_wrapper, queue_holder, after_processing, - zookeeper, - zookeeper_path, max_download_threads)); @@ -553,10 +549,8 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ else { String metadata_str = S3QueueTableMetadata(s3_configuration, *s3queue_settings).toString(); - String default_processed = mode == S3QueueMode::ORDERED ? "" : "[]"; - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", default_processed, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "processed_files\n", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( @@ -627,19 +621,27 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW std::lock_guard lock{sync_mutex}; auto it = std::make_shared( - *s3_configuration.client, s3_configuration.url, query, virtual_block, local_context, read_keys, s3_configuration.request_settings); + *s3_configuration.client, + s3_configuration.url, + query, + virtual_block, + local_context, + s3queue_settings->s3queue_polling_size.value, + read_keys, + s3_configuration.request_settings); + auto zookeeper_lock = queue_holder->AcquireLock(); S3QueueHolder::S3FilesCollection exclude = queue_holder->getExcludedFiles(); Strings processing_files; if (mode == S3QueueMode::UNORDERED) { - processing_files = it->setProcessing(mode, exclude); + processing_files = it->filterProcessingFiles(mode, exclude); } else { String max_processed_file = queue_holder->getMaxProcessedFile(); - processing_files = it->setProcessing(mode, exclude, max_processed_file); + processing_files = it->filterProcessingFiles(mode, exclude, max_processed_file); } queue_holder->setFilesProcessing(processing_files); return it; diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index b8d0ec3d5da..3276c221014 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -13,7 +13,10 @@ from helpers.s3_tools import prepare_s3_bucket @pytest.fixture(autouse=True) def s3_queue_setup_teardown(started_cluster): instance = started_cluster.instances["instance"] + instance_2 = started_cluster.instances["instance2"] + instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") + instance_2.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") minio = started_cluster.minio_client objects = list( @@ -28,7 +31,6 @@ MINIO_INTERNAL_PORT = 9001 AVAILABLE_MODES = ["ordered", "unordered"] AUTH = "'minio','minio123'," - SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -41,8 +43,12 @@ def generate_random_files( count, prefix, cluster, bucket, column_num=3, row_num=10, start_ind=0 ): total_values = [] - for i in range(start_ind, start_ind + count): - print(i) + to_generate = [ + (f"{prefix}/test_{i}.csv", i) for i in range(start_ind, start_ind + count) + ] + to_generate.sort(key=lambda x: x[0]) + + for filename, i in to_generate: rand_values = [ [random.randint(0, 50) for _ in range(column_num)] for _ in range(row_num) ] @@ -50,7 +56,6 @@ def generate_random_files( values_csv = ( "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" ).encode() - filename = f"{prefix}/test_{i}.csv" put_s3_file_content(cluster, bucket, filename, values_csv) return total_values @@ -79,6 +84,13 @@ def started_cluster(): with_zookeeper=True, main_configs=["configs/defaultS3.xml", "configs/named_collections.xml"], ) + cluster.add_instance( + "instance2", + user_configs=["configs/users.xml"], + with_minio=True, + with_zookeeper=True, + main_configs=["configs/defaultS3.xml", "configs/named_collections.xml"], + ) logging.info("Starting cluster...") cluster.start() @@ -441,3 +453,245 @@ def test_multiple_tables_meta_mismatch(started_cluster): keeper_path = '/clickhouse/test_meta'; """ ) + + +def test_max_set_age(started_cluster): + files_to_generate = 10 + max_age = 1 + prefix = f"test_multiple" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'unordered', + keeper_path = '/clickhouse/test_set_age', + s3queue_max_set_size = 10, + s3queue_max_set_age_s = {max_age}; + """ + ) + + total_values = generate_random_files( + files_to_generate, prefix, started_cluster, bucket, row_num=1 + ) + get_query = f"SELECT * FROM test.s3_queue" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + assert res1 == total_values + time.sleep(max_age + 1) + + get_query = f"SELECT * FROM test.s3_queue" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + assert res1 == total_values + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_multiple_tables_streaming_sync(started_cluster, mode): + files_to_generate = 300 + poll_size = 30 + prefix = f"test_multiple_{mode}" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + DROP TABLE IF EXISTS test.s3_queue_copy; + DROP TABLE IF EXISTS test.s3_queue_copy_2; + + DROP TABLE IF EXISTS test.s3_queue_persistent; + DROP TABLE IF EXISTS test.s3_queue_persistent_copy; + DROP TABLE IF EXISTS test.s3_queue_persistent_copy_2; + + DROP TABLE IF EXISTS test.persistent_s3_queue_mv; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy_2; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', + s3queue_polling_size = {poll_size}; + + CREATE TABLE test.s3_queue_copy ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', + s3queue_polling_size = {poll_size}; + + CREATE TABLE test.s3_queue_copy_2 ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', + s3queue_polling_size = {poll_size}; + + CREATE TABLE test.s3_queue_persistent ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue_persistent_copy ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue_persistent_copy_2 ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS + SELECT + * + FROM test.s3_queue; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy TO test.s3_queue_persistent_copy AS + SELECT + * + FROM test.s3_queue_copy; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy_2 TO test.s3_queue_persistent_copy_2 AS + SELECT + * + FROM test.s3_queue_copy_2; + """ + ) + total_values = generate_random_files( + files_to_generate, prefix, started_cluster, bucket, row_num=1 + ) + time.sleep((files_to_generate // poll_size) + 10) + + get_query = f"SELECT * FROM test.s3_queue_persistent" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + get_query_copy = f"SELECT * FROM test.s3_queue_persistent_copy" + res2 = [ + list(map(int, l.split())) + for l in run_query(instance, get_query_copy).splitlines() + ] + get_query_copy_2 = f"SELECT * FROM test.s3_queue_persistent_copy_2" + res3 = [ + list(map(int, l.split())) + for l in run_query(instance, get_query_copy_2).splitlines() + ] + + # Checking that all engines have made progress + assert len(res1) > 0 + assert len(res2) > 0 + assert len(res3) > 0 + + # Checking that all files were processed only once + assert len(res1) + len(res2) + len(res3) == files_to_generate + assert {tuple(v) for v in res1 + res2 + res3} == set( + [tuple(i) for i in total_values] + ) + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): + files_to_generate = 100 + poll_size = 10 + prefix = f"test_multiple_{mode}" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + instance_2 = started_cluster.instances["instance2"] + + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + for inst in [instance, instance_2]: + inst.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + DROP TABLE IF EXISTS test.s3_queue_persistent; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/test_multiple_consumers_{mode}', + s3queue_polling_size = {poll_size}; + + CREATE TABLE test.s3_queue_persistent ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS + SELECT + * + FROM test.s3_queue; + """ + ) + + total_values = generate_random_files( + files_to_generate, prefix, started_cluster, bucket, row_num=1 + ) + + time.sleep((files_to_generate // poll_size) + 10) + get_query = f"SELECT * FROM test.s3_queue_persistent" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + res2 = [ + list(map(int, l.split())) for l in run_query(instance_2, get_query).splitlines() + ] + + # Checking that all engines have made progress + assert len(res1) > 0 + assert len(res2) > 0 + + # Checking that all files were processed only once + assert len(res1) + len(res2) == files_to_generate + assert {tuple(v) for v in res1 + res2} == set([tuple(i) for i in total_values]) + + +def test_max_set_size(started_cluster): + files_to_generate = 10 + prefix = f"test_multiple" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'unordered', + keeper_path = '/clickhouse/test_set_size', + s3queue_max_set_size = {files_to_generate - 1}; + """ + ) + + total_values = generate_random_files( + files_to_generate, prefix, started_cluster, bucket, start_ind=0, row_num=1 + ) + get_query = f"SELECT * FROM test.s3_queue" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + assert res1 == total_values + + get_query = f"SELECT * FROM test.s3_queue" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + assert res1 == [total_values[0]] + + get_query = f"SELECT * FROM test.s3_queue" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + assert res1 == [total_values[1]] From 751337fad0c64c2db0ca401a9d36106bc50c5346 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 8 May 2023 16:31:24 +0300 Subject: [PATCH 0067/2047] reformat code --- src/Storages/S3Queue/S3QueueSource.cpp | 175 ++++-------------------- src/Storages/S3Queue/S3QueueSource.h | 20 +-- src/Storages/S3Queue/StorageS3Queue.cpp | 7 +- 3 files changed, 25 insertions(+), 177 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index b934f78c8df..0e0b00d2d3f 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -71,32 +71,6 @@ extern const Event S3ListObjects; namespace DB { -static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - -static const std::unordered_set required_configuration_keys = { - "url", -}; -static const std::unordered_set optional_configuration_keys - = {"format", - "compression", - "compression_method", - "structure", - "access_key_id", - "secret_access_key", - "filename", - "use_environment_credentials", - "max_single_read_retries", - "min_upload_part_size", - "upload_part_size_multiply_factor", - "upload_part_size_multiply_parts_count_threshold", - "max_single_part_upload_size", - "max_connections", - "expiration_window_seconds", - "no_sign_request"}; - -class IOutputFormat; -using OutputFormatPtr = std::shared_ptr; - StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( const S3::Client & client_, @@ -222,135 +196,37 @@ StorageS3QueueSource::StorageS3QueueSource( , version_id(version_id_) , format(format_) , columns_desc(columns_) - , max_block_size(max_block_size_) , request_settings(request_settings_) - , compression_hint(std::move(compression_hint_)) , client(client_) - , sample_block(sample_block_) - , format_settings(format_settings_) , queue_holder(queue_holder_) , requested_virtual_columns(requested_virtual_columns_) , file_iterator(file_iterator_) , action(action_) - , download_thread_num(download_thread_num_) - , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) - , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3QReader")) { - reader = createReader(); + internal_source = std::make_shared( + requested_virtual_columns_, + format_, + name_, + sample_block_, + context_, + format_settings_, + columns_, + max_block_size_, + request_settings_, + compression_hint_, + client_, + bucket_, + version_id_, + file_iterator, + download_thread_num_); + reader = std::move(internal_source->reader); if (reader) - reader_future = createReaderAsync(); -} - -StorageS3QueueSource::ReaderHolder StorageS3QueueSource::createReader() -{ - auto [current_key, info] = (*file_iterator)(); - if (current_key.empty()) - return {}; - - size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id, request_settings); - auto compression_method = chooseCompressionMethod(current_key, compression_hint); - - InputFormatPtr input_format; - std::unique_ptr owned_read_buf; - - auto read_buf_or_factory = createS3ReadBuffer(current_key, object_size); - if (read_buf_or_factory.buf_factory) - { - input_format = FormatFactory::instance().getInputRandomAccess( - format, - std::move(read_buf_or_factory.buf_factory), - sample_block, - getContext(), - max_block_size, - /* is_remote_fs */ true, - compression_method, - format_settings); - } - else - { - owned_read_buf = wrapReadBufferWithCompressionMethod( - std::move(read_buf_or_factory.buf), compression_method, static_cast(getContext()->getSettingsRef().zstd_window_log_max)); - input_format - = FormatFactory::instance().getInput(format, *owned_read_buf, sample_block, getContext(), max_block_size, format_settings); - } - - QueryPipelineBuilder builder; - builder.init(Pipe(input_format)); - - if (columns_desc.hasDefaults()) - { - builder.addSimpleTransform( - [&](const Block & header) - { return std::make_shared(header, columns_desc, *input_format, getContext()); }); - } - - auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - auto current_reader = std::make_unique(*pipeline); - - return ReaderHolder{fs::path(bucket) / current_key, std::move(owned_read_buf), std::move(pipeline), std::move(current_reader)}; -} - -std::future StorageS3QueueSource::createReaderAsync() -{ - return create_reader_scheduler([this] { return createReader(); }, 0); -} - -StorageS3QueueSource::ReadBufferOrFactory StorageS3QueueSource::createS3ReadBuffer(const String & key, size_t object_size) -{ - auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); - read_settings.enable_filesystem_cache = false; - auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; - const bool object_too_small = object_size <= 2 * download_buffer_size; - - // Create a read buffer that will prefetch the first ~1 MB of the file. - // When reading lots of tiny files, this prefetching almost doubles the throughput. - // For bigger files, parallel reading is more useful. - if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - { - LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size); - return {.buf = createAsyncS3ReadBuffer(key, read_settings, object_size)}; - } - - auto factory = std::make_unique(client, bucket, key, version_id, object_size, request_settings, read_settings); - return {.buf_factory = std::move(factory)}; -} - -std::unique_ptr -StorageS3QueueSource::createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size) -{ - auto read_buffer_creator = - [this, read_settings, object_size](const std::string & path, size_t read_until_position) -> std::shared_ptr - { - return std::make_shared( - client, - bucket, - path, - version_id, - request_settings, - read_settings, - /* use_external_buffer */ true, - /* offset */ 0, - read_until_position, - /* restricted_seek */ true, - object_size); - }; - - auto s3_impl = std::make_unique( - std::move(read_buffer_creator), StoredObjects{StoredObject{key, object_size}}, read_settings); - - auto & pool_reader = getContext()->getThreadPoolReader(Context::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - auto async_reader = std::make_unique(pool_reader, read_settings, std::move(s3_impl)); - - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - - return async_reader; + reader_future = std::move(internal_source->reader_future); } StorageS3QueueSource::~StorageS3QueueSource() { - create_reader_pool.wait(); + internal_source->create_reader_pool.wait(); } String StorageS3QueueSource::getName() const @@ -370,12 +246,10 @@ Chunk StorageS3QueueSource::generate() } Chunk chunk; - LOG_WARNING(log, "Try to pull new chunk"); try { if (reader->pull(chunk)) { - LOG_WARNING(log, "Success in pulling!"); UInt64 num_rows = chunk.getNumRows(); const auto & file_path = reader.getPath(); @@ -399,7 +273,6 @@ Chunk StorageS3QueueSource::generate() chunk.addColumn(column->convertToFullColumnIfConst()); } } - LOG_WARNING(log, "Set processed: {}", file_path); queue_holder->setFileProcessed(file_path); applyActionAfterProcessing(file_path); return chunk; @@ -407,9 +280,8 @@ Chunk StorageS3QueueSource::generate() } catch (const Exception & e) { - LOG_ERROR(log, "Exception: {} ", e.displayText()); + LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); const auto & failed_file_path = reader.getPath(); - LOG_WARNING(log, "Set failed: {}", failed_file_path); queue_holder->setFileFailed(failed_file_path); } @@ -419,11 +291,10 @@ Chunk StorageS3QueueSource::generate() if (!reader) break; - /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. - create_reader_pool.wait(); - reader_future = createReaderAsync(); + internal_source->create_reader_pool.wait(); + reader_future = internal_source->createReaderAsync(); } return {}; @@ -431,7 +302,7 @@ Chunk StorageS3QueueSource::generate() void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) { - LOG_WARNING(log, "Delete {} Bucke {}", file_path, bucket); + LOG_WARNING(log, "Delete {} Bucket {}", file_path, bucket); S3::DeleteObjectRequest request; request.SetBucket(bucket); request.SetKey(file_path); diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 2891a5946af..fc7ce3606b0 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -36,7 +36,6 @@ public: using DisclosedGlobIterator = StorageS3Source::DisclosedGlobIterator; using KeysWithInfo = StorageS3Source::KeysWithInfo; using KeyWithInfo = StorageS3Source::KeyWithInfo; - using ReadBufferOrFactory = StorageS3Source::ReadBufferOrFactory; class QueueGlobIterator : public IIterator { public: @@ -95,8 +94,6 @@ public: Chunk generate() override; - static std::unordered_set parseCollection(String & files); - private: String name; @@ -104,12 +101,8 @@ private: String version_id; String format; ColumnsDescription columns_desc; - UInt64 max_block_size; S3Settings::RequestSettings request_settings; - String compression_hint; std::shared_ptr client; - Block sample_block; - std::optional format_settings; std::shared_ptr queue_holder; using ReaderHolder = StorageS3Source::ReaderHolder; @@ -118,12 +111,9 @@ private: std::vector requested_virtual_columns; std::shared_ptr file_iterator; const S3QueueAction action; - size_t download_thread_num = 1; Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSource"); - ThreadPool create_reader_pool; - ThreadPoolCallbackRunner create_reader_scheduler; std::future reader_future; UInt64 total_rows_approx_max = 0; @@ -132,15 +122,7 @@ private: mutable std::mutex mutex; - - ReaderHolder createReader(); - std::future createReaderAsync(); - - ReadBufferOrFactory createS3ReadBuffer(const String & key, size_t object_size); - std::unique_ptr createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size); - - void setFileProcessed(const String & file_path); - void setFileFailed(const String & file_path); + std::shared_ptr internal_source; void applyActionAfterProcessing(const String & file_path); }; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index a5dd5b421c8..09dc094eae5 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -539,8 +539,6 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ for (size_t i = 0; i < 1000; ++i) { Coordination::Requests ops; - auto table_uuid = getStorageID().uuid; - if (zookeeper->exists(zookeeper_path + "/metadata")) { LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zookeeper_path); @@ -552,10 +550,7 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "processed_files\n", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest( - fs::path(zookeeper_path) / "processing" / toString(table_uuid), "[]", zkutil::CreateMode::Ephemeral)); - + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); From a443c7f28984e861a11407eb82d2f4da023a6f30 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Tue, 9 May 2023 06:58:29 +0300 Subject: [PATCH 0068/2047] fix --- src/Storages/S3Queue/S3QueueHolder.cpp | 4 ++-- src/Storages/S3Queue/S3QueueSource.cpp | 2 -- src/Storages/S3Queue/StorageS3Queue.cpp | 7 +++++-- src/Storages/S3Queue/StorageS3Queue.h | 1 - 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 21b2b7a45ce..1b708154bcd 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -159,7 +159,7 @@ void S3QueueHolder::setFileProcessed(const String & file_path) if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) { S3FilesCollection processing_files = parseCollection(node_data); - for (auto x : processing_files) + for (const auto & x : processing_files) { if (x != file_path) { @@ -251,7 +251,7 @@ void S3QueueHolder::setFilesProcessing(Strings & file_paths) if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) { S3FilesCollection processing_files = parseCollection(node_data); - for (auto x : processing_files) + for (const auto & x : processing_files) { if (!std::count(file_paths.begin(), file_paths.end(), x)) { diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 0e0b00d2d3f..eb49db8e84d 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -48,10 +48,8 @@ # include # include -# include # include # include -# include namespace fs = std::filesystem; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 09dc094eae5..5a011c9b51a 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -539,10 +539,13 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ for (size_t i = 0; i < 1000; ++i) { Coordination::Requests ops; + bool is_first_replica = true; if (zookeeper->exists(zookeeper_path + "/metadata")) { + if (!zookeeper->exists(zookeeper_path + "/processing")) + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zookeeper_path); - return false; + is_first_replica = false; } else { @@ -569,7 +572,7 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ zkutil::KeeperMultiException::check(code, ops, responses); } - return true; + return is_first_replica; } throw Exception( diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index ac098cdda99..73f8075b7d1 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -130,7 +130,6 @@ private: zkutil::ZooKeeperPtr getZooKeeper() const; bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); - // Return default or custom zookeeper name for table const String & getZooKeeperPath() const { return zookeeper_path; } using KeysWithInfo = StorageS3QueueSource::KeysWithInfo; From d37622434f5c5d22263120466a99d3a95b5465dc Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Tue, 9 May 2023 07:04:36 +0300 Subject: [PATCH 0069/2047] fix tests --- tests/integration/test_storage_s3_queue/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 3276c221014..bc92e809f5d 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -28,7 +28,7 @@ def s3_queue_setup_teardown(started_cluster): MINIO_INTERNAL_PORT = 9001 -AVAILABLE_MODES = ["ordered", "unordered"] +AVAILABLE_MODES = ["unordered", "ordered"] AUTH = "'minio','minio123'," SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -569,7 +569,7 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): total_values = generate_random_files( files_to_generate, prefix, started_cluster, bucket, row_num=1 ) - time.sleep((files_to_generate // poll_size) + 10) + time.sleep((files_to_generate // poll_size) * 2) get_query = f"SELECT * FROM test.s3_queue_persistent" res1 = [ @@ -637,7 +637,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): files_to_generate, prefix, started_cluster, bucket, row_num=1 ) - time.sleep((files_to_generate // poll_size) + 10) + time.sleep((files_to_generate // poll_size) * 2) get_query = f"SELECT * FROM test.s3_queue_persistent" res1 = [ list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() From 383fc06761f81bee735ec22692a2d506ca78c01e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 9 May 2023 16:10:53 +0200 Subject: [PATCH 0070/2047] Fix --- .../MaterializedPostgreSQLConsumer.cpp | 37 +++--- .../MaterializedPostgreSQLSettings.h | 3 + .../PostgreSQLReplicationHandler.cpp | 21 +-- .../PostgreSQL/PostgreSQLReplicationHandler.h | 7 +- tests/integration/helpers/postgres_utility.py | 124 +++++++++++------- .../test.py | 75 ++++++++++- 6 files changed, 183 insertions(+), 84 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index d048c94ac75..ea7009fc082 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -562,34 +562,27 @@ void MaterializedPostgreSQLConsumer::syncTables() Block result_rows = storage_data.buffer.description.sample_block.cloneWithColumns(std::move(storage_data.buffer.columns)); storage_data.buffer.columns = storage_data.buffer.description.sample_block.cloneEmptyColumns(); - try + if (result_rows.rows()) { - if (result_rows.rows()) - { - auto storage = storage_data.storage; + auto storage = storage_data.storage; - auto insert_context = Context::createCopy(context); - insert_context->setInternalQuery(true); + auto insert_context = Context::createCopy(context); + insert_context->setInternalQuery(true); - auto insert = std::make_shared(); - insert->table_id = storage->getStorageID(); - insert->columns = storage_data.buffer.columns_ast; + auto insert = std::make_shared(); + insert->table_id = storage->getStorageID(); + insert->columns = storage_data.buffer.columns_ast; - InterpreterInsertQuery interpreter(insert, insert_context, true); - auto io = interpreter.execute(); - auto input = std::make_shared( - result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); + InterpreterInsertQuery interpreter(insert, insert_context, true); + auto io = interpreter.execute(); + auto input = std::make_shared( + result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); - assertBlocksHaveEqualStructure(input->getPort().getHeader(), io.pipeline.getHeader(), "postgresql replica table sync"); - io.pipeline.complete(Pipe(std::move(input))); + assertBlocksHaveEqualStructure(input->getPort().getHeader(), io.pipeline.getHeader(), "postgresql replica table sync"); + io.pipeline.complete(Pipe(std::move(input))); - CompletedPipelineExecutor executor(io.pipeline); - executor.execute(); - } - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); + CompletedPipelineExecutor executor(io.pipeline); + executor.execute(); } } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h index e8d42ef3668..d3d2faba497 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h @@ -21,6 +21,9 @@ namespace DB M(Bool, materialized_postgresql_tables_list_with_schema, false, \ "Consider by default that if there is a dot in tables list 'name.name', " \ "then the first name is postgres schema and second is postgres table. This setting is needed to allow table names with dots", 0) \ + M(UInt64, materialized_postgresql_backoff_min_ms, 200, "Poll backoff start point", 0) \ + M(UInt64, materialized_postgresql_backoff_max_ms, 10000, "Poll backoff max point", 0) \ + M(UInt64, materialized_postgresql_backoff_factor, 2, "Poll backoff factor", 0) \ DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 998db4ea79e..f57a6a26a62 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -22,8 +22,6 @@ namespace DB { -static const auto RESCHEDULE_MS = 1000; -static const auto BACKOFF_TRESHOLD_MS = 10000; static const auto CLEANUP_RESCHEDULE_MS = 600000 * 3; /// 30 min namespace ErrorCodes @@ -80,7 +78,10 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , schema_list(replication_settings.materialized_postgresql_schema_list) , schema_as_a_part_of_table_name(!schema_list.empty() || replication_settings.materialized_postgresql_tables_list_with_schema) , user_provided_snapshot(replication_settings.materialized_postgresql_snapshot) - , milliseconds_to_wait(RESCHEDULE_MS) + , reschedule_backoff_min_ms(replication_settings.materialized_postgresql_backoff_min_ms) + , reschedule_backoff_max_ms(replication_settings.materialized_postgresql_backoff_max_ms) + , reschedule_backoff_factor(replication_settings.materialized_postgresql_backoff_factor) + , milliseconds_to_wait(reschedule_backoff_min_ms) { if (!schema_list.empty() && !tables_list.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot have schema list and tables list at the same time"); @@ -166,7 +167,7 @@ void PostgreSQLReplicationHandler::checkConnectionAndStart() throw; LOG_ERROR(log, "Unable to set up connection. Reconnection attempt will continue. Error message: {}", pqxx_error.what()); - startup_task->scheduleAfter(RESCHEDULE_MS); + startup_task->scheduleAfter(milliseconds_to_wait); } catch (...) { @@ -435,18 +436,18 @@ void PostgreSQLReplicationHandler::consumerFunc() if (schedule_now) { - milliseconds_to_wait = RESCHEDULE_MS; + milliseconds_to_wait = reschedule_backoff_min_ms; consumer_task->schedule(); LOG_DEBUG(log, "Scheduling replication thread: now"); } else { - consumer_task->scheduleAfter(milliseconds_to_wait); - if (milliseconds_to_wait < BACKOFF_TRESHOLD_MS) - milliseconds_to_wait *= 2; + if (milliseconds_to_wait < reschedule_backoff_max_ms) + milliseconds_to_wait = std::min(milliseconds_to_wait * reschedule_backoff_factor, reschedule_backoff_max_ms); LOG_DEBUG(log, "Scheduling replication thread: after {} ms", milliseconds_to_wait); + consumer_task->scheduleAfter(milliseconds_to_wait); } } @@ -892,7 +893,7 @@ void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPost catch (...) { consumer_task->activate(); - consumer_task->scheduleAfter(RESCHEDULE_MS); + consumer_task->scheduleAfter(milliseconds_to_wait); auto error_message = getCurrentExceptionMessage(false); throw Exception(ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR, @@ -922,7 +923,7 @@ void PostgreSQLReplicationHandler::removeTableFromReplication(const String & pos catch (...) { consumer_task->activate(); - consumer_task->scheduleAfter(RESCHEDULE_MS); + consumer_task->scheduleAfter(milliseconds_to_wait); auto error_message = getCurrentExceptionMessage(false); throw Exception(ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR, diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 10a196cf31b..4c16ff95692 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -140,13 +140,16 @@ private: BackgroundSchedulePool::TaskHolder consumer_task; BackgroundSchedulePool::TaskHolder cleanup_task; + const UInt64 reschedule_backoff_min_ms; + const UInt64 reschedule_backoff_max_ms; + const UInt64 reschedule_backoff_factor; + UInt64 milliseconds_to_wait; + std::atomic stop_synchronization = false; /// MaterializedPostgreSQL tables. Used for managing all operations with its internal nested tables. MaterializedStorages materialized_storages; - UInt64 milliseconds_to_wait; - bool replication_handler_initialized = false; }; diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index dfae37af434..1a00faf0f9d 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -76,16 +76,24 @@ def drop_postgres_schema(cursor, schema_name): def create_postgres_table( - cursor, table_name, replica_identity_full=False, template=postgres_table_template + cursor, + table_name, + database_name="", + replica_identity_full=False, + template=postgres_table_template, ): - drop_postgres_table(cursor, table_name) - cursor.execute(template.format(table_name)) + if database_name == "": + name = table_name + else: + name = f"{database_name}.{table_name}" + drop_postgres_table(cursor, name) + cursor.execute(template.format(name)) if replica_identity_full: - cursor.execute(f"ALTER TABLE {table_name} REPLICA IDENTITY FULL;") + cursor.execute(f"ALTER TABLE {name} REPLICA IDENTITY FULL;") -def drop_postgres_table(cursor, table_name): - cursor.execute(f"""DROP TABLE IF EXISTS "{table_name}" """) +def drop_postgres_table(cursor, name): + cursor.execute(f"""DROP TABLE IF EXISTS "{name}" """) def create_postgres_table_with_schema(cursor, schema_name, table_name): @@ -103,13 +111,16 @@ class PostgresManager: self.created_materialized_postgres_db_list = set() self.created_ch_postgres_db_list = set() - def init(self, instance, ip, port): + def init(self, instance, ip, port, default_database="postgres_database"): self.instance = instance self.ip = ip self.port = port - self.conn = get_postgres_conn(ip=self.ip, port=self.port) + self.default_database = default_database self.prepare() + def get_default_database(self): + return self.default_database + def restart(self): try: self.clear() @@ -119,10 +130,17 @@ class PostgresManager: raise ex def prepare(self): - conn = get_postgres_conn(ip=self.ip, port=self.port) - cursor = conn.cursor() - self.create_postgres_db(cursor, "postgres_database") - self.create_clickhouse_postgres_db(ip=self.ip, port=self.port) + self.conn = get_postgres_conn(ip=self.ip, port=self.port) + self.cursor = self.conn.cursor() + if self.default_database != "": + self.create_postgres_db(self.default_database) + self.conn = get_postgres_conn( + ip=self.ip, + port=self.port, + database=True, + database_name=self.default_database, + ) + self.cursor = self.conn.cursor() def clear(self): if self.conn.closed == 0: @@ -132,63 +150,76 @@ class PostgresManager: for db in self.created_ch_postgres_db_list.copy(): self.drop_clickhouse_postgres_db(db) if len(self.created_postgres_db_list) > 0: - conn = get_postgres_conn(ip=self.ip, port=self.port) - cursor = conn.cursor() + self.conn = get_postgres_conn(ip=self.ip, port=self.port) + self.cursor = self.conn.cursor() for db in self.created_postgres_db_list.copy(): - self.drop_postgres_db(cursor, db) + self.drop_postgres_db(db) - def get_db_cursor(self): - self.conn = get_postgres_conn(ip=self.ip, port=self.port, database=True) + def get_db_cursor(self, database_name=""): + if database_name == "": + database_name = self.default_database + self.conn = get_postgres_conn( + ip=self.ip, port=self.port, database=True, database_name=database_name + ) return self.conn.cursor() - def create_postgres_db(self, cursor, name="postgres_database"): - self.drop_postgres_db(cursor, name) - self.created_postgres_db_list.add(name) - cursor.execute(f"CREATE DATABASE {name}") + def database_or_default(self, database_name): + if database_name == "" and self.default_database == "": + raise Exception("Database name is empty") + if database_name == "": + database_name = self.default_database + return database_name - def drop_postgres_db(self, cursor, name="postgres_database"): - cursor.execute(f"DROP DATABASE IF EXISTS {name}") - if name in self.created_postgres_db_list: - self.created_postgres_db_list.remove(name) + def create_postgres_db(self, database_name=""): + database_name = self.database_or_default(database_name) + self.drop_postgres_db(database_name) + self.created_postgres_db_list.add(database_name) + self.cursor.execute(f"CREATE DATABASE {database_name}") + + def drop_postgres_db(self, database_name=""): + database_name = self.database_or_default(database_name) + self.cursor.execute(f"DROP DATABASE IF EXISTS {database_name}") + if database_name in self.created_postgres_db_list: + self.created_postgres_db_list.remove(database_name) def create_clickhouse_postgres_db( self, - ip, - port, - name="postgres_database", - database_name="postgres_database", + database_name="", schema_name="", ): - self.drop_clickhouse_postgres_db(name) - self.created_ch_postgres_db_list.add(name) + database_name = self.database_or_default(database_name) + self.drop_clickhouse_postgres_db(database_name) + self.created_ch_postgres_db_list.add(database_name) if len(schema_name) == 0: self.instance.query( f""" - CREATE DATABASE {name} - ENGINE = PostgreSQL('{ip}:{port}', '{database_name}', 'postgres', 'mysecretpassword')""" + CREATE DATABASE {database_name} + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword')""" ) else: self.instance.query( f""" - CREATE DATABASE {name} - ENGINE = PostgreSQL('{ip}:{port}', '{database_name}', 'postgres', 'mysecretpassword', '{schema_name}')""" + CREATE DATABASE {database_name} + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword', '{schema_name}')""" ) - def drop_clickhouse_postgres_db(self, name="postgres_database"): - self.instance.query(f"DROP DATABASE IF EXISTS {name}") - if name in self.created_ch_postgres_db_list: - self.created_ch_postgres_db_list.remove(name) + def drop_clickhouse_postgres_db(self, database_name=""): + database_name = self.database_or_default(database_name) + self.instance.query(f"DROP DATABASE IF EXISTS {database_name}") + if database_name in self.created_ch_postgres_db_list: + self.created_ch_postgres_db_list.remove(database_name) def create_materialized_db( self, ip, port, materialized_database="test_database", - postgres_database="postgres_database", + postgres_database="", settings=[], table_overrides="", ): + postgres_database = self.database_or_default(postgres_database) self.created_materialized_postgres_db_list.add(materialized_database) self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database}") @@ -207,17 +238,12 @@ class PostgresManager: self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database} SYNC") if materialized_database in self.created_materialized_postgres_db_list: self.created_materialized_postgres_db_list.remove(materialized_database) - assert materialized_database not in self.instance.query("SHOW DATABASES") - def create_and_fill_postgres_table(self, table_name): - conn = get_postgres_conn(ip=self.ip, port=self.port, database=True) - cursor = conn.cursor() - self.create_and_fill_postgres_table_from_cursor(cursor, table_name) - - def create_and_fill_postgres_table_from_cursor(self, cursor, table_name): - create_postgres_table(cursor, table_name) + def create_and_fill_postgres_table(self, table_name, database_name=""): + create_postgres_table(self.cursor, table_name, database_name) + database_name = self.database_or_default(database_name) self.instance.query( - f"INSERT INTO postgres_database.{table_name} SELECT number, number from numbers(50)" + f"INSERT INTO {database_name}.{table_name} SELECT number, number from numbers(50)" ) def create_and_fill_postgres_tables(self, tables_num, numbers=50): diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 90d19e9532c..3b5194e8806 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -46,14 +46,34 @@ instance = cluster.add_instance( stay_alive=True, ) +instance2 = cluster.add_instance( + "instance2", + main_configs=["configs/log_conf.xml", "configs/merge_tree_too_many_parts.xml"], + user_configs=["configs/users.xml"], + with_postgres=True, + stay_alive=True, +) + + pg_manager = PostgresManager() +pg_manager2 = PostgresManager() @pytest.fixture(scope="module") def started_cluster(): try: cluster.start() - pg_manager.init(instance, cluster.postgres_ip, cluster.postgres_port) + pg_manager.init( + instance, + cluster.postgres_ip, + cluster.postgres_port, + default_database="test_database", + ) + pg_manager.create_clickhouse_postgres_db() + pg_manager2.init( + instance2, cluster.postgres_ip, cluster.postgres_port, "test_database2" + ) + pg_manager2.create_clickhouse_postgres_db() yield cluster finally: @@ -649,6 +669,59 @@ def test_materialized_view(started_cluster): pg_manager.drop_materialized_db() +def test_too_many_parts(started_cluster): + table = "test_table" + pg_manager2.create_and_fill_postgres_table(table) + pg_manager2.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = 'test_table', materialized_postgresql_backoff_min_ms = 100, materialized_postgresql_backoff_max_ms = 100" + ], + ) + check_tables_are_synchronized( + instance2, "test_table", postgres_database=pg_manager2.get_default_database() + ) + assert ( + "50" == instance2.query("SELECT count() FROM test_database.test_table").strip() + ) + + instance2.query("SYSTEM STOP MERGES") + num = 50 + for i in range(10): + instance2.query( + f""" + INSERT INTO {pg_manager2.get_default_database()}.test_table SELECT {num}, {num}; + """ + ) + num = num + 1 + for i in range(30): + if num == int( + instance2.query("SELECT count() FROM test_database.test_table") + ) or instance2.contains_in_log("DB::Exception: Too many parts"): + break + time.sleep(1) + print(f"wait sync try {i}") + if instance2.contains_in_log("DB::Exception: Too many parts"): + num = num - 1 + break + assert num == int( + instance2.query("SELECT count() FROM test_database.test_table") + ) + + assert instance2.contains_in_log("DB::Exception: Too many parts") + print(num) + assert num == int(instance2.query("SELECT count() FROM test_database.test_table")) + + instance2.query("SYSTEM START MERGES") + check_tables_are_synchronized( + instance2, "test_table", postgres_database=pg_manager2.get_default_database() + ) + + # assert "200" == instance.query("SELECT count FROM test_database.test_table").strip() + pg_manager2.drop_materialized_db() + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From f0bfdb6b029748a486e5f683171f135d6a5dd957 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 22 Feb 2023 16:24:02 +0000 Subject: [PATCH 0071/2047] Refactor Query Tree visitor --- src/Analyzer/InDepthQueryTreeVisitor.h | 65 +++++++++++++++++++++++ src/Analyzer/Passes/CountDistinctPass.cpp | 34 ++++++++---- 2 files changed, 88 insertions(+), 11 deletions(-) diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index 1cc48fb1e53..ee321842ffa 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -235,4 +235,69 @@ public: template using ConstInDepthQueryTreeConditionalVisitor = InDepthQueryTreeConditionalVisitor; +template +class QueryTreeVisitor +{ +public: + explicit QueryTreeVisitor(ContextPtr context_) + : current_context(std::move(context_)) + {} + + bool needApply(QueryTreeNodePtr & node) + { + return getImpl().needApply(node); + } + + void visit(QueryTreeNodePtr & node) + { + auto current_scope_context_ptr = current_context; + SCOPE_EXIT( + current_context = std::move(current_scope_context_ptr); + ); + + if (auto * query_node = node->template as()) + current_context = query_node->getContext(); + else if (auto * union_node = node->template as()) + current_context = union_node->getContext(); + + if (!TOP_TO_BOTTOM) + visitChildren(node); + + if (needApply(node)) + getImpl().apply(node); + + if (TOP_TO_BOTTOM) + visitChildren(node); + } + + const ContextPtr & getContext() const + { + return current_context; + } + + const Settings & getSettings() const + { + return current_context->getSettingsRef(); + } +private: + + Impl & getImpl() + { + return *static_cast(this); + } + + void visitChildren(QueryTreeNodePtr & node) + { + for (auto & child : node->getChildren()) + { + if (child) + visit(child); + } + } + + static constexpr bool TOP_TO_BOTTOM = Impl::TOP_TO_BOTTOM; + + ContextPtr current_context; +}; + } diff --git a/src/Analyzer/Passes/CountDistinctPass.cpp b/src/Analyzer/Passes/CountDistinctPass.cpp index 945295f5cbc..38f7d07d052 100644 --- a/src/Analyzer/Passes/CountDistinctPass.cpp +++ b/src/Analyzer/Passes/CountDistinctPass.cpp @@ -16,16 +16,17 @@ namespace DB namespace { -class CountDistinctVisitor : public InDepthQueryTreeVisitorWithContext +class CountDistinctVisitor : public QueryTreeVisitor { public: - using Base = InDepthQueryTreeVisitorWithContext; - using Base::Base; + using QueryTreeVisitor::QueryTreeVisitor; - void visitImpl(QueryTreeNodePtr & node) + static constexpr bool TOP_TO_BOTTOM = true; + + bool needApply(QueryTreeNodePtr & node) { if (!getSettings().count_distinct_optimization) - return; + return false; auto * query_node = node->as(); @@ -33,32 +34,43 @@ public: if (!query_node || (query_node->hasWith() || query_node->hasPrewhere() || query_node->hasWhere() || query_node->hasGroupBy() || query_node->hasHaving() || query_node->hasWindow() || query_node->hasOrderBy() || query_node->hasLimitByLimit() || query_node->hasLimitByOffset() || query_node->hasLimitBy() || query_node->hasLimit() || query_node->hasOffset())) - return; + return false; /// Check that query has only single table expression auto join_tree_node_type = query_node->getJoinTree()->getNodeType(); if (join_tree_node_type == QueryTreeNodeType::JOIN || join_tree_node_type == QueryTreeNodeType::ARRAY_JOIN) - return; + return false; /// Check that query has only single node in projection auto & projection_nodes = query_node->getProjection().getNodes(); if (projection_nodes.size() != 1) - return; + return false; /// Check that query single projection node is `countDistinct` function auto & projection_node = projection_nodes[0]; auto * function_node = projection_node->as(); if (!function_node) - return; + return false; auto lower_function_name = Poco::toLower(function_node->getFunctionName()); if (lower_function_name != "countdistinct" && lower_function_name != "uniqexact") - return; + return false; /// Check that `countDistinct` function has single COLUMN argument auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); if (count_distinct_arguments_nodes.size() != 1 && count_distinct_arguments_nodes[0]->getNodeType() != QueryTreeNodeType::COLUMN) - return; + return false; + + return true; + } + + void apply(QueryTreeNodePtr & node) + { + auto * query_node = node->as(); + auto & projection_nodes = query_node->getProjection().getNodes(); + auto * function_node = projection_nodes[0]->as(); + + auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); auto & count_distinct_argument_column = count_distinct_arguments_nodes[0]; auto & count_distinct_argument_column_typed = count_distinct_argument_column->as(); From 5c34ee3019199a7e1d24730684c9c84e6c8e0615 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 9 May 2023 15:14:49 +0000 Subject: [PATCH 0072/2047] Skip unresolved table function arguments --- src/Analyzer/InDepthQueryTreeVisitor.h | 15 ++++++++++++++- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Analyzer/TableFunctionNode.cpp | 3 ++- src/Analyzer/TableFunctionNode.h | 8 +++++++- 4 files changed, 24 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index ee321842ffa..be3a760d4e6 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -7,6 +7,7 @@ #include #include +#include #include #include @@ -248,6 +249,16 @@ public: return getImpl().needApply(node); } + bool shouldSkipSubtree(QueryTreeNodePtr & parent, size_t subtree_index) + { + if (auto * table_function_node = parent->as()) + { + 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 visit(QueryTreeNodePtr & node) { auto current_scope_context_ptr = current_context; @@ -288,10 +299,12 @@ private: void visitChildren(QueryTreeNodePtr & node) { + size_t index = 0; for (auto & child : node->getChildren()) { - if (child) + if (child && !shouldSkipSubtree(node, index)) visit(child); + ++index; } } diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 7ab0261850b..aaea81dcada 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6356,7 +6356,7 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, table_function_ptr->parseArguments(table_function_ast, scope_context); auto table_function_storage = table_function_ptr->execute(table_function_ast, scope_context, table_function_ptr->getName()); - 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 diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index c130503d660..30644ad4ec4 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -27,12 +27,13 @@ TableFunctionNode::TableFunctionNode(String table_function_name_) children[arguments_child_index] = std::make_shared(); } -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 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 diff --git a/src/Analyzer/TableFunctionNode.h b/src/Analyzer/TableFunctionNode.h index 7786ba62205..69237ac8416 100644 --- a/src/Analyzer/TableFunctionNode.h +++ b/src/Analyzer/TableFunctionNode.h @@ -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 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 & 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 unresolved_arguments_indexes; std::optional table_expression_modifiers; SettingsChanges settings_changes; From 230f79a3229daf11c169c20be909c0ea4f04958e Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Wed, 10 May 2023 11:51:25 +0300 Subject: [PATCH 0073/2047] Fix bug in StorageBuffer::reschedule() --- src/Storages/StorageBuffer.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a4cb15d5711..757d3161067 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -996,8 +996,11 @@ void StorageBuffer::reschedule() std::unique_lock lock(buffer.tryLock()); if (lock.owns_lock()) { - min_first_write_time = buffer.first_write_time; - rows += buffer.data.rows(); + if (buffer.data) + { + min_first_write_time = std::min(min_first_write_time, buffer.first_write_time); + rows += buffer.data.rows(); + } } } From 446cf3c847eb77608fd01a1020fb16bda8f4ccf5 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Wed, 10 May 2023 18:17:26 +0300 Subject: [PATCH 0074/2047] handle failed files --- src/Core/Settings.h | 1 + src/Storages/S3Queue/S3QueueHolder.cpp | 269 ++++++++++-------- src/Storages/S3Queue/S3QueueHolder.h | 74 +++-- src/Storages/S3Queue/S3QueueSettings.h | 15 +- src/Storages/S3Queue/S3QueueSource.cpp | 45 ++- src/Storages/S3Queue/S3QueueSource.h | 1 + src/Storages/S3Queue/StorageS3Queue.cpp | 32 ++- .../integration/test_storage_s3_queue/test.py | 151 +++++++++- 8 files changed, 425 insertions(+), 163 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1bea2c26392..1e0e0615f66 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -94,6 +94,7 @@ class IColumn; M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ + M(String, s3queue_default_zookeeper_path, "", "Default zookeeper path prefix for S3Queue engine", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 1b708154bcd..91bf82001bb 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -16,20 +16,14 @@ namespace DB namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int NO_ZOOKEEPER; extern const int TIMEOUT_EXCEEDED; } -S3QueueHolder::ProcessedCollection::ProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) - : max_size(max_size_), max_age(max_age_) -{ -} - -void S3QueueHolder::ProcessedCollection::read(ReadBuffer & in) +void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) { files = {}; - in >> "processed_files\n"; + in >> "collection:\n"; while (!in.eof()) { String file_name; @@ -41,9 +35,9 @@ void S3QueueHolder::ProcessedCollection::read(ReadBuffer & in) } } -void S3QueueHolder::ProcessedCollection::write(WriteBuffer & out) const +void S3QueueHolder::S3QueueCollection::write(WriteBuffer & out) const { - out << "processed_files\n"; + out << "collection:\n"; for (const auto & processed_file : files) { out << processed_file.first << "\n"; @@ -51,7 +45,30 @@ void S3QueueHolder::ProcessedCollection::write(WriteBuffer & out) const } } -void S3QueueHolder::ProcessedCollection::parse(const String & s) +String S3QueueHolder::S3QueueCollection::toString() const +{ + WriteBufferFromOwnString out; + write(out); + return out.str(); +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueCollection::getFileNames() +{ + S3FilesCollection keys = {}; + for (const auto & pair : files) + { + keys.insert(pair.first); + } + return keys; +} + + +S3QueueHolder::S3QueueProcessedCollection::S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) + : max_size(max_size_), max_age(max_age_) +{ +} + +void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) { ReadBufferFromString buf(s); read(buf); @@ -69,14 +86,8 @@ void S3QueueHolder::ProcessedCollection::parse(const String & s) } } -String S3QueueHolder::ProcessedCollection::toString() const -{ - WriteBufferFromOwnString out; - write(out); - return out.str(); -} -void S3QueueHolder::ProcessedCollection::add(const String & file_name) +void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) { Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); auto pair = std::make_pair(file_name, timestamp); @@ -89,22 +100,62 @@ void S3QueueHolder::ProcessedCollection::add(const String & file_name) } } -S3QueueHolder::S3FilesCollection S3QueueHolder::ProcessedCollection::getFileNames() + +S3QueueHolder::S3QueueFailedCollection::S3QueueFailedCollection(const UInt64 & max_retries_count_) : max_retries_count(max_retries_count_) { - S3FilesCollection keys = {}; - for (auto & pair : files) - { - keys.insert(pair.first); - } - return keys; +} + +void S3QueueHolder::S3QueueFailedCollection::parse(const String & s) +{ + ReadBufferFromString buf(s); + read(buf); } +bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) +{ + auto failed_it + = std::find_if(files.begin(), files.end(), [&file_name](const std::pair & s) { return s.first == file_name; }); + if (failed_it != files.end()) + { + failed_it->second--; + if (failed_it->second == 0) + { + return false; + } + } + else + { + auto pair = std::make_pair(file_name, max_retries_count); + files.push_back(pair); + } + return true; +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueFailedCollection::getFilesWithoutRetries() +{ + S3FilesCollection failed_keys; + for (const auto & pair : files) + { + if (pair.second <= 0) + { + failed_keys.insert(pair.first); + } + } + return failed_keys; +} + S3QueueHolder::S3QueueHolder( - const String & zookeeper_path_, const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, UInt64 & max_set_age_s_) + const String & zookeeper_path_, + const S3QueueMode & mode_, + ContextPtr context_, + UInt64 & max_set_size_, + UInt64 & max_set_age_s_, + UInt64 & max_loading_retries_) : WithContext(context_) , max_set_size(max_set_size_) , max_set_age_s(max_set_age_s_) + , max_loading_retries(max_loading_retries_) , zookeeper_path(zookeeper_path_) , zookeeper_failed_path(zookeeper_path_ + "/failed") , zookeeper_processing_path(zookeeper_path_ + "/processing") @@ -140,7 +191,7 @@ void S3QueueHolder::setFileProcessed(const String & file_path) if (mode == S3QueueMode::UNORDERED) { String processed_files = zookeeper->get(zookeeper_processed_path); - auto processed = ProcessedCollection(max_set_size, max_set_age_s); + auto processed = S3QueueProcessedCollection(max_set_size, max_set_age_s); processed.parse(processed_files); processed.add(file_path); zookeeper->set(zookeeper_processed_path, processed.toString()); @@ -153,144 +204,111 @@ void S3QueueHolder::setFileProcessed(const String & file_path) zookeeper->set(zookeeper_processed_path, file_path); } } - - String node_data; - Strings file_paths; - if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) - { - S3FilesCollection processing_files = parseCollection(node_data); - for (const auto & x : processing_files) - { - if (x != file_path) - { - file_paths.push_back(x); - } - } - } - zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); + removeProcessingFile(file_path); } -void S3QueueHolder::setFileFailed(const String & file_path) +bool S3QueueHolder::markFailedAndCheckRetry(const String & file_path) { auto zookeeper = getZooKeeper(); auto lock = AcquireLock(); String failed_files = zookeeper->get(zookeeper_failed_path); - S3FilesCollection failed = parseCollection(failed_files); + auto failed_collection = S3QueueFailedCollection(max_loading_retries); + failed_collection.parse(failed_files); + bool retry_later = failed_collection.add(file_path); - failed.insert(file_path); - Strings set_failed; - set_failed.insert(set_failed.end(), failed.begin(), failed.end()); + zookeeper->set(zookeeper_failed_path, failed_collection.toString()); + removeProcessingFile(file_path); - zookeeper->set(zookeeper_failed_path, toString(set_failed)); + return retry_later; } -S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) +S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() { - ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); - Strings deserialized; - try - { - readQuoted(deserialized, rb); - } - catch (...) - { - deserialized = {}; - } - - std::unordered_set processed(deserialized.begin(), deserialized.end()); - - return processed; -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getExcludedFiles() -{ - std::unordered_set exclude_files; auto zookeeper = getZooKeeper(); + String failed_files = zookeeper->get(zookeeper_failed_path); - String failed = zookeeper->get(zookeeper_failed_path); - S3FilesCollection failed_files = parseCollection(failed); - exclude_files.merge(failed_files); + auto failed_collection = S3QueueFailedCollection(max_loading_retries); + failed_collection.parse(failed_files); - String processed = zookeeper->get(zookeeper_processed_path); - if (mode != S3QueueMode::ORDERED) - { - auto collection = ProcessedCollection(max_set_size, max_set_age_s); - collection.parse(processed); - S3FilesCollection processed_files = collection.getFileNames(); - exclude_files.merge(processed_files); - } - else - { - exclude_files.insert(processed); - } - - String processing = zookeeper->get(fs::path(zookeeper_processing_path)); - S3FilesCollection processing_files = parseCollection(processing); - exclude_files.merge(processing_files); - - return exclude_files; + return failed_collection.getFilesWithoutRetries(); } String S3QueueHolder::getMaxProcessedFile() { - if (mode != S3QueueMode::ORDERED) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getMaxProcessedFile not implemented for unordered mode"); - auto zookeeper = getZooKeeper(); - String processed = zookeeper->get(zookeeper_path + "/processed"); + String processed = zookeeper->get(zookeeper_processed_path); return processed; } +S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() +{ + auto zookeeper = getZooKeeper(); + String processing = zookeeper->get(fs::path(zookeeper_processing_path)); + return parseCollection(processing); +} + void S3QueueHolder::setFilesProcessing(Strings & file_paths) { auto zookeeper = getZooKeeper(); - String node_data; - if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) + for (const auto & x : getProcessingFiles()) { - S3FilesCollection processing_files = parseCollection(node_data); - for (const auto & x : processing_files) + if (!std::count(file_paths.begin(), file_paths.end(), x)) { - if (!std::count(file_paths.begin(), file_paths.end(), x)) - { - file_paths.push_back(x); - } + file_paths.push_back(x); } } zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); } -S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() +S3QueueHolder::S3FilesCollection S3QueueHolder::getUnorderedProcessedFiles() { auto zookeeper = getZooKeeper(); - auto lock = AcquireLock(); - String failed = zookeeper->get(zookeeper_failed_path); - return parseCollection(failed); -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessedFiles() -{ - auto zookeeper = getZooKeeper(); - - auto lock = AcquireLock(); String processed = zookeeper->get(zookeeper_processed_path); - auto collection = ProcessedCollection(max_set_size, max_set_age_s); + auto collection = S3QueueProcessedCollection(max_set_size, max_set_age_s); collection.parse(processed); + return collection.getFileNames(); } -S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() +S3QueueHolder::S3FilesCollection S3QueueHolder::getExcludedFiles() { auto zookeeper = getZooKeeper(); - auto lock = AcquireLock(); - String processing = zookeeper->get(fs::path(zookeeper_processing_path)); - return parseCollection(processing); + S3FilesCollection exclude_files = getFailedFiles(); + + if (mode == S3QueueMode::UNORDERED) + { + S3FilesCollection processed_files = getUnorderedProcessedFiles(); + exclude_files.merge(processed_files); + } + else + { + String processed = getMaxProcessedFile(); + exclude_files.insert(processed); + } + + S3FilesCollection processing_files = getProcessingFiles(); + exclude_files.merge(processing_files); + + return exclude_files; } +void S3QueueHolder::removeProcessingFile(const String & file_path) +{ + auto zookeeper = getZooKeeper(); + String node_data; + Strings file_paths; + String processing = zookeeper->get(zookeeper_processing_path); + S3FilesCollection processing_files = parseCollection(processing); + file_paths.insert(file_paths.end(), processing_files.begin(), processing_files.end()); + + file_paths.erase(std::remove(file_paths.begin(), file_paths.end(), file_path), file_paths.end()); + zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); +} std::shared_ptr S3QueueHolder::AcquireLock() { @@ -322,6 +340,23 @@ std::shared_ptr S3QueueHolder::AcquireLock() } } +S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) +{ + ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); + Strings deserialized; + try + { + readQuoted(deserialized, rb); + } + catch (...) + { + deserialized = {}; + } + + std::unordered_set processed(deserialized.begin(), deserialized.end()); + + return processed; +} } diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index 081e58a9ea2..f7fa0461a3a 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -14,44 +14,73 @@ class S3QueueHolder : public WithContext public: using S3FilesCollection = std::unordered_set; using ProcessedFiles = std::vector>; + using FailedFiles = std::vector>; S3QueueHolder( - const String & zookeeper_path_, const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, UInt64 & max_set_age_s_); + const String & zookeeper_path_, + const S3QueueMode & mode_, + ContextPtr context_, + UInt64 & max_set_size_, + UInt64 & max_set_age_s_, + UInt64 & max_loading_retries_); void setFileProcessed(const String & file_path); - void setFileFailed(const String & file_path); + bool markFailedAndCheckRetry(const String & file_path); void setFilesProcessing(Strings & file_paths); - static S3FilesCollection parseCollection(String & files); - S3FilesCollection getExcludedFiles(); String getMaxProcessedFile(); - S3FilesCollection getFailedFiles(); - S3FilesCollection getProcessedFiles(); - S3FilesCollection getProcessingFiles(); + std::shared_ptr AcquireLock(); - struct ProcessedCollection + struct S3QueueCollection { - ProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_); - - void parse(const String & s); - + public: + virtual ~S3QueueCollection() = default; String toString() const; - - void add(const String & file_name); S3FilesCollection getFileNames(); - const UInt64 max_size; - const UInt64 max_age; + + virtual void parse(const String & s) = 0; + + protected: + ProcessedFiles files; void read(ReadBuffer & in); void write(WriteBuffer & out) const; - ProcessedFiles files; }; - const UInt64 max_set_size; - const UInt64 max_set_age_s; + struct S3QueueProcessedCollection : public S3QueueCollection + { + public: + S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_); + + void parse(const String & s) override; + void add(const String & file_name); + + private: + const UInt64 max_size; + const UInt64 max_age; + }; + + struct S3QueueFailedCollection : S3QueueCollection + { + public: + S3QueueFailedCollection(const UInt64 & max_retries_count_); + + void parse(const String & s) override; + bool add(const String & file_name); + + S3FilesCollection getFilesWithoutRetries(); + + private: + const UInt64 max_retries_count; + }; + private: + const UInt64 max_set_size; + const UInt64 max_set_age_s; + const UInt64 max_loading_retries; + zkutil::ZooKeeperPtr current_zookeeper; mutable std::mutex current_zookeeper_mutex; mutable std::mutex mutex; @@ -66,6 +95,13 @@ private: zkutil::ZooKeeperPtr tryGetZooKeeper() const; zkutil::ZooKeeperPtr getZooKeeper() const; + + S3FilesCollection getFailedFiles(); + S3FilesCollection getProcessingFiles(); + S3FilesCollection getUnorderedProcessedFiles(); + void removeProcessingFile(const String & file_path); + + static S3FilesCollection parseCollection(String & files); }; diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index c2b8e51a1f8..3b4ea475887 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -14,16 +14,21 @@ class ASTStorage; M(S3QueueMode, \ mode, \ S3QueueMode::ORDERED, \ - "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", \ + "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer." \ + "With ordered mode, only the max name of the successfully consumed file stored.", \ 0) \ - M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete, keep or move file in S3 after processing", 0) \ + M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete or keep file in S3 after successful processing", 0) \ M(String, keeper_path, "", "Zookeeper node path", 0) \ - M(UInt64, s3queue_max_retries, 0, "Retry loading up to specified number of times", 0) \ + M(UInt64, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ M(UInt64, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ - M(UInt64, s3queue_polling_backoff_ms, 0, "Retry loading up to specified number of times", 0) \ + M(UInt64, s3queue_polling_backoff_ms, 0, "Polling backoff", 0) \ M(UInt64, s3queue_max_set_size, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ - M(UInt64, s3queue_max_set_age_s, 0, "Maximum number of seconds to store processed files in ZooKeeper node (forever default)", 0) \ + M(UInt64, \ + s3queue_max_set_age_s, \ + 0, \ + "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", \ + 0) \ M(UInt64, s3queue_polling_size, 50, "Maximum files to fetch from S3 with SELECT", 0) #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index eb49db8e84d..27fcdcc892c 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -69,6 +69,11 @@ extern const Event S3ListObjects; namespace DB { +namespace ErrorCodes +{ + extern const int S3_ERROR; +} + StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( const S3::Client & client_, @@ -244,13 +249,15 @@ Chunk StorageS3QueueSource::generate() } Chunk chunk; + bool success_in_pulling = false; + String file_path; try { if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); - const auto & file_path = reader.getPath(); + file_path = reader.getPath(); size_t total_size = file_iterator->getTotalSize(); if (num_rows && total_size) { @@ -271,16 +278,21 @@ Chunk StorageS3QueueSource::generate() chunk.addColumn(column->convertToFullColumnIfConst()); } } - queue_holder->setFileProcessed(file_path); - applyActionAfterProcessing(file_path); - return chunk; + success_in_pulling = true; } } catch (const Exception & e) { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); const auto & failed_file_path = reader.getPath(); - queue_holder->setFileFailed(failed_file_path); + queue_holder->markFailedAndCheckRetry(failed_file_path); + success_in_pulling = false; + } + if (success_in_pulling) + { + applyActionAfterProcessing(file_path); + queue_holder->setFileProcessed(file_path); + return chunk; } @@ -298,14 +310,28 @@ Chunk StorageS3QueueSource::generate() return {}; } + void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) { - LOG_WARNING(log, "Delete {} Bucket {}", file_path, bucket); + switch (action) + { + case S3QueueAction::DELETE: + deleteProcessedObject(file_path); + break; + case S3QueueAction::KEEP: + break; + } +} + +void StorageS3QueueSource::deleteProcessedObject(const String & file_path) +{ + LOG_WARNING(log, "Delete processed file {} from bucket {}", file_path, bucket); S3::DeleteObjectRequest request; - request.SetBucket(bucket); - request.SetKey(file_path); + String delete_key = file_path.substr(bucket.length() + 1); + + request.WithKey(delete_key).WithBucket(bucket); auto outcome = client->DeleteObject(request); - if (!outcome.IsSuccess() && !S3::isNotFoundError(outcome.GetError().GetErrorType())) + if (!outcome.IsSuccess()) { const auto & err = outcome.GetError(); LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); @@ -316,7 +342,6 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) } } - } #endif diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index fc7ce3606b0..a55dd2cbe40 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -123,6 +123,7 @@ private: mutable std::mutex mutex; std::shared_ptr internal_source; + void deleteProcessedObject(const String & file_path); void applyActionAfterProcessing(const String & file_path); }; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 5a011c9b51a..113abec4b09 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -59,11 +59,6 @@ namespace fs = std::filesystem; -//namespace CurrentMetrics -//{ -//extern const Metric S3QueueBackgroundReads; -//} - namespace ProfileEvents { extern const Event S3DeleteObjects; @@ -131,7 +126,14 @@ StorageS3Queue::StorageS3Queue( auto table_id = getStorageID(); auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); bool is_in_replicated_database = database->getEngineName() == "Replicated"; - if (is_in_replicated_database) + + auto default_path = getContext()->getSettingsRef().s3queue_default_zookeeper_path.value; + if (default_path != "") + { + zookeeper_path + = zkutil::extractZooKeeperPath(fs::path(default_path) / toString(table_id.uuid), /* check_starts_with_slash */ true, log); + } + else if (is_in_replicated_database) { LOG_INFO(log, "S3Queue engine keeper_path not specified. Use replicated database zookeeper path"); String base_zookeeper_path = assert_cast(database.get())->getZooKeeperPath(); @@ -140,7 +142,10 @@ StorageS3Queue::StorageS3Queue( } else { - throw Exception(ErrorCodes::NO_ZOOKEEPER, "S3Queue zookeeper path not specified and table not in replicated database."); + throw Exception( + ErrorCodes::NO_ZOOKEEPER, + "S3Queue keeper_path engine setting not specified, s3queue_default_zookeeper_path_prefix not specified and table not in " + "replicated database."); } } else @@ -176,7 +181,12 @@ StorageS3Queue::StorageS3Queue( } queue_holder = std::make_unique( - zookeeper_path, mode, getContext(), s3queue_settings->s3queue_max_set_size.value, s3queue_settings->s3queue_max_set_age_s.value); + zookeeper_path, + mode, + getContext(), + s3queue_settings->s3queue_max_set_size.value, + s3queue_settings->s3queue_max_set_age_s.value, + s3queue_settings->s3queue_loading_retries.value); auto default_virtuals = NamesAndTypesList{ {"_path", std::make_shared(std::make_shared())}, @@ -187,7 +197,7 @@ StorageS3Queue::StorageS3Queue( for (const auto & column : virtual_columns) virtual_block.insert({column.type->createColumn(), column.type, column.name}); - auto poll_thread = context_->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); + auto poll_thread = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); task = std::make_shared(std::move(poll_thread)); } @@ -551,8 +561,8 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ { String metadata_str = S3QueueTableMetadata(s3_configuration, *s3queue_settings).toString(); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "processed_files\n", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "collection:\n", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "collection:\n", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index bc92e809f5d..085f92035d0 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -7,7 +7,72 @@ import time import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster, ClickHouseInstance -from helpers.s3_tools import prepare_s3_bucket +import json + +""" +export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-server +export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-client +export CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-odbc-bridge +export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/home/sergey/vkr/ClickHouse/programs/server + +""" + + +def prepare_s3_bucket(started_cluster): + # Allows read-write access for bucket without authorization. + bucket_read_write_policy = { + "Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:DeleteObject", + "Resource": "arn:aws:s3:::root/*", + }, + ], + } + + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy( + started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) + ) + + started_cluster.minio_restricted_bucket = "{}-with-auth".format( + started_cluster.minio_bucket + ) + if minio_client.bucket_exists(started_cluster.minio_restricted_bucket): + minio_client.remove_bucket(started_cluster.minio_restricted_bucket) + + minio_client.make_bucket(started_cluster.minio_restricted_bucket) @pytest.fixture(autouse=True) @@ -112,6 +177,90 @@ def run_query(instance, query, stdin=None, settings=None): return result +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_delete_after_processing(started_cluster, mode): + prefix = "delete" + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + total_values = generate_random_files(5, prefix, started_cluster, bucket) + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + CREATE TABLE test.s3_queue ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/test_delete_{mode}', + s3queue_loading_retries = 3, + after_processing='delete'; + """ + ) + + get_query = f"SELECT * FROM test.s3_queue" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == total_values + minio = started_cluster.minio_client + objects = list(minio.list_objects(started_cluster.minio_bucket, recursive=True)) + assert len(objects) == 0 + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_failed_retry(started_cluster, mode): + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + values = [ + ["failed", 1, 1], + ] + values_csv = ( + "\n".join((",".join(map(str, row)) for row in values)) + "\n" + ).encode() + filename = f"test.csv" + put_s3_file_content(started_cluster, bucket, filename, values_csv) + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + CREATE TABLE test.s3_queue ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_failed_retry_{mode}', + s3queue_loading_retries = 3; + """ + ) + + # first try + get_query = f"SELECT * FROM test.s3_queue" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] + # second try + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] + # upload correct file + values = [ + [1, 1, 1], + ] + values_csv = ( + "\n".join((",".join(map(str, row)) for row in values)) + "\n" + ).encode() + put_s3_file_content(started_cluster, bucket, filename, values_csv) + + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == values + + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] + + @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_direct_select_file(started_cluster, mode): auth = "'minio','minio123'," From f9eb6ca6fd767666b7b2c8e88e92e8d02639ef99 Mon Sep 17 00:00:00 2001 From: xiebin Date: Wed, 10 May 2023 23:43:40 +0800 Subject: [PATCH 0075/2047] if the data type of numeric key is not native uint, convert to complex. --- .../getDictionaryConfigurationFromAST.cpp | 16 ++++++++++------ ...y_layout_to_complex_by_complex_keys.reference | 1 + ...tionary_layout_to_complex_by_complex_keys.sql | 5 +++++ 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 59f1a712d00..c179ce84ff9 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -19,6 +19,7 @@ #include #include #include +#include namespace DB @@ -609,17 +610,20 @@ getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr conte bool complex = DictionaryFactory::instance().isComplex(dictionary_layout->layout_type); - if (pk_attrs.size() > 1 && !complex - && DictionaryFactory::instance().convertToComplex(dictionary_layout->layout_type)) - { - complex = true; - } - auto all_attr_names_and_types = buildDictionaryAttributesConfiguration( xml_document, structure_element, query.dictionary_attributes_list, pk_attrs); checkPrimaryKey(all_attr_names_and_types, pk_attrs); + /// If the pk size is 1 and pk's DataType is not native uint(UInt8~UInt64), we should convert to complex, + /// because the data type of Numeric key(simple layout) is UInt64. + if ((pk_attrs.size() > 1 || (pk_attrs.size() == 1 && !WhichDataType(DataTypeFactory::instance().get(all_attr_names_and_types.find(pk_attrs[0])->second.type)).isNativeUInt())) + && !complex + && DictionaryFactory::instance().convertToComplex(dictionary_layout->layout_type)) + { + complex = true; + } + buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list); buildLayoutConfiguration(xml_document, current_dictionary, query.dictionary->dict_settings, dictionary_layout); diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference index 7f38556e7d1..cc9381622ec 100644 --- a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference @@ -2,3 +2,4 @@ dict_flat_simple Flat dict_hashed_simple Hashed dict_hashed_complex ComplexKeyHashed dict_hashed_simple_auto_convert ComplexKeyHashed +dict_hashed_simple_int_auto_convert ComplexKeyHashed diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql index 0fb06e5acc2..77d933afa9f 100644 --- a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql @@ -26,4 +26,9 @@ SYSTEM RELOAD DICTIONARY dict_hashed_simple_auto_convert; SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_auto_convert'; DROP DICTIONARY dict_hashed_simple_auto_convert; +CREATE DICTIONARY dict_hashed_simple_int_auto_convert (v0 Int16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple_int_auto_convert; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_int_auto_convert'; +DROP DICTIONARY dict_hashed_simple_int_auto_convert; + DROP TABLE dict_data; From 6b0bd698d36014a5eac052857bac2185a1f45f41 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 11 May 2023 04:17:53 +0200 Subject: [PATCH 0076/2047] Fix mistake --- .../02530_dictionaries_update_field.reference | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.reference b/tests/queries/0_stateless/02530_dictionaries_update_field.reference index 88c910e0313..40f2c0ee400 100644 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.reference +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.reference @@ -4,13 +4,13 @@ flat SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_flat ORDER BY key ASC; 1 First 2 SecondUpdated @@ -21,13 +21,13 @@ flat/custom SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_flat_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -38,13 +38,13 @@ hashed SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -55,13 +55,13 @@ hashed/custom SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated @@ -72,13 +72,13 @@ complex_key_hashed SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_complex_key_hashed ORDER BY key ASC; 1 First 2 SecondUpdated @@ -89,13 +89,13 @@ complex_key_hashed/custom SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First INSERT INTO table_for_update_field_dictionary VALUES (2, 'Second', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 Second INSERT INTO table_for_update_field_dictionary VALUES (2, 'SecondUpdated', now()); INSERT INTO table_for_update_field_dictionary VALUES (3, 'Third', now()); -SELECT sleepEachRow(1) FROM numbers(10) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; +SELECT sleepEachRow(1) FROM numbers(10) FORMAT Null; SELECT key, value FROM dict_complex_key_hashed_custom ORDER BY key ASC; 1 First 2 SecondUpdated From 65d28a959ff5b21199c2b20d8dcb7c7b399f314d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 11 May 2023 04:26:29 +0200 Subject: [PATCH 0077/2047] Update integration tests (1/2) --- .../configs/config.d/storage_conf.xml | 1 + .../test_merge_tree_hdfs/configs/config.d/storage_conf.xml | 1 + .../test_merge_tree_s3_failover/configs/config.xml | 4 ++++ .../test_s3_zero_copy_replication/configs/config.d/s3.xml | 1 + 4 files changed, 7 insertions(+) diff --git a/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml b/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml index cb87abcc693..d69fe96a3e2 100644 --- a/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml +++ b/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml @@ -45,5 +45,6 @@ true + 1.0 diff --git a/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml index 890c396ed95..7d59081486b 100644 --- a/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_hdfs/configs/config.d/storage_conf.xml @@ -28,5 +28,6 @@ 0 + 1.0 diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.xml index feb537ebbce..743d75d9a21 100644 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.xml +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.xml @@ -15,4 +15,8 @@ 500 ./clickhouse/ users.xml + + + 1.0 + diff --git a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml index f7d9efc2cae..55c35999703 100644 --- a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml +++ b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml @@ -70,6 +70,7 @@ 1024 1 true + 1.0 From c9e752fdc5c4cc401df240f7cd5f77586d9b542d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 11 May 2023 18:09:46 +0200 Subject: [PATCH 0078/2047] Fix --- tests/integration/helpers/postgres_utility.py | 51 ++++---- .../test.py | 113 +++++++----------- .../test.py | 64 +++++----- 3 files changed, 99 insertions(+), 129 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 1a00faf0f9d..3c8a23b15a2 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -87,7 +87,9 @@ def create_postgres_table( else: name = f"{database_name}.{table_name}" drop_postgres_table(cursor, name) - cursor.execute(template.format(name)) + query = template.format(name) + cursor.execute(query) + print(f"Query: {query}") if replica_identity_full: cursor.execute(f"ALTER TABLE {name} REPLICA IDENTITY FULL;") @@ -129,6 +131,9 @@ class PostgresManager: self.prepare() raise ex + def execute(self, query): + self.cursor.execute(query) + def prepare(self): self.conn = get_postgres_conn(ip=self.ip, port=self.port) self.cursor = self.conn.cursor() @@ -141,6 +146,7 @@ class PostgresManager: database_name=self.default_database, ) self.cursor = self.conn.cursor() + self.create_clickhouse_postgres_db() def clear(self): if self.conn.closed == 0: @@ -164,11 +170,11 @@ class PostgresManager: return self.conn.cursor() def database_or_default(self, database_name): - if database_name == "" and self.default_database == "": - raise Exception("Database name is empty") - if database_name == "": - database_name = self.default_database - return database_name + if database_name != "": + return database_name + if self.default_database != "": + return self.default_database + raise Exception("Database name is empty") def create_postgres_db(self, database_name=""): database_name = self.database_or_default(database_name) @@ -186,8 +192,11 @@ class PostgresManager: self, database_name="", schema_name="", + postgres_database="", ): database_name = self.database_or_default(database_name) + if postgres_database == "": + postgres_database = database_name self.drop_clickhouse_postgres_db(database_name) self.created_ch_postgres_db_list.add(database_name) @@ -195,13 +204,13 @@ class PostgresManager: self.instance.query( f""" CREATE DATABASE {database_name} - ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword')""" + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{postgres_database}', 'postgres', 'mysecretpassword')""" ) else: self.instance.query( f""" CREATE DATABASE {database_name} - ENGINE = PostgreSQL('{self.ip}:{self.port}', '{database_name}', 'postgres', 'mysecretpassword', '{schema_name}')""" + ENGINE = PostgreSQL('{self.ip}:{self.port}', '{postgres_database}', 'postgres', 'mysecretpassword', '{schema_name}')""" ) def drop_clickhouse_postgres_db(self, database_name=""): @@ -239,6 +248,16 @@ class PostgresManager: if materialized_database in self.created_materialized_postgres_db_list: self.created_materialized_postgres_db_list.remove(materialized_database) + def create_postgres_schema(self, name): + create_postgres_schema(self.cursor, name) + + def create_postgres_table( + self, table_name, database_name="", template=postgres_table_template + ): + create_postgres_table( + self.cursor, table_name, database_name=database_name, template=template + ) + def create_and_fill_postgres_table(self, table_name, database_name=""): create_postgres_table(self.cursor, table_name, database_name) database_name = self.database_or_default(database_name) @@ -246,22 +265,14 @@ class PostgresManager: f"INSERT INTO {database_name}.{table_name} SELECT number, number from numbers(50)" ) - def create_and_fill_postgres_tables(self, tables_num, numbers=50): - conn = get_postgres_conn(ip=self.ip, port=self.port, database=True) - cursor = conn.cursor() - self.create_and_fill_postgres_tables_from_cursor( - cursor, tables_num, numbers=numbers - ) - - def create_and_fill_postgres_tables_from_cursor( - self, cursor, tables_num, numbers=50 - ): + def create_and_fill_postgres_tables(self, tables_num, numbers=50, database_name=""): for i in range(tables_num): table_name = f"postgresql_replica_{i}" - create_postgres_table(cursor, table_name) + create_postgres_table(self.cursor, table_name, database_name) if numbers > 0: + db = self.database_or_default(database_name) self.instance.query( - f"INSERT INTO postgres_database.{table_name} SELECT number, number from numbers({numbers})" + f"INSERT INTO {db}.{table_name} SELECT number, number from numbers({numbers})" ) diff --git a/tests/integration/test_postgresql_replica_database_engine_1/test.py b/tests/integration/test_postgresql_replica_database_engine_1/test.py index 377b1c89efc..1eb2efc73a5 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_1/test.py @@ -46,7 +46,12 @@ pg_manager = PostgresManager() def started_cluster(): try: cluster.start() - pg_manager.init(instance, cluster.postgres_ip, cluster.postgres_port) + pg_manager.init( + instance, + cluster.postgres_ip, + cluster.postgres_port, + default_database="postgres_database", + ) yield cluster finally: @@ -74,16 +79,10 @@ def test_load_and_sync_all_database_tables(started_cluster): def test_replicating_dml(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() NUM_TABLES = 5 for i in range(NUM_TABLES): - create_postgres_table(cursor, "postgresql_replica_{}".format(i)) + pg_manager.create_postgres_table(f"postgresql_replica_{i}") instance.query( "INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50)".format( i, i @@ -96,39 +95,29 @@ def test_replicating_dml(started_cluster): for i in range(NUM_TABLES): instance.query( - "INSERT INTO postgres_database.postgresql_replica_{} SELECT 50 + number, {} from numbers(1000)".format( - i, i - ) + f"INSERT INTO postgres_database.postgresql_replica_{i} SELECT 50 + number, {i} from numbers(1000)" ) check_several_tables_are_synchronized(instance, NUM_TABLES) for i in range(NUM_TABLES): - cursor.execute( - "UPDATE postgresql_replica_{} SET value = {} * {} WHERE key < 50;".format( - i, i, i - ) + pg_manager.execute( + f"UPDATE postgresql_replica_{i} SET value = {i} * {i} WHERE key < 50;" ) - cursor.execute( - "UPDATE postgresql_replica_{} SET value = {} * {} * {} WHERE key >= 50;".format( - i, i, i, i - ) + pg_manager.execute( + f"UPDATE postgresql_replica_{i} SET value = {i} * {i} * {i} WHERE key >= 50;" ) + check_several_tables_are_synchronized(instance, NUM_TABLES) for i in range(NUM_TABLES): - cursor.execute( - "DELETE FROM postgresql_replica_{} WHERE (value*value + {}) % 2 = 0;".format( - i, i - ) + pg_manager.execute( + f"DELETE FROM postgresql_replica_{i} WHERE (value*value + {i}) % 2 = 0;" ) - cursor.execute( - "UPDATE postgresql_replica_{} SET value = value - (value % 7) WHERE key > 128 AND key < 512;".format( - i - ) - ) - cursor.execute( - "DELETE FROM postgresql_replica_{} WHERE key % 7 = 1;".format(i, i) + pg_manager.execute( + f"UPDATE postgresql_replica_{i} SET value = value - (value % 7) WHERE key > 128 AND key < 512;" ) + pg_manager.execute(f"DELETE FROM postgresql_replica_{i} WHERE key % 7 = 1;") + check_several_tables_are_synchronized(instance, NUM_TABLES) @@ -288,13 +277,7 @@ def test_load_and_sync_subset_of_database_tables(started_cluster): def test_changing_replica_identity_value(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") + pg_manager.create_postgres_table("postgresql_replica") instance.query( "INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, number from numbers(50)" ) @@ -307,7 +290,7 @@ def test_changing_replica_identity_value(started_cluster): "INSERT INTO postgres_database.postgresql_replica SELECT 100 + number, number from numbers(50)" ) check_tables_are_synchronized(instance, "postgresql_replica") - cursor.execute("UPDATE postgresql_replica SET key=key-25 WHERE key<100 ") + pg_manager.execute("UPDATE postgresql_replica SET key=key-25 WHERE key<100 ") check_tables_are_synchronized(instance, "postgresql_replica") @@ -331,18 +314,13 @@ def test_clickhouse_restart(started_cluster): def test_replica_identity_index(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, + pg_manager.create_postgres_table( + "postgresql_replica", template=postgres_table_template_3 ) - cursor = conn.cursor() - - create_postgres_table( - cursor, "postgresql_replica", template=postgres_table_template_3 + pg_manager.execute("CREATE unique INDEX idx on postgresql_replica(key1, key2);") + pg_manager.execute( + "ALTER TABLE postgresql_replica REPLICA IDENTITY USING INDEX idx" ) - cursor.execute("CREATE unique INDEX idx on postgresql_replica(key1, key2);") - cursor.execute("ALTER TABLE postgresql_replica REPLICA IDENTITY USING INDEX idx") instance.query( "INSERT INTO postgres_database.postgresql_replica SELECT number, number, number, number from numbers(50, 10)" ) @@ -355,35 +333,29 @@ def test_replica_identity_index(started_cluster): ) check_tables_are_synchronized(instance, "postgresql_replica", order_by="key1") - cursor.execute("UPDATE postgresql_replica SET key1=key1-25 WHERE key1<100 ") - cursor.execute("UPDATE postgresql_replica SET key2=key2-25 WHERE key2>100 ") - cursor.execute("UPDATE postgresql_replica SET value1=value1+100 WHERE key1<100 ") - cursor.execute("UPDATE postgresql_replica SET value2=value2+200 WHERE key2>100 ") + pg_manager.execute("UPDATE postgresql_replica SET key1=key1-25 WHERE key1<100 ") + pg_manager.execute("UPDATE postgresql_replica SET key2=key2-25 WHERE key2>100 ") + pg_manager.execute( + "UPDATE postgresql_replica SET value1=value1+100 WHERE key1<100 " + ) + pg_manager.execute( + "UPDATE postgresql_replica SET value2=value2+200 WHERE key2>100 " + ) check_tables_are_synchronized(instance, "postgresql_replica", order_by="key1") - cursor.execute("DELETE FROM postgresql_replica WHERE key2<75;") + pg_manager.execute("DELETE FROM postgresql_replica WHERE key2<75;") check_tables_are_synchronized(instance, "postgresql_replica", order_by="key1") def test_table_schema_changes(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() NUM_TABLES = 5 for i in range(NUM_TABLES): - create_postgres_table( - cursor, - "postgresql_replica_{}".format(i), - template=postgres_table_template_2, + pg_manager.create_postgres_table( + f"postgresql_replica_{i}", template=postgres_table_template_2 ) instance.query( - "INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format( - i, i, i, i - ) + f"INSERT INTO postgres_database.postgresql_replica_{i} SELECT number, {i}, {i}, {i} from numbers(25)" ) pg_manager.create_materialized_db( @@ -393,9 +365,7 @@ def test_table_schema_changes(started_cluster): for i in range(NUM_TABLES): instance.query( - "INSERT INTO postgres_database.postgresql_replica_{} SELECT 25 + number, {}, {}, {} from numbers(25)".format( - i, i, i, i - ) + f"INSERT INTO postgres_database.postgresql_replica_{i} SELECT 25 + number, {i}, {i}, {i} from numbers(25)" ) check_several_tables_are_synchronized(instance, NUM_TABLES) @@ -444,10 +414,7 @@ def test_many_concurrent_queries(started_cluster): port=started_cluster.postgres_port, database=True, ) - cursor = conn.cursor() - pg_manager.create_and_fill_postgres_tables_from_cursor( - cursor, NUM_TABLES, numbers=10000 - ) + pg_manager.create_and_fill_postgres_tables(NUM_TABLES, numbers=10000) def attack(thread_id): print("thread {}".format(thread_id)) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 3b5194e8806..2b17024f417 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -67,13 +67,11 @@ def started_cluster(): instance, cluster.postgres_ip, cluster.postgres_port, - default_database="test_database", + default_database="postgres_database", ) - pg_manager.create_clickhouse_postgres_db() pg_manager2.init( - instance2, cluster.postgres_ip, cluster.postgres_port, "test_database2" + instance2, cluster.postgres_ip, cluster.postgres_port, "postgres_database2" ) - pg_manager2.create_clickhouse_postgres_db() yield cluster finally: @@ -88,11 +86,10 @@ def setup_teardown(): def test_add_new_table_to_replication(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute("DROP TABLE IF EXISTS test_table") + pg_manager.execute("DROP TABLE IF EXISTS test_table") NUM_TABLES = 5 - pg_manager.create_and_fill_postgres_tables_from_cursor(cursor, NUM_TABLES, 10000) + pg_manager.create_and_fill_postgres_tables(NUM_TABLES, 10000) pg_manager.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port ) @@ -105,7 +102,7 @@ def test_add_new_table_to_replication(started_cluster): ) table_name = "postgresql_replica_5" - pg_manager.create_and_fill_postgres_table_from_cursor(cursor, table_name) + pg_manager.create_and_fill_postgres_table(table_name) result = instance.query("SHOW CREATE DATABASE test_database") assert ( @@ -158,7 +155,7 @@ def test_add_new_table_to_replication(started_cluster): ) table_name = "postgresql_replica_6" - create_postgres_table(cursor, table_name) + pg_manager.create_postgres_table(table_name) instance.query( "INSERT INTO postgres_database.{} SELECT number, number from numbers(10000)".format( table_name @@ -169,7 +166,7 @@ def test_add_new_table_to_replication(started_cluster): instance.restart_clickhouse() table_name = "postgresql_replica_7" - create_postgres_table(cursor, table_name) + pg_manager.create_postgres_table(table_name) instance.query( "INSERT INTO postgres_database.{} SELECT number, number from numbers(10000)".format( table_name @@ -271,8 +268,7 @@ def test_remove_table_from_replication(started_cluster): == ")\\nSETTINGS materialized_postgresql_tables_list = \\'postgresql_replica_0,postgresql_replica_2,postgresql_replica_3,postgresql_replica_4\\'\n" ) - cursor = pg_manager.get_db_cursor() - cursor.execute(f"drop table if exists postgresql_replica_0;") + pg_manager.execute(f"drop table if exists postgresql_replica_0;") # Removing from replication table which does not exist in PostgreSQL must be ok. instance.query("DETACH TABLE test_database.postgresql_replica_0 PERMANENTLY") @@ -282,10 +278,11 @@ def test_remove_table_from_replication(started_cluster): def test_predefined_connection_configuration(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute(f"DROP TABLE IF EXISTS test_table") - cursor.execute(f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)") - cursor.execute(f"INSERT INTO test_table SELECT 1, 2") + pg_manager.execute(f"DROP TABLE IF EXISTS test_table") + pg_manager.execute( + f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)" + ) + pg_manager.execute(f"INSERT INTO test_table SELECT 1, 2") instance.query( "CREATE DATABASE test_database ENGINE = MaterializedPostgreSQL(postgres1) SETTINGS materialized_postgresql_tables_list='test_table'" ) @@ -332,10 +329,9 @@ def test_database_with_single_non_default_schema(started_cluster): create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - ip=cluster.postgres_ip, - port=cluster.postgres_port, - name=clickhouse_postgres_db, + database_name=clickhouse_postgres_db, schema_name=schema_name, + postgres_database="postgres_database", ) for i in range(NUM_TABLES): @@ -367,7 +363,7 @@ def test_database_with_single_non_default_schema(started_cluster): check_all_tables_are_synchronized() altered_table = random.randint(0, NUM_TABLES - 1) - cursor.execute( + pg_manager.execute( "ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format( altered_table ) @@ -434,10 +430,9 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - ip=cluster.postgres_ip, - port=cluster.postgres_port, - name=clickhouse_postgres_db, + database_name=clickhouse_postgres_db, schema_name=schema_name, + postgres_database="postgres_database", ) for i in range(NUM_TABLES): @@ -472,7 +467,7 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): check_all_tables_are_synchronized() altered_table = random.randint(0, NUM_TABLES - 1) - cursor.execute( + pg_manager.execute( "ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format( altered_table ) @@ -550,10 +545,7 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): clickhouse_postgres_db = f"clickhouse_postgres_db{i}" create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - ip=cluster.postgres_ip, - port=cluster.postgres_port, - name=clickhouse_postgres_db, - schema_name=schema_name, + database_name=clickhouse_postgres_db, schema_name=schema_name, postgres_database="postgres_database", ) for ti in range(NUM_TABLES): table_name = f"postgresql_replica_{ti}" @@ -586,7 +578,7 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): altered_schema = random.randint(0, schemas_num - 1) altered_table = random.randint(0, NUM_TABLES - 1) clickhouse_postgres_db = f"clickhouse_postgres_db{altered_schema}" - cursor.execute( + pg_manager.execute( f"ALTER TABLE schema{altered_schema}.postgresql_replica_{altered_table} ADD COLUMN value2 integer" ) @@ -619,10 +611,9 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): def test_table_override(started_cluster): - cursor = pg_manager.get_db_cursor() table_name = "table_override" materialized_database = "test_database" - create_postgres_table(cursor, table_name, template=postgres_table_template_5) + pg_manager.create_postgres_table(table_name, template=postgres_table_template_5) instance.query( f"create table {table_name}(key Int32, value UUID) engine = PostgreSQL (postgres1, table={table_name})" ) @@ -649,10 +640,11 @@ def test_table_override(started_cluster): def test_materialized_view(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute(f"DROP TABLE IF EXISTS test_table") - cursor.execute(f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)") - cursor.execute(f"INSERT INTO test_table SELECT 1, 2") + pg_manager.execute(f"DROP TABLE IF EXISTS test_table") + pg_manager.execute( + f"CREATE TABLE test_table (key integer PRIMARY KEY, value integer)" + ) + pg_manager.execute(f"INSERT INTO test_table SELECT 1, 2") instance.query("DROP DATABASE IF EXISTS test_database") instance.query( "CREATE DATABASE test_database ENGINE = MaterializedPostgreSQL(postgres1) SETTINGS materialized_postgresql_tables_list='test_table'" @@ -663,7 +655,7 @@ def test_materialized_view(started_cluster): "CREATE MATERIALIZED VIEW mv ENGINE=MergeTree ORDER BY tuple() POPULATE AS SELECT * FROM test_database.test_table" ) assert "1\t2" == instance.query("SELECT * FROM mv").strip() - cursor.execute(f"INSERT INTO test_table SELECT 3, 4") + pg_manager.execute(f"INSERT INTO test_table SELECT 3, 4") check_tables_are_synchronized(instance, "test_table") assert "1\t2\n3\t4" == instance.query("SELECT * FROM mv ORDER BY 1, 2").strip() pg_manager.drop_materialized_db() From abf1d459b6d76536855ba6ad62b23bd84fe06709 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 15 May 2023 09:38:59 +0300 Subject: [PATCH 0079/2047] fix build --- src/Storages/S3Queue/StorageS3Queue.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 113abec4b09..3da15fc62b9 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -121,14 +121,14 @@ StorageS3Queue::StorageS3Queue( String setting_zookeeper_path = s3queue_settings->keeper_path; LOG_INFO(log, "Settings zookeeper_path={}", setting_zookeeper_path); - if (setting_zookeeper_path == "") + if (setting_zookeeper_path.empty()) { auto table_id = getStorageID(); auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); bool is_in_replicated_database = database->getEngineName() == "Replicated"; auto default_path = getContext()->getSettingsRef().s3queue_default_zookeeper_path.value; - if (default_path != "") + if (!default_path.empty()) { zookeeper_path = zkutil::extractZooKeeperPath(fs::path(default_path) / toString(table_id.uuid), /* check_starts_with_slash */ true, log); From 1880dedacac75432f7854a4b1a01db8b2192d42c Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 15 May 2023 11:19:50 +0300 Subject: [PATCH 0080/2047] fix build --- src/Storages/S3Queue/S3QueueSource.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 27fcdcc892c..9a9472d9578 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -51,8 +51,6 @@ # include # include -namespace fs = std::filesystem; - namespace CurrentMetrics { @@ -103,7 +101,7 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file) { - for (KeyWithInfo val : keys_buf) + for (const KeyWithInfo & val : keys_buf) { auto full_path = bucket + '/' + val.key; if (exclude_keys.find(full_path) != exclude_keys.end()) @@ -138,7 +136,7 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( } Strings keys; - for (auto v : processing_keys) + for (const auto & v : processing_keys) { keys.push_back(bucket + '/' + v.key); } From ba55c11d0a7090abf84f480189d01fa324c5d7ec Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 15 May 2023 15:26:44 +0300 Subject: [PATCH 0081/2047] add documentation --- .../table-engines/integrations/s3queue.md | 194 ++++++++++++++++++ 1 file changed, 194 insertions(+) create mode 100644 docs/en/engines/table-engines/integrations/s3queue.md diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md new file mode 100644 index 00000000000..46cecd35a48 --- /dev/null +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -0,0 +1,194 @@ +--- +slug: /en/engines/table-engines/integrations/s3queue +sidebar_position: 7 +sidebar_label: S3Queue +--- + +# S3Queue Table Engine +This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ecosystem and allows streaming import. This engine is similar to the [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) engines, but provides S3-specific features. + +## Create Table {#creating-a-table} + +``` sql +CREATE TABLE s3_queue_engine_table (name String, value UInt32) + ENGINE = S3Queue(path [, NOSIGN | aws_access_key_id, aws_secret_access_key,] format, [compression]) + [SETTINGS] + [mode = 'unordered',] + [after_processing = 'keep',] + [keeper_path = '',] + [s3queue_loading_retries = 0,] + [s3queue_polling_min_timeout_ms = 1000,] + [s3queue_polling_max_timeout_ms = 10000,] + [s3queue_polling_backoff_ms = 0,] + [s3queue_max_set_size = 1000,] + [s3queue_max_set_age_s = 0,] + [s3queue_polling_size = 50,] +``` + +**Engine parameters** + +- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). +- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. +- `format` — The [format](../../../interfaces/formats.md#formats) of the file. +- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). +- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension. + + +## Settings {#s3queue-settings} + +### mode {#mode} + +Allows to automatically close the connection after query execution, i.e. disable connection reuse. + +Possible values: + +- unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer. +- ordered — With ordered mode, only the max name of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. + +Default value: `unordered`. + +### after_processing {#after_processing} + +Delete or keep file after successful processing. +Possible values: + +- keep. +- delete. + +Default value: `keep`. + +### keeper_path {#keeper_path} + +The path in ZooKeeper can be specified as a table engine setting or default path can be formed from the global configuration-provided path and table UUID. +Possible values: + +- String. + +Default value: ``. + +### s3queue_loading_retries {#s3queue_loading_retries} + +Retry file loading up to specified number of times. By default, there are no retries. +Possible values: + +- Positive integer. + +Default value: `0`. + +### s3queue_polling_min_timeout_ms {#s3queue_polling_min_timeout_ms} + +Minimal timeout before next polling (in milliseconds). + +Possible values: + +- Positive integer. + +Default value: `1000`. + +### s3queue_polling_max_timeout_ms {#s3queue_polling_max_timeout_ms} + +Maximum timeout before next polling (in milliseconds). + +Possible values: + +- Positive integer. + +Default value: `10000`. + +### s3queue_polling_backoff_ms {#s3queue_polling_backoff_ms} + +Polling backoff (in milliseconds). + +Possible values: + +- Positive integer. + +Default value: `0`. + +### s3queue_max_set_size {#s3queue_max_set_size} + +Max set size for tracking processed files in unordered mode in ZooKeeper. + +Possible values: + +- Positive integer. + +Default value: `1000`. + +### s3queue_max_set_age_s {#s3queue_max_set_age_s} + +Maximum number of seconds to store processed files in ZooKeeper node (store forever by default). + +Possible values: + +- Positive integer. + +Default value: `0`. + +### s3queue_polling_size {#s3queue_polling_size} + +Maximum files to fetch from S3 with SELECT or in background task. + +Possible values: + +- Positive integer. + +Default value: `50`. + + +## S3-related Settings {#s3-settings} + +Engine supports all s3 related settings. For more information about S3 settings see [here](../../../engines/table-engines/integrations/s3.md). + + +## Description {#description} + +`SELECT` is not particularly useful for streaming import (except for debugging), because each file can be imported only once. It is more practical to create real-time threads using [materialized views](../../../sql-reference/statements/create/view.md). To do this: + +1. Use the engine to create a table for consuming from specified path in S3 and consider it a data stream. +2. Create a table with the desired structure. +3. Create a materialized view that converts data from the engine and puts it into a previously created table. + +When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the background. + +Example: + +``` sql + CREATE TABLE s3queue_engine_table (name String, value UInt32) + ENGINE=S3Queue('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/*', 'CSV', 'gzip') + SETTINGS + mode = 'unordred', + keeper_path = '/clickhouse/s3queue/'; + + CREATE TABLE stats (name String, value UInt32) + ENGINE = MergeTree() ORDER BY name; + + CREATE MATERIALIZED VIEW consumer TO stats + AS SELECT name, value FROM s3queue_engine_table; + + SELECT * FROM stats ORDER BY name; +``` + +## Virtual columns {#virtual-columns} + +- `_path` — Path to the file. +- `_file` — Name of the file. + +For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns). + + +## Wildcards In Path {#wildcards-in-path} + +`path` argument can specify multiple files using bash-like wildcards. For being processed file should exist and match to the whole path pattern. Listing of files is determined during `SELECT` (not at `CREATE` moment). + +- `*` — Substitutes any number of any characters except `/` including empty string. +- `?` — Substitutes any single character. +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. +- `{N..M}` — Substitutes any number in range from N to M including both borders. N and M can have leading zeroes e.g. `000..078`. + +Constructions with `{}` are similar to the [remote](../../../sql-reference/table-functions/remote.md) table function. + +:::note +If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. +::: + From d7b4d9b6c0bcb5bd5d33d11cc9471f40031e25e7 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 15 May 2023 15:27:44 +0300 Subject: [PATCH 0082/2047] add new table engine to index --- docs/en/engines/table-engines/index.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/engines/table-engines/index.md b/docs/en/engines/table-engines/index.md index d7c582164de..990e6986cbf 100644 --- a/docs/en/engines/table-engines/index.md +++ b/docs/en/engines/table-engines/index.md @@ -59,6 +59,7 @@ Engines in the family: - [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) - [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md) - [PostgreSQL](../../engines/table-engines/integrations/postgresql.md) +- [S3Queue](../../engines/table-engines/integrations/s3queue.md) ### Special Engines {#special-engines} From e2112576f04d3aeda1bc5384b5c49da4dc2a8e0c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 16 May 2023 12:21:32 +0200 Subject: [PATCH 0083/2047] Fix black check --- .../test_postgresql_replica_database_engine_2/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 2b17024f417..acec01e732b 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -545,7 +545,9 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): clickhouse_postgres_db = f"clickhouse_postgres_db{i}" create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - database_name=clickhouse_postgres_db, schema_name=schema_name, postgres_database="postgres_database", + database_name=clickhouse_postgres_db, + schema_name=schema_name, + postgres_database="postgres_database", ) for ti in range(NUM_TABLES): table_name = f"postgresql_replica_{ti}" From 1eb939766bc78a59dd11b3534f4fd7b693d75e21 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 16 May 2023 17:48:49 +0000 Subject: [PATCH 0084/2047] add test --- src/Interpreters/AsynchronousInsertQueue.cpp | 4 + src/Interpreters/AsynchronousInsertQueue.h | 1 + .../02726_async_insert_flush_stress.reference | 1 + .../02726_async_insert_flush_stress.sh | 86 +++++++++++++++++++ 4 files changed, 92 insertions(+) create mode 100644 tests/queries/0_stateless/02726_async_insert_flush_stress.reference create mode 100755 tests/queries/0_stateless/02726_async_insert_flush_stress.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 4592e92151e..e176c7afd76 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -318,6 +318,7 @@ void AsynchronousInsertQueue::flushAll() LOG_DEBUG(log, "Requested to flush asynchronous insert queue"); + /// Disable background flushes to avoid adding new elements to the queue. flush_stopped = true; std::vector queues_to_flush(pool_size); @@ -343,10 +344,13 @@ void AsynchronousInsertQueue::flushAll() } } + /// Note that jobs scheduled before the call of 'flushAll' are not counted here. LOG_DEBUG(log, "Will wait for finishing of {} flushing jobs (about {} inserts, {} bytes, {} distinct queries)", pool.active(), total_entries, total_bytes, total_queries); + /// Wait until all jobs are finished. That includes also jobs + /// that were scheduled before the call of 'flushAll'. pool.wait(); LOG_DEBUG(log, "Finished flushing of asynchronous insert queue"); diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index 455e486c798..b22b0c73907 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -38,6 +38,7 @@ public: std::unique_ptr insert_data_buffer; }; + /// Force flush the whole queue. void flushAll(); PushResult push(ASTPtr query, ContextPtr query_context); size_t getPoolSize() const { return pool_size; } diff --git a/tests/queries/0_stateless/02726_async_insert_flush_stress.reference b/tests/queries/0_stateless/02726_async_insert_flush_stress.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02726_async_insert_flush_stress.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh new file mode 100755 index 00000000000..4685e49b96d --- /dev/null +++ b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh @@ -0,0 +1,86 @@ +#!/usr/bin/env bash +# Tags: long + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +function insert1() +{ + url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" + while true; do + ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT CSV +1,"a" +2,"b" +' + done +} + +function insert2() +{ + url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" + while true; do + ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT JSONEachRow {"id": 5, "s": "e"} {"id": 6, "s": "f"}' + done +} + +function insert3() +{ + url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" + while true; do + ${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO FUNCTION remote('127.0.0.1', $CLICKHOUSE_DATABASE, async_inserts) VALUES (7, 'g') (8, 'h')" + done +} + +function select1() +{ + while true; do + ${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts FORMAT Null" + done +} + +function select2() +{ + while true; do + ${CLICKHOUSE_CLIENT} -q "SELECT * FROM system.asynchronous_inserts FORMAT Null" + done +} + +function flush1() +{ + while true; do + sleep 0.2 + ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH ASYNC INSERT QUEUE" + done +} + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE async_inserts (id UInt32, s String) ENGINE = MergeTree ORDER BY id" + +TIMEOUT=10 + +export -f insert1 +export -f insert2 +export -f insert3 +export -f select1 +export -f select2 +export -f flush1 + +for _ in {1..5}; do + timeout $TIMEOUT bash -c insert1 & + timeout $TIMEOUT bash -c insert2 & + timeout $TIMEOUT bash -c insert3 & +done + +timeout $TIMEOUT bash -c select1 & +timeout $TIMEOUT bash -c select2 & +timeout $TIMEOUT bash -c flush1 & + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH ASYNC INSERT QUEUE" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.asynchronous_inserts" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts"; From 849cddd8b281be1ecebf104f5b8f6670c1c6e916 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 May 2023 12:33:26 +0200 Subject: [PATCH 0085/2047] Add forgotten file --- .../configs/merge_tree_too_many_parts.xml | 5 +++++ .../test_postgresql_replica_database_engine_2/test.py | 7 ++----- 2 files changed, 7 insertions(+), 5 deletions(-) create mode 100644 tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml diff --git a/tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml b/tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml new file mode 100644 index 00000000000..4bc63453f55 --- /dev/null +++ b/tests/integration/test_postgresql_replica_database_engine_2/configs/merge_tree_too_many_parts.xml @@ -0,0 +1,5 @@ + + + 5 + + diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 3b5194e8806..07f356250ea 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -88,11 +88,8 @@ def setup_teardown(): def test_add_new_table_to_replication(started_cluster): - cursor = pg_manager.get_db_cursor() - cursor.execute("DROP TABLE IF EXISTS test_table") NUM_TABLES = 5 - - pg_manager.create_and_fill_postgres_tables_from_cursor(cursor, NUM_TABLES, 10000) + pg_manager.create_and_fill_postgres_tables(NUM_TABLES, 10000) pg_manager.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port ) @@ -105,7 +102,7 @@ def test_add_new_table_to_replication(started_cluster): ) table_name = "postgresql_replica_5" - pg_manager.create_and_fill_postgres_table_from_cursor(cursor, table_name) + pg_manager.create_and_fill_postgres_table(table_name) result = instance.query("SHOW CREATE DATABASE test_database") assert ( From 05a90a2e971ae7538ed72e1a3db02523c91b67d8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 17 May 2023 12:19:00 +0000 Subject: [PATCH 0086/2047] fix tests --- tests/queries/0_stateless/01271_show_privileges.reference | 1 + .../0_stateless/02117_show_create_table_system.reference | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index ec245d8b9e0..eb8b912f03b 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -133,6 +133,7 @@ SYSTEM SYNC TRANSACTION LOG ['SYNC TRANSACTION LOG'] GLOBAL SYSTEM SYSTEM SYNC FILE CACHE ['SYNC FILE CACHE'] GLOBAL SYSTEM SYSTEM FLUSH DISTRIBUTED ['FLUSH DISTRIBUTED'] TABLE SYSTEM FLUSH SYSTEM FLUSH LOGS ['FLUSH LOGS'] GLOBAL SYSTEM FLUSH +SYSTEM FLUSH ASYNC INSERT QUEUE ['FLUSH ASYNC INSERT QUEUE'] GLOBAL SYSTEM FLUSH SYSTEM FLUSH [] \N SYSTEM SYSTEM THREAD FUZZER ['SYSTEM START THREAD FUZZER','SYSTEM STOP THREAD FUZZER','START THREAD FUZZER','STOP THREAD FUZZER'] GLOBAL SYSTEM SYSTEM UNFREEZE ['SYSTEM UNFREEZE'] GLOBAL SYSTEM diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 09cc62dac00..85cdc278892 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -297,7 +297,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM' = 140, 'dictGet' = 141, 'displaySecretsInShowAndSelect' = 142, 'addressToLine' = 143, 'addressToLineWithInlines' = 144, 'addressToSymbol' = 145, 'demangle' = 146, 'INTROSPECTION' = 147, 'FILE' = 148, 'URL' = 149, 'REMOTE' = 150, 'MONGO' = 151, 'MEILISEARCH' = 152, 'MYSQL' = 153, 'POSTGRES' = 154, 'SQLITE' = 155, 'ODBC' = 156, 'JDBC' = 157, 'HDFS' = 158, 'S3' = 159, 'HIVE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -581,10 +581,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM' = 140, 'dictGet' = 141, 'displaySecretsInShowAndSelect' = 142, 'addressToLine' = 143, 'addressToLineWithInlines' = 144, 'addressToSymbol' = 145, 'demangle' = 146, 'INTROSPECTION' = 147, 'FILE' = 148, 'URL' = 149, 'REMOTE' = 150, 'MONGO' = 151, 'MEILISEARCH' = 152, 'MYSQL' = 153, 'POSTGRES' = 154, 'SQLITE' = 155, 'ODBC' = 156, 'JDBC' = 157, 'HDFS' = 158, 'S3' = 159, 'HIVE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH' = 135, 'SYSTEM THREAD FUZZER' = 136, 'SYSTEM UNFREEZE' = 137, 'SYSTEM FAILPOINT' = 138, 'SYSTEM' = 139, 'dictGet' = 140, 'displaySecretsInShowAndSelect' = 141, 'addressToLine' = 142, 'addressToLineWithInlines' = 143, 'addressToSymbol' = 144, 'demangle' = 145, 'INTROSPECTION' = 146, 'FILE' = 147, 'URL' = 148, 'REMOTE' = 149, 'MONGO' = 150, 'MEILISEARCH' = 151, 'MYSQL' = 152, 'POSTGRES' = 153, 'SQLITE' = 154, 'ODBC' = 155, 'JDBC' = 156, 'HDFS' = 157, 'S3' = 158, 'HIVE' = 159, 'SOURCES' = 160, 'CLUSTER' = 161, 'ALL' = 162, 'NONE' = 163)) + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION CONTROL' = 96, 'SYSTEM SHUTDOWN' = 97, 'SYSTEM DROP DNS CACHE' = 98, 'SYSTEM DROP MARK CACHE' = 99, 'SYSTEM DROP UNCOMPRESSED CACHE' = 100, 'SYSTEM DROP MMAP CACHE' = 101, 'SYSTEM DROP QUERY CACHE' = 102, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 103, 'SYSTEM DROP FILESYSTEM CACHE' = 104, 'SYSTEM DROP SCHEMA CACHE' = 105, 'SYSTEM DROP S3 CLIENT CACHE' = 106, 'SYSTEM DROP CACHE' = 107, 'SYSTEM RELOAD CONFIG' = 108, 'SYSTEM RELOAD USERS' = 109, 'SYSTEM RELOAD SYMBOLS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM' = 140, 'dictGet' = 141, 'displaySecretsInShowAndSelect' = 142, 'addressToLine' = 143, 'addressToLineWithInlines' = 144, 'addressToSymbol' = 145, 'demangle' = 146, 'INTROSPECTION' = 147, 'FILE' = 148, 'URL' = 149, 'REMOTE' = 150, 'MONGO' = 151, 'MEILISEARCH' = 152, 'MYSQL' = 153, 'POSTGRES' = 154, 'SQLITE' = 155, 'ODBC' = 156, 'JDBC' = 157, 'HDFS' = 158, 'S3' = 159, 'HIVE' = 160, 'SOURCES' = 161, 'CLUSTER' = 162, 'ALL' = 163, 'NONE' = 164)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' From e6d6d41f22102e3e6b3f60996e0baec34e3b0432 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 22 May 2023 12:23:31 +0800 Subject: [PATCH 0087/2047] minor improve read buffer of hdfs --- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index ee8e0764db0..fe1c2cc2815 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -41,6 +41,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory= file_size) + { + return false; + } ResourceGuard rlock(read_settings.resource_link, num_bytes_to_read); int bytes_read; From ceecb1488af0fe4413053c87d9042b9d79602371 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 22 May 2023 17:29:52 +0800 Subject: [PATCH 0088/2047] add function arrayJaccardIndex --- src/Functions/array/arrayJaccardIndex.cpp | 161 ++++++++++++++++++ .../02737_arrayJaccardIndex.reference | 32 ++++ .../0_stateless/02737_arrayJaccardIndex.sql | 26 +++ 3 files changed, 219 insertions(+) create mode 100644 src/Functions/array/arrayJaccardIndex.cpp create mode 100644 tests/queries/0_stateless/02737_arrayJaccardIndex.reference create mode 100644 tests/queries/0_stateless/02737_arrayJaccardIndex.sql diff --git a/src/Functions/array/arrayJaccardIndex.cpp b/src/Functions/array/arrayJaccardIndex.cpp new file mode 100644 index 00000000000..8cce98ab64d --- /dev/null +++ b/src/Functions/array/arrayJaccardIndex.cpp @@ -0,0 +1,161 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; +} + +class FunctionArrayJaccardIndex : public IFunction +{ +public: + using ResultType = Float64; + static constexpr auto name = "arrayJaccardIndex"; + String getName() const override { return name; } + static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } + explicit FunctionArrayJaccardIndex(ContextPtr context_) : context(context_) {} + size_t getNumberOfArguments() const override { return 2; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + DataTypes types; + for (size_t i = 0; i < 2; ++i) + { + const auto * array_type = checkAndGetDataType(arguments[i].get()); + if (!array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument {} of function {} must be array, but it has type{}.", i + 1, getName(), arguments[i]->getName()); + } + return std::make_shared>(); + } + + template + static void vector(const ColumnArray::Offsets & intersect_offsets, const ColumnArray::Offsets & left_offsets, const ColumnArray::Offsets & right_offsets, PaddedPODArray & res) + { + size_t left_size; + size_t right_size; + for (size_t i = 0; i < res.size(); ++i) + { + if constexpr (is_const_left) + left_size = left_offsets[0]; + else + left_size = left_offsets[i] - left_offsets[i - 1]; + if constexpr (is_const_right) + right_size = right_offsets[0]; + else + right_size = right_offsets[i] - right_offsets[i - 1]; + + size_t intersect_size = intersect_offsets[i] - intersect_offsets[i - 1]; + res[i] = static_cast(intersect_size) / (left_size + right_size - intersect_size); + if (unlikely(isnan(res[i]))) + res[i] = 1; + } + } + + template + static void vectorWithEmptyIntersect(const ColumnArray::Offsets & left_offsets, const ColumnArray::Offsets & right_offsets, PaddedPODArray & res) + { + size_t left_size; + size_t right_size; + for (size_t i = 0; i < res.size(); ++i) + { + if constexpr (is_const_left) + left_size = left_offsets[0]; + else + left_size = left_offsets[i] - left_offsets[i - 1]; + if constexpr (is_const_right) + right_size = right_offsets[0]; + else + right_size = right_offsets[i] - right_offsets[i - 1]; + + res[i] = static_cast(left_size + right_size == 0); + } + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + bool is_const_left; + bool is_const_right; + const ColumnArray * left_array; + const ColumnArray * right_array; + + auto cast_array = [&](const ColumnWithTypeAndName & col) + { + const ColumnArray * res; + bool is_const = false; + if (typeid_cast(col.column.get())) + { + res = checkAndGetColumn(checkAndGetColumnConst(col.column.get())->getDataColumnPtr().get()); + is_const = true; + } + else if (!(res = checkAndGetColumn(col.column.get()))) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Argument for function {} must be array but it has type {}.", + col.column->getName(), getName()); + return std::make_pair(res, is_const); + }; + + std::tie(left_array, is_const_left) = cast_array(arguments[0]); + std::tie(right_array, is_const_right) = cast_array(arguments[1]); + + auto intersect_array = FunctionFactory::instance().get("arrayIntersect", context)->build(arguments); + ColumnWithTypeAndName intersect_column; + intersect_column.type = intersect_array->getResultType(); + intersect_column.column = intersect_array->execute(arguments, intersect_column.type, input_rows_count); + const auto * return_type_intersect = checkAndGetDataType(intersect_column.type.get()); + if (!return_type_intersect) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected return type for function arrayIntersect"); + + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(input_rows_count); + +#define EXECUTE_VECTOR(is_const_left, is_const_right) \ + if (typeid_cast(return_type_intersect->getNestedType().get())) \ + vectorWithEmptyIntersect(left_array->getOffsets(), right_array->getOffsets(), vec_res); \ + else \ + { \ + const ColumnArray * col_array = checkAndGetColumn(intersect_column.column.get()); \ + vector(col_array->getOffsets(), left_array->getOffsets(), right_array->getOffsets(), vec_res); \ + } + + if (!is_const_left && !is_const_right) + EXECUTE_VECTOR(false, false) + else if (!is_const_left && is_const_right) + EXECUTE_VECTOR(false, true) + else if (is_const_left && !is_const_right) + EXECUTE_VECTOR(true, false) + else + EXECUTE_VECTOR(true, true) + +#undef EXECUTE_VECTOR + + return col_res; + } + +private: + ContextPtr context; +}; + +REGISTER_FUNCTION(ArrayJaccardIndex) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02737_arrayJaccardIndex.reference b/tests/queries/0_stateless/02737_arrayJaccardIndex.reference new file mode 100644 index 00000000000..e6934bfe092 --- /dev/null +++ b/tests/queries/0_stateless/02737_arrayJaccardIndex.reference @@ -0,0 +1,32 @@ +0 +0.5 +1 +0.67 +1 +0 +0 +0 +1 +0 +0 +0 +0 +0.5 +1 +0.67 +0.5 +0.5 +0.5 +0.5 +1 +1 +1 +1 +1 +1 +1 +1 +0.33 +0.2 +1 +1 diff --git a/tests/queries/0_stateless/02737_arrayJaccardIndex.sql b/tests/queries/0_stateless/02737_arrayJaccardIndex.sql new file mode 100644 index 00000000000..c3f04ba0b10 --- /dev/null +++ b/tests/queries/0_stateless/02737_arrayJaccardIndex.sql @@ -0,0 +1,26 @@ +drop table if exists array_jaccard_index; + +create table array_jaccard_index (arr Array(UInt8)) engine=MergeTree partition by arr order by arr; + +insert into array_jaccard_index values ([1,2,3]); +insert into array_jaccard_index values ([1,2]); +insert into array_jaccard_index values ([1]); +insert into array_jaccard_index values ([]); + +select round(arrayJaccardIndex(arr, [1,2]), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex(arr, []), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex([], arr), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex([1,2], arr), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex([1,2], [1,2,3,4]), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex([], []), 2) from array_jaccard_index order by arr; +select round(arrayJaccardIndex(arr, arr), 2) from array_jaccard_index order by arr; + +drop table if exists array_jaccard_index; + +select round(arrayJaccardIndex(['a'], ['a', 'aa', 'aaa']), 2); + +select round(arrayJaccardIndex([1, 1.1, 2.2], [2.2, 3.3, 444]), 2); + +select round(arrayJaccardIndex([], []), 2); + +select round(arrayJaccardIndex([toUInt16(1)], [toUInt32(1)]), 2); From 656d6abb2ee2d221df01367bcb53465289e4981c Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 22 May 2023 22:49:02 +0800 Subject: [PATCH 0089/2047] fix style --- src/Functions/array/arrayJaccardIndex.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayJaccardIndex.cpp b/src/Functions/array/arrayJaccardIndex.cpp index 8cce98ab64d..c1ec8b53d25 100644 --- a/src/Functions/array/arrayJaccardIndex.cpp +++ b/src/Functions/array/arrayJaccardIndex.cpp @@ -100,7 +100,7 @@ public: { const ColumnArray * res; bool is_const = false; - if (typeid_cast(col.column.get())) + if (typeid_cast(col.column.get())) { res = checkAndGetColumn(checkAndGetColumnConst(col.column.get())->getDataColumnPtr().get()); is_const = true; From b0b9f2a037918b8f745df952a9491b97de6fdada Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Tue, 23 May 2023 10:39:23 +0800 Subject: [PATCH 0090/2047] fix test --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 79a6ad1fa2d..5ef83a57ecf 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -112,6 +112,7 @@ arrayFirstIndex arrayFirstOrNull arrayFlatten arrayIntersect +arrayJaccardIndex arrayJoin arrayLast arrayLastIndex From 616904cd790473ca8075a8175a6334dd837b5bca Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 23 May 2023 15:50:52 +0000 Subject: [PATCH 0091/2047] Add encryptConfig() --- src/Common/Config/ConfigProcessor.cpp | 34 +++++++++++++++++++++++++++ src/Common/Config/ConfigProcessor.h | 5 ++++ src/Common/Config/ConfigReloader.cpp | 1 + src/Daemon/BaseDaemon.cpp | 1 + 4 files changed, 41 insertions(+) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 5bbc8eae0de..76e4ea1ebd1 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -171,6 +171,33 @@ static void mergeAttributes(Element & config_element, Element & with_element) with_element_attributes->release(); } +void ConfigProcessor::encryptRecursive(Poco::XML::Node * config_root) +{ + for (Node * node = config_root->firstChild(); node;) + { + if (node->nodeType() == Node::ELEMENT_NODE) + { + // NamedNodeMapPtr attributes = node->attributes(); + Element & element = dynamic_cast(*node); + if (element.hasAttribute("enc_codec")) + { + LOG_DEBUG(log, "Encrypted node {} value '{}'.", node->nodeName(), element.getNodeValue()); + // for (Node * child_node = node->firstChild(); child_node;) + // { + // LOG_DEBUG(log, " Child node {} value '{}'.", child_node->nodeName(), child_node->getNodeValue()); + // child_node = child_node->nextSibling(); + // } + Node * child_node = node->firstChild(); + child_node->setNodeValue("encrypted_" + child_node->getNodeValue() + "_encrypted"); + } + } + + encryptRecursive(node); + + node = node->nextSibling(); + } +} + void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root, const Node * with_root) { const NodeListPtr with_nodes = with_root->childNodes(); @@ -700,6 +727,13 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( return LoadedConfig{configuration, has_zk_includes, !processed_successfully, config_xml, path}; } +void ConfigProcessor::encryptConfig(LoadedConfig & loaded_config) +{ + Node * config_root = getRootNode(loaded_config.preprocessed_xml.get()); + encryptRecursive(config_root); + loaded_config.configuration = new Poco::Util::XMLConfiguration(loaded_config.preprocessed_xml); +} + void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir) { try diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 0ca3e46db88..2f0046bc39c 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,6 +92,9 @@ public: const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); + /// Encrypt nodes in config with specified encryption attributes + void encryptConfig(LoadedConfig & loaded_config); + /// Save preprocessed config to specified directory. /// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/ void savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir); @@ -124,6 +127,8 @@ private: using NodePtr = Poco::AutoPtr; + void encryptRecursive(Poco::XML::Node * config_root); + void mergeRecursive(XMLDocumentPtr config, Poco::XML::Node * config_root, const Poco::XML::Node * with_root); void merge(XMLDocumentPtr config, XMLDocumentPtr with); diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index de7011b67bf..896bd5949d9 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -130,6 +130,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac return; } config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); + config_processor.encryptConfig(loaded_config); /** We should remember last modification time if and only if config was successfully loaded * Otherwise a race condition could occur during config files update: diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 4780dfed4b2..2634439ee14 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -663,6 +663,7 @@ void BaseDaemon::initialize(Application & self) umask(umask_num); DB::ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, ""); + DB::ConfigProcessor(config_path).encryptConfig(loaded_config); /// Write core dump on crash. { From 15b847b41058de86ee1e85ef7acb61902c08713b Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 24 May 2023 09:49:12 +0800 Subject: [PATCH 0092/2047] init file_size as 0 --- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index fe1c2cc2815..904c1c89c16 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -41,7 +41,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory Date: Thu, 25 May 2023 00:00:32 +0000 Subject: [PATCH 0093/2047] add reading from archives --- src/Storages/StorageFile.cpp | 79 ++++++++++++++----- src/Storages/StorageFile.h | 5 +- src/TableFunctions/ITableFunctionFileLike.cpp | 9 ++- src/TableFunctions/ITableFunctionFileLike.h | 1 + src/TableFunctions/TableFunctionFile.cpp | 3 +- 5 files changed, 74 insertions(+), 23 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index e2a2f84bc72..96c6724a3d9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -15,6 +15,8 @@ #include #include +#include +#include #include #include #include @@ -248,11 +250,17 @@ std::unique_ptr createReadBuffer( const String & storage_name, int table_fd, const String & compression_method, - ContextPtr context) + ContextPtr context, + const String & path_to_archive = "auto") { CompressionMethod method; struct stat file_stat{}; + if (path_to_archive != "auto") { + auto reader = createArchiveReader(path_to_archive); + std::unique_ptr in = reader->readFile(current_path); + return in; + } if (use_table_fd) { @@ -361,7 +369,8 @@ ColumnsDescription StorageFile::getTableStructureFromFile( const std::vector & paths, const String & compression_method, const std::optional & format_settings, - ContextPtr context) + ContextPtr context, + const std::vector & paths_to_archive) { if (format == "Distributed") { @@ -382,14 +391,24 @@ ColumnsDescription StorageFile::getTableStructureFromFile( if (context->getSettingsRef().schema_inference_use_cache_for_file) columns_from_cache = tryGetColumnsFromCache(paths, format, format_settings, context); - ReadBufferIterator read_buffer_iterator = [&, it = paths.begin()](ColumnsDescription &) mutable -> std::unique_ptr - { - if (it == paths.end()) - return nullptr; + ReadBufferIterator read_buffer_iterator; + if (paths_to_archive.empty()) { + read_buffer_iterator = [&, it = paths.begin()](ColumnsDescription &) mutable -> std::unique_ptr + { + if (it == paths.end()) + return nullptr; - return createReadBuffer(*it++, false, "File", -1, compression_method, context); - }; + return createReadBuffer(*it++, false, "File", -1, compression_method, context); + }; + } else { + read_buffer_iterator = [&, it = paths_to_archive.begin()](ColumnsDescription &) mutable -> std::unique_ptr + { + if (it == paths_to_archive.end()) + return nullptr; + return createReadBuffer(paths[0], false, "File", -1, compression_method, context, *it); + }; + } ColumnsDescription columns; if (columns_from_cache) columns = *columns_from_cache; @@ -430,8 +449,13 @@ StorageFile::StorageFile(int table_fd_, CommonArguments args) StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args) : StorageFile(args) { + if (args.path_to_archive != "auto") { + paths_to_archive = getPathsList(args.path_to_archive, user_files_path, args.getContext(), total_bytes_to_read); + paths = {table_path_}; + } else { + paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read); + } is_db_table = false; - paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read); is_path_with_globs = paths.size() > 1; if (!paths.empty()) path_for_partitioned_write = paths.front(); @@ -483,7 +507,7 @@ void StorageFile::setStorageMetadata(CommonArguments args) columns = getTableStructureFromFileDescriptor(args.getContext()); else { - columns = getTableStructureFromFile(format_name, paths, compression_method, format_settings, args.getContext()); + columns = getTableStructureFromFile(format_name, paths, compression_method, format_settings, args.getContext(), paths_to_archive); if (!args.columns.empty() && args.columns != columns) throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Table structure and file structure are different"); } @@ -516,8 +540,10 @@ public: struct FilesInfo { std::vector files; + std::vector paths_to_archive; std::atomic next_file_to_read = 0; + std::atomic next_archive_to_read = 0; bool need_path_column = false; bool need_file_column = false; @@ -588,12 +614,19 @@ public: { if (!storage->use_table_fd) { - auto current_file = files_info->next_file_to_read.fetch_add(1); - if (current_file >= files_info->files.size()) - return {}; - - current_path = files_info->files[current_file]; - + size_t current_file = 0, current_archive = 0; + if (files_info->files.size() == 1 && !files_info->paths_to_archive.empty()) { + current_archive = files_info->next_archive_to_read.fetch_add(1); + if (current_archive >= files_info->paths_to_archive.size()) + return {}; + current_path = files_info->files[current_file]; + current_archive_path = files_info->paths_to_archive[current_archive]; + } else { + current_file = files_info->next_file_to_read.fetch_add(1); + if (current_file >= files_info->files.size()) + return {}; + current_path = files_info->files[current_file]; + } /// Special case for distributed format. Defaults are not needed here. if (storage->format_name == "Distributed") { @@ -603,9 +636,13 @@ public: } } - if (!read_buf) - read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context); - + if (!read_buf) { + if (files_info->paths_to_archive.empty()) { + read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context); + } else { + read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context, current_archive_path); + } + } auto format = context->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, storage->format_settings); @@ -673,6 +710,7 @@ private: StorageSnapshotPtr storage_snapshot; FilesInfoPtr files_info; String current_path; + String current_archive_path; Block sample_block; std::unique_ptr read_buf; std::unique_ptr pipeline; @@ -709,7 +747,7 @@ Pipe StorageFile::read( } else { - if (paths.size() == 1 && !fs::exists(paths[0])) + if (paths.size() == 1 && paths_to_archive.empty() && !fs::exists(paths[0])) { if (context->getSettingsRef().engine_file_empty_if_not_exists) return Pipe(std::make_shared(storage_snapshot->getSampleBlockForColumns(column_names))); @@ -720,6 +758,7 @@ Pipe StorageFile::read( auto files_info = std::make_shared(); files_info->files = paths; + files_info->paths_to_archive = paths_to_archive; files_info->total_bytes_to_read = total_bytes_to_read; for (const auto & column : column_names) diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 03b3aacb67f..e1f7c6f3bbe 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -26,6 +26,7 @@ public: const ColumnsDescription & columns; const ConstraintsDescription & constraints; const String & comment; + std::string path_to_archive = "auto"; }; /// From file descriptor @@ -85,7 +86,8 @@ public: const std::vector & paths, const String & compression_method, const std::optional & format_settings, - ContextPtr context); + ContextPtr context, + const std::vector & paths_to_archive = {"auto"}); static SchemaCache & getSchemaCache(const ContextPtr & context); @@ -118,6 +120,7 @@ private: std::string base_path; std::vector paths; + std::vector paths_to_archive; bool is_db_table = true; /// Table is stored in real database, not user's file bool use_table_fd = false; /// Use table_fd instead of path diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 8cbffc10e5a..f89cae36716 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -26,7 +26,14 @@ namespace ErrorCodes void ITableFunctionFileLike::parseFirstArguments(const ASTPtr & arg, const ContextPtr &) { - filename = checkAndGetLiteralArgument(arg, "source"); + String path = checkAndGetLiteralArgument(arg, "source"); + size_t pos = path.find(" :: "); + if (pos == String::npos) { + filename = path; + } else { + path_to_archive = path.substr(0, pos); + filename = path.substr(pos + 4, path.size() - pos - 3); + } } String ITableFunctionFileLike::getFormatFromFirstArgument() diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 589fce67638..1a1c0ca56f8 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -26,6 +26,7 @@ protected: virtual String getFormatFromFirstArgument(); String filename; + String path_to_archive = "auto"; String format = "auto"; String structure = "auto"; String compression_method = "auto"; diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index ff64bb3dc67..86c8d71cae7 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -75,10 +75,11 @@ StoragePtr TableFunctionFile::getStorage(const String & source, columns, ConstraintsDescription{}, String{}, + path_to_archive }; if (fd >= 0) return std::make_shared(fd, args); - + return std::make_shared(source, global_context->getUserFilesPath(), args); } From dd78008c9ec586a213e0e541b70dfe5055f7df0e Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 25 May 2023 09:36:41 +0000 Subject: [PATCH 0094/2047] Rename encryptConfig() into decryptConfig() --- src/Common/Config/ConfigProcessor.cpp | 12 ++++++------ src/Common/Config/ConfigProcessor.h | 6 +++--- src/Common/Config/ConfigReloader.cpp | 2 +- src/Daemon/BaseDaemon.cpp | 2 +- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 76e4ea1ebd1..3f9535205d8 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -171,7 +171,7 @@ static void mergeAttributes(Element & config_element, Element & with_element) with_element_attributes->release(); } -void ConfigProcessor::encryptRecursive(Poco::XML::Node * config_root) +void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) { for (Node * node = config_root->firstChild(); node;) { @@ -179,7 +179,7 @@ void ConfigProcessor::encryptRecursive(Poco::XML::Node * config_root) { // NamedNodeMapPtr attributes = node->attributes(); Element & element = dynamic_cast(*node); - if (element.hasAttribute("enc_codec")) + if (element.hasAttribute("encryption_codec")) { LOG_DEBUG(log, "Encrypted node {} value '{}'.", node->nodeName(), element.getNodeValue()); // for (Node * child_node = node->firstChild(); child_node;) @@ -188,11 +188,11 @@ void ConfigProcessor::encryptRecursive(Poco::XML::Node * config_root) // child_node = child_node->nextSibling(); // } Node * child_node = node->firstChild(); - child_node->setNodeValue("encrypted_" + child_node->getNodeValue() + "_encrypted"); + child_node->setNodeValue("decrypted_" + child_node->getNodeValue() + "_decrypted"); } } - encryptRecursive(node); + decryptRecursive(node); node = node->nextSibling(); } @@ -727,10 +727,10 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( return LoadedConfig{configuration, has_zk_includes, !processed_successfully, config_xml, path}; } -void ConfigProcessor::encryptConfig(LoadedConfig & loaded_config) +void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) { Node * config_root = getRootNode(loaded_config.preprocessed_xml.get()); - encryptRecursive(config_root); + decryptRecursive(config_root); loaded_config.configuration = new Poco::Util::XMLConfiguration(loaded_config.preprocessed_xml); } diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 2f0046bc39c..bc2f923f705 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,8 +92,8 @@ public: const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); - /// Encrypt nodes in config with specified encryption attributes - void encryptConfig(LoadedConfig & loaded_config); + /// Decrypt nodes in config with specified encryption attributes + void decryptConfig(LoadedConfig & loaded_config); /// Save preprocessed config to specified directory. /// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/ @@ -127,7 +127,7 @@ private: using NodePtr = Poco::AutoPtr; - void encryptRecursive(Poco::XML::Node * config_root); + void decryptRecursive(Poco::XML::Node * config_root); void mergeRecursive(XMLDocumentPtr config, Poco::XML::Node * config_root, const Poco::XML::Node * with_root); diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index 896bd5949d9..a4d2cb3d305 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -130,7 +130,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac return; } config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); - config_processor.encryptConfig(loaded_config); + config_processor.decryptConfig(loaded_config); /** We should remember last modification time if and only if config was successfully loaded * Otherwise a race condition could occur during config files update: diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 2634439ee14..4b1cd4e036e 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -663,7 +663,7 @@ void BaseDaemon::initialize(Application & self) umask(umask_num); DB::ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, ""); - DB::ConfigProcessor(config_path).encryptConfig(loaded_config); + DB::ConfigProcessor(config_path).decryptConfig(loaded_config); /// Write core dump on crash. { From 5f73681b00fb1a13873c9a8e6b07c7f57c335668 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 25 May 2023 15:51:20 +0000 Subject: [PATCH 0095/2047] Make working note descryption --- src/Common/Config/ConfigProcessor.cpp | 44 +++++++++++++++++++++++++-- src/Common/Config/ConfigProcessor.h | 2 +- utils/config-processor/CMakeLists.txt | 3 +- 3 files changed, 44 insertions(+), 5 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 3f9535205d8..fdfc6343876 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -26,6 +26,10 @@ #include #include #include +#include +#include +#include +#include #define PREPROCESSED_SUFFIX "-preprocessed" @@ -181,14 +185,47 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) Element & element = dynamic_cast(*node); if (element.hasAttribute("encryption_codec")) { - LOG_DEBUG(log, "Encrypted node {} value '{}'.", node->nodeName(), element.getNodeValue()); + LOG_DEBUG(log, "Encrypted node <{}>", node->nodeName()); // for (Node * child_node = node->firstChild(); child_node;) // { // LOG_DEBUG(log, " Child node {} value '{}'.", child_node->nodeName(), child_node->getNodeValue()); // child_node = child_node->nextSibling(); // } - Node * child_node = node->firstChild(); - child_node->setNodeValue("decrypted_" + child_node->getNodeValue() + "_decrypted"); + + Node * text_node = node->firstChild(); + auto codec_128 = DB::CompressionCodecEncrypted(DB::AES_128_GCM_SIV); + // DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, ""); + + /* + DB::Memory<> memory1; + std::string password="abcd"; + memory1.resize(password.size() + codec_128.getAdditionalSizeAtTheEndOfBuffer() + codec_128.getHeaderSize()+100); + auto bytes_written = codec_128.compress(password.data(), static_cast(password.size()), memory1.data()); + // std::string encrypted_password = std::string(memory1.data(), memory1.size()); + std::string encrypted_password = std::string(memory1.data(), bytes_written); + std::string password_hex; + boost::algorithm::hex(encrypted_password.begin(), encrypted_password.end(), std::back_inserter(password_hex)); + LOG_DEBUG(log, "Encrypted password: '{}'.", password_hex); + */ + + DB::Memory<> memory; + std::string encrypted_value; + + try + { + boost::algorithm::unhex(text_node->getNodeValue(), std::back_inserter(encrypted_value)); + // boost::algorithm::unhex(password_hex, std::back_inserter(encrypted_value)); + } + catch (const std::exception &) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read encrypted text for {}, check for valid characters [0-9a-fA-F] and length", node->nodeName()); + } + + memory.resize(codec_128.readDecompressedBlockSize(encrypted_value.data()) + codec_128.getAdditionalSizeAtTheEndOfBuffer()); + codec_128.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); + std::string decrypted_value = std::string(memory.data(), memory.size()); + LOG_DEBUG(log, "Decrypted value '{}'", decrypted_value); + text_node->setNodeValue(decrypted_value); } } @@ -729,6 +766,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) { + DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); Node * config_root = getRootNode(loaded_config.preprocessed_xml.get()); decryptRecursive(config_root); loaded_config.configuration = new Poco::Util::XMLConfiguration(loaded_config.preprocessed_xml); diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index bc2f923f705..479a0053efa 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,7 +92,7 @@ public: const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); - /// Decrypt nodes in config with specified encryption attributes + /// crypt nodes in config with specified encryption attributes void decryptConfig(LoadedConfig & loaded_config); /// Save preprocessed config to specified directory. diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt index 53b6163ba87..00cbfbba659 100644 --- a/utils/config-processor/CMakeLists.txt +++ b/utils/config-processor/CMakeLists.txt @@ -1,2 +1,3 @@ clickhouse_add_executable (config-processor config-processor.cpp) -target_link_libraries(config-processor PRIVATE clickhouse_common_config_no_zookeeper_log) +target_link_libraries(config-processor PRIVATE dbms clickhouse_common_config_no_zookeeper_log) +target_link_libraries(config-processor PUBLIC clickhouse_parsers clickhouse_common_io common ch_contrib::lz4) From bc7c67e33f20a07602715788a1426492c5a26349 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 25 May 2023 23:29:16 +0300 Subject: [PATCH 0096/2047] code review fixes --- .../table-engines/integrations/s3queue.md | 49 +++++++-- src/Core/Settings.h | 2 +- src/Storages/S3Queue/S3QueueHolder.cpp | 99 +++++++++---------- src/Storages/S3Queue/S3QueueHolder.h | 19 ++-- src/Storages/S3Queue/S3QueueSettings.h | 4 +- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 62 ++++++------ src/Storages/S3Queue/S3QueueTableMetadata.h | 7 +- src/Storages/S3Queue/StorageS3Queue.cpp | 14 +-- src/Storages/S3Queue/StorageS3Queue.h | 4 +- .../integration/test_storage_s3_queue/test.py | 6 +- 10 files changed, 149 insertions(+), 117 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 46cecd35a48..9fd6c88a966 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -20,8 +20,8 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) [s3queue_polling_min_timeout_ms = 1000,] [s3queue_polling_max_timeout_ms = 10000,] [s3queue_polling_backoff_ms = 0,] - [s3queue_max_set_size = 1000,] - [s3queue_max_set_age_s = 0,] + [s3queue_tracked_files_limit = 1000,] + [s3queue_tracked_file_ttl_sec = 0,] [s3queue_polling_size = 50,] ``` @@ -33,13 +33,40 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). - `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension. +**Example** + +```sql +CREATE TABLE s3queue_engine_table (name String, value UInt32) +ENGINE=S3Queue('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/*', 'CSV', 'gzip') +SETTINGS + mode = 'ordred'; +``` + +Using named collections: + +``` xml + + + + 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/* + test + test + + + +``` + +```sql +CREATE TABLE s3queue_engine_table (name String, value UInt32) +ENGINE=S3Queue(s3queue_conf, 'CSV', 'gzip') +SETTINGS + mode = 'ordred'; +``` ## Settings {#s3queue-settings} ### mode {#mode} -Allows to automatically close the connection after query execution, i.e. disable connection reuse. - Possible values: - unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer. @@ -64,7 +91,7 @@ Possible values: - String. -Default value: ``. +Default value: `/`. ### s3queue_loading_retries {#s3queue_loading_retries} @@ -105,9 +132,10 @@ Possible values: Default value: `0`. -### s3queue_max_set_size {#s3queue_max_set_size} +### s3queue_tracked_files_limit {#s3queue_tracked_files_limit} -Max set size for tracking processed files in unordered mode in ZooKeeper. +Allows to limit the number of Zookeeper nodes if the 'unordered' mode is used, does nothing for 'ordered' mode. +If limit reached the oldest processed files will be deleted from ZooKeeper node and processed again. Possible values: @@ -115,9 +143,10 @@ Possible values: Default value: `1000`. -### s3queue_max_set_age_s {#s3queue_max_set_age_s} +### s3queue_tracked_file_ttl_sec {#s3queue_tracked_file_ttl_sec} -Maximum number of seconds to store processed files in ZooKeeper node (store forever by default). +Maximum number of seconds to store processed files in ZooKeeper node (store forever by default) for 'unordered' mode, does nothing for 'ordered' mode. +After the specified number of seconds, the file will be re-imported. Possible values: @@ -128,6 +157,8 @@ Default value: `0`. ### s3queue_polling_size {#s3queue_polling_size} Maximum files to fetch from S3 with SELECT or in background task. +Engine takes files for processing from S3 in batches. +We limit the batch size to increase concurrency if multiple table engines with the same `keeper_path` consume files from the same path. Possible values: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 53648633630..18bbfa5d358 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -94,7 +94,7 @@ class IColumn; M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ - M(String, s3queue_default_zookeeper_path, "", "Default zookeeper path prefix for S3Queue engine", 0) \ + M(String, s3queue_default_zookeeper_path, "/", "Default zookeeper path prefix for S3Queue engine", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 91bf82001bb..03efe6ef097 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -4,6 +4,7 @@ # include # include # include +# include # include # include # include @@ -22,16 +23,20 @@ namespace ErrorCodes void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) { + assertString("collection:\n", in); + files = {}; - in >> "collection:\n"; while (!in.eof()) { String file_name; Int64 timestamp; + UInt64 retries_count; in >> file_name >> "\n"; in >> timestamp >> "\n"; - auto pair = std::make_pair(file_name, timestamp); - files.push_back(pair); + in >> retries_count >> "\n"; + + TrackedCollectionItem item = {.file_path=file_name, .timestamp=timestamp, .retries_count=retries_count}; + files.push_back(item); } } @@ -40,8 +45,9 @@ void S3QueueHolder::S3QueueCollection::write(WriteBuffer & out) const out << "collection:\n"; for (const auto & processed_file : files) { - out << processed_file.first << "\n"; - out << processed_file.second << "\n"; + out << processed_file.file_path << "\n"; + out << processed_file.timestamp << "\n"; + out << processed_file.retries_count << "\n"; } } @@ -57,7 +63,7 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueCollection::getFileNames( S3FilesCollection keys = {}; for (const auto & pair : files) { - keys.insert(pair.first); + keys.insert(pair.file_path); } return keys; } @@ -80,8 +86,8 @@ void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) auto new_end = std::remove_if( files.begin(), files.end(), - [×tamp, &max_seconds_diff](std::pair processed_file) - { return (timestamp - processed_file.second) > max_seconds_diff; }); + [×tamp, &max_seconds_diff](TrackedCollectionItem processed_file) + { return (timestamp - processed_file.timestamp) > max_seconds_diff; }); files.erase(new_end, files.end()); } } @@ -90,8 +96,9 @@ void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) { Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - auto pair = std::make_pair(file_name, timestamp); - files.push_back(pair); + + TrackedCollectionItem processed_file = {.file_path=file_name, .timestamp=timestamp}; + files.push_back(processed_file); // Check set size if (files.size() > max_size) @@ -115,19 +122,19 @@ void S3QueueHolder::S3QueueFailedCollection::parse(const String & s) bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) { auto failed_it - = std::find_if(files.begin(), files.end(), [&file_name](const std::pair & s) { return s.first == file_name; }); + = std::find_if(files.begin(), files.end(), [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); if (failed_it != files.end()) { - failed_it->second--; - if (failed_it->second == 0) + failed_it->retries_count--; + if (failed_it->retries_count == 0) { return false; } } else { - auto pair = std::make_pair(file_name, max_retries_count); - files.push_back(pair); + TrackedCollectionItem failed_file = {.file_path=file_name, .retries_count=max_retries_count}; + files.push_back(failed_file); } return true; } @@ -137,9 +144,9 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueFailedCollection::getFile S3FilesCollection failed_keys; for (const auto & pair : files) { - if (pair.second <= 0) + if (pair.retries_count <= 0) { - failed_keys.insert(pair.first); + failed_keys.insert(pair.file_path); } } return failed_keys; @@ -157,36 +164,29 @@ S3QueueHolder::S3QueueHolder( , max_set_age_s(max_set_age_s_) , max_loading_retries(max_loading_retries_) , zookeeper_path(zookeeper_path_) - , zookeeper_failed_path(zookeeper_path_ + "/failed") - , zookeeper_processing_path(zookeeper_path_ + "/processing") - , zookeeper_processed_path(zookeeper_path_ + "/processed") - , zookeeper_lock_path(zookeeper_path_ + "/lock") + , zookeeper_failed_path(fs::path(zookeeper_path_) / "failed") + , zookeeper_processing_path(fs::path(zookeeper_path_) / "processing") + , zookeeper_processed_path(fs::path(zookeeper_path_) / "processed") + , zookeeper_lock_path(fs::path(zookeeper_path_) / "lock") , mode(mode_) , log(&Poco::Logger::get("S3QueueHolder")) { current_zookeeper = getContext()->getZooKeeper(); + if (!current_zookeeper) + throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); } -zkutil::ZooKeeperPtr S3QueueHolder::tryGetZooKeeper() const +zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const { std::lock_guard lock(current_zookeeper_mutex); return current_zookeeper; } -zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const -{ - auto res = tryGetZooKeeper(); - if (!res) - throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); - return res; -} - - void S3QueueHolder::setFileProcessed(const String & file_path) { auto zookeeper = getZooKeeper(); - auto lock = AcquireLock(); + auto lock = acquireLock(); if (mode == S3QueueMode::UNORDERED) { @@ -196,9 +196,11 @@ void S3QueueHolder::setFileProcessed(const String & file_path) processed.add(file_path); zookeeper->set(zookeeper_processed_path, processed.toString()); } - else + else if (mode == S3QueueMode::ORDERED) { String max_file = getMaxProcessedFile(); + // Check that we set in ZooKeeper node only maximum processed file path. + // This check can be useful, when multiple table engines consume in ordered mode. if (max_file.compare(file_path) <= 0) { zookeeper->set(zookeeper_processed_path, file_path); @@ -211,7 +213,7 @@ void S3QueueHolder::setFileProcessed(const String & file_path) bool S3QueueHolder::markFailedAndCheckRetry(const String & file_path) { auto zookeeper = getZooKeeper(); - auto lock = AcquireLock(); + auto lock = acquireLock(); String failed_files = zookeeper->get(zookeeper_failed_path); auto failed_collection = S3QueueFailedCollection(max_loading_retries); @@ -245,22 +247,18 @@ String S3QueueHolder::getMaxProcessedFile() S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() { auto zookeeper = getZooKeeper(); - String processing = zookeeper->get(fs::path(zookeeper_processing_path)); + String processing = zookeeper->get(zookeeper_processing_path); return parseCollection(processing); } void S3QueueHolder::setFilesProcessing(Strings & file_paths) { auto zookeeper = getZooKeeper(); + std::unordered_set processing_files(file_paths.begin(), file_paths.end()); + processing_files.merge(getProcessingFiles()); - for (const auto & x : getProcessingFiles()) - { - if (!std::count(file_paths.begin(), file_paths.end(), x)) - { - file_paths.push_back(x); - } - } - zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); + Strings processing_file_paths(processing_files.begin(), processing_files.end()); + zookeeper->set(fs::path(zookeeper_processing_path), toString(processing_file_paths)); } S3QueueHolder::S3FilesCollection S3QueueHolder::getUnorderedProcessedFiles() @@ -301,16 +299,16 @@ void S3QueueHolder::removeProcessingFile(const String & file_path) { auto zookeeper = getZooKeeper(); String node_data; - Strings file_paths; String processing = zookeeper->get(zookeeper_processing_path); S3FilesCollection processing_files = parseCollection(processing); - file_paths.insert(file_paths.end(), processing_files.begin(), processing_files.end()); - file_paths.erase(std::remove(file_paths.begin(), file_paths.end(), file_path), file_paths.end()); + processing_files.erase(file_path); + + Strings file_paths(processing_files.begin(), processing_files.end()); zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); } -std::shared_ptr S3QueueHolder::AcquireLock() +std::shared_ptr S3QueueHolder::acquireLock() { auto zookeeper = getZooKeeper(); UInt32 retry_count = 200; @@ -323,7 +321,7 @@ std::shared_ptr S3QueueHolder::AcquireLock() if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { retries++; - if (retries >= retry_count) + if (retries > retry_count) { throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Can't acquire zookeeper lock"); } @@ -342,14 +340,15 @@ std::shared_ptr S3QueueHolder::AcquireLock() S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) { - ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); + ReadBufferFromString rb(files); Strings deserialized; try { readQuoted(deserialized, rb); } - catch (...) + catch (const Exception & e) { + LOG_WARNING(log, "Can't parse collection from ZooKeeper node: {}", e.displayText()); deserialized = {}; } diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index f7fa0461a3a..77719c8b941 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -12,9 +12,15 @@ namespace DB class S3QueueHolder : public WithContext { public: + struct TrackedCollectionItem + { + String file_path; + Int64 timestamp = 0; + UInt64 retries_count = 0; + }; + using S3FilesCollection = std::unordered_set; - using ProcessedFiles = std::vector>; - using FailedFiles = std::vector>; + using TrackedFiles = std::vector; S3QueueHolder( const String & zookeeper_path_, @@ -30,7 +36,7 @@ public: S3FilesCollection getExcludedFiles(); String getMaxProcessedFile(); - std::shared_ptr AcquireLock(); + std::shared_ptr acquireLock(); struct S3QueueCollection { @@ -42,7 +48,7 @@ public: virtual void parse(const String & s) = 0; protected: - ProcessedFiles files; + TrackedFiles files; void read(ReadBuffer & in); void write(WriteBuffer & out) const; @@ -72,7 +78,7 @@ public: S3FilesCollection getFilesWithoutRetries(); private: - const UInt64 max_retries_count; + UInt64 max_retries_count; }; @@ -93,7 +99,6 @@ private: const UUID table_uuid; Poco::Logger * log; - zkutil::ZooKeeperPtr tryGetZooKeeper() const; zkutil::ZooKeeperPtr getZooKeeper() const; S3FilesCollection getFailedFiles(); @@ -101,7 +106,7 @@ private: S3FilesCollection getUnorderedProcessedFiles(); void removeProcessingFile(const String & file_path); - static S3FilesCollection parseCollection(String & files); + S3FilesCollection parseCollection(String & files); }; diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 3b4ea475887..75defc4a57f 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -23,9 +23,9 @@ class ASTStorage; M(UInt64, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ M(UInt64, s3queue_polling_backoff_ms, 0, "Polling backoff", 0) \ - M(UInt64, s3queue_max_set_size, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ + M(UInt64, s3queue_tracked_files_limit, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ M(UInt64, \ - s3queue_max_set_age_s, \ + s3queue_tracked_file_ttl_sec, \ 0, \ "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", \ 0) \ diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 340890f75de..5cfd1ee2d35 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -2,8 +2,9 @@ #if USE_AWS_S3 -# include -# include +# include +# include +# include # include # include # include @@ -22,42 +23,41 @@ S3QueueTableMetadata::S3QueueTableMetadata(const StorageS3::Configuration & conf format_name = configuration.format; after_processing = engine_settings.after_processing.toString(); mode = engine_settings.mode.toString(); - s3queue_max_set_size = engine_settings.s3queue_max_set_size; - s3queue_max_set_age_s = engine_settings.s3queue_max_set_age_s; + s3queue_tracked_files_limit = engine_settings.s3queue_tracked_files_limit; + s3queue_tracked_file_ttl_sec = engine_settings.s3queue_tracked_file_ttl_sec; } -void S3QueueTableMetadata::write(WriteBuffer & out) const -{ - out << "metadata format version: 1\n" - << "after processing: " << after_processing << "\n" - << "mode: " << mode << "\n" - << "s3queue_max_set_size: " << s3queue_max_set_size << "\n" - << "s3queue_max_set_age_s: " << s3queue_max_set_age_s << "\n" - << "format name: " << format_name << "\n"; -} String S3QueueTableMetadata::toString() const { - WriteBufferFromOwnString out; - write(out); - return out.str(); + Poco::JSON::Object json; + json.set("after_processing", after_processing); + json.set("mode", mode); + json.set("s3queue_tracked_files_limit", s3queue_tracked_files_limit); + json.set("s3queue_tracked_file_ttl_sec", s3queue_tracked_file_ttl_sec); + json.set("format_name", format_name); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); } -void S3QueueTableMetadata::read(ReadBuffer & in) +void S3QueueTableMetadata::read(const String & s) { - in >> "metadata format version: 1\n"; - in >> "after processing: " >> after_processing >> "\n"; - in >> "mode: " >> mode >> "\n"; - in >> "s3queue_max_set_size: " >> s3queue_max_set_size >> "\n"; - in >> "s3queue_max_set_age_s: " >> s3queue_max_set_age_s >> "\n"; - in >> "format name: " >> format_name >> "\n"; + Poco::JSON::Parser parser; + auto json = parser.parse(s).extract(); + after_processing = json->getValue("after_processing"); + mode = json->getValue("mode"); + s3queue_tracked_files_limit = json->getValue("s3queue_tracked_files_limit"); + s3queue_tracked_file_ttl_sec = json->getValue("s3queue_tracked_file_ttl_sec"); + format_name = json->getValue("format_name"); } S3QueueTableMetadata S3QueueTableMetadata::parse(const String & s) { S3QueueTableMetadata metadata; - ReadBufferFromString buf(s); - metadata.read(buf); + metadata.read(s); return metadata; } @@ -80,21 +80,21 @@ void S3QueueTableMetadata::checkImmutableFieldsEquals(const S3QueueTableMetadata DB::toString(from_zk.after_processing), DB::toString(after_processing)); - if (s3queue_max_set_size != from_zk.s3queue_max_set_size) + if (s3queue_tracked_files_limit != from_zk.s3queue_tracked_files_limit) throw Exception( ErrorCodes::METADATA_MISMATCH, "Existing table metadata in ZooKeeper differs in max set size. " "Stored in ZooKeeper: {}, local: {}", - from_zk.s3queue_max_set_size, - s3queue_max_set_size); + from_zk.s3queue_tracked_files_limit, + s3queue_tracked_files_limit); - if (s3queue_max_set_age_s != from_zk.s3queue_max_set_age_s) + if (s3queue_tracked_file_ttl_sec != from_zk.s3queue_tracked_file_ttl_sec) throw Exception( ErrorCodes::METADATA_MISMATCH, "Existing table metadata in ZooKeeper differs in max set age. " "Stored in ZooKeeper: {}, local: {}", - from_zk.s3queue_max_set_age_s, - s3queue_max_set_age_s); + from_zk.s3queue_tracked_file_ttl_sec, + s3queue_tracked_file_ttl_sec); if (format_name != from_zk.format_name) throw Exception( diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 6e87528db37..a0a768b64f2 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -20,16 +20,15 @@ struct S3QueueTableMetadata String format_name; String after_processing; String mode; - UInt64 s3queue_max_set_size; - UInt64 s3queue_max_set_age_s; + UInt64 s3queue_tracked_files_limit; + UInt64 s3queue_tracked_file_ttl_sec; S3QueueTableMetadata() = default; S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings); - void read(ReadBuffer & in); + void read(const String & in); static S3QueueTableMetadata parse(const String & s); - void write(WriteBuffer & out) const; String toString() const; void checkEquals(const S3QueueTableMetadata & from_zk) const; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 3da15fc62b9..df99167f180 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -112,8 +112,6 @@ StorageS3Queue::StorageS3Queue( , is_key_with_globs(s3_configuration.url.key.find_first_of("*?{") != std::string::npos) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { - LOG_INFO(log, "Init engine"); - if (!is_key_with_globs) { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); @@ -138,7 +136,7 @@ StorageS3Queue::StorageS3Queue( LOG_INFO(log, "S3Queue engine keeper_path not specified. Use replicated database zookeeper path"); String base_zookeeper_path = assert_cast(database.get())->getZooKeeperPath(); zookeeper_path = zkutil::extractZooKeeperPath( - fs::path(base_zookeeper_path) / toString(table_id.uuid), /* check_starts_with_slash */ true, log); + fs::path(base_zookeeper_path) / "s3queue" / toString(table_id.uuid), /* check_starts_with_slash */ true, log); } else { @@ -184,8 +182,8 @@ StorageS3Queue::StorageS3Queue( zookeeper_path, mode, getContext(), - s3queue_settings->s3queue_max_set_size.value, - s3queue_settings->s3queue_max_set_age_s.value, + s3queue_settings->s3queue_tracked_files_limit.value, + s3queue_settings->s3queue_tracked_file_ttl_sec.value, s3queue_settings->s3queue_loading_retries.value); auto default_virtuals = NamesAndTypesList{ @@ -506,8 +504,6 @@ void StorageS3Queue::streamToViews() std::atomic_size_t rows = 0; { block_io.pipeline.complete(std::move(pipe)); - block_io.pipeline.setNumThreads(1); - block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); @@ -546,7 +542,7 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ auto zookeeper = getZooKeeper(); zookeeper->createAncestors(zookeeper_path); - for (size_t i = 0; i < 1000; ++i) + for (size_t i = 0; i < zk_create_table_retries; ++i) { Coordination::Requests ops; bool is_first_replica = true; @@ -638,7 +634,7 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW read_keys, s3_configuration.request_settings); - auto zookeeper_lock = queue_holder->AcquireLock(); + auto zookeeper_lock = queue_holder->acquireLock(); S3QueueHolder::S3FilesCollection exclude = queue_holder->getExcludedFiles(); Strings processing_files; diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 73f8075b7d1..b9aeac95f68 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -11,9 +11,9 @@ # include # include +# include # include # include -# include # include # include @@ -128,6 +128,8 @@ private: void setZooKeeper(); zkutil::ZooKeeperPtr tryGetZooKeeper() const; zkutil::ZooKeeperPtr getZooKeeper() const; + + const UInt32 zk_create_table_retries = 1000; bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); const String & getZooKeeperPath() const { return zookeeper_path; } diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 085f92035d0..798c5952682 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -621,8 +621,8 @@ def test_max_set_age(started_cluster): SETTINGS mode = 'unordered', keeper_path = '/clickhouse/test_set_age', - s3queue_max_set_size = 10, - s3queue_max_set_age_s = {max_age}; + s3queue_tracked_files_limit = 10, + s3queue_tracked_file_ttl_sec = {max_age}; """ ) @@ -820,7 +820,7 @@ def test_max_set_size(started_cluster): SETTINGS mode = 'unordered', keeper_path = '/clickhouse/test_set_size', - s3queue_max_set_size = {files_to_generate - 1}; + s3queue_tracked_files_limit = {files_to_generate - 1}; """ ) From 528fa9ae04ce5894b48d45dfabf089427205079a Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 25 May 2023 23:40:26 +0300 Subject: [PATCH 0097/2047] fix --- src/Storages/S3Queue/StorageS3Queue.cpp | 3 +-- src/Storages/S3Queue/StorageS3Queue.h | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index df99167f180..593d5637ad9 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -109,10 +109,9 @@ StorageS3Queue::StorageS3Queue( , distributed_processing(distributed_processing_) , format_settings(format_settings_) , partition_by(partition_by_) - , is_key_with_globs(s3_configuration.url.key.find_first_of("*?{") != std::string::npos) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { - if (!is_key_with_globs) + if (!withGlobs()) { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); } diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index b9aeac95f68..60857b6edad 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -95,9 +95,9 @@ private: const bool distributed_processing; std::optional format_settings; ASTPtr partition_by; - bool is_key_with_globs = false; bool supportsSubcolumns() const override; + bool withGlobs() const { return s3_configuration.url.key.find_first_of("*?{") != std::string::npos; } void threadFunc(); size_t getTableDependentCount() const; From 3629e0c980f967b41f6ddb1411d6954a147cad65 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Fri, 26 May 2023 10:09:49 +0300 Subject: [PATCH 0098/2047] fix --- src/Storages/S3Queue/S3QueueSource.cpp | 6 ------ src/Storages/S3Queue/StorageS3Queue.cpp | 6 ------ src/Storages/S3Queue/StorageS3Queue.h | 3 +-- 3 files changed, 1 insertion(+), 14 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 9a9472d9578..85a1db2ad35 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -28,12 +28,6 @@ # include # include -# include -# include -# include - -# include - # include # include diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 593d5637ad9..69c6601365f 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -37,9 +37,6 @@ # include -# include -# include - # include # include @@ -93,7 +90,6 @@ StorageS3Queue::StorageS3Queue( const String & comment, ContextPtr context_, std::optional format_settings_, - bool distributed_processing_, ASTPtr partition_by_) : IStorage(table_id_) , WithContext(context_) @@ -106,7 +102,6 @@ StorageS3Queue::StorageS3Queue( , format_name(configuration_.format) , compression_method(configuration_.compression_method) , name(s3_configuration.url.storage_name) - , distributed_processing(distributed_processing_) , format_settings(format_settings_) , partition_by(partition_by_) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) @@ -709,7 +704,6 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) args.comment, args.getContext(), format_settings, - /* distributed_processing_ */ false, partition_by); }, { diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 60857b6edad..9dd1c6029f5 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -50,7 +50,6 @@ public: const String & comment, ContextPtr context_, std::optional format_settings_, - bool distributed_processing_ = false, ASTPtr partition_by_ = nullptr); String getName() const override { return "S3Queue"; } @@ -92,7 +91,7 @@ private: String format_name; String compression_method; String name; - const bool distributed_processing; + std::optional format_settings; ASTPtr partition_by; From 3e39ce79cf47c4f721a1cd066c9a29e5c6c66def Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Fri, 26 May 2023 21:02:32 +0300 Subject: [PATCH 0099/2047] fix build --- src/Storages/S3Queue/S3QueueTableMetadata.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index a0a768b64f2..d11b3cad00a 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -26,7 +26,7 @@ struct S3QueueTableMetadata S3QueueTableMetadata() = default; S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings); - void read(const String & in); + void read(const String & s); static S3QueueTableMetadata parse(const String & s); String toString() const; From f07999699ff28a0708ca05bf9a1dd7e6d3ddad5a Mon Sep 17 00:00:00 2001 From: Nikita Keba Date: Mon, 29 May 2023 17:00:38 +0000 Subject: [PATCH 0100/2047] add libarchive support --- .gitmodules | 4 + contrib/CMakeLists.txt | 1 + contrib/libarchive | 1 + contrib/libarchive-cmake/CMakeLists.txt | 149 +++ contrib/libarchive-cmake/config.h | 1349 +++++++++++++++++++++++ src/CMakeLists.txt | 2 + 6 files changed, 1506 insertions(+) create mode 160000 contrib/libarchive create mode 100644 contrib/libarchive-cmake/CMakeLists.txt create mode 100644 contrib/libarchive-cmake/config.h diff --git a/.gitmodules b/.gitmodules index 0b88bd616fb..3aaf5518c02 100644 --- a/.gitmodules +++ b/.gitmodules @@ -332,3 +332,7 @@ [submodule "contrib/liburing"] path = contrib/liburing url = https://github.com/axboe/liburing +[submodule "contrib/libarchive"] + path = contrib/libarchive + url = https://github.com/libarchive/libarchive.git + ignore = dirty diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 7d22f54bf89..7fcff84b8da 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -94,6 +94,7 @@ add_contrib (protobuf-cmake protobuf) add_contrib (openldap-cmake openldap) add_contrib (grpc-cmake grpc) add_contrib (msgpack-c-cmake msgpack-c) +add_contrib (libarchive-cmake libarchive) add_contrib (corrosion-cmake corrosion) diff --git a/contrib/libarchive b/contrib/libarchive new file mode 160000 index 00000000000..1f3c62ebf4d --- /dev/null +++ b/contrib/libarchive @@ -0,0 +1 @@ +Subproject commit 1f3c62ebf4d492ac21d3099b3b064993100dd997 diff --git a/contrib/libarchive-cmake/CMakeLists.txt b/contrib/libarchive-cmake/CMakeLists.txt new file mode 100644 index 00000000000..0bf53e737db --- /dev/null +++ b/contrib/libarchive-cmake/CMakeLists.txt @@ -0,0 +1,149 @@ +set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/libarchive") + +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-reserved-id-macro") + +set(SRCS + "${LIBRARY_DIR}/libarchive/archive_acl.c" + "${LIBRARY_DIR}/libarchive/archive_blake2sp_ref.c" + "${LIBRARY_DIR}/libarchive/archive_blake2s_ref.c" + "${LIBRARY_DIR}/libarchive/archive_check_magic.c" + "${LIBRARY_DIR}/libarchive/archive_cmdline.c" + "${LIBRARY_DIR}/libarchive/archive_cryptor.c" + "${LIBRARY_DIR}/libarchive/archive_digest.c" + "${LIBRARY_DIR}/libarchive/archive_disk_acl_darwin.c" + "${LIBRARY_DIR}/libarchive/archive_disk_acl_freebsd.c" + "${LIBRARY_DIR}/libarchive/archive_disk_acl_linux.c" + "${LIBRARY_DIR}/libarchive/archive_disk_acl_sunos.c" + "${LIBRARY_DIR}/libarchive/archive_entry.c" + "${LIBRARY_DIR}/libarchive/archive_entry_copy_bhfi.c" + "${LIBRARY_DIR}/libarchive/archive_entry_copy_stat.c" + "${LIBRARY_DIR}/libarchive/archive_entry_link_resolver.c" + "${LIBRARY_DIR}/libarchive/archive_entry_sparse.c" + "${LIBRARY_DIR}/libarchive/archive_entry_stat.c" + "${LIBRARY_DIR}/libarchive/archive_entry_strmode.c" + "${LIBRARY_DIR}/libarchive/archive_entry_xattr.c" + "${LIBRARY_DIR}/libarchive/archive_getdate.c" + "${LIBRARY_DIR}/libarchive/archive_hmac.c" + "${LIBRARY_DIR}/libarchive/archive_match.c" + "${LIBRARY_DIR}/libarchive/archive_options.c" + "${LIBRARY_DIR}/libarchive/archive_pack_dev.c" + "${LIBRARY_DIR}/libarchive/archive_pathmatch.c" + "${LIBRARY_DIR}/libarchive/archive_ppmd7.c" + "${LIBRARY_DIR}/libarchive/archive_ppmd8.c" + "${LIBRARY_DIR}/libarchive/archive_random.c" + "${LIBRARY_DIR}/libarchive/archive_rb.c" + "${LIBRARY_DIR}/libarchive/archive_read_add_passphrase.c" + "${LIBRARY_DIR}/libarchive/archive_read_append_filter.c" + "${LIBRARY_DIR}/libarchive/archive_read.c" + "${LIBRARY_DIR}/libarchive/archive_read_data_into_fd.c" + "${LIBRARY_DIR}/libarchive/archive_read_disk_entry_from_file.c" + "${LIBRARY_DIR}/libarchive/archive_read_disk_posix.c" + "${LIBRARY_DIR}/libarchive/archive_read_disk_set_standard_lookup.c" + "${LIBRARY_DIR}/libarchive/archive_read_disk_windows.c" + "${LIBRARY_DIR}/libarchive/archive_read_extract2.c" + "${LIBRARY_DIR}/libarchive/archive_read_extract.c" + "${LIBRARY_DIR}/libarchive/archive_read_open_fd.c" + "${LIBRARY_DIR}/libarchive/archive_read_open_file.c" + "${LIBRARY_DIR}/libarchive/archive_read_open_filename.c" + "${LIBRARY_DIR}/libarchive/archive_read_open_memory.c" + "${LIBRARY_DIR}/libarchive/archive_read_set_format.c" + "${LIBRARY_DIR}/libarchive/archive_read_set_options.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_all.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_by_code.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_bzip2.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_compress.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_grzip.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_gzip.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_lrzip.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_lz4.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_lzop.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_none.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_program.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_rpm.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_uu.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_xz.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_filter_zstd.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_7zip.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_all.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_ar.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_by_code.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_cab.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_cpio.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_empty.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_iso9660.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_lha.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_mtree.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_rar5.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_rar.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_raw.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_tar.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_warc.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_xar.c" + "${LIBRARY_DIR}/libarchive/archive_read_support_format_zip.c" + "${LIBRARY_DIR}/libarchive/archive_string.c" + "${LIBRARY_DIR}/libarchive/archive_string_sprintf.c" + "${LIBRARY_DIR}/libarchive/archive_util.c" + "${LIBRARY_DIR}/libarchive/archive_version_details.c" + "${LIBRARY_DIR}/libarchive/archive_virtual.c" + "${LIBRARY_DIR}/libarchive/archive_windows.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_b64encode.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_by_name.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_bzip2.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_compress.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_grzip.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_gzip.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_lrzip.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_lz4.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_lzop.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_none.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_program.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_uuencode.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_xz.c" + "${LIBRARY_DIR}/libarchive/archive_write_add_filter_zstd.c" + "${LIBRARY_DIR}/libarchive/archive_write.c" + "${LIBRARY_DIR}/libarchive/archive_write_disk_posix.c" + "${LIBRARY_DIR}/libarchive/archive_write_disk_set_standard_lookup.c" + "${LIBRARY_DIR}/libarchive/archive_write_disk_windows.c" + "${LIBRARY_DIR}/libarchive/archive_write_open_fd.c" + "${LIBRARY_DIR}/libarchive/archive_write_open_file.c" + "${LIBRARY_DIR}/libarchive/archive_write_open_filename.c" + "${LIBRARY_DIR}/libarchive/archive_write_open_memory.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_7zip.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_ar.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_by_name.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_cpio_binary.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_cpio.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_cpio_newc.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_cpio_odc.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_filter_by_ext.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_gnutar.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_iso9660.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_mtree.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_pax.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_raw.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_shar.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_ustar.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_v7tar.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_warc.c" + # "${LIBRARY_DIR}/libarchive/archive_write_set_format_xar.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_zip.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_options.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_passphrase.c" + "${LIBRARY_DIR}/libarchive/filter_fork_posix.c" + "${LIBRARY_DIR}/libarchive/filter_fork_windows.c" + "${LIBRARY_DIR}/libarchive/xxhash.c" +) + +add_library(_libarchive ${SRCS}) +target_include_directories(_libarchive PUBLIC + ${CMAKE_CURRENT_SOURCE_DIR} + "${LIBRARY_DIR}/libarchive" +) + +target_compile_definitions(_libarchive PUBLIC + HAVE_CONFIG_H +) + +add_library(ch_contrib::libarchive ALIAS _libarchive) \ No newline at end of file diff --git a/contrib/libarchive-cmake/config.h b/contrib/libarchive-cmake/config.h new file mode 100644 index 00000000000..0b54bf63ddd --- /dev/null +++ b/contrib/libarchive-cmake/config.h @@ -0,0 +1,1349 @@ +/* config.h. Generated from build/cmake/config.h.in by cmake configure */ +#define __LIBARCHIVE_CONFIG_H_INCLUDED 1 + +/* + * Ensure we have C99-style int64_t, etc, all defined. + */ + +/* First, we need to know if the system has already defined them. */ +#define HAVE_INT16_T +#define HAVE_INT32_T +#define HAVE_INT64_T +#define HAVE_INTMAX_T + +#define HAVE_UINT8_T +#define HAVE_UINT16_T +#define HAVE_UINT32_T +#define HAVE_UINT64_T +#define HAVE_UINTMAX_T + +/* We might have the types we want under other spellings. */ +/* #undef HAVE___INT64 */ +/* #undef HAVE_U_INT64_T */ +/* #undef HAVE_UNSIGNED___INT64 */ + +/* The sizes of various standard integer types. */ +#define SIZEOF_SHORT 2 +#define SIZEOF_INT 4 +#define SIZEOF_LONG 8 +#define SIZEOF_LONG_LONG 8 +#define SIZEOF_UNSIGNED_SHORT 2 +#define SIZEOF_UNSIGNED 4 +#define SIZEOF_UNSIGNED_LONG 8 +#define SIZEOF_UNSIGNED_LONG_LONG 8 + +/* + * If we lack int64_t, define it to the first of __int64, int, long, and long long + * that exists and is the right size. + */ +#if !defined(HAVE_INT64_T) && defined(HAVE___INT64) +typedef __int64 int64_t; +#define HAVE_INT64_T +#endif + +#if !defined(HAVE_INT64_T) && SIZEOF_INT == 8 +typedef int int64_t; +#define HAVE_INT64_T +#endif + +#if !defined(HAVE_INT64_T) && SIZEOF_LONG == 8 +typedef long int64_t; +#define HAVE_INT64_T +#endif + +#if !defined(HAVE_INT64_T) && SIZEOF_LONG_LONG == 8 +typedef long long int64_t; +#define HAVE_INT64_T +#endif + +#if !defined(HAVE_INT64_T) +#error No 64-bit integer type was found. +#endif + +/* + * Similarly for int32_t + */ +#if !defined(HAVE_INT32_T) && SIZEOF_INT == 4 +typedef int int32_t; +#define HAVE_INT32_T +#endif + +#if !defined(HAVE_INT32_T) && SIZEOF_LONG == 4 +typedef long int32_t; +#define HAVE_INT32_T +#endif + +#if !defined(HAVE_INT32_T) +#error No 32-bit integer type was found. +#endif + +/* + * Similarly for int16_t + */ +#if !defined(HAVE_INT16_T) && SIZEOF_INT == 2 +typedef int int16_t; +#define HAVE_INT16_T +#endif + +#if !defined(HAVE_INT16_T) && SIZEOF_SHORT == 2 +typedef short int16_t; +#define HAVE_INT16_T +#endif + +#if !defined(HAVE_INT16_T) +#error No 16-bit integer type was found. +#endif + +/* + * Similarly for uint64_t + */ +#if !defined(HAVE_UINT64_T) && defined(HAVE_UNSIGNED___INT64) +typedef unsigned __int64 uint64_t; +#define HAVE_UINT64_T +#endif + +#if !defined(HAVE_UINT64_T) && SIZEOF_UNSIGNED == 8 +typedef unsigned uint64_t; +#define HAVE_UINT64_T +#endif + +#if !defined(HAVE_UINT64_T) && SIZEOF_UNSIGNED_LONG == 8 +typedef unsigned long uint64_t; +#define HAVE_UINT64_T +#endif + +#if !defined(HAVE_UINT64_T) && SIZEOF_UNSIGNED_LONG_LONG == 8 +typedef unsigned long long uint64_t; +#define HAVE_UINT64_T +#endif + +#if !defined(HAVE_UINT64_T) +#error No 64-bit unsigned integer type was found. +#endif + + +/* + * Similarly for uint32_t + */ +#if !defined(HAVE_UINT32_T) && SIZEOF_UNSIGNED == 4 +typedef unsigned uint32_t; +#define HAVE_UINT32_T +#endif + +#if !defined(HAVE_UINT32_T) && SIZEOF_UNSIGNED_LONG == 4 +typedef unsigned long uint32_t; +#define HAVE_UINT32_T +#endif + +#if !defined(HAVE_UINT32_T) +#error No 32-bit unsigned integer type was found. +#endif + +/* + * Similarly for uint16_t + */ +#if !defined(HAVE_UINT16_T) && SIZEOF_UNSIGNED == 2 +typedef unsigned uint16_t; +#define HAVE_UINT16_T +#endif + +#if !defined(HAVE_UINT16_T) && SIZEOF_UNSIGNED_SHORT == 2 +typedef unsigned short uint16_t; +#define HAVE_UINT16_T +#endif + +#if !defined(HAVE_UINT16_T) +#error No 16-bit unsigned integer type was found. +#endif + +/* + * Similarly for uint8_t + */ +#if !defined(HAVE_UINT8_T) +typedef unsigned char uint8_t; +#define HAVE_UINT8_T +#endif + +#if !defined(HAVE_UINT8_T) +#error No 8-bit unsigned integer type was found. +#endif + +/* Define intmax_t and uintmax_t if they are not already defined. */ +#if !defined(HAVE_INTMAX_T) +typedef int64_t intmax_t; +#endif + +#if !defined(HAVE_UINTMAX_T) +typedef uint64_t uintmax_t; +#endif + +/* Define ZLIB_WINAPI if zlib was built on Visual Studio. */ +/* #undef ZLIB_WINAPI */ + +/* Darwin ACL support */ +/* #undef ARCHIVE_ACL_DARWIN */ + +/* FreeBSD ACL support */ +/* #undef ARCHIVE_ACL_FREEBSD */ + +/* FreeBSD NFSv4 ACL support */ +/* #undef ARCHIVE_ACL_FREEBSD_NFS4 */ + +/* Linux POSIX.1e ACL support via libacl */ +/* #undef ARCHIVE_ACL_LIBACL */ + +/* Linux NFSv4 ACL support via librichacl */ +/* #undef ARCHIVE_ACL_LIBRICHACL */ + +/* Solaris ACL support */ +/* #undef ARCHIVE_ACL_SUNOS */ + +/* Solaris NFSv4 ACL support */ +/* #undef ARCHIVE_ACL_SUNOS_NFS4 */ + +/* MD5 via ARCHIVE_CRYPTO_MD5_LIBC supported. */ +/* #undef ARCHIVE_CRYPTO_MD5_LIBC */ + +/* MD5 via ARCHIVE_CRYPTO_MD5_LIBSYSTEM supported. */ +/* #undef ARCHIVE_CRYPTO_MD5_LIBSYSTEM */ + +/* MD5 via ARCHIVE_CRYPTO_MD5_NETTLE supported. */ +/* #undef ARCHIVE_CRYPTO_MD5_NETTLE */ + +/* MD5 via ARCHIVE_CRYPTO_MD5_OPENSSL supported. */ +/* #undef ARCHIVE_CRYPTO_MD5_OPENSSL */ + +/* MD5 via ARCHIVE_CRYPTO_MD5_WIN supported. */ +/* #undef ARCHIVE_CRYPTO_MD5_WIN */ + +/* RMD160 via ARCHIVE_CRYPTO_RMD160_LIBC supported. */ +/* #undef ARCHIVE_CRYPTO_RMD160_LIBC */ + +/* RMD160 via ARCHIVE_CRYPTO_RMD160_NETTLE supported. */ +/* #undef ARCHIVE_CRYPTO_RMD160_NETTLE */ + +/* RMD160 via ARCHIVE_CRYPTO_RMD160_OPENSSL supported. */ +/* #undef ARCHIVE_CRYPTO_RMD160_OPENSSL */ + +/* SHA1 via ARCHIVE_CRYPTO_SHA1_LIBC supported. */ +/* #undef ARCHIVE_CRYPTO_SHA1_LIBC */ + +/* SHA1 via ARCHIVE_CRYPTO_SHA1_LIBSYSTEM supported. */ +/* #undef ARCHIVE_CRYPTO_SHA1_LIBSYSTEM */ + +/* SHA1 via ARCHIVE_CRYPTO_SHA1_NETTLE supported. */ +/* #undef ARCHIVE_CRYPTO_SHA1_NETTLE */ + +/* SHA1 via ARCHIVE_CRYPTO_SHA1_OPENSSL supported. */ +/* #undef ARCHIVE_CRYPTO_SHA1_OPENSSL */ + +/* SHA1 via ARCHIVE_CRYPTO_SHA1_WIN supported. */ +/* #undef ARCHIVE_CRYPTO_SHA1_WIN */ + +/* SHA256 via ARCHIVE_CRYPTO_SHA256_LIBC supported. */ +/* #undef ARCHIVE_CRYPTO_SHA256_LIBC */ + +/* SHA256 via ARCHIVE_CRYPTO_SHA256_LIBC2 supported. */ +/* #undef ARCHIVE_CRYPTO_SHA256_LIBC2 */ + +/* SHA256 via ARCHIVE_CRYPTO_SHA256_LIBC3 supported. */ +/* #undef ARCHIVE_CRYPTO_SHA256_LIBC3 */ + +/* SHA256 via ARCHIVE_CRYPTO_SHA256_LIBSYSTEM supported. */ +/* #undef ARCHIVE_CRYPTO_SHA256_LIBSYSTEM */ + +/* SHA256 via ARCHIVE_CRYPTO_SHA256_NETTLE supported. */ +/* #undef ARCHIVE_CRYPTO_SHA256_NETTLE */ + +/* SHA256 via ARCHIVE_CRYPTO_SHA256_OPENSSL supported. */ +/* #undef ARCHIVE_CRYPTO_SHA256_OPENSSL */ + +/* SHA256 via ARCHIVE_CRYPTO_SHA256_WIN supported. */ +/* #undef ARCHIVE_CRYPTO_SHA256_WIN */ + +/* SHA384 via ARCHIVE_CRYPTO_SHA384_LIBC supported. */ +/* #undef ARCHIVE_CRYPTO_SHA384_LIBC */ + +/* SHA384 via ARCHIVE_CRYPTO_SHA384_LIBC2 supported. */ +/* #undef ARCHIVE_CRYPTO_SHA384_LIBC2 */ + +/* SHA384 via ARCHIVE_CRYPTO_SHA384_LIBC3 supported. */ +/* #undef ARCHIVE_CRYPTO_SHA384_LIBC3 */ + +/* SHA384 via ARCHIVE_CRYPTO_SHA384_LIBSYSTEM supported. */ +/* #undef ARCHIVE_CRYPTO_SHA384_LIBSYSTEM */ + +/* SHA384 via ARCHIVE_CRYPTO_SHA384_NETTLE supported. */ +/* #undef ARCHIVE_CRYPTO_SHA384_NETTLE */ + +/* SHA384 via ARCHIVE_CRYPTO_SHA384_OPENSSL supported. */ +/* #undef ARCHIVE_CRYPTO_SHA384_OPENSSL */ + +/* SHA384 via ARCHIVE_CRYPTO_SHA384_WIN supported. */ +/* #undef ARCHIVE_CRYPTO_SHA384_WIN */ + +/* SHA512 via ARCHIVE_CRYPTO_SHA512_LIBC supported. */ +/* #undef ARCHIVE_CRYPTO_SHA512_LIBC */ + +/* SHA512 via ARCHIVE_CRYPTO_SHA512_LIBC2 supported. */ +/* #undef ARCHIVE_CRYPTO_SHA512_LIBC2 */ + +/* SHA512 via ARCHIVE_CRYPTO_SHA512_LIBC3 supported. */ +/* #undef ARCHIVE_CRYPTO_SHA512_LIBC3 */ + +/* SHA512 via ARCHIVE_CRYPTO_SHA512_LIBSYSTEM supported. */ +/* #undef ARCHIVE_CRYPTO_SHA512_LIBSYSTEM */ + +/* SHA512 via ARCHIVE_CRYPTO_SHA512_NETTLE supported. */ +/* #undef ARCHIVE_CRYPTO_SHA512_NETTLE */ + +/* SHA512 via ARCHIVE_CRYPTO_SHA512_OPENSSL supported. */ +/* #undef ARCHIVE_CRYPTO_SHA512_OPENSSL */ + +/* SHA512 via ARCHIVE_CRYPTO_SHA512_WIN supported. */ +/* #undef ARCHIVE_CRYPTO_SHA512_WIN */ + +/* AIX xattr support */ +/* #undef ARCHIVE_XATTR_AIX */ + +/* Darwin xattr support */ +/* #undef ARCHIVE_XATTR_DARWIN */ + +/* FreeBSD xattr support */ +/* #undef ARCHIVE_XATTR_FREEBSD */ + +/* Linux xattr support */ +#define ARCHIVE_XATTR_LINUX 1 + +/* Version number of bsdcpio */ +#define BSDCPIO_VERSION_STRING "3.6.3" + +/* Version number of bsdtar */ +#define BSDTAR_VERSION_STRING "3.6.3" + +/* Version number of bsdcat */ +#define BSDCAT_VERSION_STRING "3.6.3" + +/* Define to 1 if you have the `acl_create_entry' function. */ +/* #undef HAVE_ACL_CREATE_ENTRY */ + +/* Define to 1 if you have the `acl_get_fd_np' function. */ +/* #undef HAVE_ACL_GET_FD_NP */ + +/* Define to 1 if you have the `acl_get_link' function. */ +/* #undef HAVE_ACL_GET_LINK */ + +/* Define to 1 if you have the `acl_get_link_np' function. */ +/* #undef HAVE_ACL_GET_LINK_NP */ + +/* Define to 1 if you have the `acl_get_perm' function. */ +/* #undef HAVE_ACL_GET_PERM */ + +/* Define to 1 if you have the `acl_get_perm_np' function. */ +/* #undef HAVE_ACL_GET_PERM_NP */ + +/* Define to 1 if you have the `acl_init' function. */ +/* #undef HAVE_ACL_INIT */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_ACL_LIBACL_H */ + +/* Define to 1 if the system has the type `acl_permset_t'. */ +/* #undef HAVE_ACL_PERMSET_T */ + +/* Define to 1 if you have the `acl_set_fd' function. */ +/* #undef HAVE_ACL_SET_FD */ + +/* Define to 1 if you have the `acl_set_fd_np' function. */ +/* #undef HAVE_ACL_SET_FD_NP */ + +/* Define to 1 if you have the `acl_set_file' function. */ +/* #undef HAVE_ACL_SET_FILE */ + +/* Define to 1 if you have the `arc4random_buf' function. */ +/* #undef HAVE_ARC4RANDOM_BUF */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_ATTR_XATTR_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_BCRYPT_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_BSDXML_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_BZLIB_H */ + +/* Define to 1 if you have the `chflags' function. */ +/* #undef HAVE_CHFLAGS */ + +/* Define to 1 if you have the `chown' function. */ +#define HAVE_CHOWN 1 + +/* Define to 1 if you have the `chroot' function. */ +#define HAVE_CHROOT 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_COPYFILE_H */ + +/* Define to 1 if you have the `ctime_r' function. */ +#define HAVE_CTIME_R 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_CTYPE_H 1 + +/* Define to 1 if you have the `cygwin_conv_path' function. */ +/* #undef HAVE_CYGWIN_CONV_PATH */ + +/* Define to 1 if you have the declaration of `ACE_GETACL', and to 0 if you + don't. */ +/* #undef HAVE_DECL_ACE_GETACL */ + +/* Define to 1 if you have the declaration of `ACE_GETACLCNT', and to 0 if you + don't. */ +/* #undef HAVE_DECL_ACE_GETACLCNT */ + +/* Define to 1 if you have the declaration of `ACE_SETACL', and to 0 if you + don't. */ +/* #undef HAVE_DECL_ACE_SETACL */ + +/* Define to 1 if you have the declaration of `ACL_SYNCHRONIZE', and to 0 if + you don't. */ +/* #undef HAVE_DECL_ACL_SYNCHRONIZE */ + +/* Define to 1 if you have the declaration of `ACL_TYPE_EXTENDED', and to 0 if + you don't. */ +/* #undef HAVE_DECL_ACL_TYPE_EXTENDED */ + +/* Define to 1 if you have the declaration of `ACL_TYPE_NFS4', and to 0 if you + don't. */ +/* #undef HAVE_DECL_ACL_TYPE_NFS4 */ + +/* Define to 1 if you have the declaration of `ACL_USER', and to 0 if you + don't. */ +/* #undef HAVE_DECL_ACL_USER */ + +/* Define to 1 if you have the declaration of `INT32_MAX', and to 0 if you + don't. */ +#define HAVE_DECL_INT32_MAX 1 + +/* Define to 1 if you have the declaration of `INT32_MIN', and to 0 if you + don't. */ +#define HAVE_DECL_INT32_MIN 1 + +/* Define to 1 if you have the declaration of `INT64_MAX', and to 0 if you + don't. */ +#define HAVE_DECL_INT64_MAX 1 + +/* Define to 1 if you have the declaration of `INT64_MIN', and to 0 if you + don't. */ +#define HAVE_DECL_INT64_MIN 1 + +/* Define to 1 if you have the declaration of `INTMAX_MAX', and to 0 if you + don't. */ +#define HAVE_DECL_INTMAX_MAX 1 + +/* Define to 1 if you have the declaration of `INTMAX_MIN', and to 0 if you + don't. */ +#define HAVE_DECL_INTMAX_MIN 1 + +/* Define to 1 if you have the declaration of `SETACL', and to 0 if you don't. + */ +/* #undef HAVE_DECL_SETACL */ + +/* Define to 1 if you have the declaration of `SIZE_MAX', and to 0 if you + don't. */ +#define HAVE_DECL_SIZE_MAX 1 + +/* Define to 1 if you have the declaration of `SSIZE_MAX', and to 0 if you + don't. */ +#define HAVE_DECL_SSIZE_MAX 1 + +/* Define to 1 if you have the declaration of `strerror_r', and to 0 if you + don't. */ +#define HAVE_DECL_STRERROR_R 1 + +/* Define to 1 if you have the declaration of `UINT32_MAX', and to 0 if you + don't. */ +#define HAVE_DECL_UINT32_MAX 1 + +/* Define to 1 if you have the declaration of `UINT64_MAX', and to 0 if you + don't. */ +#define HAVE_DECL_UINT64_MAX 1 + +/* Define to 1 if you have the declaration of `UINTMAX_MAX', and to 0 if you + don't. */ +#define HAVE_DECL_UINTMAX_MAX 1 + +/* Define to 1 if you have the declaration of `XATTR_NOFOLLOW', and to 0 if + you don't. */ +/* #undef HAVE_DECL_XATTR_NOFOLLOW */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_DIRECT_H */ + +/* Define to 1 if you have the header file, and it defines `DIR'. + */ +#define HAVE_DIRENT_H 1 + +/* Define to 1 if you have the `dirfd' function. */ +#define HAVE_DIRFD 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_DLFCN_H 1 + +/* Define to 1 if you don't have `vprintf' but do have `_doprnt.' */ +/* #undef HAVE_DOPRNT */ + +/* Define to 1 if nl_langinfo supports D_MD_ORDER */ +/* #undef HAVE_D_MD_ORDER */ + +/* A possible errno value for invalid file format errors */ +/* #undef HAVE_EFTYPE */ + +/* A possible errno value for invalid file format errors */ +#define HAVE_EILSEQ 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_ERRNO_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_EXPAT_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_EXT2FS_EXT2_FS_H */ + +/* Define to 1 if you have the `extattr_get_file' function. */ +/* #undef HAVE_EXTATTR_GET_FILE */ + +/* Define to 1 if you have the `extattr_list_file' function. */ +/* #undef HAVE_EXTATTR_LIST_FILE */ + +/* Define to 1 if you have the `extattr_set_fd' function. */ +/* #undef HAVE_EXTATTR_SET_FD */ + +/* Define to 1 if you have the `extattr_set_file' function. */ +/* #undef HAVE_EXTATTR_SET_FILE */ + +/* Define to 1 if EXTATTR_NAMESPACE_USER is defined in sys/extattr.h. */ +/* #undef HAVE_DECL_EXTATTR_NAMESPACE_USER */ + +/* Define to 1 if you have the declaration of `GETACL', and to 0 if you don't. + */ +/* #undef HAVE_DECL_GETACL */ + +/* Define to 1 if you have the declaration of `GETACLCNT', and to 0 if you + don't. */ +/* #undef HAVE_DECL_GETACLCNT */ + +/* Define to 1 if you have the `fchdir' function. */ +#define HAVE_FCHDIR 1 + +/* Define to 1 if you have the `fchflags' function. */ +/* #undef HAVE_FCHFLAGS */ + +/* Define to 1 if you have the `fchmod' function. */ +#define HAVE_FCHMOD 1 + +/* Define to 1 if you have the `fchown' function. */ +#define HAVE_FCHOWN 1 + +/* Define to 1 if you have the `fcntl' function. */ +#define HAVE_FCNTL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_FCNTL_H 1 + +/* Define to 1 if you have the `fdopendir' function. */ +#define HAVE_FDOPENDIR 1 + +/* Define to 1 if you have the `fgetea' function. */ +/* #undef HAVE_FGETEA */ + +/* Define to 1 if you have the `fgetxattr' function. */ +#define HAVE_FGETXATTR 1 + +/* Define to 1 if you have the `flistea' function. */ +/* #undef HAVE_FLISTEA */ + +/* Define to 1 if you have the `flistxattr' function. */ +#define HAVE_FLISTXATTR 1 + +/* Define to 1 if you have the `fork' function. */ +#define HAVE_FORK 1 + +/* Define to 1 if fseeko (and presumably ftello) exists and is declared. */ +#define HAVE_FSEEKO 1 + +/* Define to 1 if you have the `fsetea' function. */ +/* #undef HAVE_FSETEA */ + +/* Define to 1 if you have the `fsetxattr' function. */ +#define HAVE_FSETXATTR 1 + +/* Define to 1 if you have the `fstat' function. */ +#define HAVE_FSTAT 1 + +/* Define to 1 if you have the `fstatat' function. */ +#define HAVE_FSTATAT 1 + +/* Define to 1 if you have the `fstatfs' function. */ +#define HAVE_FSTATFS 1 + +/* Define to 1 if you have the `fstatvfs' function. */ +#define HAVE_FSTATVFS 1 + +/* Define to 1 if you have the `ftruncate' function. */ +#define HAVE_FTRUNCATE 1 + +/* Define to 1 if you have the `futimens' function. */ +#define HAVE_FUTIMENS 1 + +/* Define to 1 if you have the `futimes' function. */ +#define HAVE_FUTIMES 1 + +/* Define to 1 if you have the `futimesat' function. */ +#define HAVE_FUTIMESAT 1 + +/* Define to 1 if you have the `getea' function. */ +/* #undef HAVE_GETEA */ + +/* Define to 1 if you have the `geteuid' function. */ +#define HAVE_GETEUID 1 + +/* Define to 1 if you have the `getgrgid_r' function. */ +#define HAVE_GETGRGID_R 1 + +/* Define to 1 if you have the `getgrnam_r' function. */ +#define HAVE_GETGRNAM_R 1 + +/* Define to 1 if you have the `getpid' function. */ +#define HAVE_GETPID 1 + +/* Define to 1 if you have the `getpwnam_r' function. */ +#define HAVE_GETPWNAM_R 1 + +/* Define to 1 if you have the `getpwuid_r' function. */ +#define HAVE_GETPWUID_R 1 + +/* Define to 1 if you have the `getvfsbyname' function. */ +/* #undef HAVE_GETVFSBYNAME */ + +/* Define to 1 if you have the `getxattr' function. */ +#define HAVE_GETXATTR 1 + +/* Define to 1 if you have the `gmtime_r' function. */ +#define HAVE_GMTIME_R 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_GRP_H 1 + +/* Define to 1 if you have the `iconv' function. */ +#define HAVE_ICONV 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_ICONV_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_INTTYPES_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_IO_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_LANGINFO_H 1 + +/* Define to 1 if you have the `lchflags' function. */ +/* #undef HAVE_LCHFLAGS */ + +/* Define to 1 if you have the `lchmod' function. */ +#define HAVE_LCHMOD 1 + +/* Define to 1 if you have the `lchown' function. */ +#define HAVE_LCHOWN 1 + +/* Define to 1 if you have the `lgetea' function. */ +/* #undef HAVE_LGETEA */ + +/* Define to 1 if you have the `lgetxattr' function. */ +#define HAVE_LGETXATTR 1 + +/* Define to 1 if you have the `acl' library (-lacl). */ +/* #undef HAVE_LIBACL */ + +/* Define to 1 if you have the `attr' library (-lattr). */ +/* #undef HAVE_LIBATTR */ + +/* Define to 1 if you have the `bsdxml' library (-lbsdxml). */ +/* #undef HAVE_LIBBSDXML */ + +/* Define to 1 if you have the `bz2' library (-lbz2). */ +/* #undef HAVE_LIBBZ2 */ + +/* Define to 1 if you have the `b2' library (-lb2). */ +/* #undef HAVE_LIBB2 */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_BLAKE2_H */ + +/* Define to 1 if you have the `charset' library (-lcharset). */ +/* #undef HAVE_LIBCHARSET */ + +/* Define to 1 if you have the `crypto' library (-lcrypto). */ +/* #undef HAVE_LIBCRYPTO */ + +/* Define to 1 if you have the `expat' library (-lexpat). */ +/* #undef HAVE_LIBEXPAT */ + +/* Define to 1 if you have the `gcc' library (-lgcc). */ +/* #undef HAVE_LIBGCC */ + +/* Define to 1 if you have the `lz4' library (-llz4). */ +/* #undef HAVE_LIBLZ4 */ + +/* Define to 1 if you have the `lzma' library (-llzma). */ +/* #undef HAVE_LIBLZMA */ + +/* Define to 1 if you have the `lzmadec' library (-llzmadec). */ +/* #undef HAVE_LIBLZMADEC */ + +/* Define to 1 if you have the `lzo2' library (-llzo2). */ +/* #undef HAVE_LIBLZO2 */ + +/* Define to 1 if you have the `mbedcrypto' library (-lmbedcrypto). */ +/* #undef HAVE_LIBMBEDCRYPTO */ + +/* Define to 1 if you have the `nettle' library (-lnettle). */ +/* #undef HAVE_LIBNETTLE */ + +/* Define to 1 if you have the `pcre' library (-lpcre). */ +/* #undef HAVE_LIBPCRE */ + +/* Define to 1 if you have the `pcreposix' library (-lpcreposix). */ +/* #undef HAVE_LIBPCREPOSIX */ + +/* Define to 1 if you have the `xml2' library (-lxml2). */ +#define HAVE_LIBXML2 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_LIBXML_XMLREADER_H 0 + +/* Define to 1 if you have the header file. */ +#define HAVE_LIBXML_XMLWRITER_H 1 + +/* Define to 1 if you have the `z' library (-lz). */ +/* #undef HAVE_LIBZ */ + +/* Define to 1 if you have the `zstd' library (-lzstd). */ +/* #undef HAVE_LIBZSTD */ + +/* Define to 1 if you have the `zstd' library (-lzstd) with compression + support. */ +/* #undef HAVE_LIBZSTD_COMPRESSOR */ + +/* Define to 1 if you have the header file. */ +#define HAVE_LIMITS_H 1 + +/* Define to 1 if you have the `link' function. */ +#define HAVE_LINK 1 + +/* Define to 1 if you have the `linkat' function. */ +#define HAVE_LINKAT 1 + +/* Define to 1 if you have the header file. */ +// #define HAVE_LINUX_FIEMAP_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_LINUX_FS_H 1 + +/* Define to 1 if you have the header file. */ +// #define HAVE_LINUX_MAGIC_H 0 + +/* Define to 1 if you have the header file. */ +#define HAVE_LINUX_TYPES_H 1 + +/* Define to 1 if you have the `listea' function. */ +/* #undef HAVE_LISTEA */ + +/* Define to 1 if you have the `listxattr' function. */ +#define HAVE_LISTXATTR 1 + +/* Define to 1 if you have the `llistea' function. */ +/* #undef HAVE_LLISTEA */ + +/* Define to 1 if you have the `llistxattr' function. */ +#define HAVE_LLISTXATTR 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_LOCALCHARSET_H */ + +/* Define to 1 if you have the `locale_charset' function. */ +/* #undef HAVE_LOCALE_CHARSET */ + +/* Define to 1 if you have the header file. */ +#define HAVE_LOCALE_H 1 + +/* Define to 1 if you have the `localtime_r' function. */ +#define HAVE_LOCALTIME_R 1 + +/* Define to 1 if the system has the type `long long int'. */ +/* #undef HAVE_LONG_LONG_INT */ + +/* Define to 1 if you have the `lsetea' function. */ +/* #undef HAVE_LSETEA */ + +/* Define to 1 if you have the `lsetxattr' function. */ +#define HAVE_LSETXATTR 1 + +/* Define to 1 if you have the `lstat' function. */ +#define HAVE_LSTAT 1 + +/* Define to 1 if `lstat' has the bug that it succeeds when given the + zero-length file name argument. */ +/* #undef HAVE_LSTAT_EMPTY_STRING_BUG */ + +/* Define to 1 if you have the `lutimes' function. */ +#define HAVE_LUTIMES 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_LZ4HC_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_LZ4_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_LZMADEC_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_LZMA_H */ + +/* Define to 1 if you have a working `lzma_stream_encoder_mt' function. */ +/* #undef HAVE_LZMA_STREAM_ENCODER_MT */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_LZO_LZO1X_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_LZO_LZOCONF_H */ + +/* Define to 1 if you have the `mbrtowc' function. */ +// #define HAVE_MBRTOWC 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_MEMBERSHIP_H */ + +/* Define to 1 if you have the `memmove' function. */ +#define HAVE_MEMMOVE 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_MEMORY_H 1 + +/* Define to 1 if you have the `mkdir' function. */ +#define HAVE_MKDIR 1 + +/* Define to 1 if you have the `mkfifo' function. */ +#define HAVE_MKFIFO 1 + +/* Define to 1 if you have the `mknod' function. */ +#define HAVE_MKNOD 1 + +/* Define to 1 if you have the `mkstemp' function. */ +#define HAVE_MKSTEMP 1 + +/* Define to 1 if you have the header file, and it defines `DIR'. */ +/* #undef HAVE_NDIR_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_NETTLE_AES_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_NETTLE_HMAC_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_NETTLE_MD5_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_NETTLE_PBKDF2_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_NETTLE_RIPEMD160_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_NETTLE_SHA_H */ + +/* Define to 1 if you have the `nl_langinfo' function. */ +#define HAVE_NL_LANGINFO 1 + +/* Define to 1 if you have the `openat' function. */ +#define HAVE_OPENAT 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_PATHS_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_PCREPOSIX_H */ + +/* Define to 1 if you have the `pipe' function. */ +#define HAVE_PIPE 1 + +/* Define to 1 if you have the `PKCS5_PBKDF2_HMAC_SHA1' function. */ +/* #undef HAVE_PKCS5_PBKDF2_HMAC_SHA1 */ + +/* Define to 1 if you have the `poll' function. */ +#define HAVE_POLL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_POLL_H 1 + +/* Define to 1 if you have the `posix_spawnp' function. */ +#define HAVE_POSIX_SPAWNP 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_PROCESS_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_PTHREAD_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_PWD_H 1 + +/* Define to 1 if you have the `readdir_r' function. */ +/* #undef HAVE_READDIR_R */ + +/* Define to 1 if you have the `readlink' function. */ +#define HAVE_READLINK 1 + +/* Define to 1 if you have the `readlinkat' function. */ +#define HAVE_READLINKAT 1 + +/* Define to 1 if you have the `readpassphrase' function. */ +/* #undef HAVE_READPASSPHRASE */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_READPASSPHRASE_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_REGEX_H 1 + +/* Define to 1 if you have the `select' function. */ +#define HAVE_SELECT 1 + +/* Define to 1 if you have the `setenv' function. */ +#define HAVE_SETENV 1 + +/* Define to 1 if you have the `setlocale' function. */ +#define HAVE_SETLOCALE 1 + +/* Define to 1 if you have the `sigaction' function. */ +#define HAVE_SIGACTION 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SIGNAL_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SPAWN_H 1 + +/* Define to 1 if you have the `statfs' function. */ +#define HAVE_STATFS 1 + +/* Define to 1 if you have the `statvfs' function. */ +#define HAVE_STATVFS 1 + +/* Define to 1 if `stat' has the bug that it succeeds when given the + zero-length file name argument. */ +/* #undef HAVE_STAT_EMPTY_STRING_BUG */ + +/* Define to 1 if you have the header file. */ +#define HAVE_STDARG_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDINT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDLIB_H 1 + +/* Define to 1 if you have the `strchr' function. */ +#define HAVE_STRCHR 1 + +/* Define to 1 if you have the `strnlen' function. */ +#define HAVE_STRNLEN 1 + +/* Define to 1 if you have the `strdup' function. */ +#define HAVE_STRDUP 1 + +/* Define to 1 if you have the `strerror' function. */ +#define HAVE_STRERROR 1 + +/* Define to 1 if you have the `strerror_r' function. */ +#define HAVE_STRERROR_R 1 + +/* Define to 1 if you have the `strftime' function. */ +#define HAVE_STRFTIME 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRINGS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRING_H 1 + +/* Define to 1 if you have the `strrchr' function. */ +#define HAVE_STRRCHR 1 + +/* Define to 1 if `f_namemax' is a member of `struct statfs'. */ +/* #undef HAVE_STRUCT_STATFS_F_NAMEMAX */ + +/* Define to 1 if `f_iosize' is a member of `struct statvfs'. */ +/* #undef HAVE_STRUCT_STATVFS_F_IOSIZE */ + +/* Define to 1 if `st_birthtime' is a member of `struct stat'. */ +/* #undef HAVE_STRUCT_STAT_ST_BIRTHTIME */ + +/* Define to 1 if `st_birthtimespec.tv_nsec' is a member of `struct stat'. */ +/* #undef HAVE_STRUCT_STAT_ST_BIRTHTIMESPEC_TV_NSEC */ + +/* Define to 1 if `st_blksize' is a member of `struct stat'. */ +#define HAVE_STRUCT_STAT_ST_BLKSIZE 1 + +/* Define to 1 if `st_flags' is a member of `struct stat'. */ +/* #undef HAVE_STRUCT_STAT_ST_FLAGS */ + +/* Define to 1 if `st_mtimespec.tv_nsec' is a member of `struct stat'. */ +/* #undef HAVE_STRUCT_STAT_ST_MTIMESPEC_TV_NSEC */ + +/* Define to 1 if `st_mtime_n' is a member of `struct stat'. */ +/* #undef HAVE_STRUCT_STAT_ST_MTIME_N */ + +/* Define to 1 if `st_mtime_usec' is a member of `struct stat'. */ +/* #undef HAVE_STRUCT_STAT_ST_MTIME_USEC */ + +/* Define to 1 if `st_mtim.tv_nsec' is a member of `struct stat'. */ +#define HAVE_STRUCT_STAT_ST_MTIM_TV_NSEC 1 + +/* Define to 1 if `st_umtime' is a member of `struct stat'. */ +/* #undef HAVE_STRUCT_STAT_ST_UMTIME */ + +/* Define to 1 if `tm_gmtoff' is a member of `struct tm'. */ +#define HAVE_STRUCT_TM_TM_GMTOFF 1 + +/* Define to 1 if `__tm_gmtoff' is a member of `struct tm'. */ +/* #undef HAVE_STRUCT_TM___TM_GMTOFF */ + +/* Define to 1 if you have `struct vfsconf'. */ +/* #undef HAVE_STRUCT_VFSCONF */ + +/* Define to 1 if you have `struct xvfsconf'. */ +/* #undef HAVE_STRUCT_XVFSCONF */ + +/* Define to 1 if you have the `symlink' function. */ +#define HAVE_SYMLINK 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_ACL_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_CDEFS_H 1 + +/* Define to 1 if you have the header file, and it defines `DIR'. + */ +/* #undef HAVE_SYS_DIR_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_EA_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_EXTATTR_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_IOCTL_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_MKDEV_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_MOUNT_H */ + +/* Define to 1 if you have the header file, and it defines `DIR'. + */ +/* #undef HAVE_SYS_NDIR_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_PARAM_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_POLL_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_RICHACL_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SELECT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_STATFS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_STATVFS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_STAT_H 1 + + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SYSMACROS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_TIME_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_TYPES_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_UTIME_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_UTSNAME_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_VFS_H 1 + +/* Define to 1 if you have that is POSIX.1 compatible. */ +#define HAVE_SYS_WAIT_H 1 + +/* Define to 1 if you have the header file. */ +// #define HAVE_SYS_XATTR_H 0 + +/* Define to 1 if you have the `timegm' function. */ +#define HAVE_TIMEGM 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_TIME_H 1 + +/* Define to 1 if you have the `tzset' function. */ +#define HAVE_TZSET 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_UNISTD_H 1 + +/* Define to 1 if you have the `unlinkat' function. */ +#define HAVE_UNLINKAT 1 + +/* Define to 1 if you have the `unsetenv' function. */ +#define HAVE_UNSETENV 1 + +/* Define to 1 if the system has the type `unsigned long long'. */ +/* #undef HAVE_UNSIGNED_LONG_LONG */ + +/* Define to 1 if the system has the type `unsigned long long int'. */ +/* #undef HAVE_UNSIGNED_LONG_LONG_INT */ + +/* Define to 1 if you have the `utime' function. */ +#define HAVE_UTIME 1 + +/* Define to 1 if you have the `utimensat' function. */ +#define HAVE_UTIMENSAT 1 + +/* Define to 1 if you have the `utimes' function. */ +#define HAVE_UTIMES 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_UTIME_H 1 + +/* Define to 1 if you have the `vfork' function. */ +#define HAVE_VFORK 1 + +/* Define to 1 if you have the `vprintf' function. */ +#define HAVE_VPRINTF 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_WCHAR_H 1 + +/* Define to 1 if the system has the type `wchar_t'. */ +#define HAVE_WCHAR_T 1 + +/* Define to 1 if you have the `wcrtomb' function. */ +#define HAVE_WCRTOMB 1 + +/* Define to 1 if you have the `wcscmp' function. */ +#define HAVE_WCSCMP 1 + +/* Define to 1 if you have the `wcscpy' function. */ +#define HAVE_WCSCPY 1 + +/* Define to 1 if you have the `wcslen' function. */ +#define HAVE_WCSLEN 1 + +/* Define to 1 if you have the `wctomb' function. */ +#define HAVE_WCTOMB 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_WCTYPE_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_WINCRYPT_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_WINDOWS_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_WINIOCTL_H */ + +/* Define to 1 if you have _CrtSetReportMode in */ +/* #undef HAVE__CrtSetReportMode */ + +/* Define to 1 if you have the `wmemcmp' function. */ +#define HAVE_WMEMCMP 1 + +/* Define to 1 if you have the `wmemcpy' function. */ +#define HAVE_WMEMCPY 1 + +/* Define to 1 if you have the `wmemmove' function. */ +#define HAVE_WMEMMOVE 1 + +/* Define to 1 if you have a working EXT2_IOC_GETFLAGS */ +/* #undef HAVE_WORKING_EXT2_IOC_GETFLAGS */ + +/* Define to 1 if you have a working FS_IOC_GETFLAGS */ +#define HAVE_WORKING_FS_IOC_GETFLAGS 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_ZLIB_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_ZSTD_H */ + +/* Define to 1 if you have the `ctime_s' function. */ +/* #undef HAVE_CTIME_S */ + +/* Define to 1 if you have the `_fseeki64' function. */ +/* #undef HAVE__FSEEKI64 */ + +/* Define to 1 if you have the `_get_timezone' function. */ +/* #undef HAVE__GET_TIMEZONE */ + +/* Define to 1 if you have the `gmtime_s' function. */ +/* #undef HAVE_GMTIME_S */ + +/* Define to 1 if you have the `localtime_s' function. */ +/* #undef HAVE_LOCALTIME_S */ + +/* Define to 1 if you have the `_mkgmtime' function. */ +/* #undef HAVE__MKGMTIME */ + +/* Define as const if the declaration of iconv() needs const. */ +#define ICONV_CONST + +/* Version number of libarchive as a single integer */ +#define LIBARCHIVE_VERSION_NUMBER "3006003" + +/* Version number of libarchive */ +#define LIBARCHIVE_VERSION_STRING "3.6.3" + +/* Define to 1 if `lstat' dereferences a symlink specified with a trailing + slash. */ +/* #undef LSTAT_FOLLOWS_SLASHED_SYMLINK */ + +/* Define to 1 if `major', `minor', and `makedev' are declared in . + */ +/* #undef MAJOR_IN_MKDEV */ + +/* Define to 1 if `major', `minor', and `makedev' are declared in + . */ +#define MAJOR_IN_SYSMACROS 1 + +/* Define to 1 if your C compiler doesn't accept -c and -o together. */ +/* #undef NO_MINUS_C_MINUS_O */ + +/* The size of `wchar_t', as computed by sizeof. */ +#define SIZEOF_WCHAR_T 4 + +/* Define to 1 if strerror_r returns char *. */ +/* #undef STRERROR_R_CHAR_P */ + +/* Define to 1 if you can safely include both and . */ +/* #undef TIME_WITH_SYS_TIME */ + +/* + * Some platform requires a macro to use extension functions. + */ +#define SAFE_TO_DEFINE_EXTENSIONS 1 +#ifdef SAFE_TO_DEFINE_EXTENSIONS +/* Enable extensions on AIX 3, Interix. */ +#ifndef _ALL_SOURCE +# define _ALL_SOURCE 1 +#endif +/* Enable GNU extensions on systems that have them. */ +#ifndef _GNU_SOURCE +# define _GNU_SOURCE 1 +#endif +/* Enable threading extensions on Solaris. */ +#ifndef _POSIX_PTHREAD_SEMANTICS +# define _POSIX_PTHREAD_SEMANTICS 1 +#endif +/* Enable extensions on HP NonStop. */ +#ifndef _TANDEM_SOURCE +# define _TANDEM_SOURCE 1 +#endif +/* Enable general extensions on Solaris. */ +#ifndef __EXTENSIONS__ +# define __EXTENSIONS__ 1 +#endif +#endif /* SAFE_TO_DEFINE_EXTENSIONS */ + +/* Version number of package */ +#define VERSION "3.6.3" + +/* Number of bits in a file offset, on hosts where this is settable. */ +/* #undef _FILE_OFFSET_BITS */ + +/* Define to 1 to make fseeko visible on some hosts (e.g. glibc 2.2). */ +/* #undef _LARGEFILE_SOURCE */ + +/* Define for large files, on AIX-style hosts. */ +/* #undef _LARGE_FILES */ + +/* Define to control Windows SDK version */ +#ifndef NTDDI_VERSION +/* #undef NTDDI_VERSION */ +#endif // NTDDI_VERSION + +#ifndef _WIN32_WINNT +/* #undef _WIN32_WINNT */ +#endif // _WIN32_WINNT + +#ifndef WINVER +/* #undef WINVER */ +#endif // WINVER + +/* Define to empty if `const' does not conform to ANSI C. */ +/* #undef const */ + +/* Define to `int' if doesn't define. */ +/* #undef gid_t */ + +/* Define to `unsigned long' if does not define. */ +/* #undef id_t */ + +/* Define to `int' if does not define. */ +/* #undef mode_t */ + +/* Define to `long long' if does not define. */ +/* #undef off_t */ + +/* Define to `int' if doesn't define. */ +/* #undef pid_t */ + +/* Define to `unsigned int' if does not define. */ +/* #undef size_t */ + +/* Define to `int' if does not define. */ +/* #undef ssize_t */ + +/* Define to `int' if doesn't define. */ +/* #undef uid_t */ + +/* Define to `int' if does not define. */ +/* #undef intptr_t */ + +/* Define to `unsigned int' if does not define. */ +/* #undef uintptr_t */ diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index a21c9e422ab..2f2bc0f278e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -554,6 +554,8 @@ if (TARGET ch_contrib::bzip2) target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::bzip2) endif() +target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::libarchive) + if (TARGET ch_contrib::minizip) target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::minizip) endif () From 895e78dae39d968040d2e97e4ba4a28c70380eeb Mon Sep 17 00:00:00 2001 From: Nikita Keba Date: Mon, 29 May 2023 17:10:03 +0000 Subject: [PATCH 0101/2047] add SevenZipArchiveReader/TarArchiveReader --- src/IO/Archives/SevenZipArchiveReader.cpp | 172 ++++++++++++++++++++++ src/IO/Archives/SevenZipArchiveReader.h | 62 ++++++++ src/IO/Archives/TarArchiveReader.cpp | 165 +++++++++++++++++++++ src/IO/Archives/TarArchiveReader.h | 59 ++++++++ src/IO/Archives/createArchiveReader.cpp | 6 + 5 files changed, 464 insertions(+) create mode 100644 src/IO/Archives/SevenZipArchiveReader.cpp create mode 100644 src/IO/Archives/SevenZipArchiveReader.h create mode 100644 src/IO/Archives/TarArchiveReader.cpp create mode 100644 src/IO/Archives/TarArchiveReader.h diff --git a/src/IO/Archives/SevenZipArchiveReader.cpp b/src/IO/Archives/SevenZipArchiveReader.cpp new file mode 100644 index 00000000000..0a3fc716b3d --- /dev/null +++ b/src/IO/Archives/SevenZipArchiveReader.cpp @@ -0,0 +1,172 @@ +#include "SevenZipArchiveReader.h" +#include +#include + +#include + +namespace DB{ + namespace ErrorCodes + { + extern const int CANNOT_UNPACK_ARCHIVE; + extern const int LOGICAL_ERROR; + extern const int SEEK_POSITION_OUT_OF_BOUND; + } +class SevenZipArchiveReader::Handle { +public: + Handle(const String & path_to_archive_) + : path_to_archive(path_to_archive_) { + archive = archive_read_new(); + archive_read_support_filter_all(archive); + archive_read_support_format_all(archive); + if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) { + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open 7z archive {}", quoteString(path_to_archive)); + } + entry = archive_entry_new(); + std::cout << "NIKITAKEBA Succesfully created handle " << path_to_archive << std::endl; + } + ~Handle() { + archive_read_close(archive); + archive_read_free(archive); + } + + bool locateFile(const String &filename) { + std::cout << "NIKITAKEBA LOCATE" << filename << std::endl; + while (archive_read_next_header(archive, &entry) == ARCHIVE_OK) + { + std::cout << "NIKITAKEBA FILENAME " << archive_entry_pathname(entry) << std::endl; + if (archive_entry_pathname(entry) == filename) + return true; + } + return false; + } + +struct archive* archive; +struct archive_entry* entry; + +private: + const String path_to_archive; +}; + +class SevenZipArchiveReader::ReadBufferFromSevenZipArchive : public ReadBufferFromFileBase +{ +public: + explicit ReadBufferFromSevenZipArchive(const String & path_to_archive_, const String & filename_) + : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0), handle(path_to_archive_), path_to_archive(path_to_archive_), filename(filename_) + { + handle.locateFile(filename_); + } + + off_t seek(off_t off, int whence) override + { + off_t current_pos = getPosition(); + off_t new_pos; + if (whence == SEEK_SET) + new_pos = off; + else if (whence == SEEK_CUR) + new_pos = off + current_pos; + else + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Only SEEK_SET and SEEK_CUR seek modes allowed."); + + if (new_pos == current_pos) + return current_pos; /// The position is the same. + + if (new_pos < 0) + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); + + off_t working_buffer_start_pos = current_pos - offset(); + off_t working_buffer_end_pos = current_pos + available(); + + if ((working_buffer_start_pos <= new_pos) && (new_pos <= working_buffer_end_pos)) + { + /// The new position is still inside the buffer. + position() += new_pos - current_pos; + return new_pos; + } + + /// Check that the new position is now beyond the end of the file. + if (new_pos > archive_entry_size(handle.entry)) + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); + + ignore(new_pos - current_pos); + return new_pos; + } + + off_t getPosition() override + { + return archive_entry_size(handle.entry) - available(); + } + + String getFileName() const override { return filename; } + + +private: + bool nextImpl() override + { + auto bytes_read = archive_read_data(handle.archive, internal_buffer.begin(), static_cast(internal_buffer.size())); + + if (!bytes_read) + return false; + + working_buffer = internal_buffer; + working_buffer.resize(bytes_read); + return true; + } + Handle handle; + const String path_to_archive; + const String filename; +}; + + SevenZipArchiveReader::SevenZipArchiveReader(const String & path_to_archive_) + : path_to_archive(path_to_archive_) { + } + + SevenZipArchiveReader::SevenZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_): path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), archive_size(archive_size_) { + std::cout << "NIKITAKEBA CREATE 7z\n";} + + SevenZipArchiveReader::~SevenZipArchiveReader() {} + bool SevenZipArchiveReader::fileExists(const String& filename) + { + check_file_in_7z(path_to_archive.c_str(), filename.c_str()); + Handle handle(path_to_archive); + return handle.locateFile(filename); + } + + SevenZipArchiveReader::FileInfo SevenZipArchiveReader::getFileInfo(const String & filename) { + + Handle handle(path_to_archive); + + handle.locateFile(filename); + FileInfo info; + info.uncompressed_size = archive_entry_size(handle.entry); + info.compressed_size = archive_entry_size(handle.entry); + info.is_encrypted = false; + + return info; + } + + std::unique_ptr SevenZipArchiveReader::firstFile() { + return nullptr; + } + + std::unique_ptr SevenZipArchiveReader::readFile(const String & filename) { + + Handle handle(path_to_archive); + handle.locateFile(filename); + + return std::make_unique(path_to_archive, filename); + } + + std::unique_ptr SevenZipArchiveReader::readFile([[maybe_unused]] std::unique_ptr enumerator) { + return nullptr; + } + + std::unique_ptr SevenZipArchiveReader::nextFile([[maybe_unused]] std::unique_ptr read_buffer) { + return nullptr; + } + + + void SevenZipArchiveReader::setPassword([[maybe_unused]] const String & password_) { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to .7z archive"); + } + +} diff --git a/src/IO/Archives/SevenZipArchiveReader.h b/src/IO/Archives/SevenZipArchiveReader.h new file mode 100644 index 00000000000..196d640f70b --- /dev/null +++ b/src/IO/Archives/SevenZipArchiveReader.h @@ -0,0 +1,62 @@ +#pragma once + +#include + +#include +#include +#include + + +namespace DB +{ +class ReadBuffer; +class ReadBufferFromFileBase; +class SeekableReadBuffer; + +/// Implementation of IArchiveReader for reading SevenZip archives. +class SevenZipArchiveReader : public IArchiveReader +{ +public: + + /// Constructs an archive's reader that will read from a file in the local filesystem. + explicit SevenZipArchiveReader(const String & path_to_archive_); + + /// Constructs an archive's reader that will read by making a read buffer by using + /// a specified function. + SevenZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_); + + ~SevenZipArchiveReader() override; + + /// Returns true if there is a specified file in the archive. + bool fileExists(const String & filename) override; + + /// Returns the information about a file stored in the archive. + FileInfo getFileInfo(const String & filename) override; + + /// Starts enumerating files in the archive. + std::unique_ptr firstFile() override; + + /// Starts reading a file from the archive. The function returns a read buffer, + /// you can read that buffer to extract uncompressed data from the archive. + /// Several read buffers can be used at the same time in parallel. + std::unique_ptr readFile(const String & filename) override; + + /// It's possible to convert a file enumerator to a read buffer and vice versa. + [[maybe_unused]] std::unique_ptr readFile(std::unique_ptr enumerator) override; + [[maybe_unused]] std::unique_ptr nextFile(std::unique_ptr read_buffer) override; + + /// Sets password used to decrypt the contents of the files in the archive. + void setPassword([[maybe_unused]] const String & password_) override; + +private: + + class ReadBufferFromSevenZipArchive; + class Handle; + + const String path_to_archive; + String password; + const ReadArchiveFunction archive_read_function; + [[maybe_unused]] const UInt64 archive_size = 0; +}; + +} diff --git a/src/IO/Archives/TarArchiveReader.cpp b/src/IO/Archives/TarArchiveReader.cpp new file mode 100644 index 00000000000..aade03f4b87 --- /dev/null +++ b/src/IO/Archives/TarArchiveReader.cpp @@ -0,0 +1,165 @@ +#include "TarArchiveReader.h" +#include +#include + + +namespace DB{ + namespace ErrorCodes + { + extern const int CANNOT_UNPACK_ARCHIVE; + extern const int LOGICAL_ERROR; + extern const int SEEK_POSITION_OUT_OF_BOUND; + } +class TarArchiveReader::Handle { +public: + Handle(const String & path_to_archive_) + : path_to_archive(path_to_archive_) { + archive = archive_read_new(); + archive_read_support_filter_all(archive); + archive_read_support_format_all(archive); + if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) { + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open tar archive {}", quoteString(path_to_archive)); + } + entry = archive_entry_new(); + } + ~Handle() { + archive_read_close(archive); + archive_read_free(archive); + } + + bool locateFile(const String &filename) { + while (archive_read_next_header(archive, &entry) == ARCHIVE_OK) + { + if (archive_entry_pathname(entry) == filename) + return true; + } + return false; + } + +struct archive* archive; +struct archive_entry* entry; + +private: + const String path_to_archive; +}; + +class TarArchiveReader::ReadBufferFromTarArchive : public ReadBufferFromFileBase +{ +public: + explicit ReadBufferFromTarArchive(const String & path_to_archive_, const String & filename_) + : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0), handle(path_to_archive_), path_to_archive(path_to_archive_), filename(filename_) + { + handle.locateFile(filename_); + } + + off_t seek(off_t off, int whence) override + { + off_t current_pos = getPosition(); + off_t new_pos; + if (whence == SEEK_SET) + new_pos = off; + else if (whence == SEEK_CUR) + new_pos = off + current_pos; + else + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Only SEEK_SET and SEEK_CUR seek modes allowed."); + + if (new_pos == current_pos) + return current_pos; /// The position is the same. + + if (new_pos < 0) + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); + + off_t working_buffer_start_pos = current_pos - offset(); + off_t working_buffer_end_pos = current_pos + available(); + + if ((working_buffer_start_pos <= new_pos) && (new_pos <= working_buffer_end_pos)) + { + /// The new position is still inside the buffer. + position() += new_pos - current_pos; + return new_pos; + } + + /// Check that the new position is now beyond the end of the file. + if (new_pos > archive_entry_size(handle.entry)) + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); + + ignore(new_pos - current_pos); + return new_pos; + } + + off_t getPosition() override + { + return archive_entry_size(handle.entry) - available(); + } + + String getFileName() const override { return filename; } + + +private: + bool nextImpl() override + { + auto bytes_read = archive_read_data(handle.archive, internal_buffer.begin(), static_cast(internal_buffer.size())); + + if (!bytes_read) + return false; + + working_buffer = internal_buffer; + working_buffer.resize(bytes_read); + return true; + } + Handle handle; + const String path_to_archive; + const String filename; +}; + + TarArchiveReader::TarArchiveReader(const String & path_to_archive_) + : path_to_archive(path_to_archive_) {} + + TarArchiveReader::TarArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_): path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), archive_size(archive_size_) {} + + TarArchiveReader::~TarArchiveReader() {} + bool TarArchiveReader::fileExists(const String& filename) + { + Handle handle(path_to_archive); + return handle.locateFile(filename); + } + + TarArchiveReader::FileInfo TarArchiveReader::getFileInfo(const String & filename) { + + Handle handle(path_to_archive); + + handle.locateFile(filename); + FileInfo info; + info.uncompressed_size = archive_entry_size(handle.entry); + info.compressed_size = archive_entry_size(handle.entry); + info.is_encrypted = false; + + return info; + } + + std::unique_ptr TarArchiveReader::firstFile() { + return nullptr; + } + + std::unique_ptr TarArchiveReader::readFile(const String & filename) { + + Handle handle(path_to_archive); + handle.locateFile(filename); + + return std::make_unique(path_to_archive, filename); + } + + std::unique_ptr TarArchiveReader::readFile([[maybe_unused]] std::unique_ptr enumerator) { + return nullptr; + } + + std::unique_ptr TarArchiveReader::nextFile([[maybe_unused]] std::unique_ptr read_buffer) { + return nullptr; + } + + + void TarArchiveReader::setPassword([[maybe_unused]] const String & password_) { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to .tar archive"); + } + +} diff --git a/src/IO/Archives/TarArchiveReader.h b/src/IO/Archives/TarArchiveReader.h new file mode 100644 index 00000000000..ff4217678fe --- /dev/null +++ b/src/IO/Archives/TarArchiveReader.h @@ -0,0 +1,59 @@ +#pragma once + +#include + +#include +#include + + +namespace DB +{ +class ReadBuffer; +class ReadBufferFromFileBase; +class SeekableReadBuffer; + +/// Implementation of IArchiveReader for reading tar archives. +class TarArchiveReader : public IArchiveReader +{ +public: + /// Constructs an archive's reader that will read from a file in the local filesystem. + explicit TarArchiveReader(const String & path_to_archive_); + + /// Constructs an archive's reader that will read by making a read buffer by using + /// a specified function. + TarArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_); + + ~TarArchiveReader() override; + + /// Returns true if there is a specified file in the archive. + bool fileExists(const String & filename) override; + + /// Returns the information about a file stored in the archive. + FileInfo getFileInfo(const String & filename) override; + + /// Starts enumerating files in the archive. + std::unique_ptr firstFile() override; + + /// Starts reading a file from the archive. The function returns a read buffer, + /// you can read that buffer to extract uncompressed data from the archive. + /// Several read buffers can be used at the same time in parallel. + std::unique_ptr readFile(const String & filename) override; + + /// It's possible to convert a file enumerator to a read buffer and vice versa. + [[maybe_unused]] std::unique_ptr readFile(std::unique_ptr enumerator) override; + [[maybe_unused]] std::unique_ptr nextFile(std::unique_ptr read_buffer) override; + + /// Sets password used to decrypt the contents of the files in the archive. + void setPassword([[maybe_unused]] const String & password_) override; + +private: + + class ReadBufferFromTarArchive; + class Handle; + + const String path_to_archive; + const ReadArchiveFunction archive_read_function; + [[maybe_unused]] const UInt64 archive_size = 0; +}; + +} diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index 3cb4802792b..df6b0d15ce4 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include #include @@ -30,6 +32,10 @@ std::shared_ptr createArchiveReader( #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "minizip library is disabled"); #endif + } else if (path_to_archive.ends_with(".tar")) { + return std::make_shared(path_to_archive, archive_read_function, archive_size); + } else if (path_to_archive.ends_with(".7z")) { + return std::make_shared(path_to_archive, archive_read_function, archive_size); } else throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Cannot determine the type of archive {}", path_to_archive); From 19a0fbeccc1107c42c27929bd90328cc80625391 Mon Sep 17 00:00:00 2001 From: Nikita Keba Date: Mon, 29 May 2023 17:10:03 +0000 Subject: [PATCH 0102/2047] add SevenZipArchiveReader/TarArchiveReader --- src/IO/Archives/SevenZipArchiveReader.cpp | 166 ++++++++++++++++++++++ src/IO/Archives/SevenZipArchiveReader.h | 62 ++++++++ src/IO/Archives/TarArchiveReader.cpp | 165 +++++++++++++++++++++ src/IO/Archives/TarArchiveReader.h | 59 ++++++++ src/IO/Archives/createArchiveReader.cpp | 6 + 5 files changed, 458 insertions(+) create mode 100644 src/IO/Archives/SevenZipArchiveReader.cpp create mode 100644 src/IO/Archives/SevenZipArchiveReader.h create mode 100644 src/IO/Archives/TarArchiveReader.cpp create mode 100644 src/IO/Archives/TarArchiveReader.h diff --git a/src/IO/Archives/SevenZipArchiveReader.cpp b/src/IO/Archives/SevenZipArchiveReader.cpp new file mode 100644 index 00000000000..05cb8d8396e --- /dev/null +++ b/src/IO/Archives/SevenZipArchiveReader.cpp @@ -0,0 +1,166 @@ +#include "SevenZipArchiveReader.h" +#include +#include + + +namespace DB{ + namespace ErrorCodes + { + extern const int CANNOT_UNPACK_ARCHIVE; + extern const int LOGICAL_ERROR; + extern const int SEEK_POSITION_OUT_OF_BOUND; + } +class SevenZipArchiveReader::Handle { +public: + Handle(const String & path_to_archive_) + : path_to_archive(path_to_archive_) { + archive = archive_read_new(); + archive_read_support_filter_all(archive); + archive_read_support_format_all(archive); + if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) { + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open 7z archive {}", quoteString(path_to_archive)); + } + entry = archive_entry_new(); + } + ~Handle() { + archive_read_close(archive); + archive_read_free(archive); + } + + bool locateFile(const String &filename) { + while (archive_read_next_header(archive, &entry) == ARCHIVE_OK) + { + if (archive_entry_pathname(entry) == filename) + return true; + } + return false; + } + +struct archive* archive; +struct archive_entry* entry; + +private: + const String path_to_archive; +}; + +class SevenZipArchiveReader::ReadBufferFromSevenZipArchive : public ReadBufferFromFileBase +{ +public: + explicit ReadBufferFromSevenZipArchive(const String & path_to_archive_, const String & filename_) + : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0), handle(path_to_archive_), path_to_archive(path_to_archive_), filename(filename_) + { + handle.locateFile(filename_); + } + + off_t seek(off_t off, int whence) override + { + off_t current_pos = getPosition(); + off_t new_pos; + if (whence == SEEK_SET) + new_pos = off; + else if (whence == SEEK_CUR) + new_pos = off + current_pos; + else + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Only SEEK_SET and SEEK_CUR seek modes allowed."); + + if (new_pos == current_pos) + return current_pos; /// The position is the same. + + if (new_pos < 0) + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); + + off_t working_buffer_start_pos = current_pos - offset(); + off_t working_buffer_end_pos = current_pos + available(); + + if ((working_buffer_start_pos <= new_pos) && (new_pos <= working_buffer_end_pos)) + { + /// The new position is still inside the buffer. + position() += new_pos - current_pos; + return new_pos; + } + + /// Check that the new position is now beyond the end of the file. + if (new_pos > archive_entry_size(handle.entry)) + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); + + ignore(new_pos - current_pos); + return new_pos; + } + + off_t getPosition() override + { + return archive_entry_size(handle.entry) - available(); + } + + String getFileName() const override { return filename; } + + +private: + bool nextImpl() override + { + auto bytes_read = archive_read_data(handle.archive, internal_buffer.begin(), static_cast(internal_buffer.size())); + + if (!bytes_read) + return false; + + working_buffer = internal_buffer; + working_buffer.resize(bytes_read); + return true; + } + Handle handle; + const String path_to_archive; + const String filename; +}; + + SevenZipArchiveReader::SevenZipArchiveReader(const String & path_to_archive_) + : path_to_archive(path_to_archive_) { + } + + SevenZipArchiveReader::SevenZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_): path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), archive_size(archive_size_) {} + + SevenZipArchiveReader::~SevenZipArchiveReader() {} + bool SevenZipArchiveReader::fileExists(const String& filename) + { + Handle handle(path_to_archive); + return handle.locateFile(filename); + } + + SevenZipArchiveReader::FileInfo SevenZipArchiveReader::getFileInfo(const String & filename) { + + Handle handle(path_to_archive); + + handle.locateFile(filename); + FileInfo info; + info.uncompressed_size = archive_entry_size(handle.entry); + info.compressed_size = archive_entry_size(handle.entry); + info.is_encrypted = false; + + return info; + } + + std::unique_ptr SevenZipArchiveReader::firstFile() { + return nullptr; + } + + std::unique_ptr SevenZipArchiveReader::readFile(const String & filename) { + + Handle handle(path_to_archive); + handle.locateFile(filename); + + return std::make_unique(path_to_archive, filename); + } + + std::unique_ptr SevenZipArchiveReader::readFile([[maybe_unused]] std::unique_ptr enumerator) { + return nullptr; + } + + std::unique_ptr SevenZipArchiveReader::nextFile([[maybe_unused]] std::unique_ptr read_buffer) { + return nullptr; + } + + + void SevenZipArchiveReader::setPassword([[maybe_unused]] const String & password_) { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to .7z archive"); + } + +} diff --git a/src/IO/Archives/SevenZipArchiveReader.h b/src/IO/Archives/SevenZipArchiveReader.h new file mode 100644 index 00000000000..196d640f70b --- /dev/null +++ b/src/IO/Archives/SevenZipArchiveReader.h @@ -0,0 +1,62 @@ +#pragma once + +#include + +#include +#include +#include + + +namespace DB +{ +class ReadBuffer; +class ReadBufferFromFileBase; +class SeekableReadBuffer; + +/// Implementation of IArchiveReader for reading SevenZip archives. +class SevenZipArchiveReader : public IArchiveReader +{ +public: + + /// Constructs an archive's reader that will read from a file in the local filesystem. + explicit SevenZipArchiveReader(const String & path_to_archive_); + + /// Constructs an archive's reader that will read by making a read buffer by using + /// a specified function. + SevenZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_); + + ~SevenZipArchiveReader() override; + + /// Returns true if there is a specified file in the archive. + bool fileExists(const String & filename) override; + + /// Returns the information about a file stored in the archive. + FileInfo getFileInfo(const String & filename) override; + + /// Starts enumerating files in the archive. + std::unique_ptr firstFile() override; + + /// Starts reading a file from the archive. The function returns a read buffer, + /// you can read that buffer to extract uncompressed data from the archive. + /// Several read buffers can be used at the same time in parallel. + std::unique_ptr readFile(const String & filename) override; + + /// It's possible to convert a file enumerator to a read buffer and vice versa. + [[maybe_unused]] std::unique_ptr readFile(std::unique_ptr enumerator) override; + [[maybe_unused]] std::unique_ptr nextFile(std::unique_ptr read_buffer) override; + + /// Sets password used to decrypt the contents of the files in the archive. + void setPassword([[maybe_unused]] const String & password_) override; + +private: + + class ReadBufferFromSevenZipArchive; + class Handle; + + const String path_to_archive; + String password; + const ReadArchiveFunction archive_read_function; + [[maybe_unused]] const UInt64 archive_size = 0; +}; + +} diff --git a/src/IO/Archives/TarArchiveReader.cpp b/src/IO/Archives/TarArchiveReader.cpp new file mode 100644 index 00000000000..aade03f4b87 --- /dev/null +++ b/src/IO/Archives/TarArchiveReader.cpp @@ -0,0 +1,165 @@ +#include "TarArchiveReader.h" +#include +#include + + +namespace DB{ + namespace ErrorCodes + { + extern const int CANNOT_UNPACK_ARCHIVE; + extern const int LOGICAL_ERROR; + extern const int SEEK_POSITION_OUT_OF_BOUND; + } +class TarArchiveReader::Handle { +public: + Handle(const String & path_to_archive_) + : path_to_archive(path_to_archive_) { + archive = archive_read_new(); + archive_read_support_filter_all(archive); + archive_read_support_format_all(archive); + if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) { + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open tar archive {}", quoteString(path_to_archive)); + } + entry = archive_entry_new(); + } + ~Handle() { + archive_read_close(archive); + archive_read_free(archive); + } + + bool locateFile(const String &filename) { + while (archive_read_next_header(archive, &entry) == ARCHIVE_OK) + { + if (archive_entry_pathname(entry) == filename) + return true; + } + return false; + } + +struct archive* archive; +struct archive_entry* entry; + +private: + const String path_to_archive; +}; + +class TarArchiveReader::ReadBufferFromTarArchive : public ReadBufferFromFileBase +{ +public: + explicit ReadBufferFromTarArchive(const String & path_to_archive_, const String & filename_) + : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0), handle(path_to_archive_), path_to_archive(path_to_archive_), filename(filename_) + { + handle.locateFile(filename_); + } + + off_t seek(off_t off, int whence) override + { + off_t current_pos = getPosition(); + off_t new_pos; + if (whence == SEEK_SET) + new_pos = off; + else if (whence == SEEK_CUR) + new_pos = off + current_pos; + else + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Only SEEK_SET and SEEK_CUR seek modes allowed."); + + if (new_pos == current_pos) + return current_pos; /// The position is the same. + + if (new_pos < 0) + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); + + off_t working_buffer_start_pos = current_pos - offset(); + off_t working_buffer_end_pos = current_pos + available(); + + if ((working_buffer_start_pos <= new_pos) && (new_pos <= working_buffer_end_pos)) + { + /// The new position is still inside the buffer. + position() += new_pos - current_pos; + return new_pos; + } + + /// Check that the new position is now beyond the end of the file. + if (new_pos > archive_entry_size(handle.entry)) + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); + + ignore(new_pos - current_pos); + return new_pos; + } + + off_t getPosition() override + { + return archive_entry_size(handle.entry) - available(); + } + + String getFileName() const override { return filename; } + + +private: + bool nextImpl() override + { + auto bytes_read = archive_read_data(handle.archive, internal_buffer.begin(), static_cast(internal_buffer.size())); + + if (!bytes_read) + return false; + + working_buffer = internal_buffer; + working_buffer.resize(bytes_read); + return true; + } + Handle handle; + const String path_to_archive; + const String filename; +}; + + TarArchiveReader::TarArchiveReader(const String & path_to_archive_) + : path_to_archive(path_to_archive_) {} + + TarArchiveReader::TarArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_): path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), archive_size(archive_size_) {} + + TarArchiveReader::~TarArchiveReader() {} + bool TarArchiveReader::fileExists(const String& filename) + { + Handle handle(path_to_archive); + return handle.locateFile(filename); + } + + TarArchiveReader::FileInfo TarArchiveReader::getFileInfo(const String & filename) { + + Handle handle(path_to_archive); + + handle.locateFile(filename); + FileInfo info; + info.uncompressed_size = archive_entry_size(handle.entry); + info.compressed_size = archive_entry_size(handle.entry); + info.is_encrypted = false; + + return info; + } + + std::unique_ptr TarArchiveReader::firstFile() { + return nullptr; + } + + std::unique_ptr TarArchiveReader::readFile(const String & filename) { + + Handle handle(path_to_archive); + handle.locateFile(filename); + + return std::make_unique(path_to_archive, filename); + } + + std::unique_ptr TarArchiveReader::readFile([[maybe_unused]] std::unique_ptr enumerator) { + return nullptr; + } + + std::unique_ptr TarArchiveReader::nextFile([[maybe_unused]] std::unique_ptr read_buffer) { + return nullptr; + } + + + void TarArchiveReader::setPassword([[maybe_unused]] const String & password_) { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to .tar archive"); + } + +} diff --git a/src/IO/Archives/TarArchiveReader.h b/src/IO/Archives/TarArchiveReader.h new file mode 100644 index 00000000000..ff4217678fe --- /dev/null +++ b/src/IO/Archives/TarArchiveReader.h @@ -0,0 +1,59 @@ +#pragma once + +#include + +#include +#include + + +namespace DB +{ +class ReadBuffer; +class ReadBufferFromFileBase; +class SeekableReadBuffer; + +/// Implementation of IArchiveReader for reading tar archives. +class TarArchiveReader : public IArchiveReader +{ +public: + /// Constructs an archive's reader that will read from a file in the local filesystem. + explicit TarArchiveReader(const String & path_to_archive_); + + /// Constructs an archive's reader that will read by making a read buffer by using + /// a specified function. + TarArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_); + + ~TarArchiveReader() override; + + /// Returns true if there is a specified file in the archive. + bool fileExists(const String & filename) override; + + /// Returns the information about a file stored in the archive. + FileInfo getFileInfo(const String & filename) override; + + /// Starts enumerating files in the archive. + std::unique_ptr firstFile() override; + + /// Starts reading a file from the archive. The function returns a read buffer, + /// you can read that buffer to extract uncompressed data from the archive. + /// Several read buffers can be used at the same time in parallel. + std::unique_ptr readFile(const String & filename) override; + + /// It's possible to convert a file enumerator to a read buffer and vice versa. + [[maybe_unused]] std::unique_ptr readFile(std::unique_ptr enumerator) override; + [[maybe_unused]] std::unique_ptr nextFile(std::unique_ptr read_buffer) override; + + /// Sets password used to decrypt the contents of the files in the archive. + void setPassword([[maybe_unused]] const String & password_) override; + +private: + + class ReadBufferFromTarArchive; + class Handle; + + const String path_to_archive; + const ReadArchiveFunction archive_read_function; + [[maybe_unused]] const UInt64 archive_size = 0; +}; + +} diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index 3cb4802792b..df6b0d15ce4 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include #include @@ -30,6 +32,10 @@ std::shared_ptr createArchiveReader( #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "minizip library is disabled"); #endif + } else if (path_to_archive.ends_with(".tar")) { + return std::make_shared(path_to_archive, archive_read_function, archive_size); + } else if (path_to_archive.ends_with(".7z")) { + return std::make_shared(path_to_archive, archive_read_function, archive_size); } else throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Cannot determine the type of archive {}", path_to_archive); From 1556010166a383262c884f3b4fb4232a2b3d7f50 Mon Sep 17 00:00:00 2001 From: Nikita Keba Date: Mon, 29 May 2023 17:36:34 +0000 Subject: [PATCH 0103/2047] add doc for reading from archives --- docs/en/sql-reference/table-functions/file.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index d2ef66dde73..ae917fd6d32 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -13,12 +13,14 @@ Creates a table from a file. This table function is similar to [url](../../sql-r **Syntax** ``` sql -file(path [,format] [,structure]) +file([path_to_archive ::] path [,format] [,structure]) ``` **Parameters** - `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. +- `path_to_archive` - The relative path to zip/tar/7z archive. Path to archive support the same globs. + - `format` — The [format](../../interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format: `'column1_name column1_type, column2_name column2_type, ...'`. @@ -73,6 +75,11 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U └─────────┴─────────┴─────────┘ ``` +Getting data from table in table.csv, located in archive1.zip or(and) archive2.zip +``` sql +SELECT * FROM file('user_files/archives/archive{1..2}.zip :: table.csv'); +``` + ## Globs in Path Multiple path components can have globs. For being processed file must exist and match to the whole path pattern (not only suffix or prefix). From 0bb5af1381679ebe3edd7a76b9be8c3dd2d82f7d Mon Sep 17 00:00:00 2001 From: Nikita Keba Date: Mon, 29 May 2023 18:20:03 +0000 Subject: [PATCH 0104/2047] add func test select_from_table_in_archive --- ...02661_select_from_table_in_archive.reference | 7 +++++++ .../02661_select_from_table_in_archive.sh | 17 +++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/02661_select_from_table_in_archive.reference create mode 100755 tests/queries/0_stateless/02661_select_from_table_in_archive.sh diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.reference b/tests/queries/0_stateless/02661_select_from_table_in_archive.reference new file mode 100644 index 00000000000..9869a226367 --- /dev/null +++ b/tests/queries/0_stateless/02661_select_from_table_in_archive.reference @@ -0,0 +1,7 @@ +1 2 +3 4 +1 +3 +1 +3 + diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh new file mode 100755 index 00000000000..f72bba719a5 --- /dev/null +++ b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo -e "1,2\n3,4" > 02661_data.csv +zip archive1.zip 02661_data.csv > /dev/null +zip archive2.zip 02661_data.csv > /dev/null + +$CLICKHOUSE_LOCAL --query "SELECT * FROM file('archive1.zip :: 02661_data.csv')" +$CLICKHOUSE_LOCAL --query "SELECT c1 FROM file('archive{1..2}.zip :: 02661_data.csv')" + +rm 02661_data.csv +rm archive1.zip +rm archive2.zip From 636d50caa0a2d4ac35d528f75e2ac51a99a4288a Mon Sep 17 00:00:00 2001 From: Nikita Keba Date: Mon, 29 May 2023 19:35:24 +0000 Subject: [PATCH 0105/2047] fix cmake + add unit tests for TarArchiveReader --- contrib/libarchive-cmake/CMakeLists.txt | 2 +- contrib/libarchive-cmake/config.h | 2 +- src/IO/Archives/util/tar_archive_writer.h | 27 ++++++++++ .../tests/gtest_archive_reader_and_writer.cpp | 50 ++++++++++++++++++- 4 files changed, 78 insertions(+), 3 deletions(-) create mode 100644 src/IO/Archives/util/tar_archive_writer.h diff --git a/contrib/libarchive-cmake/CMakeLists.txt b/contrib/libarchive-cmake/CMakeLists.txt index 0bf53e737db..4593f7f96c8 100644 --- a/contrib/libarchive-cmake/CMakeLists.txt +++ b/contrib/libarchive-cmake/CMakeLists.txt @@ -127,7 +127,7 @@ set(SRCS "${LIBRARY_DIR}/libarchive/archive_write_set_format_ustar.c" "${LIBRARY_DIR}/libarchive/archive_write_set_format_v7tar.c" "${LIBRARY_DIR}/libarchive/archive_write_set_format_warc.c" - # "${LIBRARY_DIR}/libarchive/archive_write_set_format_xar.c" + "${LIBRARY_DIR}/libarchive/archive_write_set_format_xar.c" "${LIBRARY_DIR}/libarchive/archive_write_set_format_zip.c" "${LIBRARY_DIR}/libarchive/archive_write_set_options.c" "${LIBRARY_DIR}/libarchive/archive_write_set_passphrase.c" diff --git a/contrib/libarchive-cmake/config.h b/contrib/libarchive-cmake/config.h index 0b54bf63ddd..2fa1bb8945d 100644 --- a/contrib/libarchive-cmake/config.h +++ b/contrib/libarchive-cmake/config.h @@ -730,7 +730,7 @@ typedef uint64_t uintmax_t; #define HAVE_LIBXML_XMLREADER_H 0 /* Define to 1 if you have the header file. */ -#define HAVE_LIBXML_XMLWRITER_H 1 +// #define HAVE_LIBXML_XMLWRITER_H 1 /* Define to 1 if you have the `z' library (-lz). */ /* #undef HAVE_LIBZ */ diff --git a/src/IO/Archives/util/tar_archive_writer.h b/src/IO/Archives/util/tar_archive_writer.h new file mode 100644 index 00000000000..138bcb036a6 --- /dev/null +++ b/src/IO/Archives/util/tar_archive_writer.h @@ -0,0 +1,27 @@ +#include +#include + +bool create_tar_with_file(const std::string &archivename, std::map files) { + struct archive *a; + struct archive_entry *entry; + + a = archive_write_new(); + archive_write_set_format_pax_restricted(a); + archive_write_open_filename(a, archivename.c_str()); + + for (auto &[filename, content] : files) { + entry = archive_entry_new(); + archive_entry_set_pathname(entry, filename.c_str()); + archive_entry_set_size(entry, content.size()); + archive_entry_set_mode(entry, S_IFREG | 0644); // regular file with rw-r--r-- permissions + archive_entry_set_mtime(entry, time(nullptr), 0); + archive_write_header(a, entry); + archive_write_data(a, content.c_str(), content.size()); + archive_entry_free(entry); + } + + archive_write_close(a); + archive_write_free(a); + + return true; +} diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index 3bc9d670f05..186fdb89532 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -327,6 +328,53 @@ TEST_P(ArchiveReaderAndWriterTest, ArchiveNotExist) [&]{ createArchiveReader(getPathToArchive()); }); } +TEST(TarArchiveReaderTest, FileExists) { + String archive_path = "archive.tar"; + String filename = "file.txt"; + String contents = "test"; + bool created = create_tar_with_file(archive_path, {{filename, contents}}); + EXPECT_EQ(created, true); + auto reader = createArchiveReader(archive_path); + EXPECT_EQ(reader->fileExists(filename), true); + fs::remove(archive_path); +} + +TEST(TarArchiveReaderTest, ReadFile) { + String archive_path = "archive.tar"; + String filename = "file.txt"; + String contents = "test"; + bool created = create_tar_with_file(archive_path, {{filename, contents}}); + EXPECT_EQ(created, true); + auto reader = createArchiveReader(archive_path); + auto in = reader->readFile(filename); + String str; + readStringUntilEOF(str, *in); + EXPECT_EQ(str, contents); + fs::remove(archive_path); +} + +TEST(TarArchiveReaderTest, ReadTwoFiles) { + String archive_path = "archive.tar"; + String file1 = "file1.txt"; + String contents1 = "test1"; + String file2 = "file2.txt"; + String contents2 = "test2"; + bool created = create_tar_with_file(archive_path, {{file1, contents1}, {file2, contents2}}); + EXPECT_EQ(created, true); + auto reader = createArchiveReader(archive_path); + EXPECT_EQ(reader->fileExists(file1), true); + EXPECT_EQ(reader->fileExists(file2), true); + auto in = reader->readFile(file1); + String str; + readStringUntilEOF(str, *in); + EXPECT_EQ(str, contents1); + in = reader->readFile(file2); + + readStringUntilEOF(str, *in); + EXPECT_EQ(str, contents2); + fs::remove(archive_path); +} + #if USE_MINIZIP @@ -334,7 +382,7 @@ namespace { const char * supported_archive_file_exts[] = { - ".zip", + ".zip" }; } From 8cf79cdb6c69396a224307d8dda73897755c1965 Mon Sep 17 00:00:00 2001 From: Nikita Keba Date: Mon, 29 May 2023 19:55:46 +0000 Subject: [PATCH 0106/2047] add SevenZipArchiveReader unit tests --- ...tar_archive_writer.h => archive_writers.h} | 25 ++++++++++ .../tests/gtest_archive_reader_and_writer.cpp | 49 ++++++++++++++++++- 2 files changed, 73 insertions(+), 1 deletion(-) rename src/IO/Archives/util/{tar_archive_writer.h => archive_writers.h} (51%) diff --git a/src/IO/Archives/util/tar_archive_writer.h b/src/IO/Archives/util/archive_writers.h similarity index 51% rename from src/IO/Archives/util/tar_archive_writer.h rename to src/IO/Archives/util/archive_writers.h index 138bcb036a6..a340565756f 100644 --- a/src/IO/Archives/util/tar_archive_writer.h +++ b/src/IO/Archives/util/archive_writers.h @@ -25,3 +25,28 @@ bool create_tar_with_file(const std::string &archivename, std::map files) { + struct archive *a; + struct archive_entry *entry; + + a = archive_write_new(); + archive_write_set_format_7zip(a); + archive_write_open_filename(a, archivename.c_str()); + + for (auto &[filename, content] : files) { + entry = archive_entry_new(); + archive_entry_set_pathname(entry, filename.c_str()); + archive_entry_set_size(entry, content.size()); + archive_entry_set_mode(entry, S_IFREG | 0644); // regular file with rw-r--r-- permissions + archive_entry_set_mtime(entry, time(nullptr), 0); + archive_write_header(a, entry); + archive_write_data(a, content.c_str(), content.size()); + archive_entry_free(entry); + } + + archive_write_close(a); + archive_write_free(a); + + return true; +} diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index 186fdb89532..4f3ea8eba30 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include @@ -375,6 +375,53 @@ TEST(TarArchiveReaderTest, ReadTwoFiles) { fs::remove(archive_path); } +TEST(SevenZipArchiveReaderTest, FileExists) { + String archive_path = "archive.7z"; + String filename = "file.txt"; + String contents = "test"; + bool created = create_7z_with_file(archive_path, {{filename, contents}}); + EXPECT_EQ(created, true); + auto reader = createArchiveReader(archive_path); + EXPECT_EQ(reader->fileExists(filename), true); + fs::remove(archive_path); +} + +TEST(SevenZipArchiveReaderTest, ReadFile) { + String archive_path = "archive.7z"; + String filename = "file.txt"; + String contents = "test"; + bool created = create_7z_with_file(archive_path, {{filename, contents}}); + EXPECT_EQ(created, true); + auto reader = createArchiveReader(archive_path); + auto in = reader->readFile(filename); + String str; + readStringUntilEOF(str, *in); + EXPECT_EQ(str, contents); + fs::remove(archive_path); +} + +TEST(SevenZipArchiveReaderTest, ReadTwoFiles) { + String archive_path = "archive.7z"; + String file1 = "file1.txt"; + String contents1 = "test1"; + String file2 = "file2.txt"; + String contents2 = "test2"; + bool created = create_7z_with_file(archive_path, {{file1, contents1}, {file2, contents2}}); + EXPECT_EQ(created, true); + auto reader = createArchiveReader(archive_path); + EXPECT_EQ(reader->fileExists(file1), true); + EXPECT_EQ(reader->fileExists(file2), true); + auto in = reader->readFile(file1); + String str; + readStringUntilEOF(str, *in); + EXPECT_EQ(str, contents1); + in = reader->readFile(file2); + + readStringUntilEOF(str, *in); + EXPECT_EQ(str, contents2); + fs::remove(archive_path); +} + #if USE_MINIZIP From c18bff58b3694590eabbee5b369093c1410f8ef9 Mon Sep 17 00:00:00 2001 From: Nikita Keba Date: Mon, 29 May 2023 20:08:18 +0000 Subject: [PATCH 0107/2047] fix style --- contrib/libarchive | 2 +- src/IO/Archives/SevenZipArchiveReader.cpp | 134 +++--- src/IO/Archives/SevenZipArchiveReader.h | 4 +- src/IO/Archives/TarArchiveReader.cpp | 132 +++--- src/IO/Archives/TarArchiveReader.h | 1 - src/Storages/StorageFile.cpp | 552 +++++++++++----------- src/Storages/StorageFile.h | 16 +- 7 files changed, 445 insertions(+), 396 deletions(-) diff --git a/contrib/libarchive b/contrib/libarchive index 1f3c62ebf4d..30a8610f4d0 160000 --- a/contrib/libarchive +++ b/contrib/libarchive @@ -1 +1 @@ -Subproject commit 1f3c62ebf4d492ac21d3099b3b064993100dd997 +Subproject commit 30a8610f4d05141d85bb9b123cdec16906a02c59 diff --git a/src/IO/Archives/SevenZipArchiveReader.cpp b/src/IO/Archives/SevenZipArchiveReader.cpp index 05cb8d8396e..dc3daa4cccc 100644 --- a/src/IO/Archives/SevenZipArchiveReader.cpp +++ b/src/IO/Archives/SevenZipArchiveReader.cpp @@ -3,31 +3,36 @@ #include -namespace DB{ - namespace ErrorCodes - { - extern const int CANNOT_UNPACK_ARCHIVE; - extern const int LOGICAL_ERROR; - extern const int SEEK_POSITION_OUT_OF_BOUND; - } -class SevenZipArchiveReader::Handle { +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_UNPACK_ARCHIVE; + extern const int LOGICAL_ERROR; + extern const int SEEK_POSITION_OUT_OF_BOUND; +} +class SevenZipArchiveReader::Handle +{ public: - Handle(const String & path_to_archive_) - : path_to_archive(path_to_archive_) { + Handle(const String & path_to_archive_) : path_to_archive(path_to_archive_) + { archive = archive_read_new(); archive_read_support_filter_all(archive); archive_read_support_format_all(archive); - if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) { + if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) + { throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open 7z archive {}", quoteString(path_to_archive)); } entry = archive_entry_new(); } - ~Handle() { + ~Handle() + { archive_read_close(archive); archive_read_free(archive); } - bool locateFile(const String &filename) { + bool locateFile(const String & filename) + { while (archive_read_next_header(archive, &entry) == ARCHIVE_OK) { if (archive_entry_pathname(entry) == filename) @@ -36,8 +41,8 @@ public: return false; } -struct archive* archive; -struct archive_entry* entry; + struct archive * archive; + struct archive_entry * entry; private: const String path_to_archive; @@ -47,7 +52,10 @@ class SevenZipArchiveReader::ReadBufferFromSevenZipArchive : public ReadBufferFr { public: explicit ReadBufferFromSevenZipArchive(const String & path_to_archive_, const String & filename_) - : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0), handle(path_to_archive_), path_to_archive(path_to_archive_), filename(filename_) + : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0) + , handle(path_to_archive_) + , path_to_archive(path_to_archive_) + , filename(filename_) { handle.locateFile(filename_); } @@ -87,10 +95,7 @@ public: return new_pos; } - off_t getPosition() override - { - return archive_entry_size(handle.entry) - available(); - } + off_t getPosition() override { return archive_entry_size(handle.entry) - available(); } String getFileName() const override { return filename; } @@ -112,55 +117,66 @@ private: const String filename; }; - SevenZipArchiveReader::SevenZipArchiveReader(const String & path_to_archive_) - : path_to_archive(path_to_archive_) { - } +SevenZipArchiveReader::SevenZipArchiveReader(const String & path_to_archive_) : path_to_archive(path_to_archive_) +{ +} - SevenZipArchiveReader::SevenZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_): path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), archive_size(archive_size_) {} +SevenZipArchiveReader::SevenZipArchiveReader( + const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_) + : path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), archive_size(archive_size_) +{ +} - SevenZipArchiveReader::~SevenZipArchiveReader() {} - bool SevenZipArchiveReader::fileExists(const String& filename) - { - Handle handle(path_to_archive); - return handle.locateFile(filename); - } +SevenZipArchiveReader::~SevenZipArchiveReader() +{ +} +bool SevenZipArchiveReader::fileExists(const String & filename) +{ + Handle handle(path_to_archive); + return handle.locateFile(filename); +} - SevenZipArchiveReader::FileInfo SevenZipArchiveReader::getFileInfo(const String & filename) { - - Handle handle(path_to_archive); - - handle.locateFile(filename); - FileInfo info; - info.uncompressed_size = archive_entry_size(handle.entry); - info.compressed_size = archive_entry_size(handle.entry); - info.is_encrypted = false; +SevenZipArchiveReader::FileInfo SevenZipArchiveReader::getFileInfo(const String & filename) +{ + Handle handle(path_to_archive); - return info; - } + handle.locateFile(filename); + FileInfo info; + info.uncompressed_size = archive_entry_size(handle.entry); + info.compressed_size = archive_entry_size(handle.entry); + info.is_encrypted = false; - std::unique_ptr SevenZipArchiveReader::firstFile() { - return nullptr; - } + return info; +} - std::unique_ptr SevenZipArchiveReader::readFile(const String & filename) { - - Handle handle(path_to_archive); - handle.locateFile(filename); +std::unique_ptr SevenZipArchiveReader::firstFile() +{ + return nullptr; +} - return std::make_unique(path_to_archive, filename); - } +std::unique_ptr SevenZipArchiveReader::readFile(const String & filename) +{ + Handle handle(path_to_archive); + handle.locateFile(filename); - std::unique_ptr SevenZipArchiveReader::readFile([[maybe_unused]] std::unique_ptr enumerator) { - return nullptr; - } + return std::make_unique(path_to_archive, filename); +} - std::unique_ptr SevenZipArchiveReader::nextFile([[maybe_unused]] std::unique_ptr read_buffer) { - return nullptr; - } +std::unique_ptr SevenZipArchiveReader::readFile([[maybe_unused]] std::unique_ptr enumerator) +{ + return nullptr; +} + +std::unique_ptr +SevenZipArchiveReader::nextFile([[maybe_unused]] std::unique_ptr read_buffer) +{ + return nullptr; +} - void SevenZipArchiveReader::setPassword([[maybe_unused]] const String & password_) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to .7z archive"); - } +void SevenZipArchiveReader::setPassword([[maybe_unused]] const String & password_) +{ + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to .7z archive"); +} } diff --git a/src/IO/Archives/SevenZipArchiveReader.h b/src/IO/Archives/SevenZipArchiveReader.h index 196d640f70b..62ea4daff9e 100644 --- a/src/IO/Archives/SevenZipArchiveReader.h +++ b/src/IO/Archives/SevenZipArchiveReader.h @@ -2,9 +2,9 @@ #include +#include #include #include -#include namespace DB @@ -17,7 +17,6 @@ class SeekableReadBuffer; class SevenZipArchiveReader : public IArchiveReader { public: - /// Constructs an archive's reader that will read from a file in the local filesystem. explicit SevenZipArchiveReader(const String & path_to_archive_); @@ -49,7 +48,6 @@ public: void setPassword([[maybe_unused]] const String & password_) override; private: - class ReadBufferFromSevenZipArchive; class Handle; diff --git a/src/IO/Archives/TarArchiveReader.cpp b/src/IO/Archives/TarArchiveReader.cpp index aade03f4b87..b47b90b04aa 100644 --- a/src/IO/Archives/TarArchiveReader.cpp +++ b/src/IO/Archives/TarArchiveReader.cpp @@ -3,31 +3,36 @@ #include -namespace DB{ - namespace ErrorCodes - { - extern const int CANNOT_UNPACK_ARCHIVE; - extern const int LOGICAL_ERROR; - extern const int SEEK_POSITION_OUT_OF_BOUND; - } -class TarArchiveReader::Handle { +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_UNPACK_ARCHIVE; + extern const int LOGICAL_ERROR; + extern const int SEEK_POSITION_OUT_OF_BOUND; +} +class TarArchiveReader::Handle +{ public: - Handle(const String & path_to_archive_) - : path_to_archive(path_to_archive_) { + Handle(const String & path_to_archive_) : path_to_archive(path_to_archive_) + { archive = archive_read_new(); archive_read_support_filter_all(archive); archive_read_support_format_all(archive); - if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) { + if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) + { throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open tar archive {}", quoteString(path_to_archive)); } entry = archive_entry_new(); } - ~Handle() { + ~Handle() + { archive_read_close(archive); archive_read_free(archive); } - bool locateFile(const String &filename) { + bool locateFile(const String & filename) + { while (archive_read_next_header(archive, &entry) == ARCHIVE_OK) { if (archive_entry_pathname(entry) == filename) @@ -36,8 +41,8 @@ public: return false; } -struct archive* archive; -struct archive_entry* entry; + struct archive * archive; + struct archive_entry * entry; private: const String path_to_archive; @@ -47,7 +52,10 @@ class TarArchiveReader::ReadBufferFromTarArchive : public ReadBufferFromFileBase { public: explicit ReadBufferFromTarArchive(const String & path_to_archive_, const String & filename_) - : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0), handle(path_to_archive_), path_to_archive(path_to_archive_), filename(filename_) + : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0) + , handle(path_to_archive_) + , path_to_archive(path_to_archive_) + , filename(filename_) { handle.locateFile(filename_); } @@ -87,10 +95,7 @@ public: return new_pos; } - off_t getPosition() override - { - return archive_entry_size(handle.entry) - available(); - } + off_t getPosition() override { return archive_entry_size(handle.entry) - available(); } String getFileName() const override { return filename; } @@ -112,54 +117,65 @@ private: const String filename; }; - TarArchiveReader::TarArchiveReader(const String & path_to_archive_) - : path_to_archive(path_to_archive_) {} +TarArchiveReader::TarArchiveReader(const String & path_to_archive_) : path_to_archive(path_to_archive_) +{ +} - TarArchiveReader::TarArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_): path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), archive_size(archive_size_) {} +TarArchiveReader::TarArchiveReader( + const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_) + : path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), archive_size(archive_size_) +{ +} - TarArchiveReader::~TarArchiveReader() {} - bool TarArchiveReader::fileExists(const String& filename) - { - Handle handle(path_to_archive); - return handle.locateFile(filename); - } +TarArchiveReader::~TarArchiveReader() +{ +} +bool TarArchiveReader::fileExists(const String & filename) +{ + Handle handle(path_to_archive); + return handle.locateFile(filename); +} - TarArchiveReader::FileInfo TarArchiveReader::getFileInfo(const String & filename) { - - Handle handle(path_to_archive); - - handle.locateFile(filename); - FileInfo info; - info.uncompressed_size = archive_entry_size(handle.entry); - info.compressed_size = archive_entry_size(handle.entry); - info.is_encrypted = false; +TarArchiveReader::FileInfo TarArchiveReader::getFileInfo(const String & filename) +{ + Handle handle(path_to_archive); - return info; - } + handle.locateFile(filename); + FileInfo info; + info.uncompressed_size = archive_entry_size(handle.entry); + info.compressed_size = archive_entry_size(handle.entry); + info.is_encrypted = false; - std::unique_ptr TarArchiveReader::firstFile() { - return nullptr; - } + return info; +} - std::unique_ptr TarArchiveReader::readFile(const String & filename) { - - Handle handle(path_to_archive); - handle.locateFile(filename); +std::unique_ptr TarArchiveReader::firstFile() +{ + return nullptr; +} - return std::make_unique(path_to_archive, filename); - } +std::unique_ptr TarArchiveReader::readFile(const String & filename) +{ + Handle handle(path_to_archive); + handle.locateFile(filename); - std::unique_ptr TarArchiveReader::readFile([[maybe_unused]] std::unique_ptr enumerator) { - return nullptr; - } + return std::make_unique(path_to_archive, filename); +} - std::unique_ptr TarArchiveReader::nextFile([[maybe_unused]] std::unique_ptr read_buffer) { - return nullptr; - } +std::unique_ptr TarArchiveReader::readFile([[maybe_unused]] std::unique_ptr enumerator) +{ + return nullptr; +} + +std::unique_ptr TarArchiveReader::nextFile([[maybe_unused]] std::unique_ptr read_buffer) +{ + return nullptr; +} - void TarArchiveReader::setPassword([[maybe_unused]] const String & password_) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to .tar archive"); - } +void TarArchiveReader::setPassword([[maybe_unused]] const String & password_) +{ + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to .tar archive"); +} } diff --git a/src/IO/Archives/TarArchiveReader.h b/src/IO/Archives/TarArchiveReader.h index ff4217678fe..644ae806d75 100644 --- a/src/IO/Archives/TarArchiveReader.h +++ b/src/IO/Archives/TarArchiveReader.h @@ -47,7 +47,6 @@ public: void setPassword([[maybe_unused]] const String & password_) override; private: - class ReadBufferFromTarArchive; class Handle; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 96c6724a3d9..51fab74f446 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1,11 +1,11 @@ -#include -#include #include -#include -#include #include -#include +#include #include +#include +#include +#include +#include #include #include @@ -28,37 +28,37 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include -#include -#include -#include -#include #include +#include +#include +#include +#include #include #include -#include +#include +#include #include #include #include -#include -#include +#include namespace ProfileEvents { - extern const Event CreatedReadBufferOrdinary; - extern const Event CreatedReadBufferMMap; - extern const Event CreatedReadBufferMMapFailed; +extern const Event CreatedReadBufferOrdinary; +extern const Event CreatedReadBufferMMap; +extern const Event CreatedReadBufferMMapFailed; } namespace fs = std::filesystem; @@ -89,213 +89,211 @@ namespace ErrorCodes namespace { -/* Recursive directory listing with matched paths as a result. + /* Recursive directory listing with matched paths as a result. * Have the same method in StorageHDFS. */ -void listFilesWithRegexpMatchingImpl( - const std::string & path_for_ls, - const std::string & for_match, - size_t & total_bytes_to_read, - std::vector & result, - bool recursive = false) -{ - const size_t first_glob = for_match.find_first_of("*?{"); - - const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); - const std::string suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - - const size_t next_slash = suffix_with_globs.find('/', 1); - const std::string current_glob = suffix_with_globs.substr(0, next_slash); - auto regexp = makeRegexpPatternFromGlobs(current_glob); - - re2::RE2 matcher(regexp); - if (!matcher.ok()) - throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from glob ({}): {}", for_match, matcher.error()); - - bool skip_regex = current_glob == "/*" ? true : false; - if (!recursive) - recursive = current_glob == "/**" ; - - const std::string prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); - - if (!fs::exists(prefix_without_globs)) - return; - - const fs::directory_iterator end; - for (fs::directory_iterator it(prefix_without_globs); it != end; ++it) + void listFilesWithRegexpMatchingImpl( + const std::string & path_for_ls, + const std::string & for_match, + size_t & total_bytes_to_read, + std::vector & result, + bool recursive = false) { - const std::string full_path = it->path().string(); - const size_t last_slash = full_path.rfind('/'); - const String file_name = full_path.substr(last_slash); - const bool looking_for_directory = next_slash != std::string::npos; + const size_t first_glob = for_match.find_first_of("*?{"); - /// Condition is_directory means what kind of path is it in current iteration of ls - if (!it->is_directory() && !looking_for_directory) + const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + const std::string suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' + + const size_t next_slash = suffix_with_globs.find('/', 1); + const std::string current_glob = suffix_with_globs.substr(0, next_slash); + auto regexp = makeRegexpPatternFromGlobs(current_glob); + + re2::RE2 matcher(regexp); + if (!matcher.ok()) + throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", for_match, matcher.error()); + + bool skip_regex = current_glob == "/*" ? true : false; + if (!recursive) + recursive = current_glob == "/**"; + + const std::string prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); + + if (!fs::exists(prefix_without_globs)) + return; + + const fs::directory_iterator end; + for (fs::directory_iterator it(prefix_without_globs); it != end; ++it) { - if (skip_regex || re2::RE2::FullMatch(file_name, matcher)) + const std::string full_path = it->path().string(); + const size_t last_slash = full_path.rfind('/'); + const String file_name = full_path.substr(last_slash); + const bool looking_for_directory = next_slash != std::string::npos; + + /// Condition is_directory means what kind of path is it in current iteration of ls + if (!it->is_directory() && !looking_for_directory) { - total_bytes_to_read += it->file_size(); - result.push_back(it->path().string()); + if (skip_regex || re2::RE2::FullMatch(file_name, matcher)) + { + total_bytes_to_read += it->file_size(); + result.push_back(it->path().string()); + } } - } - else if (it->is_directory()) - { - if (recursive) + else if (it->is_directory()) { - listFilesWithRegexpMatchingImpl(fs::path(full_path).append(it->path().string()) / "" , - looking_for_directory ? suffix_with_globs.substr(next_slash) : current_glob , - total_bytes_to_read, result, recursive); - } - else if (looking_for_directory && re2::RE2::FullMatch(file_name, matcher)) - { - /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. - listFilesWithRegexpMatchingImpl(fs::path(full_path) / "", suffix_with_globs.substr(next_slash), total_bytes_to_read, result); + if (recursive) + { + listFilesWithRegexpMatchingImpl( + fs::path(full_path).append(it->path().string()) / "", + looking_for_directory ? suffix_with_globs.substr(next_slash) : current_glob, + total_bytes_to_read, + result, + recursive); + } + else if (looking_for_directory && re2::RE2::FullMatch(file_name, matcher)) + { + /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. + listFilesWithRegexpMatchingImpl( + fs::path(full_path) / "", suffix_with_globs.substr(next_slash), total_bytes_to_read, result); + } } } } -} -std::vector listFilesWithRegexpMatching( - const std::string & path_for_ls, - const std::string & for_match, - size_t & total_bytes_to_read) -{ - std::vector result; - listFilesWithRegexpMatchingImpl(path_for_ls, for_match, total_bytes_to_read, result); - return result; -} - -std::string getTablePath(const std::string & table_dir_path, const std::string & format_name) -{ - return table_dir_path + "/data." + escapeForFileName(format_name); -} - -/// Both db_dir_path and table_path must be converted to absolute paths (in particular, path cannot contain '..'). -void checkCreationIsAllowed( - ContextPtr context_global, - const std::string & db_dir_path, - const std::string & table_path, - bool can_be_directory) -{ - if (context_global->getApplicationType() != Context::ApplicationType::SERVER) - return; - - /// "/dev/null" is allowed for perf testing - if (!fileOrSymlinkPathStartsWith(table_path, db_dir_path) && table_path != "/dev/null") - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "File `{}` is not inside `{}`", table_path, db_dir_path); - - if (can_be_directory) + std::vector + listFilesWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match, size_t & total_bytes_to_read) { - auto table_path_stat = fs::status(table_path); - if (fs::exists(table_path_stat) && fs::is_directory(table_path_stat)) - throw Exception(ErrorCodes::INCORRECT_FILE_NAME, "File must not be a directory"); + std::vector result; + listFilesWithRegexpMatchingImpl(path_for_ls, for_match, total_bytes_to_read, result); + return result; } -} -std::unique_ptr selectReadBuffer( - const String & current_path, - bool use_table_fd, - int table_fd, - const struct stat & file_stat, - ContextPtr context) -{ - auto read_method = context->getSettingsRef().storage_file_read_method; - - if (S_ISREG(file_stat.st_mode) && read_method == LocalFSReadMethod::mmap) + std::string getTablePath(const std::string & table_dir_path, const std::string & format_name) { - try + return table_dir_path + "/data." + escapeForFileName(format_name); + } + + /// Both db_dir_path and table_path must be converted to absolute paths (in particular, path cannot contain '..'). + void checkCreationIsAllowed( + ContextPtr context_global, const std::string & db_dir_path, const std::string & table_path, bool can_be_directory) + { + if (context_global->getApplicationType() != Context::ApplicationType::SERVER) + return; + + /// "/dev/null" is allowed for perf testing + if (!fileOrSymlinkPathStartsWith(table_path, db_dir_path) && table_path != "/dev/null") + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "File `{}` is not inside `{}`", table_path, db_dir_path); + + if (can_be_directory) + { + auto table_path_stat = fs::status(table_path); + if (fs::exists(table_path_stat) && fs::is_directory(table_path_stat)) + throw Exception(ErrorCodes::INCORRECT_FILE_NAME, "File must not be a directory"); + } + } + + std::unique_ptr + selectReadBuffer(const String & current_path, bool use_table_fd, int table_fd, const struct stat & file_stat, ContextPtr context) + { + auto read_method = context->getSettingsRef().storage_file_read_method; + + if (S_ISREG(file_stat.st_mode) && read_method == LocalFSReadMethod::mmap) + { + try + { + std::unique_ptr res; + if (use_table_fd) + res = std::make_unique(table_fd, 0); + else + res = std::make_unique(current_path, 0); + + ProfileEvents::increment(ProfileEvents::CreatedReadBufferMMap); + return res; + } + catch (const ErrnoException &) + { + /// Fallback if mmap is not supported. + ProfileEvents::increment(ProfileEvents::CreatedReadBufferMMapFailed); + } + } + + std::unique_ptr res; + if (S_ISREG(file_stat.st_mode) && (read_method == LocalFSReadMethod::pread || read_method == LocalFSReadMethod::mmap)) { - std::unique_ptr res; if (use_table_fd) - res = std::make_unique(table_fd, 0); + res = std::make_unique(table_fd); else - res = std::make_unique(current_path, 0); + res = std::make_unique(current_path, context->getSettingsRef().max_read_buffer_size); - ProfileEvents::increment(ProfileEvents::CreatedReadBufferMMap); - return res; + ProfileEvents::increment(ProfileEvents::CreatedReadBufferOrdinary); } - catch (const ErrnoException &) + else { - /// Fallback if mmap is not supported. - ProfileEvents::increment(ProfileEvents::CreatedReadBufferMMapFailed); + if (use_table_fd) + res = std::make_unique(table_fd); + else + res = std::make_unique(current_path, context->getSettingsRef().max_read_buffer_size); + + ProfileEvents::increment(ProfileEvents::CreatedReadBufferOrdinary); } + return res; } - std::unique_ptr res; - if (S_ISREG(file_stat.st_mode) && (read_method == LocalFSReadMethod::pread || read_method == LocalFSReadMethod::mmap)) + std::unique_ptr createReadBuffer( + const String & current_path, + bool use_table_fd, + const String & storage_name, + int table_fd, + const String & compression_method, + ContextPtr context, + const String & path_to_archive = "auto") { + CompressionMethod method; + + struct stat file_stat + { + }; + if (path_to_archive != "auto") + { + auto reader = createArchiveReader(path_to_archive); + std::unique_ptr in = reader->readFile(current_path); + return in; + } + if (use_table_fd) - res = std::make_unique(table_fd); + { + /// Check if file descriptor allows random reads (and reading it twice). + if (0 != fstat(table_fd, &file_stat)) + throwFromErrno("Cannot stat table file descriptor, inside " + storage_name, ErrorCodes::CANNOT_STAT); + + method = chooseCompressionMethod("", compression_method); + } else - res = std::make_unique(current_path, context->getSettingsRef().max_read_buffer_size); + { + /// Check if file descriptor allows random reads (and reading it twice). + if (0 != stat(current_path.c_str(), &file_stat)) + throwFromErrno("Cannot stat file " + current_path, ErrorCodes::CANNOT_STAT); - ProfileEvents::increment(ProfileEvents::CreatedReadBufferOrdinary); + method = chooseCompressionMethod(current_path, compression_method); + } + + std::unique_ptr nested_buffer = selectReadBuffer(current_path, use_table_fd, table_fd, file_stat, context); + + /// For clickhouse-local and clickhouse-client add progress callback to display progress bar. + if (context->getApplicationType() == Context::ApplicationType::LOCAL + || context->getApplicationType() == Context::ApplicationType::CLIENT) + { + auto & in = static_cast(*nested_buffer); + in.setProgressCallback(context); + } + + int zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); + return wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method, zstd_window_log_max); } - else - { - if (use_table_fd) - res = std::make_unique(table_fd); - else - res = std::make_unique(current_path, context->getSettingsRef().max_read_buffer_size); - - ProfileEvents::increment(ProfileEvents::CreatedReadBufferOrdinary); - } - return res; -} - -std::unique_ptr createReadBuffer( - const String & current_path, - bool use_table_fd, - const String & storage_name, - int table_fd, - const String & compression_method, - ContextPtr context, - const String & path_to_archive = "auto") -{ - CompressionMethod method; - - struct stat file_stat{}; - if (path_to_archive != "auto") { - auto reader = createArchiveReader(path_to_archive); - std::unique_ptr in = reader->readFile(current_path); - return in; - } - - if (use_table_fd) - { - /// Check if file descriptor allows random reads (and reading it twice). - if (0 != fstat(table_fd, &file_stat)) - throwFromErrno("Cannot stat table file descriptor, inside " + storage_name, ErrorCodes::CANNOT_STAT); - - method = chooseCompressionMethod("", compression_method); - } - else - { - /// Check if file descriptor allows random reads (and reading it twice). - if (0 != stat(current_path.c_str(), &file_stat)) - throwFromErrno("Cannot stat file " + current_path, ErrorCodes::CANNOT_STAT); - - method = chooseCompressionMethod(current_path, compression_method); - } - - std::unique_ptr nested_buffer = selectReadBuffer(current_path, use_table_fd, table_fd, file_stat, context); - - /// For clickhouse-local and clickhouse-client add progress callback to display progress bar. - if (context->getApplicationType() == Context::ApplicationType::LOCAL - || context->getApplicationType() == Context::ApplicationType::CLIENT) - { - auto & in = static_cast(*nested_buffer); - in.setProgressCallback(context); - } - - int zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); - return wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method, zstd_window_log_max); -} } -Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read) +Strings +StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read) { fs::path user_files_absolute_path = fs::weakly_canonical(user_files_path); fs::path fs_table_path(table_path); @@ -385,14 +383,16 @@ ColumnsDescription StorageFile::getTableStructureFromFile( throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file, because there are no files with provided path. " - "You must specify table structure manually", format); + "You must specify table structure manually", + format); std::optional columns_from_cache; if (context->getSettingsRef().schema_inference_use_cache_for_file) columns_from_cache = tryGetColumnsFromCache(paths, format, format_settings, context); ReadBufferIterator read_buffer_iterator; - if (paths_to_archive.empty()) { + if (paths_to_archive.empty()) + { read_buffer_iterator = [&, it = paths.begin()](ColumnsDescription &) mutable -> std::unique_ptr { if (it == paths.end()) @@ -400,7 +400,9 @@ ColumnsDescription StorageFile::getTableStructureFromFile( return createReadBuffer(*it++, false, "File", -1, compression_method, context); }; - } else { + } + else + { read_buffer_iterator = [&, it = paths_to_archive.begin()](ColumnsDescription &) mutable -> std::unique_ptr { if (it == paths_to_archive.end()) @@ -426,8 +428,7 @@ bool StorageFile::supportsSubsetOfColumns() const return format_name != "Distributed" && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name); } -StorageFile::StorageFile(int table_fd_, CommonArguments args) - : StorageFile(args) +StorageFile::StorageFile(int table_fd_, CommonArguments args) : StorageFile(args) { struct stat buf; int res = fstat(table_fd_, &buf); @@ -446,13 +447,15 @@ StorageFile::StorageFile(int table_fd_, CommonArguments args) setStorageMetadata(args); } -StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args) - : StorageFile(args) +StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args) : StorageFile(args) { - if (args.path_to_archive != "auto") { + if (args.path_to_archive != "auto") + { paths_to_archive = getPathsList(args.path_to_archive, user_files_path, args.getContext(), total_bytes_to_read); paths = {table_path_}; - } else { + } + else + { paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read); } is_db_table = false; @@ -465,8 +468,7 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us setStorageMetadata(args); } -StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArguments args) - : StorageFile(args) +StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArguments args) : StorageFile(args) { if (relative_table_dir_path.empty()) throw Exception(ErrorCodes::INCORRECT_FILE_NAME, "Storage {} requires data path", getName()); @@ -507,7 +509,8 @@ void StorageFile::setStorageMetadata(CommonArguments args) columns = getTableStructureFromFileDescriptor(args.getContext()); else { - columns = getTableStructureFromFile(format_name, paths, compression_method, format_settings, args.getContext(), paths_to_archive); + columns + = getTableStructureFromFile(format_name, paths, compression_method, format_settings, args.getContext(), paths_to_archive); if (!args.columns.empty() && args.columns != columns) throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Table structure and file structure are different"); } @@ -600,10 +603,7 @@ public: } } - String getName() const override - { - return storage->getName(); - } + String getName() const override { return storage->getName(); } Chunk generate() override { @@ -615,13 +615,16 @@ public: if (!storage->use_table_fd) { size_t current_file = 0, current_archive = 0; - if (files_info->files.size() == 1 && !files_info->paths_to_archive.empty()) { + if (files_info->files.size() == 1 && !files_info->paths_to_archive.empty()) + { current_archive = files_info->next_archive_to_read.fetch_add(1); if (current_archive >= files_info->paths_to_archive.size()) return {}; current_path = files_info->files[current_file]; current_archive_path = files_info->paths_to_archive[current_archive]; - } else { + } + else + { current_file = files_info->next_file_to_read.fetch_add(1); if (current_file >= files_info->files.size()) return {}; @@ -636,11 +639,28 @@ public: } } - if (!read_buf) { - if (files_info->paths_to_archive.empty()) { - read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context); - } else { - read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context, current_archive_path); + if (!read_buf) + { + if (files_info->paths_to_archive.empty()) + { + read_buf = createReadBuffer( + current_path, + storage->use_table_fd, + storage->getName(), + storage->table_fd, + storage->compression_method, + context); + } + else + { + read_buf = createReadBuffer( + current_path, + storage->use_table_fd, + storage->getName(), + storage->table_fd, + storage->compression_method, + context, + current_archive_path); } } auto format @@ -651,10 +671,9 @@ public: if (columns_description.hasDefaults()) { - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, columns_description, *format, context); - }); + builder.addSimpleTransform( + [&](const Block & header) + { return std::make_shared(header, columns_description, *format, context); }); } pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); @@ -679,14 +698,20 @@ public: size_t last_slash_pos = current_path.find_last_of('/'); auto file_name = current_path.substr(last_slash_pos + 1); - auto column = DataTypeLowCardinality{std::make_shared()}.createColumnConst(num_rows, std::move(file_name)); + auto column + = DataTypeLowCardinality{std::make_shared()}.createColumnConst(num_rows, std::move(file_name)); chunk.addColumn(column->convertToFullColumnIfConst()); } if (num_rows) { updateRowsProgressApprox( - *this, chunk, files_info->total_bytes_to_read, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + *this, + chunk, + files_info->total_bytes_to_read, + total_rows_approx_accumulated, + total_rows_count_times, + total_rows_approx_max); } return chunk; } @@ -719,7 +744,7 @@ private: ColumnsDescription columns_description; Block block_for_format; - ContextPtr context; /// TODO Untangle potential issues with context lifetime. + ContextPtr context; /// TODO Untangle potential issues with context lifetime. UInt64 max_block_size; bool finished_generate = false; @@ -743,7 +768,7 @@ Pipe StorageFile::read( { if (use_table_fd) { - paths = {""}; /// when use fd, paths are empty + paths = {""}; /// when use fd, paths are empty } else { @@ -793,14 +818,14 @@ Pipe StorageFile::read( const auto & virtuals = getVirtuals(); std::erase_if( fetch_columns, - [&](const String & col) - { + [&](const String & col) { return std::any_of( virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); }); if (fetch_columns.empty()) - fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); + fetch_columns.push_back( + ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); } else @@ -914,8 +939,8 @@ public: write_buf = wrapWriteBufferWithCompressionMethod(std::move(naked_buffer), compression_method, 3); - writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format_name, - *write_buf, metadata_snapshot->getSampleBlock(), context, format_settings); + writer = FormatFactory::instance().getOutputFormatParallelIfPossible( + format_name, *write_buf, metadata_snapshot->getSampleBlock(), context, format_settings); if (do_not_write_prefix) writer->doNotWritePrefix(); @@ -1025,12 +1050,12 @@ public: { auto partition_path = PartitionedSink::replaceWildcards(path, partition_id); PartitionedSink::validatePartitionKey(partition_path, true); - checkCreationIsAllowed(context, context->getUserFilesPath(), partition_path, /*can_be_directory=*/ true); + checkCreationIsAllowed(context, context->getUserFilesPath(), partition_path, /*can_be_directory=*/true); return std::make_shared( metadata_snapshot, table_name_for_log, -1, - /* use_table_fd */false, + /* use_table_fd */ false, base_path, partition_path, compression_method, @@ -1056,10 +1081,7 @@ private: }; -SinkToStoragePtr StorageFile::write( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr context) +SinkToStoragePtr StorageFile::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) { if (format_name == "Distributed") throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is not implemented for Distributed format"); @@ -1099,9 +1121,10 @@ SinkToStoragePtr StorageFile::write( if (!paths.empty()) { if (is_path_with_globs) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "Table '{}' is in readonly mode because of globs in filepath", - getStorageID().getNameForLogs()); + throw Exception( + ErrorCodes::DATABASE_ACCESS_DENIED, + "Table '{}' is in readonly mode because of globs in filepath", + getStorageID().getNameForLogs()); path = paths.back(); fs::create_directories(fs::path(path).parent_path()); @@ -1118,10 +1141,10 @@ SinkToStoragePtr StorageFile::write( String new_path; do { - new_path = paths[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : paths[0].substr(pos)); + new_path = paths[0].substr(0, pos) + "." + std::to_string(index) + + (pos == std::string::npos ? "" : paths[0].substr(pos)); ++index; - } - while (fs::exists(new_path)); + } while (fs::exists(new_path)); paths.push_back(new_path); path = new_path; } @@ -1166,8 +1189,10 @@ Strings StorageFile::getDataPaths() const void StorageFile::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { if (!is_db_table) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "Can't rename table {} bounded to user-defined file (or FD)", getStorageID().getNameForLogs()); + throw Exception( + ErrorCodes::DATABASE_ACCESS_DENIED, + "Can't rename table {} bounded to user-defined file (or FD)", + getStorageID().getNameForLogs()); if (paths.size() != 1) throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "Can't rename table {} in readonly mode", getStorageID().getNameForLogs()); @@ -1184,10 +1209,7 @@ void StorageFile::rename(const String & new_path_to_table_data, const StorageID } void StorageFile::truncate( - const ASTPtr & /*query*/, - const StorageMetadataPtr & /* metadata_snapshot */, - ContextPtr /* context */, - TableExclusiveLockHolder &) + const ASTPtr & /*query*/, const StorageMetadataPtr & /* metadata_snapshot */, ContextPtr /* context */, TableExclusiveLockHolder &) { if (is_path_with_globs) throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "Can't truncate table '{}' in readonly mode", getStorageID().getNameForLogs()); @@ -1223,8 +1245,7 @@ void registerStorageFile(StorageFactory & factory) "File", [](const StorageFactory::Arguments & factory_args) { - StorageFile::CommonArguments storage_args - { + StorageFile::CommonArguments storage_args{ WithContext(factory_args.getContext()), factory_args.table_id, {}, @@ -1238,9 +1259,10 @@ void registerStorageFile(StorageFactory & factory) ASTs & engine_args_ast = factory_args.engine_args; if (!(engine_args_ast.size() >= 1 && engine_args_ast.size() <= 3)) // NOLINT - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage File requires from 1 to 3 arguments: " - "name of used format, source and compression_method."); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage File requires from 1 to 3 arguments: " + "name of used format, source and compression_method."); engine_args_ast[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args_ast[0], factory_args.getLocalContext()); storage_args.format_name = checkAndGetLiteralArgument(engine_args_ast[0], "format_name"); @@ -1264,16 +1286,13 @@ void registerStorageFile(StorageFactory & factory) } // Apply changes from SETTINGS clause, with validation. - user_format_settings.applyChanges( - factory_args.storage_def->settings->changes); + user_format_settings.applyChanges(factory_args.storage_def->settings->changes); - storage_args.format_settings = getFormatSettings( - factory_args.getContext(), user_format_settings); + storage_args.format_settings = getFormatSettings(factory_args.getContext(), user_format_settings); } else { - storage_args.format_settings = getFormatSettings( - factory_args.getContext()); + storage_args.format_settings = getFormatSettings(factory_args.getContext()); } if (engine_args_ast.size() == 1) /// Table in database @@ -1292,8 +1311,8 @@ void registerStorageFile(StorageFactory & factory) else if (*opt_name == "stderr") source_fd = STDERR_FILENO; else - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier '{}' in second arg of File storage constructor", - *opt_name); + throw Exception( + ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier '{}' in second arg of File storage constructor", *opt_name); } else if (const auto * literal = engine_args_ast[1]->as()) { @@ -1334,7 +1353,8 @@ NamesAndTypesList StorageFile::getVirtuals() const SchemaCache & StorageFile::getSchemaCache(const ContextPtr & context) { - static SchemaCache schema_cache(context->getConfigRef().getUInt("schema_inference_cache_max_elements_for_file", DEFAULT_SCHEMA_CACHE_ELEMENTS)); + static SchemaCache schema_cache( + context->getConfigRef().getUInt("schema_inference_cache_max_elements_for_file", DEFAULT_SCHEMA_CACHE_ELEMENTS)); return schema_cache; } @@ -1343,7 +1363,9 @@ std::optional StorageFile::tryGetColumnsFromCache( { /// Check if the cache contains one of the paths. auto & schema_cache = getSchemaCache(context); - struct stat file_stat{}; + struct stat file_stat + { + }; for (const auto & path : paths) { auto get_last_mod_time = [&]() -> std::optional diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index e1f7c6f3bbe..4e8b34cd082 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -1,7 +1,7 @@ #pragma once -#include #include +#include #include @@ -14,7 +14,7 @@ namespace DB class StorageFile final : public IStorage { -friend class partitionedstoragefilesink; + friend class partitionedstoragefilesink; public: struct CommonArguments : public WithContext @@ -51,10 +51,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write( - const ASTPtr & query, - const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; void truncate( const ASTPtr & /*query*/, @@ -69,7 +66,8 @@ public: NamesAndTypesList getVirtuals() const override; - static Strings getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read); + static Strings + getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read); /// Check if the format supports reading only some subset of columns. /// Is is useful because such formats could effectively skip unknown columns @@ -122,8 +120,8 @@ private: std::vector paths; std::vector paths_to_archive; - bool is_db_table = true; /// Table is stored in real database, not user's file - bool use_table_fd = false; /// Use table_fd instead of path + bool is_db_table = true; /// Table is stored in real database, not user's file + bool use_table_fd = false; /// Use table_fd instead of path mutable std::shared_timed_mutex rwlock; From 04450a2042d95b473bb7ebd2e169cd3c07a67b22 Mon Sep 17 00:00:00 2001 From: Nikita Keba Date: Mon, 29 May 2023 20:28:15 +0000 Subject: [PATCH 0108/2047] add CheckFileInfo Unit Tests --- .../tests/gtest_archive_reader_and_writer.cpp | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index 4f3ea8eba30..4d05a31af1f 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -375,6 +375,20 @@ TEST(TarArchiveReaderTest, ReadTwoFiles) { fs::remove(archive_path); } + +TEST(TarArchiveReaderTest, CheckFileInfo) { + String archive_path = "archive.tar"; + String filename = "file.txt"; + String contents = "test"; + bool created = create_tar_with_file(archive_path, {{filename, contents}}); + EXPECT_EQ(created, true); + auto reader = createArchiveReader(archive_path); + auto info = reader->getFileInfo(filename); + EXPECT_EQ(info.uncompressed_size, contents.size()); + EXPECT_GT(info.compressed_size, 0); + fs::remove(archive_path); +} + TEST(SevenZipArchiveReaderTest, FileExists) { String archive_path = "archive.7z"; String filename = "file.txt"; @@ -400,6 +414,19 @@ TEST(SevenZipArchiveReaderTest, ReadFile) { fs::remove(archive_path); } +TEST(SevenZipArchiveReaderTest, CheckFileInfo) { + String archive_path = "archive.7z"; + String filename = "file.txt"; + String contents = "test"; + bool created = create_7z_with_file(archive_path, {{filename, contents}}); + EXPECT_EQ(created, true); + auto reader = createArchiveReader(archive_path); + auto info = reader->getFileInfo(filename); + EXPECT_EQ(info.uncompressed_size, contents.size()); + EXPECT_GT(info.compressed_size, 0); + fs::remove(archive_path); +} + TEST(SevenZipArchiveReaderTest, ReadTwoFiles) { String archive_path = "archive.7z"; String file1 = "file1.txt"; From beb3690c7ecf95f496f4cea2be7d3b19ae24046c Mon Sep 17 00:00:00 2001 From: xiebin Date: Tue, 30 May 2023 16:09:01 +0800 Subject: [PATCH 0109/2047] if dictionary id is number, do not convert layout to complex --- src/Dictionaries/getDictionaryConfigurationFromAST.cpp | 7 ++++--- .../queries/0_stateless/02391_hashed_dictionary_shards.sql | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index c179ce84ff9..5f8e0bc27b4 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -615,9 +615,10 @@ getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr conte checkPrimaryKey(all_attr_names_and_types, pk_attrs); - /// If the pk size is 1 and pk's DataType is not native uint(UInt8~UInt64), we should convert to complex, - /// because the data type of Numeric key(simple layout) is UInt64. - if ((pk_attrs.size() > 1 || (pk_attrs.size() == 1 && !WhichDataType(DataTypeFactory::instance().get(all_attr_names_and_types.find(pk_attrs[0])->second.type)).isNativeUInt())) + /// If the pk size is 1 and pk's DataType is not number, we should convert to complex. + /// NOTE: the data type of Numeric key(simple layout) is UInt64, so if the type is not under UInt64, type casting will lead to precision loss. + DataTypePtr first_key_type = DataTypeFactory::instance().get(all_attr_names_and_types.find(pk_attrs[0])->second.type); + if ((pk_attrs.size() > 1 || (pk_attrs.size() == 1 && !isNumber(first_key_type))) && !complex && DictionaryFactory::instance().convertToComplex(dictionary_layout->layout_type)) { diff --git a/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql b/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql index ac43c12afc0..018f6b2cf4f 100644 --- a/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql +++ b/tests/queries/0_stateless/02391_hashed_dictionary_shards.sql @@ -89,7 +89,7 @@ SOURCE(CLICKHOUSE(TABLE test_table_string)) LAYOUT(SPARSE_HASHED(SHARDS 10)) LIFETIME(0); -SYSTEM RELOAD DICTIONARY test_dictionary_10_shards_string; -- { serverError CANNOT_PARSE_TEXT } +SYSTEM RELOAD DICTIONARY test_dictionary_10_shards_string; DROP DICTIONARY test_dictionary_10_shards_string; From b66881b95a3698c050ef0ff9bff2f570fa01fba8 Mon Sep 17 00:00:00 2001 From: xiebin Date: Tue, 30 May 2023 17:59:46 +0800 Subject: [PATCH 0110/2047] refine functional test --- ...ayout_to_complex_by_complex_keys.reference | 6 ++-- ...nary_layout_to_complex_by_complex_keys.sql | 35 ++++++++++--------- 2 files changed, 21 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference index cc9381622ec..7616c59e4fd 100644 --- a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.reference @@ -1,5 +1,5 @@ dict_flat_simple Flat -dict_hashed_simple Hashed -dict_hashed_complex ComplexKeyHashed +dict_hashed_simple_Decimal128 Hashed +dict_hashed_simple_Float32 Hashed +dict_hashed_simple_String ComplexKeyHashed dict_hashed_simple_auto_convert ComplexKeyHashed -dict_hashed_simple_int_auto_convert ComplexKeyHashed diff --git a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql index 77d933afa9f..753b9f663b5 100644 --- a/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql +++ b/tests/queries/0_stateless/02731_auto_convert_dictionary_layout_to_complex_by_complex_keys.sql @@ -1,34 +1,35 @@ DROP DICTIONARY IF EXISTS dict_flat_simple; -DROP DICTIONARY IF EXISTS dict_hashed_simple; -DROP DICTIONARY IF EXISTS dict_hashed_complex; +DROP DICTIONARY IF EXISTS dict_hashed_simple_Decimal128; +DROP DICTIONARY IF EXISTS dict_hashed_simple_Float32; +DROP DICTIONARY IF EXISTS dict_hashed_simple_String; DROP DICTIONARY IF EXISTS dict_hashed_simple_auto_convert; DROP TABLE IF EXISTS dict_data; -CREATE TABLE dict_data (v0 UInt16, v1 UInt16, v2 UInt16, v3 UInt16, v4 UInt16) engine=Memory(); +CREATE TABLE dict_data (v0 UInt16, v1 Int16, v2 Float32, v3 Decimal128(10), v4 String) engine=Memory() AS SELECT number, number%65535, number*1.1, number*1.1, 'foo' FROM numbers(10);; CREATE DICTIONARY dict_flat_simple (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(flat()); SYSTEM RELOAD DICTIONARY dict_flat_simple; SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_flat_simple'; DROP DICTIONARY dict_flat_simple; -CREATE DICTIONARY dict_hashed_simple (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); -SYSTEM RELOAD DICTIONARY dict_hashed_simple; -SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple'; -DROP DICTIONARY dict_hashed_simple; +CREATE DICTIONARY dict_hashed_simple_Decimal128 (v3 Decimal128(10), v1 UInt16, v2 Float32) PRIMARY KEY v3 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple_Decimal128; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_Decimal128'; +DROP DICTIONARY dict_hashed_simple_Decimal128; -CREATE DICTIONARY dict_hashed_complex (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0,v1 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(complex_key_hashed()); -SYSTEM RELOAD DICTIONARY dict_hashed_complex; -SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_complex'; -DROP DICTIONARY dict_hashed_complex; +CREATE DICTIONARY dict_hashed_simple_Float32 (v2 Float32, v3 Decimal128(10), v4 String) PRIMARY KEY v2 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple_Float32; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_Float32'; +DROP DICTIONARY dict_hashed_simple_Float32; -CREATE DICTIONARY dict_hashed_simple_auto_convert (v0 UInt16, v1 UInt16, v2 UInt16) PRIMARY KEY v0,v1 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +CREATE DICTIONARY dict_hashed_simple_String (v4 String, v3 Decimal128(10), v2 Float32) PRIMARY KEY v4 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); +SYSTEM RELOAD DICTIONARY dict_hashed_simple_String; +SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_String'; +DROP DICTIONARY dict_hashed_simple_String; + +CREATE DICTIONARY dict_hashed_simple_auto_convert (v0 UInt16, v1 Int16, v2 UInt16) PRIMARY KEY v0,v1 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); SYSTEM RELOAD DICTIONARY dict_hashed_simple_auto_convert; SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_auto_convert'; DROP DICTIONARY dict_hashed_simple_auto_convert; -CREATE DICTIONARY dict_hashed_simple_int_auto_convert (v0 Int16, v1 UInt16, v2 UInt16) PRIMARY KEY v0 SOURCE(CLICKHOUSE(TABLE 'dict_data')) LIFETIME(0) LAYOUT(hashed()); -SYSTEM RELOAD DICTIONARY dict_hashed_simple_int_auto_convert; -SELECT name, type FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict_hashed_simple_int_auto_convert'; -DROP DICTIONARY dict_hashed_simple_int_auto_convert; - DROP TABLE dict_data; From cd8eb44f0c54945f4777ed3e50e08b057ee41f43 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 31 May 2023 14:03:11 +0000 Subject: [PATCH 0111/2047] Add encryptValue(), decryptValue() and exceptions --- src/Common/Config/ConfigProcessor.cpp | 94 ++++++++++++++++----------- src/Common/Config/ConfigProcessor.h | 8 ++- 2 files changed, 63 insertions(+), 39 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index fdfc6343876..b6db53018f4 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -46,6 +46,17 @@ namespace ErrorCodes extern const int CANNOT_LOAD_CONFIG; } +/// Get method for string name. Throw exception for wrong name +EncryptionMethod getEncryptionMethod(const std::string & name) +{ + if (name == "AES_128_GCM_SIV") + return AES_128_GCM_SIV; + else if (name == "AES_256_GCM_SIV") + return AES_256_GCM_SIV; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", name); +} + /// For cutting preprocessed path to this base static std::string main_config_path; @@ -175,57 +186,64 @@ static void mergeAttributes(Element & config_element, Element & with_element) with_element_attributes->release(); } +std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) +{ + auto codec = DB::CompressionCodecEncrypted(getEncryptionMethod(codec_name)); + + DB::Memory<> memory1; + memory1.resize(value.size() + codec.getAdditionalSizeAtTheEndOfBuffer() + codec.getHeaderSize()+100); + auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory1.data()); + std::string encrypted_value = std::string(memory1.data(), bytes_written); + std::string hex_value; + boost::algorithm::hex(encrypted_value.begin(), encrypted_value.end(), std::back_inserter(hex_value)); + LOG_DEBUG(log, "Encrypted value: '{}'.", hex_value); + return hex_value; +} + +std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) +{ + auto codec = DB::CompressionCodecEncrypted(getEncryptionMethod(codec_name)); + + DB::Memory<> memory; + std::string encrypted_value; + + try + { + boost::algorithm::unhex(value, std::back_inserter(encrypted_value)); + } + catch (const std::exception &) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read encrypted text, check for valid characters [0-9a-fA-F] and length"); + } + + memory.resize(codec.readDecompressedBlockSize(encrypted_value.data()) + codec.getAdditionalSizeAtTheEndOfBuffer()); + codec.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); + std::string decrypted_value = std::string(memory.data(), memory.size()); + LOG_DEBUG(log, "Decrypted value '{}'", decrypted_value); + return decrypted_value; +} + void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) { for (Node * node = config_root->firstChild(); node;) { if (node->nodeType() == Node::ELEMENT_NODE) { - // NamedNodeMapPtr attributes = node->attributes(); Element & element = dynamic_cast(*node); if (element.hasAttribute("encryption_codec")) { LOG_DEBUG(log, "Encrypted node <{}>", node->nodeName()); - // for (Node * child_node = node->firstChild(); child_node;) - // { - // LOG_DEBUG(log, " Child node {} value '{}'.", child_node->nodeName(), child_node->getNodeValue()); - // child_node = child_node->nextSibling(); - // } + + const NodeListPtr children = element.childNodes(); + if (children->length() != 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} should have only one text node", node->nodeName()); Node * text_node = node->firstChild(); - auto codec_128 = DB::CompressionCodecEncrypted(DB::AES_128_GCM_SIV); - // DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, ""); + if (text_node->nodeType() != Node::TEXT_NODE) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} should have text node", node->nodeName()); - /* - DB::Memory<> memory1; - std::string password="abcd"; - memory1.resize(password.size() + codec_128.getAdditionalSizeAtTheEndOfBuffer() + codec_128.getHeaderSize()+100); - auto bytes_written = codec_128.compress(password.data(), static_cast(password.size()), memory1.data()); - // std::string encrypted_password = std::string(memory1.data(), memory1.size()); - std::string encrypted_password = std::string(memory1.data(), bytes_written); - std::string password_hex; - boost::algorithm::hex(encrypted_password.begin(), encrypted_password.end(), std::back_inserter(password_hex)); - LOG_DEBUG(log, "Encrypted password: '{}'.", password_hex); - */ - - DB::Memory<> memory; - std::string encrypted_value; - - try - { - boost::algorithm::unhex(text_node->getNodeValue(), std::back_inserter(encrypted_value)); - // boost::algorithm::unhex(password_hex, std::back_inserter(encrypted_value)); - } - catch (const std::exception &) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read encrypted text for {}, check for valid characters [0-9a-fA-F] and length", node->nodeName()); - } - - memory.resize(codec_128.readDecompressedBlockSize(encrypted_value.data()) + codec_128.getAdditionalSizeAtTheEndOfBuffer()); - codec_128.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); - std::string decrypted_value = std::string(memory.data(), memory.size()); - LOG_DEBUG(log, "Decrypted value '{}'", decrypted_value); - text_node->setNodeValue(decrypted_value); + auto encryption_codec = element.getAttribute("encryption_codec"); + text_node->setNodeValue(decryptValue(encryption_codec, text_node->getNodeValue())); } } diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 479a0053efa..c9b227863f0 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,7 +92,13 @@ public: const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); - /// crypt nodes in config with specified encryption attributes + /// Encrypt text value + std::string encryptValue(const std::string & codec_name, const std::string & value); + + /// Decrypt value + std::string decryptValue(const std::string & codec_name, const std::string & value); + + /// Decrypt nodes in config with specified encryption attributes void decryptConfig(LoadedConfig & loaded_config); /// Save preprocessed config to specified directory. From fd8c5992889728c76d231a4f96c577bc6578017d Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 31 May 2023 15:16:18 +0000 Subject: [PATCH 0112/2047] Add encrypt_decrypt example --- src/Common/examples/CMakeLists.txt | 3 ++ src/Common/examples/encrypt_decrypt.cpp | 50 +++++++++++++++++++++++++ utils/config-processor/CMakeLists.txt | 3 +- 3 files changed, 54 insertions(+), 2 deletions(-) create mode 100644 src/Common/examples/encrypt_decrypt.cpp diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index d095ab3a1be..12a2b59ff77 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -82,3 +82,6 @@ endif() clickhouse_add_executable (interval_tree interval_tree.cpp) target_link_libraries (interval_tree PRIVATE dbms) + +clickhouse_add_executable (encrypt_decrypt encrypt_decrypt.cpp) +target_link_libraries (encrypt_decrypt PRIVATE dbms) diff --git a/src/Common/examples/encrypt_decrypt.cpp b/src/Common/examples/encrypt_decrypt.cpp new file mode 100644 index 00000000000..cd48963c47a --- /dev/null +++ b/src/Common/examples/encrypt_decrypt.cpp @@ -0,0 +1,50 @@ +#include +#include +#include +#include + + +int main(int argc, char ** argv) +{ + try + { + if (argc != 5) + { + std::cerr << "usage: " << argv[0] << " path action codec value" << std::endl; + return 3; + } + + std::string action = argv[2]; + std::string codec_name = argv[3]; + std::string value = argv[4]; + DB::ConfigProcessor processor(argv[1], false, true); + + auto loaded_config = processor.loadConfig(); + + DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); + + if (action == "-e") + std::cout << processor.encryptValue(codec_name, value) << std::endl; + else if (action == "-d") + std::cout << processor.decryptValue(codec_name, value) << std::endl; + else + std::cerr << "Unknown action: " << action << std::endl; + } + catch (Poco::Exception & e) + { + std::cerr << "Exception: " << e.displayText() << std::endl; + return 1; + } + catch (std::exception & e) + { + std::cerr << "std::exception: " << e.what() << std::endl; + return 3; + } + catch (...) + { + std::cerr << "Some exception" << std::endl; + return 2; + } + + return 0; +} diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt index 00cbfbba659..80c3535ef4e 100644 --- a/utils/config-processor/CMakeLists.txt +++ b/utils/config-processor/CMakeLists.txt @@ -1,3 +1,2 @@ clickhouse_add_executable (config-processor config-processor.cpp) -target_link_libraries(config-processor PRIVATE dbms clickhouse_common_config_no_zookeeper_log) -target_link_libraries(config-processor PUBLIC clickhouse_parsers clickhouse_common_io common ch_contrib::lz4) +target_link_libraries(config-processor PRIVATE dbms) From 0708caeb770e88a4805e084eeb01465c85fa45e2 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 1 Jun 2023 09:01:01 +0000 Subject: [PATCH 0113/2047] Fix style --- src/Common/Config/ConfigProcessor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index b6db53018f4..055a497fb38 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -44,6 +44,7 @@ namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; extern const int CANNOT_LOAD_CONFIG; + extern const int BAD_ARGUMENTS; } /// Get method for string name. Throw exception for wrong name From 2ccec017717e57b0eb1bdfb573f6f09e5201446d Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 1 Jun 2023 09:53:31 +0000 Subject: [PATCH 0114/2047] Set correct memory size for encrypt/decrypt --- src/Common/Config/ConfigProcessor.cpp | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 055a497fb38..99bea019c3b 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -191,13 +191,12 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const { auto codec = DB::CompressionCodecEncrypted(getEncryptionMethod(codec_name)); - DB::Memory<> memory1; - memory1.resize(value.size() + codec.getAdditionalSizeAtTheEndOfBuffer() + codec.getHeaderSize()+100); - auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory1.data()); - std::string encrypted_value = std::string(memory1.data(), bytes_written); + DB::Memory<> memory; + memory.resize(codec.getCompressedReserveSize(static_cast(value.size()))); + auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory.data()); + std::string encrypted_value = std::string(memory.data(), bytes_written); std::string hex_value; boost::algorithm::hex(encrypted_value.begin(), encrypted_value.end(), std::back_inserter(hex_value)); - LOG_DEBUG(log, "Encrypted value: '{}'.", hex_value); return hex_value; } @@ -217,10 +216,9 @@ std::string ConfigProcessor::decryptValue(const std::string & codec_name, const throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read encrypted text, check for valid characters [0-9a-fA-F] and length"); } - memory.resize(codec.readDecompressedBlockSize(encrypted_value.data()) + codec.getAdditionalSizeAtTheEndOfBuffer()); + memory.resize(codec.readDecompressedBlockSize(encrypted_value.data())); codec.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); std::string decrypted_value = std::string(memory.data(), memory.size()); - LOG_DEBUG(log, "Decrypted value '{}'", decrypted_value); return decrypted_value; } From d5add614daa2e6f7f0a18eaada22f5c43a057934 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 1 Jun 2023 12:48:45 +0000 Subject: [PATCH 0115/2047] Add text memo for encrypt_decrypt --- src/Common/examples/encrypt_decrypt.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Common/examples/encrypt_decrypt.cpp b/src/Common/examples/encrypt_decrypt.cpp index cd48963c47a..542e173deb9 100644 --- a/src/Common/examples/encrypt_decrypt.cpp +++ b/src/Common/examples/encrypt_decrypt.cpp @@ -3,6 +3,12 @@ #include #include +/** This test program encrypts or decrypts text values using AES_128_GCM_SIV or AES_256_GCM_SIV codecs. + * Keys for codecs are loaded from section of configuration file. + * + * How to use: + * ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV test + */ int main(int argc, char ** argv) { @@ -10,17 +16,22 @@ int main(int argc, char ** argv) { if (argc != 5) { - std::cerr << "usage: " << argv[0] << " path action codec value" << std::endl; + std::cerr << "Usage:" << std::endl + << " " << argv[0] << " path action codec value" << std::endl + << "path: path to configuration file." << std::endl + << "action: -e for encryption and -d for decryption." << std::endl + << "codec: AES_128_GCM_SIV or AES_256_GCM_SIV." << std::endl << std::endl + << "Example:" << std::endl + << " ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV test"; return 3; } std::string action = argv[2]; std::string codec_name = argv[3]; std::string value = argv[4]; + DB::ConfigProcessor processor(argv[1], false, true); - auto loaded_config = processor.loadConfig(); - DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); if (action == "-e") From e269235dbcf32f7e507370e1bff74a202a33446c Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 1 Jun 2023 13:09:21 +0000 Subject: [PATCH 0116/2047] Make decryptRecursive() go through element nodes only --- src/Common/Config/ConfigProcessor.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 99bea019c3b..df25a9a3825 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -231,8 +231,6 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) Element & element = dynamic_cast(*node); if (element.hasAttribute("encryption_codec")) { - LOG_DEBUG(log, "Encrypted node <{}>", node->nodeName()); - const NodeListPtr children = element.childNodes(); if (children->length() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} should have only one text node", node->nodeName()); @@ -244,10 +242,8 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) auto encryption_codec = element.getAttribute("encryption_codec"); text_node->setNodeValue(decryptValue(encryption_codec, text_node->getNodeValue())); } + decryptRecursive(node); } - - decryptRecursive(node); - node = node->nextSibling(); } } From 88bf4e49d4767b6a3a3ccfcc383a42ca90ae12f1 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 1 Jun 2023 17:40:40 -0400 Subject: [PATCH 0117/2047] update for min_chunk_bytes_for_parallel_parsing --- docs/en/sql-reference/transactions.md | 53 +++++++++++++++++++-------- 1 file changed, 37 insertions(+), 16 deletions(-) diff --git a/docs/en/sql-reference/transactions.md b/docs/en/sql-reference/transactions.md index 68fbfe0b22a..1ca2db44b13 100644 --- a/docs/en/sql-reference/transactions.md +++ b/docs/en/sql-reference/transactions.md @@ -3,23 +3,44 @@ slug: /en/guides/developer/transactional --- # Transactional (ACID) support -INSERT into one partition* in one table* of MergeTree* family up to max_insert_block_size rows* is transactional (ACID): -- Atomic: INSERT is succeeded or rejected as a whole: if confirmation is sent to the client, all rows INSERTed; if error is sent to the client, no rows INSERTed. +## Case 1: INSERT into one partition, of one table, of the MergeTree* family + +This is transactional (ACID) if the number of rows inserted is less than or equal to `max_insert_block_size rows`, and in the case of data in TSV, TKSV, CSV, or JSONEachRow format if the number of bytes is less than `min_chunk_bytes_for_parallel_parsing`: +- Atomic: an INSERT succeeds or is rejected as a whole: if a confirmation is sent to the client, then all rows were inserted; if an error is sent to the client, then no rows were inserted. - Consistent: if there are no table constraints violated, then all rows in an INSERT are inserted and the INSERT succeeds; if constraints are violated, then no rows are inserted. -- Isolated: concurrent clients observe a consistent snapshot of the table–the state of the table either as if before INSERT or after successful INSERT; no partial state is seen; -- Durable: successful INSERT is written to the filesystem before answering to the client, on single replica or multiple replicas (controlled by the `insert_quorum` setting), and ClickHouse can ask the OS to sync the filesystem data on the storage media (controlled by the `fsync_after_insert` setting). -* If table has many partitions and INSERT covers many partitions–then insertion into every partition is transactional on its own; -* INSERT into multiple tables with one statement is possible if materialized views are involved; -* INSERT into Distributed table is not transactional as a whole, while insertion into every shard is transactional; -* another example: insert into Buffer tables is neither atomic nor isolated or consistent or durable; -* atomicity is ensured even if `async_insert` is enabled, but it can be turned off by the wait_for_async_insert setting; -* max_insert_block_size is 1 000 000 by default and can be adjusted as needed; -* if client did not receive the answer from the server, the client does not know if transaction succeeded, and it can repeat the transaction, using exactly-once insertion properties; -* ClickHouse is using MVCC with snapshot isolation internally; -* all ACID properties are valid even in case of server kill / crash; -* either insert_quorum into different AZ or fsync should be enabled to ensure durable inserts in typical setup; -* "consistency" in ACID terms does not cover the semantics of distributed systems, see https://jepsen.io/consistency which is controlled by different settings (select_sequential_consistency) -* this explanation does not cover a new transactions feature that allow to have full-featured transactions over multiple tables, materialized views, for multiple SELECTs, etc. +- Isolated: concurrent clients observe a consistent snapshot of the table–the state of the table either as it was before the INSERT attempt, or after the successful INSERT; no partial state is seen +- Durable: a successful INSERT is written to the filesystem before answering to the client, on a single replica or multiple replicas (controlled by the `insert_quorum` setting), and ClickHouse can ask the OS to sync the filesystem data on the storage media (controlled by the `fsync_after_insert` setting). +- INSERT into multiple tables with one statement is possible if materialized views are involved (the INSERT from the client is to a table which has associate materialized views). + +## Case 2: INSERT into multiple partitions, of one table, of the MergeTree* family + +Same as Case 1 above, with this detail: +- If table has many partitions and INSERT covers many partitions–then insertion into every partition is transactional on its own + + +## Case 3: INSERT into one distributed table of the MergeTree* family + +Same as Case 1 above, with this detail: +- INSERT into Distributed table is not transactional as a whole, while insertion into every shard is transactional + +## Case 4: Using a Buffer table + +- insert into Buffer tables is neither atomic nor isolated nor consistent nor durable + +## Case 5: Using async_insert + +Same as Case 1 above, with this detail: +- atomicity is ensured even if `async_insert` is enabled and `wait_for_async_insert` is set to 1 (the default), but if `wait_for_async_insert` is set to 0, then atomicity is not ensured. + +## Notes +- `max_insert_block_size` is 1 000 000 by default and can be adjusted as needed +- `min_chunk_bytes_for_parallel_parsing` is 1 000 000 by default and can be adjusted as needed +- if the client did not receive an answer from the server, the client does not know if the transaction succeeded, and it can repeat the transaction, using exactly-once insertion properties +- ClickHouse is using MVCC with snapshot isolation internally +- all ACID properties are valid even in the case of server kill/crash +- either insert_quorum into different AZ or fsync should be enabled to ensure durable inserts in the typical setup +- "consistency" in ACID terms does not cover the semantics of distributed systems, see https://jepsen.io/consistency which is controlled by different settings (select_sequential_consistency) +- this explanation does not cover a new transactions feature that allow to have full-featured transactions over multiple tables, materialized views, for multiple SELECTs, etc. (see the next section on Transactions, Commit, and Rollback). ## Transactions, Commit, and Rollback From d868e35863c3a80c9924b347ac017e9e93c33ba2 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 1 Jun 2023 19:08:44 -0400 Subject: [PATCH 0118/2047] update spelling list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ded7a4643a9..0787ead76cf 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -169,6 +169,7 @@ SelfManaged Stateful Submodules Subqueries +TKSV TSVRaw TSan TabItem From 1a361ef3060a4d271e6e27bb816df1d18cffaa02 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 5 Jun 2023 03:21:43 +0200 Subject: [PATCH 0119/2047] works for file --- src/Storages/StorageFile.cpp | 103 +++++++++++++++++++++++++++++++---- 1 file changed, 93 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 647f9511052..53da509d383 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -90,6 +90,57 @@ namespace ErrorCodes namespace { +/// Forward-declare to use in listFilesWithFoldedRegexpMatchingImpl() +void listFilesWithRegexpMatchingImpl( + const std::string & path_for_ls, + const std::string & for_match, + size_t & total_bytes_to_read, + std::vector & result, + bool recursive = false); + +/// When `{...}` has any `/`s, it must be processed in a different way +void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const std::string & processed_suffix, + const std::string & suffix_with_globs, + const std::string & glob, re2::RE2 & matcher, size_t & total_bytes_to_read, + const size_t max_depth, const size_t next_slash_after_glob_pos, + std::vector & result) +{ + /// We don't need to go all the way in every directory if max_depth is reached + /// as it is upper limit of depth by simply counting `/`s in curly braces + if (!max_depth) + return; + + const fs::directory_iterator end; + for (fs::directory_iterator it(start_dir + processed_suffix); it != end; ++it) + { + const std::string full_path = it->path().string(); + const size_t last_slash = full_path.rfind('/'); + const String dir_or_file_name = full_path.substr(last_slash); + + if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) + { + if (next_slash_after_glob_pos == std::string::npos) + { + total_bytes_to_read += it->file_size(); + result.push_back(it->path().string()); + } + else + { + listFilesWithRegexpMatchingImpl(fs::path(full_path).append(processed_suffix).append(it->path().string()) / "" , + suffix_with_globs.substr(next_slash_after_glob_pos), + total_bytes_to_read, result); + } + } + else if (it->is_directory()) + { + listFilesWithFoldedRegexpMatchingImpl(start_dir, processed_suffix + dir_or_file_name, suffix_with_globs, + glob, matcher, total_bytes_to_read, max_depth - 1, + next_slash_after_glob_pos, result); + } + + } +} + /* Recursive directory listing with matched paths as a result. * Have the same method in StorageHDFS. */ @@ -98,15 +149,40 @@ void listFilesWithRegexpMatchingImpl( const std::string & for_match, size_t & total_bytes_to_read, std::vector & result, - bool recursive = false) + bool recursive) { - const size_t first_glob = for_match.find_first_of("*?{"); + const size_t first_glob_pos = for_match.find_first_of("*?{"); + const bool has_glob = first_glob_pos != std::string::npos; - const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + const size_t end_of_path_without_globs = for_match.substr(0, first_glob_pos).rfind('/'); const std::string suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - const size_t next_slash = suffix_with_globs.find('/', 1); - const std::string current_glob = suffix_with_globs.substr(0, next_slash); + /// slashes_in_glob counter is a upper-bound estimate of recursion depth + /// needed to process complex cases when `/` is included into glob, e.g. /pa{th1/a,th2/b}.csv + size_t slashes_in_glob = 0; + const size_t next_slash_after_glob_pos = [&](){ + if (!has_glob) + return suffix_with_globs.find('/', 1); + + size_t in_curly = 0; + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + if (*it == '{') + ++in_curly; + else if (*it == '/') + { + if (in_curly) + ++slashes_in_glob; + else + return size_t(std::distance(suffix_with_globs.begin(), it)); + } + else if (*it == '}') + --in_curly; + } + return std::string::npos; + }(); + + const std::string current_glob = suffix_with_globs.substr(0, next_slash_after_glob_pos); + auto regexp = makeRegexpPatternFromGlobs(current_glob); re2::RE2 matcher(regexp); @@ -123,13 +199,22 @@ void listFilesWithRegexpMatchingImpl( if (!fs::exists(prefix_without_globs)) return; + const bool looking_for_directory = next_slash_after_glob_pos != std::string::npos; + + if (slashes_in_glob) + { + listFilesWithFoldedRegexpMatchingImpl(prefix_without_globs, "", suffix_with_globs, + current_glob, matcher, total_bytes_to_read, slashes_in_glob, + next_slash_after_glob_pos, result); + return; + } + const fs::directory_iterator end; for (fs::directory_iterator it(prefix_without_globs); it != end; ++it) { const std::string full_path = it->path().string(); const size_t last_slash = full_path.rfind('/'); const String file_name = full_path.substr(last_slash); - const bool looking_for_directory = next_slash != std::string::npos; /// Condition is_directory means what kind of path is it in current iteration of ls if (!it->is_directory() && !looking_for_directory) @@ -145,14 +230,12 @@ void listFilesWithRegexpMatchingImpl( if (recursive) { listFilesWithRegexpMatchingImpl(fs::path(full_path).append(it->path().string()) / "" , - looking_for_directory ? suffix_with_globs.substr(next_slash) : current_glob , + looking_for_directory ? suffix_with_globs.substr(next_slash_after_glob_pos) : current_glob , total_bytes_to_read, result, recursive); } else if (looking_for_directory && re2::RE2::FullMatch(file_name, matcher)) - { /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. - listFilesWithRegexpMatchingImpl(fs::path(full_path) / "", suffix_with_globs.substr(next_slash), total_bytes_to_read, result); - } + listFilesWithRegexpMatchingImpl(fs::path(full_path) / "", suffix_with_globs.substr(next_slash_after_glob_pos), total_bytes_to_read, result); } } } From 28fb99cd260dcf7d6c36386364645f19c3ef7ab6 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Mon, 5 Jun 2023 13:12:19 -0700 Subject: [PATCH 0120/2047] Maintain same aggregate function mergeY behavior for small and big endian machine --- src/AggregateFunctions/UniquesHashSet.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/UniquesHashSet.h b/src/AggregateFunctions/UniquesHashSet.h index ca6d31a716d..369fce9374f 100644 --- a/src/AggregateFunctions/UniquesHashSet.h +++ b/src/AggregateFunctions/UniquesHashSet.h @@ -108,7 +108,13 @@ private: inline size_t buf_size() const { return 1ULL << size_degree; } /// NOLINT inline size_t max_fill() const { return 1ULL << (size_degree - 1); } /// NOLINT inline size_t mask() const { return buf_size() - 1; } - inline size_t place(HashValue x) const { return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); } + + inline size_t place(HashValue x) const { + if constexpr (std::endian::native == std::endian::little) + return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); + else + return (std::byteswap(x) >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); + } /// The value is divided by 2 ^ skip_degree inline bool good(HashValue hash) const From d316add2f1f6ffa9cf6f2a1107a4d7d69960c72a Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 6 Jun 2023 15:24:29 +0000 Subject: [PATCH 0121/2047] Add integration test test_config_decryption --- .../test_config_decryption/__init__.py | 0 .../test_config_decryption/configs/config.xml | 12 +++++++ .../test_config_decryption/test.py | 31 +++++++++++++++++++ 3 files changed, 43 insertions(+) create mode 100644 tests/integration/test_config_decryption/__init__.py create mode 100644 tests/integration/test_config_decryption/configs/config.xml create mode 100644 tests/integration/test_config_decryption/test.py diff --git a/tests/integration/test_config_decryption/__init__.py b/tests/integration/test_config_decryption/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_config_decryption/configs/config.xml b/tests/integration/test_config_decryption/configs/config.xml new file mode 100644 index 00000000000..5c274128e39 --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config.xml @@ -0,0 +1,12 @@ + + + + 00112233445566778899aabbccddeeff + + + 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff + + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/test.py b/tests/integration/test_config_decryption/test.py new file mode 100644 index 00000000000..a3cb1bb57f3 --- /dev/null +++ b/tests/integration/test_config_decryption/test.py @@ -0,0 +1,31 @@ +import pytest +import os +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", main_configs=["configs/config.xml"]) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_successful_decryption(started_cluster): + assert ( + node.query( + "select value from system.server_settings where name ='max_table_size_to_drop'" + ) + == "60000000000\n" + ) + assert ( + node.query( + "select value from system.server_settings where name ='max_partition_size_to_drop'" + ) + == "40000000000\n" + ) From 9cd0d5e6db0ce68ffdc320e2f73d17531fb54ec0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 7 Jun 2023 11:49:41 +0000 Subject: [PATCH 0122/2047] move settings to server_settings --- programs/server/Server.cpp | 9 +++------ src/Core/ServerSettings.h | 2 ++ 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index a64676cfa01..5496720e5dc 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1438,15 +1438,12 @@ try /// This is needed to load proper values of background_pool_size etc. global_context->initializeBackgroundExecutorsIfNeeded(); - size_t async_insert_threads = config().getUInt("async_insert_threads", 16); - bool async_insert_queue_flush_on_shutdown = config().getBool("async_insert_queue_flush_on_shutdown", false); - - if (async_insert_threads) + if (server_settings.async_insert_threads) { global_context->setAsynchronousInsertQueue(std::make_shared( global_context, - async_insert_threads, - async_insert_queue_flush_on_shutdown)); + server_settings.async_insert_threads, + server_settings.async_insert_queue_flush_on_shutdown)); } size_t mark_cache_size = server_settings.mark_cache_size; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 1a9f226041b..ca27cbdbf19 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -48,6 +48,8 @@ namespace DB M(UInt64, merges_mutations_memory_usage_soft_limit, 0, "Limit on total memory usage for merges and mutations. Zero means Unlimited.", 0) \ M(Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to ram ratio. Allows to lower memory limit on low-memory systems.", 0) \ M(Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0) \ + M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ + M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \ \ M(UInt64, max_concurrent_queries, 0, "Limit on total number of concurrently executed queries. Zero means Unlimited.", 0) \ M(UInt64, max_concurrent_insert_queries, 0, "Limit on total number of concurrently insert queries. Zero means Unlimited.", 0) \ From ece53808519ce6e42bbff2f2f3ac7d9ffccabe63 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 7 Jun 2023 15:48:38 +0200 Subject: [PATCH 0123/2047] add simple stateless for file() --- .../02771_complex_globs_in_storage_file_path.reference | 4 ++++ .../02771_complex_globs_in_storage_file_path.sql | 8 ++++++++ 2 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference create mode 100644 tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql diff --git a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference new file mode 100644 index 00000000000..e1b420ecf37 --- /dev/null +++ b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.reference @@ -0,0 +1,4 @@ +This is file data1 data1.csv +This is file data2 data2.csv +This is file data1 data1.csv +This is file data2 data2.csv diff --git a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql new file mode 100644 index 00000000000..c579c8d8698 --- /dev/null +++ b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql @@ -0,0 +1,8 @@ +-- Tags: no-replicated-database, no-parallel + +INSERT INTO TABLE FUNCTION file('02771/dir1/subdir11/data1.csv', 'CSV', 's String') SELECT 'This is file data1' SETTINGS engine_file_truncate_on_insert=1; +INSERT INTO TABLE FUNCTION file('02771/dir2/subdir22/data2.csv', 'CSV', 's String') SELECT 'This is file data2' SETTINGS engine_file_truncate_on_insert=1; + +SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV); +SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta2.csv', CSV); +SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV); From bfdb18619c3b31044eba28f15cd9a0d7141b790e Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 7 Jun 2023 19:25:33 +0200 Subject: [PATCH 0124/2047] small docs udpate --- docs/en/sql-reference/table-functions/file.md | 2 +- docs/ru/sql-reference/table-functions/file.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 28c2dc9f1f3..09e3df7cc6a 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -134,7 +134,7 @@ Multiple path components can have globs. For being processed file must exist and - `*` — Substitutes any number of any characters except `/` including empty string. - `?` — Substitutes any single character. -- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`, including `/`. - `{N..M}` — Substitutes any number in range from N to M including both borders. - `**` - Fetches all files inside the folder recursively. diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 94bc734a8fb..f0fea630ad2 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -79,7 +79,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U - `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов. - `?` — заменяет ровно один любой символ. -- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. +- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`, причём строка может содержать `/`. - `{N..M}` — заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). Конструкция с `{}` аналогична табличной функции [remote](remote.md). From ff1b069cdb9301f5c13bed69d70c484210693dea Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 7 Jun 2023 21:15:35 +0000 Subject: [PATCH 0125/2047] fix test --- tests/queries/0_stateless/02726_async_insert_flush_queue.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql index 33f40eef14e..98e78045b85 100644 --- a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql +++ b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + DROP TABLE IF EXISTS t_async_inserts_flush; CREATE TABLE t_async_inserts_flush (a UInt64) ENGINE = Memory; From 60499164b3acc8663a10836233926c0f997ac381 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Jun 2023 14:35:11 +0000 Subject: [PATCH 0126/2047] fix tests --- tests/queries/0_stateless/02726_async_insert_flush_stress.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh index 4685e49b96d..5fafb773d16 100755 --- a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh +++ b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-parallel set -e From 6328811097204e87abc31973adcec84ddf8f0fc4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 8 Jun 2023 09:26:30 +0200 Subject: [PATCH 0127/2047] Added first draft of azure blob storage cluster --- src/Storages/StorageAzureBlob.h | 1 - src/Storages/StorageAzureBlobCluster.cpp | 105 ++++++++++++++++++ src/Storages/StorageAzureBlobCluster.h | 56 ++++++++++ .../TableFunctionAzureBlobStorage.cpp | 49 ++++++-- .../TableFunctionAzureBlobStorage.h | 4 +- .../TableFunctionAzureBlobStorageCluster.cpp | 79 +++++++++++++ .../TableFunctionAzureBlobStorageCluster.h | 54 +++++++++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + .../configs/cluster.xml | 23 ++++ .../test_cluster.py | 102 +++++++++++++++++ 11 files changed, 463 insertions(+), 12 deletions(-) create mode 100644 src/Storages/StorageAzureBlobCluster.cpp create mode 100644 src/Storages/StorageAzureBlobCluster.h create mode 100644 src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp create mode 100644 src/TableFunctions/TableFunctionAzureBlobStorageCluster.h create mode 100644 tests/integration/test_storage_azure_blob_storage/configs/cluster.xml create mode 100644 tests/integration/test_storage_azure_blob_storage/test_cluster.py diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 1f91e47ddbe..cbea0b1e26c 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -136,7 +136,6 @@ private: const String & format_name, const ContextPtr & ctx); - }; class StorageAzureBlobSource : public ISource, WithContext diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp new file mode 100644 index 00000000000..203c8cbc12d --- /dev/null +++ b/src/Storages/StorageAzureBlobCluster.cpp @@ -0,0 +1,105 @@ +#include "Storages/StorageAzureBlobCluster.h" + +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +StorageAzureBlobCluster::StorageAzureBlobCluster( + const String & cluster_name_, + const StorageAzureBlob::Configuration & configuration_, + std::unique_ptr && object_storage_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + ContextPtr context_, + bool structure_argument_was_provided_) + : IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageAzureBlobCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) + , configuration{configuration_} + , object_storage(std::move(object_storage_)) +{ + context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.getConnectionURL()); + StorageInMemoryMetadata storage_metadata; + updateConfigurationIfChanged(context_); + + if (columns_.empty()) + { + /// `format_settings` is set to std::nullopt, because StorageAzureBlobCluster is used only as table function + auto columns = StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, /*format_settings=*/std::nullopt, context_); + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); + + auto default_virtuals = NamesAndTypesList{ + {"_path", std::make_shared(std::make_shared())}, + {"_file", std::make_shared(std::make_shared())}}; + + auto columns = storage_metadata.getSampleBlock().getNamesAndTypesList(); + virtual_columns = getVirtualsForStorage(columns, default_virtuals); + for (const auto & column : virtual_columns) + virtual_block.insert({column.type->createColumn(), column.type, column.name}); +} + +void StorageAzureBlobCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) +{ + ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); + if (!expression_list) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function s3Cluster, got '{}'", queryToString(query)); + + TableFunctionAzureBlobStorageCluster::addColumnsStructureToArguments(expression_list->children, structure, context); +} + +void StorageAzureBlobCluster::updateConfigurationIfChanged(ContextPtr /*local_context*/) +{ +// configuration.update(local_context); +} + +RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const +{ + auto iterator = std::make_shared( + object_storage.get(), configuration.container, std::nullopt, + configuration.blob_path, query, virtual_block, context, nullptr); + auto callback = std::make_shared>([iterator]() mutable -> String { return iterator->next().relative_path; }); + return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; +} + +NamesAndTypesList StorageAzureBlobCluster::getVirtuals() const +{ + return virtual_columns; +} + + +} + +#endif diff --git a/src/Storages/StorageAzureBlobCluster.h b/src/Storages/StorageAzureBlobCluster.h new file mode 100644 index 00000000000..015452e641a --- /dev/null +++ b/src/Storages/StorageAzureBlobCluster.h @@ -0,0 +1,56 @@ +#pragma once + +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +#include +#include + +#include "Client/Connection.h" +#include +#include +#include + +namespace DB +{ + +class Context; + +class StorageAzureBlobCluster : public IStorageCluster +{ +public: + StorageAzureBlobCluster( + const String & cluster_name_, + const StorageAzureBlob::Configuration & configuration_, + std::unique_ptr && object_storage_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + ContextPtr context_, + bool structure_argument_was_provided_); + + std::string getName() const override { return "AzureBlobStorageCluster"; } + + NamesAndTypesList getVirtuals() const override; + + RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; + +protected: + void updateConfigurationIfChanged(ContextPtr local_context); + +private: + void updateBeforeRead(const ContextPtr & context) override { updateConfigurationIfChanged(context); } + + void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override; + + StorageAzureBlob::Configuration configuration; + NamesAndTypesList virtual_columns; + Block virtual_block; + std::unique_ptr object_storage; +}; + + +} + +#endif diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 38d9362894a..1b29e313c50 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -44,10 +44,8 @@ bool isConnectionString(const std::string & candidate) } -StorageAzureBlob::Configuration TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context, bool get_format_from_file) +void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context) { - StorageAzureBlob::Configuration configuration; - /// Supported signatures: /// /// AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) @@ -59,10 +57,8 @@ StorageAzureBlob::Configuration TableFunctionAzureBlobStorage::parseArgumentsImp configuration.blobs_paths = {configuration.blob_path}; - if (configuration.format == "auto" && get_format_from_file) + if (configuration.format == "auto") configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); - - return configuration; } if (engine_args.size() < 3 || engine_args.size() > 8) @@ -172,10 +168,8 @@ StorageAzureBlob::Configuration TableFunctionAzureBlobStorage::parseArgumentsImp configuration.blobs_paths = {configuration.blob_path}; - if (configuration.format == "auto" && get_format_from_file) + if (configuration.format == "auto") configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); - - return configuration; } void TableFunctionAzureBlobStorage::parseArguments(const ASTPtr & ast_function, ContextPtr context) @@ -190,9 +184,44 @@ void TableFunctionAzureBlobStorage::parseArguments(const ASTPtr & ast_function, auto & args = args_func.at(0)->children; - configuration = parseArgumentsImpl(args, context); + parseArgumentsImpl(args, context); + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "CONFIGURATION {}", configuration.connection_url); } + +void TableFunctionAzureBlobStorage::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context) +{ + if (tryGetNamedCollectionWithOverrides(args, context)) + { + /// In case of named collection, just add key-value pair "structure='...'" + /// at the end of arguments to override existed structure. + ASTs equal_func_args = {std::make_shared("structure"), std::make_shared(structure)}; + auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); + args.push_back(equal_func); + } + else + { + if (args.size() < 3 || args.size() > 8) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage Azure requires 3 to 7 arguments: " + "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); + + auto structure_literal = std::make_shared(structure); + + if (args.size() == 3) + { + /// Add format=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + else if (args.size() == 4) + { + args.push_back(structure_literal); + } + } +} + + ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(ContextPtr context) const { if (configuration.structure == "auto") diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.h b/src/TableFunctions/TableFunctionAzureBlobStorage.h index 0bb872de3f3..a473b969a20 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.h +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.h @@ -46,7 +46,9 @@ public: return {"_path", "_file"}; } - static StorageAzureBlob::Configuration parseArgumentsImpl(ASTs & args, const ContextPtr & context, bool get_format_from_file = true); + virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); + + static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context); protected: diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp new file mode 100644 index 00000000000..47b03e30621 --- /dev/null +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp @@ -0,0 +1,79 @@ +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +#include +#include +#include +#include + +#include "registerTableFunctions.h" + +#include + + +namespace DB +{ + +StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( + const ASTPtr & /*function*/, ContextPtr context, + const std::string & table_name, ColumnsDescription /*cached_columns*/) const +{ + StoragePtr storage; + ColumnsDescription columns; + bool structure_argument_was_provided = configuration.structure != "auto"; + + if (structure_argument_was_provided) + { + columns = parseColumnsListFromString(configuration.structure, context); + } + else if (!structure_hint.empty()) + { + columns = structure_hint; + } + + auto client = StorageAzureBlob::createClient(configuration); + auto settings = StorageAzureBlob::createSettings(context); + + if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) + { + /// On worker node this filename won't contains globs + storage = std::make_shared( + configuration, + std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings)), + context, + StorageID(getDatabaseName(), table_name), + columns, + ConstraintsDescription{}, + /* comment */String{}, + /* format_settings */std::nullopt, /// No format_settings for S3Cluster + /*partition_by_=*/nullptr); + } + else + { + storage = std::make_shared( + cluster_name, + configuration, + std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings)), + StorageID(getDatabaseName(), table_name), + columns, + ConstraintsDescription{}, + context, + structure_argument_was_provided); + } + + storage->startup(); + + return storage; +} + + +void registerTableFunctionAzureBlobStorageCluster(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} + + +} + +#endif diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h new file mode 100644 index 00000000000..af4f57f235e --- /dev/null +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h @@ -0,0 +1,54 @@ +#pragma once + +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +#include +#include +#include +#include + + +namespace DB +{ + +class Context; + +/** + * azure_blob_storage_cluster(cluster_name, source, [access_key_id, secret_access_key,] format, structure, compression_method) + * A table function, which allows to process many files from Azure Blob Storage on a specific cluster + * On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks + * in Azure Blob Storage file path and dispatch each file dynamically. + * On worker node it asks initiator about next task to process, processes it. + * This is repeated until the tasks are finished. + */ +class TableFunctionAzureBlobStorageCluster : public ITableFunctionCluster +{ +public: + static constexpr auto name = "azure_blob_storage_cluster"; + static constexpr auto signature = " - cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]"; + + String getName() const override + { + return name; + } + + String getSignature() const override + { + return signature; + } + +protected: + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns) const override; + + const char * getStorageTypeName() const override { return "AzureBlobStorageCluster"; } +}; + +} + +#endif diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 0499524a912..8b684c102fa 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -74,6 +74,7 @@ void registerTableFunctions() #if USE_AZURE_BLOB_STORAGE registerTableFunctionAzureBlobStorage(factory); + registerTableFunctionAzureBlobStorageCluster(factory); #endif diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 393bc080a3d..2e5ef926984 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -71,6 +71,7 @@ void registerTableFunctionExplain(TableFunctionFactory & factory); #if USE_AZURE_BLOB_STORAGE void registerTableFunctionAzureBlobStorage(TableFunctionFactory & factory); +void registerTableFunctionAzureBlobStorageCluster(TableFunctionFactory & factory); #endif void registerTableFunctions(); diff --git a/tests/integration/test_storage_azure_blob_storage/configs/cluster.xml b/tests/integration/test_storage_azure_blob_storage/configs/cluster.xml new file mode 100644 index 00000000000..43df7b46f3f --- /dev/null +++ b/tests/integration/test_storage_azure_blob_storage/configs/cluster.xml @@ -0,0 +1,23 @@ + + + + + + node_0 + 9000 + + + node_1 + 9000 + + + node_2 + 9000 + + + + + + simple_cluster + + \ No newline at end of file diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py new file mode 100644 index 00000000000..1d551a9a3c3 --- /dev/null +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -0,0 +1,102 @@ +#!/usr/bin/env python3 + +import gzip +import json +import logging +import os +import io +import random +import threading +import time + +from azure.storage.blob import BlobServiceClient +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.test_tools import TSV +from helpers.network import PartitionManager +from helpers.mock_servers import start_mock_servers +from helpers.test_tools import exec_query_with_retry + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node_0", + main_configs=["configs/named_collections.xml", "configs/cluster.xml"], + with_azurite=True, + ) + cluster.add_instance( + "node_1", + main_configs=["configs/named_collections.xml", "configs/cluster.xml"], + with_azurite=True, + ) + cluster.start() + + yield cluster + finally: + cluster.shutdown() + +def azure_query(node, query, try_num=3, settings={}): + for i in range(try_num): + try: + return node.query(query, settings=settings) + except Exception as ex: + retriable_errors = [ + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response" + ] + retry = False + for error in retriable_errors: + if error in str(ex): + retry = True + print(f"Try num: {i}. Having retriable error: {ex}") + time.sleep(i) + break + if not retry or i == try_num - 1: + raise Exception(ex) + continue + + +def get_azure_file_content(filename): + container_name = "cont" + connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://127.0.0.1:10000/devstoreaccount1;" + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + container_client = blob_service_client.get_container_client(container_name) + blob_client = container_client.get_blob_client(filename) + download_stream = blob_client.download_blob() + return download_stream.readall().decode("utf-8") + + + +def test_simple_write_account_string_table_function(cluster): + node = cluster.instances["node_0"] + azure_query( + node, + "INSERT INTO TABLE FUNCTION azure_blob_storage(" + "'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', " + "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', " + "'auto', 'key UInt64, data String') VALUES (1, 'a')", + ) + print(get_azure_file_content("test_simple_write_tf.csv")) + assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' + + pure_azure = node.query( + """ + SELECT * from azure_blob_storage( + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', + 'auto', 'key UInt64, data String')""" + ) + print(pure_azure) + distributed_azure = node.query( + """ + SELECT * from azure_blob_storage_cluster( + 'simple_cluster', 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', + 'auto', 'key UInt64, data String')""" + ) + print(distributed_azure) + + assert TSV(pure_azure) == TSV(distributed_azure) From 590236030dadec86df0f176486cfa6be38b8351e Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 9 Jun 2023 15:17:08 +0200 Subject: [PATCH 0128/2047] Fixed cluster with distributed_processing --- src/Storages/StorageAzureBlob.cpp | 33 +++++++++++----- src/Storages/StorageAzureBlob.h | 38 +++++++++++++++---- src/Storages/StorageAzureBlobCluster.cpp | 14 ++----- src/Storages/StorageAzureBlobCluster.h | 5 +-- .../TableFunctionAzureBlobStorage.cpp | 3 +- .../TableFunctionAzureBlobStorageCluster.cpp | 7 ++-- .../test_storage_azure_blob_storage/test.py | 2 +- .../test_cluster.py | 17 ++++++--- 8 files changed, 77 insertions(+), 42 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 3ee176a68b7..68b21d93ad9 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -299,6 +299,7 @@ void registerStorageAzureBlob(StorageFactory & factory) args.constraints, args.comment, format_settings, + /* distributed_processing */ false, partition_by); }, { @@ -392,12 +393,13 @@ StorageAzureBlob::StorageAzureBlob( const ConstraintsDescription & constraints_, const String & comment, std::optional format_settings_, + bool distributed_processing_, ASTPtr partition_by_) : IStorage(table_id_) , name("AzureBlobStorage") , configuration(configuration_) , object_storage(std::move(object_storage_)) - , distributed_processing(false) + , distributed_processing(distributed_processing_) , format_settings(format_settings_) , partition_by(partition_by_) { @@ -407,7 +409,7 @@ StorageAzureBlob::StorageAzureBlob( StorageInMemoryMetadata storage_metadata; if (columns_.empty()) { - auto columns = getTableStructureFromData(object_storage.get(), configuration, format_settings, context); + auto columns = getTableStructureFromData(object_storage.get(), configuration, format_settings, context, distributed_processing); storage_metadata.setColumns(columns); } else @@ -611,8 +613,13 @@ Pipe StorageAzureBlob::read( requested_virtual_columns.push_back(virtual_column); } - std::shared_ptr iterator_wrapper; - if (configuration.withGlobs()) + std::shared_ptr iterator_wrapper; + if (distributed_processing) + { + iterator_wrapper = std::make_shared(local_context, + local_context->getReadTaskCallback()); + } + else if (configuration.withGlobs()) { /// Iterate through disclosed globs and make a source for each file iterator_wrapper = std::make_shared( @@ -795,7 +802,7 @@ StorageAzureBlobSource::Iterator::Iterator( const Block & virtual_header_, ContextPtr context_, RelativePathsWithMetadata * outer_blobs_) - : WithContext(context_) + : IIterator(context_) , object_storage(object_storage_) , container(container_) , keys(keys_) @@ -1073,7 +1080,7 @@ StorageAzureBlobSource::StorageAzureBlobSource( String compression_hint_, AzureObjectStorage * object_storage_, const String & container_, - std::shared_ptr file_iterator_) + std::shared_ptr file_iterator_) :ISource(getHeader(sample_block_, requested_virtual_columns_)) , WithContext(context_) , requested_virtual_columns(requested_virtual_columns_) @@ -1165,11 +1172,17 @@ ColumnsDescription StorageAzureBlob::getTableStructureFromData( AzureObjectStorage * object_storage, const Configuration & configuration, const std::optional & format_settings, - ContextPtr ctx) + ContextPtr ctx, + bool distributed_processing) { RelativePathsWithMetadata read_keys; - std::shared_ptr file_iterator; - if (configuration.withGlobs()) + std::shared_ptr file_iterator; + if (distributed_processing) + { + file_iterator = std::make_shared(ctx , + ctx->getReadTaskCallback()); + } + else if (configuration.withGlobs()) { file_iterator = std::make_shared( object_storage, configuration.container, std::nullopt, @@ -1202,7 +1215,7 @@ ColumnsDescription StorageAzureBlob::getTableStructureFromData( return nullptr; } - /// S3 file iterator could get new keys after new iteration, check them in schema cache. + /// Azure file iterator could get new keys after new iteration, check them in schema cache. if (ctx->getSettingsRef().schema_inference_use_cache_for_azure && read_keys.size() > prev_read_keys_size) { columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), configuration, format_settings, ctx); diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 04b7e736aea..3bb6f747ecf 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -62,6 +62,7 @@ public: const ConstraintsDescription & constraints_, const String & comment, std::optional format_settings_, + bool distributed_processing_, ASTPtr partition_by_); static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context); @@ -107,7 +108,8 @@ public: AzureObjectStorage * object_storage, const Configuration & configuration, const std::optional & format_settings, - ContextPtr ctx); + ContextPtr ctx, + bool distributed_processing = false); private: std::string name; @@ -141,7 +143,18 @@ private: class StorageAzureBlobSource : public ISource, WithContext { public: - class Iterator : WithContext + class IIterator : public WithContext + { + public: + IIterator(ContextPtr context_):WithContext(context_) {} + virtual ~IIterator() = default; + virtual RelativePathWithMetadata next() = 0; + virtual size_t getTotalSize() const = 0; + + RelativePathWithMetadata operator ()() { return next(); } + }; + + class Iterator : public IIterator { public: Iterator( @@ -154,9 +167,9 @@ public: ContextPtr context_, RelativePathsWithMetadata * outer_blobs_); - RelativePathWithMetadata next(); - size_t getTotalSize() const; - ~Iterator() = default; + RelativePathWithMetadata next() override; + size_t getTotalSize() const override; + ~Iterator() override = default; private: AzureObjectStorage * object_storage; @@ -183,6 +196,17 @@ public: std::mutex next_mutex; }; + class ReadIterator : public IIterator + { + public: + explicit ReadIterator(ContextPtr context_, const ReadTaskCallback & callback_) + : IIterator(context_), callback(callback_) {} + RelativePathWithMetadata next() override { return {callback(), {}}; } + size_t getTotalSize() const override { return 0; } + private: + ReadTaskCallback callback; + }; + StorageAzureBlobSource( const std::vector & requested_virtual_columns_, const String & format_, @@ -195,7 +219,7 @@ public: String compression_hint_, AzureObjectStorage * object_storage_, const String & container_, - std::shared_ptr file_iterator_); + std::shared_ptr file_iterator_); ~StorageAzureBlobSource() override; @@ -216,7 +240,7 @@ private: String compression_hint; AzureObjectStorage * object_storage; String container; - std::shared_ptr file_iterator; + std::shared_ptr file_iterator; struct ReaderHolder { diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp index 203c8cbc12d..9296043aaeb 100644 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ b/src/Storages/StorageAzureBlobCluster.cpp @@ -47,12 +47,11 @@ StorageAzureBlobCluster::StorageAzureBlobCluster( { context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.getConnectionURL()); StorageInMemoryMetadata storage_metadata; - updateConfigurationIfChanged(context_); if (columns_.empty()) { /// `format_settings` is set to std::nullopt, because StorageAzureBlobCluster is used only as table function - auto columns = StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, /*format_settings=*/std::nullopt, context_); + auto columns = StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, /*format_settings=*/std::nullopt, context_, false); storage_metadata.setColumns(columns); } else @@ -80,17 +79,12 @@ void StorageAzureBlobCluster::addColumnsStructureToQuery(ASTPtr & query, const S TableFunctionAzureBlobStorageCluster::addColumnsStructureToArguments(expression_list->children, structure, context); } -void StorageAzureBlobCluster::updateConfigurationIfChanged(ContextPtr /*local_context*/) -{ -// configuration.update(local_context); -} - RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const { auto iterator = std::make_shared( - object_storage.get(), configuration.container, std::nullopt, - configuration.blob_path, query, virtual_block, context, nullptr); - auto callback = std::make_shared>([iterator]() mutable -> String { return iterator->next().relative_path; }); + object_storage.get(), configuration.container, configuration.blobs_paths, + std::nullopt, query, virtual_block, context, nullptr); + auto callback = std::make_shared>([iterator]() mutable -> String {return iterator->next().relative_path;}); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } diff --git a/src/Storages/StorageAzureBlobCluster.h b/src/Storages/StorageAzureBlobCluster.h index 015452e641a..6ef994fb873 100644 --- a/src/Storages/StorageAzureBlobCluster.h +++ b/src/Storages/StorageAzureBlobCluster.h @@ -36,11 +36,8 @@ public: RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; -protected: - void updateConfigurationIfChanged(ContextPtr local_context); - private: - void updateBeforeRead(const ContextPtr & context) override { updateConfigurationIfChanged(context); } + void updateBeforeRead(const ContextPtr & /*context*/) override {} void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override; diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 1b29e313c50..9d9699a51a1 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -231,7 +231,7 @@ ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(Contex auto settings = StorageAzureBlob::createSettings(context); auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings)); - return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context); + return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context, false); } return parseColumnsListFromString(configuration.structure, context); @@ -263,6 +263,7 @@ StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_funct String{}, /// No format_settings for table function Azure std::nullopt, + /* distributed_processing */ false, nullptr); storage->startup(); diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp index 47b03e30621..b3b18c7ec0d 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp @@ -40,13 +40,14 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( /// On worker node this filename won't contains globs storage = std::make_shared( configuration, - std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings)), + std::make_unique(table_name, std::move(client), std::move(settings)), context, StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, /* comment */String{}, - /* format_settings */std::nullopt, /// No format_settings for S3Cluster + /* format_settings */std::nullopt, /// No format_settings + /* distributed_processing */ true, /*partition_by_=*/nullptr); } else @@ -54,7 +55,7 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( storage = std::make_shared( cluster_name, configuration, - std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings)), + std::make_unique(table_name, std::move(client), std::move(settings)), StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f0934d3aa80..fc859479409 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -591,4 +591,4 @@ def test_partition_by_tf(cluster): assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv") assert "3,2,1\n" == get_azure_file_content("test_tf_1.csv") - assert "78,43,45\n" == get_azure_file_content("test_tf_45.csv") + assert "78,43,45\n" == get_azure_file_content("test_tf_45.csv") \ No newline at end of file diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index 1d551a9a3c3..212d21e8d2d 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -33,6 +33,11 @@ def cluster(): main_configs=["configs/named_collections.xml", "configs/cluster.xml"], with_azurite=True, ) + cluster.add_instance( + "node_2", + main_configs=["configs/named_collections.xml", "configs/cluster.xml"], + with_azurite=True, + ) cluster.start() yield cluster @@ -77,25 +82,25 @@ def test_simple_write_account_string_table_function(cluster): "INSERT INTO TABLE FUNCTION azure_blob_storage(" "'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', " "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', " - "'auto', 'key UInt64, data String') VALUES (1, 'a')", + "'auto', 'key UInt64') VALUES (1), (2)", ) print(get_azure_file_content("test_simple_write_tf.csv")) - assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' + #assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' pure_azure = node.query( """ - SELECT * from azure_blob_storage( + SELECT count(*) from azure_blob_storage( 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', - 'auto', 'key UInt64, data String')""" + 'auto', 'key UInt64')""" ) print(pure_azure) distributed_azure = node.query( """ - SELECT * from azure_blob_storage_cluster( + SELECT count(*) from azure_blob_storage_cluster( 'simple_cluster', 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', - 'auto', 'key UInt64, data String')""" + 'auto', 'key UInt64')""" ) print(distributed_azure) From 6213b1b04fdff796ca69685d15a8700f240fec91 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 9 Jun 2023 13:27:46 +0000 Subject: [PATCH 0129/2047] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- .../test_storage_azure_blob_storage/test_cluster.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index fc859479409..f0934d3aa80 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -591,4 +591,4 @@ def test_partition_by_tf(cluster): assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv") assert "3,2,1\n" == get_azure_file_content("test_tf_1.csv") - assert "78,43,45\n" == get_azure_file_content("test_tf_45.csv") \ No newline at end of file + assert "78,43,45\n" == get_azure_file_content("test_tf_45.csv") diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index 212d21e8d2d..776944c2c53 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -44,6 +44,7 @@ def cluster(): finally: cluster.shutdown() + def azure_query(node, query, try_num=3, settings={}): for i in range(try_num): try: @@ -74,7 +75,6 @@ def get_azure_file_content(filename): return download_stream.readall().decode("utf-8") - def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node_0"] azure_query( @@ -85,7 +85,7 @@ def test_simple_write_account_string_table_function(cluster): "'auto', 'key UInt64') VALUES (1), (2)", ) print(get_azure_file_content("test_simple_write_tf.csv")) - #assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' + # assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' pure_azure = node.query( """ From 1bce32c1cc1d9e2b0aeea93c01947646e18c52b3 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 9 Jun 2023 15:18:46 +0000 Subject: [PATCH 0130/2047] Add tests for wrong settings --- .../configs/config_invalid_chars.xml | 12 +++++++ .../configs/config_no_encryption_codecs.xml | 4 +++ .../configs/config_subnodes.xml | 10 ++++++ .../configs/config_wrong_method.xml | 12 +++++++ .../test_wrong_settings.py | 34 +++++++++++++++++++ 5 files changed, 72 insertions(+) create mode 100644 tests/integration/test_config_decryption/configs/config_invalid_chars.xml create mode 100644 tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml create mode 100644 tests/integration/test_config_decryption/configs/config_subnodes.xml create mode 100644 tests/integration/test_config_decryption/configs/config_wrong_method.xml create mode 100644 tests/integration/test_config_decryption/test_wrong_settings.py diff --git a/tests/integration/test_config_decryption/configs/config_invalid_chars.xml b/tests/integration/test_config_decryption/configs/config_invalid_chars.xml new file mode 100644 index 00000000000..49bf51b5bad --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config_invalid_chars.xml @@ -0,0 +1,12 @@ + + + + 00112233445566778899aabbccddeeff + + + 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff + + + --96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml b/tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml new file mode 100644 index 00000000000..07bf69d17c8 --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml @@ -0,0 +1,4 @@ + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/configs/config_subnodes.xml b/tests/integration/test_config_decryption/configs/config_subnodes.xml new file mode 100644 index 00000000000..b0e519ff546 --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config_subnodes.xml @@ -0,0 +1,10 @@ + + + + 00112233445566778899aabbccddeeff + + + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + + diff --git a/tests/integration/test_config_decryption/configs/config_wrong_method.xml b/tests/integration/test_config_decryption/configs/config_wrong_method.xml new file mode 100644 index 00000000000..b452ce6374c --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config_wrong_method.xml @@ -0,0 +1,12 @@ + + + + 00112233445566778899aabbccddeeff + + + 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff + + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py new file mode 100644 index 00000000000..c01f5050b00 --- /dev/null +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -0,0 +1,34 @@ +import pytest +from helpers.cluster import ClickHouseCluster + + +def start_clickhouse(config, err_msg): + cluster = ClickHouseCluster(__file__) + node = cluster.add_instance("node", main_configs=[config]) + caught_exception = "" + try: + cluster.start() + except Exception as e: + caught_exception = str(e) + assert caught_exception.find(err_msg) != -1 + + +def test_wrong_method(): + start_clickhouse("configs/config_wrong_method.xml", "Wrong encryption Method") + + +def test_invalid_chars(): + start_clickhouse( + "configs/config_invalid_chars.xml", + "Cannot read encrypted text, check for valid characters", + ) + + +def test_no_encryption_codecs(): + start_clickhouse( + "configs/config_no_encryption_codecs.xml", "There is no key 0 in config" + ) + + +def test_subnodes(): + start_clickhouse("configs/config_subnodes.xml", "should have only one text node") From 4a7761c16210c7e2eccc0b26e172ec8dc7e6c183 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 10 Jun 2023 08:26:32 +0000 Subject: [PATCH 0131/2047] Add column is_obsolete for system.settings table and related system tables --- src/Interpreters/Context.cpp | 6 ++++-- src/Storages/System/StorageSystemMergeTreeSettings.cpp | 2 ++ src/Storages/System/StorageSystemServerSettings.cpp | 2 ++ src/Storages/System/StorageSystemSettings.cpp | 2 ++ tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- tests/queries/0_stateless/01945_system_warnings.reference | 2 +- tests/queries/0_stateless/01945_system_warnings.sh | 4 ++-- 7 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 995e78d8f0b..a12117b7677 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -781,8 +781,10 @@ Strings Context::getWarnings() const { if (setting.isValueChanged() && setting.isObsolete()) { - common_warnings.emplace_back("Some obsolete setting is changed. " - "Check 'select * from system.settings where changed' and read the changelog."); + common_warnings.emplace_back( + "Obsolete setting `" + setting.getName() + + "` is changed. " + "Check 'select * from system.settings where changed' and read the changelog."); break; } } diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.cpp b/src/Storages/System/StorageSystemMergeTreeSettings.cpp index 6de3fb800f4..0ddd4546208 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.cpp +++ b/src/Storages/System/StorageSystemMergeTreeSettings.cpp @@ -21,6 +21,7 @@ NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes() {"max", std::make_shared(std::make_shared())}, {"readonly", std::make_shared()}, {"type", std::make_shared()}, + {"is_obsolete", std::make_shared()}, }; } @@ -52,6 +53,7 @@ void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, res_columns[5]->insert(max); res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); + res_columns[8]->insert(setting.isObsolete()); } } diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index ad52c6896ac..290b575465c 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -15,6 +15,7 @@ NamesAndTypesList StorageSystemServerSettings::getNamesAndTypes() {"changed", std::make_shared()}, {"description", std::make_shared()}, {"type", std::make_shared()}, + {"is_obsolete", std::make_shared()}, }; } @@ -33,6 +34,7 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context res_columns[3]->insert(setting.isValueChanged()); res_columns[4]->insert(setting.getDescription()); res_columns[5]->insert(setting.getTypeName()); + res_columns[6]->insert(setting.isObsolete()); } } diff --git a/src/Storages/System/StorageSystemSettings.cpp b/src/Storages/System/StorageSystemSettings.cpp index c54f7eef25f..dcb54eac0a0 100644 --- a/src/Storages/System/StorageSystemSettings.cpp +++ b/src/Storages/System/StorageSystemSettings.cpp @@ -21,6 +21,7 @@ NamesAndTypesList StorageSystemSettings::getNamesAndTypes() {"type", std::make_shared()}, {"default", std::make_shared()}, {"alias_for", std::make_shared()}, + {"is_obsolete", std::make_shared()}, }; } @@ -51,6 +52,7 @@ void StorageSystemSettings::fillData(MutableColumns & res_columns, ContextPtr co res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); res_columns[8]->insert(setting.getDefaultValueString()); + res_columns[10]->insert(setting.isObsolete()); }; const auto & settings_to_aliases = Settings::Traits::settingsToAliases(); diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 4e6dd3e1b0f..3d5b1ca99a5 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Some obsolete setting is changed." +expect " * Obsolete setting `max_memory_usage_for_all_queries` is changed." expect ":) " send -- "q\r" expect eof diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index 296a03447db..d6ae567289c 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Some obsolete setting is changed. Check \'select * from system.settings where changed\' and read the changelog. +Obsolete setting `multiple_joins_rewriter_version` is changed. Check \'select * from system.settings where changed\' and read the changelog. 1 1 diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index bf11cee2911..112baab614e 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -14,8 +14,8 @@ else echo "Server was built in debug mode. It will work slowly." fi -${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.warnings WHERE message LIKE '%obsolete setting%'" -${CLICKHOUSE_CLIENT} --multiple_joins_rewriter_version=42 -q "SELECT message FROM system.warnings WHERE message LIKE '%obsolete setting%'" +${CLICKHOUSE_CLIENT} -q "SELECT count() FROM system.warnings WHERE message LIKE '%Obsolete setting%'" +${CLICKHOUSE_CLIENT} --multiple_joins_rewriter_version=42 -q "SELECT message FROM system.warnings WHERE message LIKE '%Obsolete setting%'" # Avoid duplicated warnings ${CLICKHOUSE_CLIENT} -q "SELECT count() = countDistinct(message) FROM system.warnings" From 820673a5cf3f3f1c17b781496b3ab56f72f72c08 Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 10 Jun 2023 10:16:53 +0000 Subject: [PATCH 0132/2047] update test --- .../queries/0_stateless/01221_system_settings.reference | 4 ++-- .../0_stateless/02117_show_create_table_system.reference | 9 ++++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01221_system_settings.reference b/tests/queries/0_stateless/01221_system_settings.reference index 399b3778b66..e9c2f3fec32 100644 --- a/tests/queries/0_stateless/01221_system_settings.reference +++ b/tests/queries/0_stateless/01221_system_settings.reference @@ -1,4 +1,4 @@ -send_timeout 300 0 Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the \'receive_timeout\' for the socket will be also set on the corresponding connection end on the server. \N \N 0 Seconds 300 -storage_policy default 0 Name of storage disk policy \N \N 0 String +send_timeout 300 0 Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the \'receive_timeout\' for the socket will be also set on the corresponding connection end on the server. \N \N 0 Seconds 300 0 +storage_policy default 0 Name of storage disk policy \N \N 0 String 0 1 1 diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index e864ba85018..38d00c15725 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -346,7 +346,8 @@ CREATE TABLE system.merge_tree_settings `min` Nullable(String), `max` Nullable(String), `readonly` UInt8, - `type` String + `type` String, + `is_obsolete` UInt8 ) ENGINE = SystemMergeTreeSettings COMMENT 'SYSTEM TABLE is built on the fly.' @@ -918,7 +919,8 @@ CREATE TABLE system.replicated_merge_tree_settings `min` Nullable(String), `max` Nullable(String), `readonly` UInt8, - `type` String + `type` String, + `is_obsolete` UInt8 ) ENGINE = SystemReplicatedMergeTreeSettings COMMENT 'SYSTEM TABLE is built on the fly.' @@ -993,7 +995,8 @@ CREATE TABLE system.settings `readonly` UInt8, `type` String, `default` String, - `alias_for` String + `alias_for` String, + `is_obsolete` UInt8 ) ENGINE = SystemSettings COMMENT 'SYSTEM TABLE is built on the fly.' From 2c97a94892358f7e6b4a7aa4b25c4d46dd0f0fc0 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 11 Jun 2023 01:50:17 +0200 Subject: [PATCH 0133/2047] fix hdfs + style update --- src/Storages/HDFS/StorageHDFS.cpp | 115 ++++++++++++++++++++++++++++-- src/Storages/StorageFile.cpp | 20 +++--- 2 files changed, 120 insertions(+), 15 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 19c0840149b..379ee395939 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -63,23 +63,124 @@ namespace ErrorCodes } namespace { + Strings LSWithRegexpMatching(const String & path_for_ls, + const HDFSFSPtr & fs, + const String & for_match, + std::unordered_map * last_mod_times); + + /// When `{...}` has any `/`s, it must be processed in a different way + Strings LSWithFoldedRegexpMatching(const String & path_for_ls, + const HDFSFSPtr & fs, + std::unordered_map * last_mod_times, + const String & processed_suffix, + const String & suffix_with_globs, + const String & current_glob, + re2::RE2 & matcher, + const size_t max_depth, + const size_t next_slash_after_glob_pos) + { + /// We don't need to go all the way in every directory if max_depth is reached + /// as it is upper limit of depth by simply counting `/`s in curly braces + if (!max_depth) + return {}; + + HDFSFileInfo ls; + ls.file_info = hdfsListDirectory(fs.get(), path_for_ls.data(), &ls.length); + if (ls.file_info == nullptr && errno != ENOENT) // NOLINT + { + // ignore file not found exception, keep throw other exception, libhdfs3 doesn't have function to get exception type, so use errno. + throw Exception( + ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", path_for_ls, String(hdfsGetLastError())); + } + + Strings result; + if (!ls.file_info && ls.length > 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "file_info shouldn't be null"); + + for (int i = 0; i < ls.length; ++i) + { + const String full_path = String(ls.file_info[i].mName); + const size_t last_slash = full_path.rfind('/'); + const String dir_or_file_name = full_path.substr(last_slash); + const bool is_directory = ls.file_info[i].mKind == 'D'; + + if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) + { + if (next_slash_after_glob_pos == std::string::npos) + { + result.push_back(String(ls.file_info[i].mName)); + if (last_mod_times) + (*last_mod_times)[result.back()] = ls.file_info[i].mLastMod; + } + else + { + Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "" , fs, + suffix_with_globs.substr(next_slash_after_glob_pos), last_mod_times); + std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); + } + } + else if (is_directory) + { + Strings result_part = LSWithFoldedRegexpMatching(fs::path(full_path).append(processed_suffix), + fs, last_mod_times, processed_suffix + dir_or_file_name, suffix_with_globs, current_glob, matcher, + max_depth - 1, next_slash_after_glob_pos); + std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); + } + } + return result; + } + /* Recursive directory listing with matched paths as a result. * Have the same method in StorageFile. */ - Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match, std::unordered_map * last_mod_times) + Strings LSWithRegexpMatching(const String & path_for_ls, + const HDFSFSPtr & fs, + const String & for_match, + std::unordered_map * last_mod_times) { - const size_t first_glob = for_match.find_first_of("*?{"); + const size_t first_glob_pos = for_match.find_first_of("*?{"); + const bool has_glob = first_glob_pos != std::string::npos; - const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + const size_t end_of_path_without_globs = for_match.substr(0, first_glob_pos).rfind('/'); const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' - const size_t next_slash = suffix_with_globs.find('/', 1); - re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash))); + size_t slashes_in_glob = 0; + const size_t next_slash_after_glob_pos = [&](){ + if (!has_glob) + return suffix_with_globs.find('/', 1); + + size_t in_curly = 0; + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + if (*it == '{') + ++in_curly; + else if (*it == '/') + { + if (in_curly) + ++slashes_in_glob; + else + return size_t(std::distance(suffix_with_globs.begin(), it)); + } + else if (*it == '}') + --in_curly; + } + return std::string::npos; + }(); + + const std::string current_glob = suffix_with_globs.substr(0, next_slash_after_glob_pos); + + re2::RE2 matcher(makeRegexpPatternFromGlobs(current_glob)); if (!matcher.ok()) throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", for_match, matcher.error()); + if (slashes_in_glob) + { + return LSWithFoldedRegexpMatching(fs::path(prefix_without_globs), fs, last_mod_times, + "", suffix_with_globs, current_glob, matcher, + slashes_in_glob, next_slash_after_glob_pos); + } + HDFSFileInfo ls; ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length); if (ls.file_info == nullptr && errno != ENOENT) // NOLINT @@ -96,7 +197,7 @@ namespace const String full_path = String(ls.file_info[i].mName); const size_t last_slash = full_path.rfind('/'); const String file_name = full_path.substr(last_slash); - const bool looking_for_directory = next_slash != std::string::npos; + const bool looking_for_directory = next_slash_after_glob_pos != std::string::npos; const bool is_directory = ls.file_info[i].mKind == 'D'; /// Condition with type of current file_info means what kind of path is it in current iteration of ls if (!is_directory && !looking_for_directory) @@ -112,7 +213,7 @@ namespace { if (re2::RE2::FullMatch(file_name, matcher)) { - Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash), last_mod_times); + Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash_after_glob_pos), last_mod_times); /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 53da509d383..74303b16ee9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -99,10 +99,14 @@ void listFilesWithRegexpMatchingImpl( bool recursive = false); /// When `{...}` has any `/`s, it must be processed in a different way -void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const std::string & processed_suffix, +void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, + const std::string & processed_suffix, const std::string & suffix_with_globs, - const std::string & glob, re2::RE2 & matcher, size_t & total_bytes_to_read, - const size_t max_depth, const size_t next_slash_after_glob_pos, + const std::string & current_glob, + re2::RE2 & matcher, + size_t & total_bytes_to_read, + const size_t max_depth, + const size_t next_slash_after_glob_pos, std::vector & result) { /// We don't need to go all the way in every directory if max_depth is reached @@ -111,7 +115,7 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const return; const fs::directory_iterator end; - for (fs::directory_iterator it(start_dir + processed_suffix); it != end; ++it) + for (fs::directory_iterator it(path_for_ls); it != end; ++it) { const std::string full_path = it->path().string(); const size_t last_slash = full_path.rfind('/'); @@ -133,9 +137,9 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & start_dir, const } else if (it->is_directory()) { - listFilesWithFoldedRegexpMatchingImpl(start_dir, processed_suffix + dir_or_file_name, suffix_with_globs, - glob, matcher, total_bytes_to_read, max_depth - 1, - next_slash_after_glob_pos, result); + listFilesWithFoldedRegexpMatchingImpl(fs::path(full_path).append(processed_suffix), processed_suffix + dir_or_file_name, + suffix_with_globs, current_glob, matcher, + total_bytes_to_read, max_depth - 1, next_slash_after_glob_pos, result); } } @@ -203,7 +207,7 @@ void listFilesWithRegexpMatchingImpl( if (slashes_in_glob) { - listFilesWithFoldedRegexpMatchingImpl(prefix_without_globs, "", suffix_with_globs, + listFilesWithFoldedRegexpMatchingImpl(fs::path(prefix_without_globs), "", suffix_with_globs, current_glob, matcher, total_bytes_to_read, slashes_in_glob, next_slash_after_glob_pos, result); return; From aad7712b18fa358f60c59565d3914fafc2a3f759 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 11 Jun 2023 02:09:05 +0200 Subject: [PATCH 0134/2047] add existing test --- tests/integration/test_storage_hdfs/test.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index edf5344e887..0b18c0180cc 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -85,6 +85,15 @@ def test_read_write_storage_with_globs(started_cluster): assert "in readonly mode" in str(ex) +def test_storage_with_multidirectory_glob(started_cluster): + hdfs_api = started_cluster.hdfs_api + for i in ["1", "2"]: + hdfs_api.write_data(f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n") + assert hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") == f"File{i}\t{i}{i}\n" + + assert node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") == f"\File1\t11\nFile2\t22\n" + + def test_read_write_table(started_cluster): hdfs_api = started_cluster.hdfs_api From 427c5cb1bafef9b52011f9d77e725b7a5ba85553 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Sun, 11 Jun 2023 18:42:10 +0200 Subject: [PATCH 0135/2047] fix integration test --- tests/integration/test_storage_hdfs/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 0b18c0180cc..dde3dd257b1 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -91,7 +91,8 @@ def test_storage_with_multidirectory_glob(started_cluster): hdfs_api.write_data(f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n") assert hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") == f"File{i}\t{i}{i}\n" - assert node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") == f"\File1\t11\nFile2\t22\n" + r = node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") + assert (r == f"File1\t11\nFile2\t22\n") or (r == f"File2\t22\nFile1\t11\n") def test_read_write_table(started_cluster): From 95d447d81fad93770cdddf1ba508466ac7a1d886 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 12 Jun 2023 10:10:09 +0200 Subject: [PATCH 0136/2047] Updated name to azureBlobStorageCluster --- src/TableFunctions/TableFunctionAzureBlobStorageCluster.h | 4 ++-- .../test_storage_azure_blob_storage/test_cluster.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h index af4f57f235e..c263fd6ca0c 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h @@ -16,7 +16,7 @@ namespace DB class Context; /** - * azure_blob_storage_cluster(cluster_name, source, [access_key_id, secret_access_key,] format, structure, compression_method) + * azureBlobStorageCluster(cluster_name, source, [access_key_id, secret_access_key,] format, compression_method, structure) * A table function, which allows to process many files from Azure Blob Storage on a specific cluster * On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks * in Azure Blob Storage file path and dispatch each file dynamically. @@ -26,7 +26,7 @@ class Context; class TableFunctionAzureBlobStorageCluster : public ITableFunctionCluster { public: - static constexpr auto name = "azure_blob_storage_cluster"; + static constexpr auto name = "azureBlobStorageCluster"; static constexpr auto signature = " - cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]"; String getName() const override diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index 776944c2c53..ca6fd7433f4 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -97,7 +97,7 @@ def test_simple_write_account_string_table_function(cluster): print(pure_azure) distributed_azure = node.query( """ - SELECT count(*) from azure_blob_storage_cluster( + SELECT count(*) from azureBlobStorageCluster( 'simple_cluster', 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64')""" From e1395860221e22c343e3cc786cc34d1b15f4f0fd Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 12 Jun 2023 10:31:32 +0200 Subject: [PATCH 0137/2047] Fixed style fails --- src/Storages/StorageAzureBlobCluster.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp index 9296043aaeb..b111262c498 100644 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ b/src/Storages/StorageAzureBlobCluster.cpp @@ -83,7 +83,7 @@ RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension { auto iterator = std::make_shared( object_storage.get(), configuration.container, configuration.blobs_paths, - std::nullopt, query, virtual_block, context, nullptr); + std::nullopt, query, virtual_block, context, nullptr); auto callback = std::make_shared>([iterator]() mutable -> String {return iterator->next().relative_path;}); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } From eb9cdbcf7d5d1cdcc3b45936b4045dd8eda8f818 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 12 Jun 2023 11:41:36 +0200 Subject: [PATCH 0138/2047] fix File test being flaky --- src/Storages/StorageFile.cpp | 15 +++++++++++---- .../02771_complex_globs_in_storage_file_path.sql | 4 +++- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 74303b16ee9..c7a57b7d4c9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -109,8 +109,14 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t next_slash_after_glob_pos, std::vector & result) { - /// We don't need to go all the way in every directory if max_depth is reached - /// as it is upper limit of depth by simply counting `/`s in curly braces + /* + * When `{...}` has any `/`s, it must be processed in a different way: + * Basically, a path with globs is processed by LSWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, + * LSWithFoldedRegexpMatching is in charge from now on. + * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. + * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. + * StorageHDFS.cpp has the same logic. + */ if (!max_depth) return; @@ -121,6 +127,7 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t last_slash = full_path.rfind('/'); const String dir_or_file_name = full_path.substr(last_slash); + std::cerr << "\nprocessing file (full_path): " << full_path << "\n"; if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) { if (next_slash_after_glob_pos == std::string::npos) @@ -130,14 +137,14 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, } else { - listFilesWithRegexpMatchingImpl(fs::path(full_path).append(processed_suffix).append(it->path().string()) / "" , + listFilesWithRegexpMatchingImpl(fs::path(full_path) / "" , suffix_with_globs.substr(next_slash_after_glob_pos), total_bytes_to_read, result); } } else if (it->is_directory()) { - listFilesWithFoldedRegexpMatchingImpl(fs::path(full_path).append(processed_suffix), processed_suffix + dir_or_file_name, + listFilesWithFoldedRegexpMatchingImpl(fs::path(full_path), processed_suffix + dir_or_file_name, suffix_with_globs, current_glob, matcher, total_bytes_to_read, max_depth - 1, next_slash_after_glob_pos, result); } diff --git a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql index c579c8d8698..41d7d6fcc3f 100644 --- a/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql +++ b/tests/queries/0_stateless/02771_complex_globs_in_storage_file_path.sql @@ -5,4 +5,6 @@ INSERT INTO TABLE FUNCTION file('02771/dir2/subdir22/data2.csv', 'CSV', 's Strin SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta1.csv', CSV); SELECT *, _file FROM file('02771/dir{?/subdir?1/da,2/subdir2?/da}ta2.csv', CSV); -SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV); + +SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data1.csv'; +SELECT *, _file FROM file('02771/dir?/{subdir?1/data1,subdir2?/data2}.csv', CSV) WHERE _file == 'data2.csv'; From 7d7bd5bb66fbf1e0d3a1e35a0fe74231d6e5e7ed Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 12 Jun 2023 11:50:23 +0200 Subject: [PATCH 0139/2047] update comment describing workflow --- src/Storages/HDFS/StorageHDFS.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 379ee395939..bd50b66ede5 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -63,12 +63,20 @@ namespace ErrorCodes } namespace { + /// Forward-declared to use in LSWithFoldedRegexpMatching w/o circular dependency. Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match, std::unordered_map * last_mod_times); - /// When `{...}` has any `/`s, it must be processed in a different way + /* + * When `{...}` has any `/`s, it must be processed in a different way: + * Basically, a path with globs is processed by LSWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, + * LSWithFoldedRegexpMatching is in charge from now on. + * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. + * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. + * StorageFile.cpp has the same logic. + */ Strings LSWithFoldedRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, std::unordered_map * last_mod_times, From 3e6d393e17b0913d2664838a98a8f946bc8b7695 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 12 Jun 2023 12:06:21 +0200 Subject: [PATCH 0140/2047] remove debug cerr --- src/Storages/StorageFile.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index c7a57b7d4c9..93228cf4d39 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -127,7 +127,6 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t last_slash = full_path.rfind('/'); const String dir_or_file_name = full_path.substr(last_slash); - std::cerr << "\nprocessing file (full_path): " << full_path << "\n"; if (re2::RE2::FullMatch(processed_suffix + dir_or_file_name, matcher)) { if (next_slash_after_glob_pos == std::string::npos) From cd1a3916a6ea755b24b475983f6f67447cebdd6a Mon Sep 17 00:00:00 2001 From: Andrey Zvonov Date: Mon, 12 Jun 2023 13:28:17 +0200 Subject: [PATCH 0141/2047] cleanup HDFS --- src/Storages/HDFS/StorageHDFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index bd50b66ede5..7e836c028a0 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -129,7 +129,7 @@ namespace } else if (is_directory) { - Strings result_part = LSWithFoldedRegexpMatching(fs::path(full_path).append(processed_suffix), + Strings result_part = LSWithFoldedRegexpMatching(fs::path(full_path), fs, last_mod_times, processed_suffix + dir_or_file_name, suffix_with_globs, current_glob, matcher, max_depth - 1, next_slash_after_glob_pos); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); From 2d0152e93d0bb64d077b510886cdb26924c9711a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 12 Jun 2023 13:38:01 +0200 Subject: [PATCH 0142/2047] Fixed issue with named collection --- .../TableFunctionAzureBlobStorage.cpp | 257 +++++++++++------- .../test_cluster.py | 4 +- 2 files changed, 154 insertions(+), 107 deletions(-) diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 9a48bacd3db..4a9e13a9abf 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -60,116 +60,116 @@ void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const if (configuration.format == "auto") configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); } - - if (engine_args.size() < 3 || engine_args.size() > 8) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage Azure requires 3 to 7 arguments: " - "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); - - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); - - std::unordered_map engine_args_to_idx; - - configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); - configuration.is_connection_string = isConnectionString(configuration.connection_url); - - configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); - configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); - - auto is_format_arg = [] (const std::string & s) -> bool + else { - return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); - }; + if (engine_args.size() < 3 || engine_args.size() > 8) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage Azure requires 3 to 7 arguments: " + "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); - if (engine_args.size() == 4) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name/structure"); - if (is_format_arg(fourth_arg)) + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); + + std::unordered_map engine_args_to_idx; + + configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); + configuration.is_connection_string = isConnectionString(configuration.connection_url); + + configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); + configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); + + auto is_format_arg + = [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); }; + + if (engine_args.size() == 4) { - configuration.format = fourth_arg; + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name/structure"); + if (is_format_arg(fourth_arg)) + { + configuration.format = fourth_arg; + } + else + { + configuration.structure = fourth_arg; + } } - else + else if (engine_args.size() == 5) { - configuration.structure = fourth_arg; + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + configuration.format = fourth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); + } + else + { + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + } } + else if (engine_args.size() == 6) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + configuration.format = fourth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); + configuration.structure = checkAndGetLiteralArgument(engine_args[5], "structure"); + } + else + { + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; + } + } + else if (engine_args.size() == 7) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format, compression and structure must be last arguments"); + } + else + { + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); + } + } + else if (engine_args.size() == 8) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); + } + else + { + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); + configuration.structure = checkAndGetLiteralArgument(engine_args[7], "structure"); + } + } + + configuration.blobs_paths = {configuration.blob_path}; + + if (configuration.format == "auto") + configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); } - else if (engine_args.size() == 5) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - } - } - else if (engine_args.size() == 6) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); - configuration.structure = checkAndGetLiteralArgument(engine_args[5], "structure"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); - if (!is_format_arg(sixth_arg)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - configuration.format = sixth_arg; - } - } - else if (engine_args.size() == 7) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format, compression and structure must be last arguments"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); - if (!is_format_arg(sixth_arg)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - configuration.format = sixth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); - } - } - else if (engine_args.size() == 8) - { - - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); - if (!is_format_arg(sixth_arg)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - configuration.format = sixth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); - configuration.structure = checkAndGetLiteralArgument(engine_args[7], "structure"); - } - } - - configuration.blobs_paths = {configuration.blob_path}; - - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); } void TableFunctionAzureBlobStorage::parseArguments(const ASTPtr & ast_function, ContextPtr context) @@ -185,7 +185,6 @@ void TableFunctionAzureBlobStorage::parseArguments(const ASTPtr & ast_function, auto & args = args_func.at(0)->children; parseArgumentsImpl(args, context); - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "CONFIGURATION {}", configuration.connection_url); } @@ -208,16 +207,64 @@ void TableFunctionAzureBlobStorage::addColumnsStructureToArguments(ASTs & args, auto structure_literal = std::make_shared(structure); + auto is_format_arg + = [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); }; + + if (args.size() == 3) { - /// Add format=auto before structure argument. + /// Add format=auto & compression=auto before structure argument. + args.push_back(std::make_shared("auto")); args.push_back(std::make_shared("auto")); args.push_back(structure_literal); } else if (args.size() == 4) { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name/structure"); + if (is_format_arg(fourth_arg)) + { + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + else + { + args.back() = structure_literal; + } + } + else if (args.size() == 5) + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); + if (!is_format_arg(fourth_arg)) + { + /// Add format=auto & compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(std::make_shared("auto")); + } args.push_back(structure_literal); } + else if (args.size() == 6) + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); + if (!is_format_arg(fourth_arg)) + { + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + else + { + args.back() = structure_literal; + } + } + else if (args.size() == 7) + { + args.push_back(structure_literal); + } + else if (args.size() == 8) + { + args.back() = structure_literal; + } } } diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index ca6fd7433f4..e0edbd743fc 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -79,7 +79,7 @@ def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node_0"] azure_query( node, - "INSERT INTO TABLE FUNCTION azure_blob_storage(" + "INSERT INTO TABLE FUNCTION azureBlobStorage(" "'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', " "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', " "'auto', 'key UInt64') VALUES (1), (2)", @@ -89,7 +89,7 @@ def test_simple_write_account_string_table_function(cluster): pure_azure = node.query( """ - SELECT count(*) from azure_blob_storage( + SELECT count(*) from azureBlobStorage( 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64')""" From e9763caa0eb7078cd28e3765d0da1e0a9b4a204b Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 12 Jun 2023 14:21:58 +0000 Subject: [PATCH 0143/2047] fix --- src/Interpreters/Context.cpp | 18 +++++++++++++----- .../01945_system_warnings.reference | 2 +- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a12117b7677..823c3d678df 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -777,17 +777,25 @@ Strings Context::getWarnings() const auto lock = getLock(); common_warnings = shared->warnings; } + String res = "Obsolete settings ["; + size_t obsolete_settings_count = 0; for (const auto & setting : settings) { if (setting.isValueChanged() && setting.isObsolete()) { - common_warnings.emplace_back( - "Obsolete setting `" + setting.getName() - + "` is changed. " - "Check 'select * from system.settings where changed' and read the changelog."); - break; + res += (obsolete_settings_count ? ", `" : "`") + setting.getName() + "`"; + ++obsolete_settings_count; } } + + if (obsolete_settings_count) + { + res = res + "]" + (obsolete_settings_count == 1 ? " is" : " are") + + " changed. " + "Please check 'select * from system.settings where changed and is_obsolete' and read the changelog."; + common_warnings.emplace_back(res); + } + return common_warnings; } diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index d6ae567289c..3e7edacd275 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Obsolete setting `multiple_joins_rewriter_version` is changed. Check \'select * from system.settings where changed\' and read the changelog. +Obsolete settings [`multiple_joins_rewriter_version`] is changed. Check \'select * from system.settings where changed\' and read the changelog. 1 1 From 18f4f1a5238c64f3b45e1d6781ef2c7104ab842d Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 12 Jun 2023 15:11:19 +0000 Subject: [PATCH 0144/2047] udpate test --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- tests/queries/0_stateless/01945_system_warnings.reference | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 3d5b1ca99a5..f0c97acb1f5 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete setting `max_memory_usage_for_all_queries` is changed." +expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed." expect ":) " send -- "q\r" expect eof diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index 3e7edacd275..0c05d5d7049 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Obsolete settings [`multiple_joins_rewriter_version`] is changed. Check \'select * from system.settings where changed\' and read the changelog. +Obsolete settings [`multiple_joins_rewriter_version`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. 1 1 From b76ba13250ad5b0abe728875be0e41667450cd5f Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 12 Jun 2023 15:41:46 +0000 Subject: [PATCH 0145/2047] fix --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index f0c97acb1f5..617e54a375e 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed." +expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." expect ":) " send -- "q\r" expect eof From 2148f29a40f44f387b2cfbd9d3496bf9bc0b7e8d Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 8 Jun 2023 20:29:27 -0400 Subject: [PATCH 0146/2047] More accurate DNS resolve for the keeper connection --- src/Common/ZooKeeper/ZooKeeper.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index a587ad6caf4..e078470476a 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -13,6 +13,7 @@ #include #include #include "Common/ZooKeeper/IKeeper.h" +#include #include #include #include @@ -80,8 +81,12 @@ void ZooKeeper::init(ZooKeeperArgs args_) if (secure) host_string.erase(0, strlen("secure://")); - LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, Poco::Net::SocketAddress{host_string}.toString()); - nodes.emplace_back(Coordination::ZooKeeper::Node{Poco::Net::SocketAddress{host_string}, secure}); + /// We want to resolve all hosts without DNS cache for keeper connection. + Coordination::DNSResolver::instance().removeHostFromCache(host_string); + + auto address = Coordination::DNSResolver::instance().resolveAddress(host_string); + LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, address.toString()); + nodes.emplace_back(Coordination::ZooKeeper::Node{address, secure}); } catch (const Poco::Net::HostNotFoundException & e) { From bbf0548007432dc5482cd28fda4c31e57dd5c24f Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 02:48:28 +0000 Subject: [PATCH 0147/2047] fix test --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 617e54a375e..9a8e22aa26f 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [`max_memory_usage_for_all_queries`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." +expect " * Obsolete settings [\`max_memory_usage_for_all_queries\`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." expect ":) " send -- "q\r" expect eof From 6ad6c6afa3bdf1cd95e1454bad9e7eb75db7b0ab Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 04:13:16 +0000 Subject: [PATCH 0148/2047] fix --- src/Interpreters/Context.cpp | 2 +- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- tests/queries/0_stateless/01945_system_warnings.reference | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 823c3d678df..1b8c52ee06b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -783,7 +783,7 @@ Strings Context::getWarnings() const { if (setting.isValueChanged() && setting.isObsolete()) { - res += (obsolete_settings_count ? ", `" : "`") + setting.getName() + "`"; + res += (obsolete_settings_count ? ", '" : "'") + setting.getName() + "'"; ++obsolete_settings_count; } } diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 9a8e22aa26f..5315c56bde8 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [\`max_memory_usage_for_all_queries\`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." +expect " * Obsolete settings [\'max_memory_usage_for_all_queries\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." expect ":) " send -- "q\r" expect eof diff --git a/tests/queries/0_stateless/01945_system_warnings.reference b/tests/queries/0_stateless/01945_system_warnings.reference index 0c05d5d7049..dcb296c61aa 100644 --- a/tests/queries/0_stateless/01945_system_warnings.reference +++ b/tests/queries/0_stateless/01945_system_warnings.reference @@ -1,5 +1,5 @@ Server was built in debug mode. It will work slowly. 0 -Obsolete settings [`multiple_joins_rewriter_version`] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. +Obsolete settings [\'multiple_joins_rewriter_version\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog. 1 1 From 404bfe773ef726b63e944b70a8b4253907637b8c Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 06:28:47 +0000 Subject: [PATCH 0149/2047] fix --- tests/queries/0_stateless/01945_show_debug_warning.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01945_show_debug_warning.expect b/tests/queries/0_stateless/01945_show_debug_warning.expect index 5315c56bde8..9be0eb6e399 100755 --- a/tests/queries/0_stateless/01945_show_debug_warning.expect +++ b/tests/queries/0_stateless/01945_show_debug_warning.expect @@ -55,7 +55,7 @@ expect eof spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion --max_memory_usage_for_all_queries=123 --history_file=$history_file" expect "Warnings:" -expect " * Obsolete settings [\'max_memory_usage_for_all_queries\'] is changed. Please check \'select * from system.settings where changed and is_obsolete\' and read the changelog." +expect " * Obsolete settings" expect ":) " send -- "q\r" expect eof From e7d1dfb704caa283174823ba8ff59b6c10ae0e1d Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 13 Jun 2023 08:30:07 +0000 Subject: [PATCH 0150/2047] fix --- tests/queries/0_stateless/01945_system_warnings.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index 112baab614e..e44fe0ad6b5 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 8650586131f8cedabb682d4c4191442d8cd30f3e Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 10:51:35 +0200 Subject: [PATCH 0151/2047] Added tests and docs for azureBlobStorageCluster --- .../table-functions/azureBlobStorage.md | 4 +- .../azureBlobStorageCluster.md | 47 +++++ .../TableFunctionAzureBlobStorageCluster.cpp | 7 +- .../configs/cluster.xml | 16 ++ .../test_cluster.py | 163 +++++++++++++++++- 5 files changed, 227 insertions(+), 10 deletions(-) create mode 100644 docs/en/sql-reference/table-functions/azureBlobStorageCluster.md diff --git a/docs/en/sql-reference/table-functions/azureBlobStorage.md b/docs/en/sql-reference/table-functions/azureBlobStorage.md index 369bf7a964d..7a362710b9c 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorage.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorage.md @@ -1,10 +1,10 @@ --- slug: /en/sql-reference/table-functions/azure_blob_storage -sidebar_label: azure_blob_storage +sidebar_label: azureBlobStorage keywords: [azure blob storage] --- -# azure\_blob\_storage Table Function +# azureBlobStorage Table Function Provides a table-like interface to select/insert files in [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs). This table function is similar to the [s3 function](../../sql-reference/table-functions/s3.md). diff --git a/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md b/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md new file mode 100644 index 00000000000..7718be74428 --- /dev/null +++ b/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md @@ -0,0 +1,47 @@ +--- +slug: /en/sql-reference/table-functions/azureBlobStorageCluster +sidebar_position: 55 +sidebar_label: azureBlobStorageCluster +title: "azureBlobStorageCluster Table Function" +--- + +Allows processing files from [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs). in parallel from many nodes in a specified cluster. On initiator it creates a connection to all nodes in the cluster, discloses asterisks in S3 file path, and dispatches each file dynamically. On the worker node it asks the initiator about the next task to process and processes it. This is repeated until all tasks are finished. +This table function is similar to the [s3Cluster function](../../sql-reference/table-functions/s3Cluster.md). + +**Syntax** + +``` sql +azureBlobStorageCluster(cluster_name, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) +``` + +**Arguments** + +- `cluster_name` — Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. +- `connection_string|storage_account_url` — connection_string includes account name & key ([Create connection string](https://learn.microsoft.com/en-us/azure/storage/common/storage-configure-connection-string?toc=%2Fazure%2Fstorage%2Fblobs%2Ftoc.json&bc=%2Fazure%2Fstorage%2Fblobs%2Fbreadcrumb%2Ftoc.json#configure-a-connection-string-for-an-azure-storage-account)) or you could also provide the storage account url here and account name & account key as separate parameters (see parameters account_name & account_key) +- `container_name` - Container name +- `blobpath` - file path. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. +- `account_name` - if storage_account_url is used, then account name can be specified here +- `account_key` - if storage_account_url is used, then account key can be specified here +- `format` — The [format](../../interfaces/formats.md#formats) of the file. +- `compression` — Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. By default, it will autodetect compression by file extension. (same as setting to `auto`). +- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. + +**Returned value** + +A table with the specified structure for reading or writing data in the specified file. + +**Examples** + +Select the count for the file `test_cluster_*.csv`, using all the nodes in the `cluster_simple` cluster: + +``` sql +SELECT count(*) from azureBlobStorageCluster( + 'cluster_simple', 'http://azurite1:10000/devstoreaccount1', 'test_container', 'test_cluster_count.csv', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', + 'auto', 'key UInt64') +``` + +**See Also** + +- [AzureBlobStorage engine](../../engines/table-engines/integrations/azureBlobStorage.md) +- [azureBlobStorage table function](../../sql-reference/table-functions/azureBlobStorage.md) diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp index b3b18c7ec0d..e4c5d25492b 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp @@ -71,7 +71,12 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( void registerTableFunctionAzureBlobStorageCluster(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction( + {.documentation + = {.description=R"(The table function can be used to read the data stored on Azure Blob Storage in parallel for many nodes in a specified cluster.)", + .examples{{"azureBlobStorageCluster", "SELECT * FROM azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])", ""}}}, + .allow_readonly = false} + ); } diff --git a/tests/integration/test_storage_azure_blob_storage/configs/cluster.xml b/tests/integration/test_storage_azure_blob_storage/configs/cluster.xml index 43df7b46f3f..ffa4673c9ee 100644 --- a/tests/integration/test_storage_azure_blob_storage/configs/cluster.xml +++ b/tests/integration/test_storage_azure_blob_storage/configs/cluster.xml @@ -16,6 +16,22 @@ + + + + + node_0 + 9000 + + + + + node_1 + 19000 + + + + simple_cluster diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index e0edbd743fc..953d7bca14c 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -75,22 +75,51 @@ def get_azure_file_content(filename): return download_stream.readall().decode("utf-8") -def test_simple_write_account_string_table_function(cluster): +def test_select_all(cluster): node = cluster.instances["node_0"] azure_query( node, "INSERT INTO TABLE FUNCTION azureBlobStorage(" - "'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', " + "'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1', " + "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', " + "'auto', 'key UInt64, data String') VALUES (1, 'a'), (2, 'b')", + ) + print(get_azure_file_content("test_cluster_select_all.csv")) + + pure_azure = node.query( + """ + SELECT * from azureBlobStorage( + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', + 'auto')""" + ) + print(pure_azure) + distributed_azure = node.query( + """ + SELECT * from azureBlobStorageCluster( + 'simple_cluster', 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', + 'auto')""" + ) + print(distributed_azure) + assert TSV(pure_azure) == TSV(distributed_azure) + + +def test_count(cluster): + node = cluster.instances["node_0"] + azure_query( + node, + "INSERT INTO TABLE FUNCTION azureBlobStorage(" + "'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', " "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', " "'auto', 'key UInt64') VALUES (1), (2)", ) - print(get_azure_file_content("test_simple_write_tf.csv")) - # assert get_azure_file_content("test_simple_write_tf.csv") == '1,"a"\n' + print(get_azure_file_content("test_cluster_count.csv")) pure_azure = node.query( """ SELECT count(*) from azureBlobStorage( - 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64')""" ) @@ -98,10 +127,130 @@ def test_simple_write_account_string_table_function(cluster): distributed_azure = node.query( """ SELECT count(*) from azureBlobStorageCluster( - 'simple_cluster', 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_simple_write_tf.csv', 'devstoreaccount1', + 'simple_cluster', 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', 'key UInt64')""" ) print(distributed_azure) - assert TSV(pure_azure) == TSV(distributed_azure) + + +def test_union_all(cluster): + node = cluster.instances["node_0"] + azure_query( + node, + "INSERT INTO TABLE FUNCTION azureBlobStorage(" + "'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', " + "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', " + "'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", + ) + + pure_azure = node.query( + """ + SELECT * FROM + ( + SELECT * from azureBlobStorage( + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', + 'auto', 'a Int32, b String') + UNION ALL + SELECT * from azureBlobStorage( + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', + 'auto', 'a Int32, b String') + ) + ORDER BY (a) + """ + ) + azure_distributed = node.query( + """ + SELECT * FROM + ( + SELECT * from azureBlobStorageCluster( + 'simple_cluster', + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', + 'auto', 'a Int32, b String') + UNION ALL + SELECT * from azureBlobStorageCluster( + 'simple_cluster', + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', + 'auto', 'a Int32, b String') + ) + ORDER BY (a) + """ + ) + + assert TSV(pure_azure) == TSV(azure_distributed) + +def test_skip_unavailable_shards(cluster): + node = cluster.instances["node_0"] + azure_query( + node, + "INSERT INTO TABLE FUNCTION azureBlobStorage(" + "'http://azurite1:10000/devstoreaccount1', 'cont', 'test_skip_unavailable.csv', 'devstoreaccount1', " + "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " + "'auto', 'a UInt64') VALUES (1), (2)", + ) + result = node.query( + """ + SELECT count(*) from azureBlobStorageCluster( + 'cluster_non_existent_port', + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_skip_unavailable.csv', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') + SETTINGS skip_unavailable_shards = 1 + """ + ) + + assert result == "2\n" + + +def test_unset_skip_unavailable_shards(cluster): + # Although skip_unavailable_shards is not set, cluster table functions should always skip unavailable shards. + node = cluster.instances["node_0"] + azure_query( + node, + "INSERT INTO TABLE FUNCTION azureBlobStorage(" + "'http://azurite1:10000/devstoreaccount1', 'cont', 'test_unset_skip_unavailable.csv', 'devstoreaccount1', " + "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " + "'auto', 'a UInt64') VALUES (1), (2)", + ) + result = node.query( + """ + SELECT count(*) from azureBlobStorageCluster( + 'cluster_non_existent_port', + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_skip_unavailable.csv', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') + """ + ) + + assert result == "2\n" + +def test_cluster_with_named_collection(cluster): + node = cluster.instances["node_0"] + + azure_query( + node, + "INSERT INTO TABLE FUNCTION azureBlobStorage(" + "'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_with_named_collection.csv', 'devstoreaccount1', " + "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " + "'auto', 'a UInt64') VALUES (1), (2)", + ) + + pure_azure = node.query( + """ + SELECT * from azureBlobStorage( + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_with_named_collection.csv', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') + """ + ) + + azure_cluster = node.query( + """ + SELECT * from azureBlobStorageCluster( + 'simple_cluster', azure_conf2, container='cont', blob_path='test_cluster_with_named_collection.csv') + """ + ) + + assert TSV(pure_azure) == TSV(azure_cluster) \ No newline at end of file From 3c5fb65d6dce01136c9a27ad8c265daedc840c5b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 13 Jun 2023 09:10:04 +0000 Subject: [PATCH 0152/2047] Automatic style fix --- .../test_storage_azure_blob_storage/test_cluster.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index 953d7bca14c..c42aac35071 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -184,6 +184,7 @@ def test_union_all(cluster): assert TSV(pure_azure) == TSV(azure_distributed) + def test_skip_unavailable_shards(cluster): node = cluster.instances["node_0"] azure_query( @@ -227,6 +228,7 @@ def test_unset_skip_unavailable_shards(cluster): assert result == "2\n" + def test_cluster_with_named_collection(cluster): node = cluster.instances["node_0"] @@ -253,4 +255,4 @@ def test_cluster_with_named_collection(cluster): """ ) - assert TSV(pure_azure) == TSV(azure_cluster) \ No newline at end of file + assert TSV(pure_azure) == TSV(azure_cluster) From 9e8ca5f5ae8c4bde3e7aab7854a59c2f1d85e472 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 13 Jun 2023 13:09:39 +0000 Subject: [PATCH 0153/2047] Add integration test check with enabled analyzer --- .github/workflows/pull_request.yml | 210 ++++++++++++++++++ tests/ci/integration_test_check.py | 2 + .../helpers/0_common_enable_analyzer.xml | 7 + tests/integration/helpers/cluster.py | 2 + 4 files changed, 221 insertions(+) create mode 100644 tests/integration/helpers/0_common_enable_analyzer.xml diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index afc08f3e637..9a39b1177cf 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3861,6 +3861,216 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan0: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan1: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan2: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan3: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan4: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=4 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan5: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=5 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" IntegrationTestsTsan0: needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 8ef6244a1c5..523b1cfaab5 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -226,6 +226,8 @@ def main(): download_all_deb_packages(check_name, reports_path, build_path) my_env = get_env_for_runner(build_path, repo_path, result_path, work_path) + if "analyzer" in check_name.lower(): + my_env["USE_NEW_ANALYZER"] = "1" json_path = os.path.join(work_path, "params.json") with open(json_path, "w", encoding="utf-8") as json_params: diff --git a/tests/integration/helpers/0_common_enable_analyzer.xml b/tests/integration/helpers/0_common_enable_analyzer.xml new file mode 100644 index 00000000000..aa374364ef0 --- /dev/null +++ b/tests/integration/helpers/0_common_enable_analyzer.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f57ebf40e54..6d66a539cdc 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4187,6 +4187,8 @@ class ClickHouseInstance: ) write_embedded_config("0_common_instance_users.xml", users_d_dir) + if os.environ.get('USE_NEW_ANALYZER') is not None: + write_embedded_config("0_common_enable_analyzer.xml", users_d_dir) if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) From 5d541332482ae8fc37bacd4f8db340b599779c92 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 13 Jun 2023 13:31:14 +0000 Subject: [PATCH 0154/2047] Automatic style fix --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6d66a539cdc..8c54a37cf60 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4187,7 +4187,7 @@ class ClickHouseInstance: ) write_embedded_config("0_common_instance_users.xml", users_d_dir) - if os.environ.get('USE_NEW_ANALYZER') is not None: + if os.environ.get("USE_NEW_ANALYZER") is not None: write_embedded_config("0_common_enable_analyzer.xml", users_d_dir) if len(self.custom_dictionaries_paths): From ed4455e2915968c664a2498f6d9503f358e6109e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 13 Jun 2023 15:44:30 +0200 Subject: [PATCH 0155/2047] Update tests/integration/helpers/cluster.py From 1d8013074de7eb5a35460fef94bc82b8872531c3 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 13 Jun 2023 18:47:02 +0200 Subject: [PATCH 0156/2047] fix style & black --- src/Storages/HDFS/StorageHDFS.cpp | 6 ++++-- src/Storages/StorageFile.cpp | 6 ++++-- tests/integration/test_storage_hdfs/test.py | 13 ++++++++++--- 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 7e836c028a0..2503b0e60f2 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -154,12 +154,14 @@ namespace const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' size_t slashes_in_glob = 0; - const size_t next_slash_after_glob_pos = [&](){ + const size_t next_slash_after_glob_pos = [&]() + { if (!has_glob) return suffix_with_globs.find('/', 1); size_t in_curly = 0; - for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) + { if (*it == '{') ++in_curly; else if (*it == '/') diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 93228cf4d39..dc7cfe0de29 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -170,12 +170,14 @@ void listFilesWithRegexpMatchingImpl( /// slashes_in_glob counter is a upper-bound estimate of recursion depth /// needed to process complex cases when `/` is included into glob, e.g. /pa{th1/a,th2/b}.csv size_t slashes_in_glob = 0; - const size_t next_slash_after_glob_pos = [&](){ + const size_t next_slash_after_glob_pos = [&]() + { if (!has_glob) return suffix_with_globs.find('/', 1); size_t in_curly = 0; - for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) { + for (std::string::const_iterator it = ++suffix_with_globs.begin(); it != suffix_with_globs.end(); it++) + { if (*it == '{') ++in_curly; else if (*it == '/') diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index dde3dd257b1..789e8e11035 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -88,10 +88,17 @@ def test_read_write_storage_with_globs(started_cluster): def test_storage_with_multidirectory_glob(started_cluster): hdfs_api = started_cluster.hdfs_api for i in ["1", "2"]: - hdfs_api.write_data(f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n") - assert hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") == f"File{i}\t{i}{i}\n" + hdfs_api.write_data( + f"/multiglob/p{i}/path{i}/postfix/data{i}", f"File{i}\t{i}{i}\n" + ) + assert ( + hdfs_api.read_data(f"/multiglob/p{i}/path{i}/postfix/data{i}") + == f"File{i}\t{i}{i}\n" + ) - r = node1.query("SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)") + r = node1.query( + "SELECT * FROM hdfs('hdfs://hdfs1:9000/multiglob/{p1/path1,p2/path2}/postfix/data{1,2}', TSV)" + ) assert (r == f"File1\t11\nFile2\t22\n") or (r == f"File2\t22\nFile1\t11\n") From 906db0318dee9d08a8af603ab0400143578e4f3d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Jun 2023 19:45:43 +0000 Subject: [PATCH 0157/2047] Remove AST based optimization duplicate_order_by_and_distinct It was quite some time ago since it was replaced by plan level optimizations: - query_plan_remove_redundant_sorting - query_plan_remove_redundant_distinct --- src/Core/Settings.h | 2 +- src/Interpreters/TreeOptimizer.cpp | 53 -------- ..._duplicate_order_by_and_distinct.reference | 58 --------- .../01305_duplicate_order_by_and_distinct.sql | 123 ------------------ ...t_optimize_for_distributed_table.reference | 4 - ...istinct_optimize_for_distributed_table.sql | 46 ------- 6 files changed, 1 insertion(+), 285 deletions(-) delete mode 100644 tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference delete mode 100644 tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.sql delete mode 100644 tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.reference delete mode 100644 tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3a23127e2fd..c53bed2007a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -526,7 +526,6 @@ class IColumn; M(Bool, convert_query_to_cnf, false, "Convert SELECT query to CNF", 0) \ M(Bool, optimize_or_like_chain, false, "Optimize multiple OR LIKE into multiMatchAny. This optimization should not be enabled by default, because it defies index analysis in some cases.", 0) \ M(Bool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \ - M(Bool, optimize_duplicate_order_by_and_distinct, false, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \ M(Bool, optimize_redundant_functions_in_order_by, true, "Remove functions from ORDER BY if its argument is also in ORDER BY", 0) \ M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ M(Bool, optimize_multiif_to_if, true, "Replace 'multiIf' with only one condition to 'if'.", 0) \ @@ -818,6 +817,7 @@ class IColumn; MAKE_OBSOLETE(M, Seconds, drain_timeout, 3) \ MAKE_OBSOLETE(M, UInt64, backup_threads, 16) \ MAKE_OBSOLETE(M, UInt64, restore_threads, 16) \ + MAKE_OBSOLETE(M, Bool, optimize_duplicate_order_by_and_distinct, false) \ /** The section above is for obsolete settings. Do not add anything there. */ diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index c38b3c79026..b6b45c664f9 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -288,13 +288,6 @@ void optimizeDuplicatesInOrderBy(const ASTSelectQuery * select_query) elems = std::move(unique_elems); } -/// Optimize duplicate ORDER BY -void optimizeDuplicateOrderBy(ASTPtr & query, ContextPtr context) -{ - DuplicateOrderByVisitor::Data order_by_data{context}; - DuplicateOrderByVisitor(order_by_data).visit(query); -} - /// Return simple subselect (without UNIONs or JOINs or SETTINGS) if any const ASTSelectQuery * getSimpleSubselect(const ASTSelectQuery & select) { @@ -378,41 +371,6 @@ std::unordered_set getDistinctNames(const ASTSelectQuery & select) return names; } -/// Remove DISTINCT from query if columns are known as DISTINCT from subquery -void optimizeDuplicateDistinct(ASTSelectQuery & select) -{ - if (!select.select() || select.select()->children.empty()) - return; - - const ASTSelectQuery * subselect = getSimpleSubselect(select); - if (!subselect) - return; - - std::unordered_set distinct_names = getDistinctNames(*subselect); - std::unordered_set selected_names; - - /// Check source column names from select list (ignore aliases and table names) - for (const auto & id : select.select()->children) - { - const auto * identifier = id->as(); - if (!identifier) - return; - - const String & name = identifier->shortName(); - if (!distinct_names.contains(name)) - return; /// Not a distinct column, keep DISTINCT for it. - - selected_names.emplace(name); - } - - /// select columns list != distinct columns list - /// SELECT DISTINCT a FROM (SELECT DISTINCT a, b FROM ...)) -- cannot remove DISTINCT - if (selected_names.size() != distinct_names.size()) - return; - - select.distinct = false; -} - /// Replace monotonous functions in ORDER BY if they don't participate in GROUP BY expression, /// has a single argument and not an aggregate functions. void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, ContextPtr context, @@ -811,17 +769,6 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, && !select_query->group_by_with_cube) optimizeAggregateFunctionsOfGroupByKeys(select_query, query); - /// Remove duplicate ORDER BY and DISTINCT from subqueries. - if (settings.optimize_duplicate_order_by_and_distinct) - { - optimizeDuplicateOrderBy(query, context); - - /// DISTINCT has special meaning in Distributed query with enabled distributed_group_by_no_merge - /// TODO: disable Distributed/remote() tables only - if (!settings.distributed_group_by_no_merge) - optimizeDuplicateDistinct(*select_query); - } - /// Remove functions from ORDER BY if its argument is also in ORDER BY if (settings.optimize_redundant_functions_in_order_by) optimizeRedundantFunctionsInOrderBy(select_query, context); diff --git a/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference b/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference deleted file mode 100644 index 10f8bbfd392..00000000000 --- a/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference +++ /dev/null @@ -1,58 +0,0 @@ -SELECT number -FROM -( - SELECT number - FROM - ( - SELECT DISTINCT number - FROM numbers(3) - ) -) -ORDER BY number ASC -0 -1 -2 -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM - ( - SELECT DISTINCT number - FROM numbers(3) - ORDER BY number ASC - ) - ORDER BY number ASC -) -ORDER BY number ASC -0 -1 -2 -SELECT number -FROM -( - SELECT number - FROM - ( - SELECT DISTINCT number % 2 AS number - FROM numbers(3) - ) -) -ORDER BY number ASC -0 -1 -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM - ( - SELECT DISTINCT number % 2 AS number - FROM numbers(3) - ORDER BY number ASC - ) - ORDER BY number ASC -) -ORDER BY number ASC -0 -1 diff --git a/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.sql b/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.sql deleted file mode 100644 index 3b13b208eb5..00000000000 --- a/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.sql +++ /dev/null @@ -1,123 +0,0 @@ -set optimize_duplicate_order_by_and_distinct = 1; - -EXPLAIN SYNTAX SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT * - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT * - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -set optimize_duplicate_order_by_and_distinct = 0; - -EXPLAIN SYNTAX SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT * - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT * - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -set optimize_duplicate_order_by_and_distinct = 1; - -EXPLAIN SYNTAX SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT number % 2 - AS number - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT number % 2 - AS number - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -set optimize_duplicate_order_by_and_distinct = 0; - -EXPLAIN SYNTAX SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT number % 2 - AS number - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; - -SELECT DISTINCT * -FROM -( - SELECT DISTINCT * - FROM - ( - SELECT DISTINCT number % 2 - AS number - FROM numbers(3) - ORDER BY number - ) - ORDER BY number -) -ORDER BY number; diff --git a/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.reference b/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.reference deleted file mode 100644 index 44e0be8e356..00000000000 --- a/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.reference +++ /dev/null @@ -1,4 +0,0 @@ -0 -0 -0 -0 diff --git a/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.sql b/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.sql deleted file mode 100644 index 8ef1273c855..00000000000 --- a/tests/queries/0_stateless/01306_disable_duplicate_order_by_and_distinct_optimize_for_distributed_table.sql +++ /dev/null @@ -1,46 +0,0 @@ --- Tags: distributed - -set query_plan_remove_redundant_distinct = 1; -set optimize_duplicate_order_by_and_distinct = 0; -SET distributed_group_by_no_merge = 0; - -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM remote('127.0.0.{1,2}', system.numbers) - LIMIT 1 - SETTINGS distributed_group_by_no_merge = 1 -); - -SET distributed_group_by_no_merge = 1; - -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM remote('127.0.0.{1,2}', system.numbers) - LIMIT 1 -); - -set optimize_duplicate_order_by_and_distinct = 0; -SET distributed_group_by_no_merge = 0; - -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM remote('127.0.0.{1,2}', system.numbers) - LIMIT 1 - SETTINGS distributed_group_by_no_merge = 1 -); - -SET distributed_group_by_no_merge = 1; -set optimize_duplicate_order_by_and_distinct = 0; -SELECT DISTINCT number -FROM -( - SELECT DISTINCT number - FROM remote('127.0.0.{1,2}', system.numbers) - LIMIT 1 -); From 9652b38a6cd51f8c0f5a65cc70108d126589c793 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 13 Jun 2023 23:10:03 +0000 Subject: [PATCH 0158/2047] Fix CHECK_NAME --- .github/workflows/pull_request.yml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 9a39b1177cf..59beddac8d5 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3870,7 +3870,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan) + CHECK_NAME=Integration tests (asan, analyzer) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=0 RUN_BY_HASH_TOTAL=6 @@ -3905,7 +3905,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan) + CHECK_NAME=Integration tests (asan, analyzer) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=1 RUN_BY_HASH_TOTAL=6 @@ -3940,7 +3940,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan) + CHECK_NAME=Integration tests (asan, analyzer) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=2 RUN_BY_HASH_TOTAL=6 @@ -3975,7 +3975,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan) + CHECK_NAME=Integration tests (asan, analyzer) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=3 RUN_BY_HASH_TOTAL=6 @@ -4010,7 +4010,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan) + CHECK_NAME=Integration tests (asan, analyzer) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=4 RUN_BY_HASH_TOTAL=6 @@ -4045,7 +4045,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/integration_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan) + CHECK_NAME=Integration tests (asan, analyzer) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse RUN_BY_HASH_NUM=5 RUN_BY_HASH_TOTAL=6 From ed318d10353101c76a4493ccd9fa6c239868abd3 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 14 Jun 2023 10:35:36 +0000 Subject: [PATCH 0159/2047] Add input_format_csv_ignore_extra_columns setting (prototype) --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 15 ++++++++++++++- tests/queries/0_stateless/00301_csv.reference | 4 ++++ tests/queries/0_stateless/00301_csv.sh | 10 ++++++++++ 6 files changed, 31 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bc879b9bdf6..d38f7767252 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -835,6 +835,7 @@ class IColumn; M(Bool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \ M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, -WithNames, -WithNamesAndTypes formats).", IMPORTANT) \ M(Bool, input_format_csv_empty_as_default, true, "Treat empty fields in CSV input as default values.", 0) \ + M(Bool, input_format_csv_ignore_extra_columns, false, "", 0) \ M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ M(Bool, input_format_tsv_enum_as_number, false, "Treat inserted enum values in TSV formats as enum indices.", 0) \ M(Bool, input_format_null_as_default, true, "Initialize null fields with default values if the data type of this field is not nullable and it is supported by the input format", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index c235afae57e..0218d268c51 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -63,6 +63,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.delimiter = settings.format_csv_delimiter; format_settings.csv.tuple_delimiter = settings.format_csv_delimiter; format_settings.csv.empty_as_default = settings.input_format_csv_empty_as_default; + format_settings.csv.ignore_extra_columns = settings.input_format_csv_ignore_extra_columns; format_settings.csv.enum_as_number = settings.input_format_csv_enum_as_number; format_settings.csv.null_representation = settings.format_csv_null_representation; format_settings.csv.arrays_as_nested_csv = settings.input_format_csv_arrays_as_nested_csv; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 787c1a64759..3bc53140fe5 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -128,6 +128,7 @@ struct FormatSettings bool allow_single_quotes = true; bool allow_double_quotes = true; bool empty_as_default = false; + bool ignore_extra_columns = false; bool crlf_end_of_line = false; bool enum_as_number = false; bool arrays_as_nested_csv = false; diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index ae75240e0ee..0cc5889b732 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -302,14 +302,27 @@ bool CSVFormatReader::readField( return false; } + auto skip_all = [&]() + { + if (!is_last_file_column || !format_settings.csv.ignore_extra_columns) + { + return; + } + //std::cout << "skip !!!" << std::endl; + buf->position() = find_first_symbols<'\n'>(buf->position(), buf->buffer().end()); + }; if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) { /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); + bool res = SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); + skip_all(); + return res; } /// Read the column normally. serialization->deserializeTextCSV(column, *buf, format_settings); + + skip_all(); return true; } diff --git a/tests/queries/0_stateless/00301_csv.reference b/tests/queries/0_stateless/00301_csv.reference index 9863da4b640..61279f3b84a 100644 --- a/tests/queries/0_stateless/00301_csv.reference +++ b/tests/queries/0_stateless/00301_csv.reference @@ -11,3 +11,7 @@ default-eof 1 2019-06-19 2016-01-01 01:02:03 NUL 2016-01-02 01:02:03 Nhello \N \N +Hello world 1 2016-01-01 +Hello world 2 2016-01-02 +Hello world 3 2016-01-03 +Hello world 4 2016-01-04 diff --git a/tests/queries/0_stateless/00301_csv.sh b/tests/queries/0_stateless/00301_csv.sh index b2618343dc0..e99c39a0f6f 100755 --- a/tests/queries/0_stateless/00301_csv.sh +++ b/tests/queries/0_stateless/00301_csv.sh @@ -37,3 +37,13 @@ echo 'NULL, NULL $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s NULLS LAST"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; + + +$CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64 DEFAULT 1, d Date DEFAULT '2019-06-19') ENGINE = Memory"; + +echo 'Hello world, 1, 2016-01-01 +Hello world, 2 ,2016-01-02, +Hello world, 3 ,2016-01-03, 2016-01-13 +Hello world, 4 ,2016-01-04, 2016-01-14, 2016-01-15' | $CLICKHOUSE_CLIENT --input_format_csv_empty_as_default=1 --input_format_csv_ignore_extra_columns=1 --query="INSERT INTO csv FORMAT CSV"; +$CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s, n"; +$CLICKHOUSE_CLIENT --query="DROP TABLE csv"; \ No newline at end of file From a4e982442f4a3d6b3007b432f8e0b6211e9aa4e7 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 14 Jun 2023 11:13:59 +0000 Subject: [PATCH 0160/2047] Update documentation --- docs/en/operations/configuration-files.md | 36 +++++++++++++++++++++++ docs/ru/operations/configuration-files.md | 36 +++++++++++++++++++++++ 2 files changed, 72 insertions(+) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index b3583e156ad..b5d52acca49 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -54,6 +54,42 @@ XML substitution example: Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. +## Decryption {#decryption} + +Elements with text nodes may be encrypted with [encryption codecs](../../sql-reference/statements/create/table.md#encryption-codecs). In this case `` section should be included in configuration file and each element node with encrypted text should have `encryption_codec` attribute with name of codec. + +Example: + +```xml + + + + 00112233445566778899aabbccddeeff + + + + admin + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + + +``` + +To get the encrypted value `encrypt_decrypt` example application may be used. + +Example: + +``` bash +./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV abcd +``` + +``` text +961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 +``` + +:::note +The decryption is executed after creation of preprocessed configuration file. It means that elements with `encryption_codec` attribute in the preprocessed configuration file are encrypted. But the values of corresponding parameters in server's memory are decrypted. +::: + ## User Settings {#user-settings} The `config.xml` file can specify a separate config with user settings, profiles, and quotas. The relative path to this config is set in the `users_config` element. By default, it is `users.xml`. If `users_config` is omitted, the user settings, profiles, and quotas are specified directly in `config.xml`. diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index 2b824ce91bd..96512fbbe23 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -85,6 +85,42 @@ $ cat /etc/clickhouse-server/users.d/alice.xml Сервер следит за изменениями конфигурационных файлов, а также файлов и ZooKeeper-узлов, которые были использованы при выполнении подстановок и переопределений, и перезагружает настройки пользователей и кластеров на лету. То есть, можно изменять кластера, пользователей и их настройки без перезапуска сервера. +## Расшифровка {#decryption} + +Элементы с текстовыми узлами могут быть зашифрован с помощью [кодеков шифрования](../../sql-reference/statements/create/table.md#encryption-codecs). В этом случае секция `` должна быть включена в конфигурационный файл и каждый элемент с зашифрованным текстом должен иметь аттрибут `encryption_codec` с именем кодека. + +Пример: + +```xml + + + + 00112233445566778899aabbccddeeff + + + + admin + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + + +``` + +Чтобы получить зашифрованное значение может быть использовано приложение-пример `encrypt_decrypt` . + +Пример: + +``` bash +./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV abcd +``` + +``` text +961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 +``` + +:::note +Расшифровка выполняется после создания конфигурационного файла предобработки. Это означает что элементы с аттрибутом `encryption_codec` в конфигурационном файле предобработки зашифрованы. Но значения соответствующих параметров в памяти сервера расшифрованы. +::: + ## Примеры записи конфигурации на YAML {#example} Здесь можно рассмотреть пример реальной конфигурации записанной на YAML: [config.yaml.example](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/config.yaml.example). From b5d4ad583f3741f87843f51c56ccc41b91833523 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 14 Jun 2023 11:35:55 +0000 Subject: [PATCH 0161/2047] Small code style improvements --- src/Common/Config/ConfigProcessor.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index df25a9a3825..9548bf33b7b 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -26,9 +26,9 @@ #include #include #include +#include #include #include -#include #include #define PREPROCESSED_SUFFIX "-preprocessed" @@ -194,7 +194,7 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const DB::Memory<> memory; memory.resize(codec.getCompressedReserveSize(static_cast(value.size()))); auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory.data()); - std::string encrypted_value = std::string(memory.data(), bytes_written); + auto encrypted_value = std::string(memory.data(), bytes_written); std::string hex_value; boost::algorithm::hex(encrypted_value.begin(), encrypted_value.end(), std::back_inserter(hex_value)); return hex_value; @@ -224,7 +224,7 @@ std::string ConfigProcessor::decryptValue(const std::string & codec_name, const void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) { - for (Node * node = config_root->firstChild(); node;) + for (Node * node = config_root->firstChild(); node; node = node->nextSibling()) { if (node->nodeType() == Node::ELEMENT_NODE) { @@ -244,7 +244,6 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) } decryptRecursive(node); } - node = node->nextSibling(); } } From f55623aa2d23fda63f2b19720f4035568a4595a4 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 14 Jun 2023 11:46:43 +0000 Subject: [PATCH 0162/2047] Use anonymous namespace for getEncryptionMethod() --- src/Common/Config/ConfigProcessor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 9548bf33b7b..17abc3d161d 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -47,6 +47,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ + /// Get method for string name. Throw exception for wrong name EncryptionMethod getEncryptionMethod(const std::string & name) { @@ -58,6 +61,8 @@ EncryptionMethod getEncryptionMethod(const std::string & name) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", name); } +} + /// For cutting preprocessed path to this base static std::string main_config_path; From afcc3aca363ff6cee0cb7f2417b711e08854d96c Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 14 Jun 2023 13:14:59 +0000 Subject: [PATCH 0163/2047] Update ci_config.py --- tests/ci/ci_config.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index d829115cfe1..36bca9d741d 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -325,6 +325,9 @@ CI_CONFIG = { "Integration tests (asan)": { "required_build": "package_asan", }, + "Integration tests (asan, analyzer)": { + "required_build": "package_asan", + }, "Integration tests (tsan)": { "required_build": "package_tsan", }, From fe8172fbd9c58fadc5c0523c69e5adce05887dd2 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 14 Jun 2023 13:17:04 +0000 Subject: [PATCH 0164/2047] Review fixes --- tests/ci/integration_test_check.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 523b1cfaab5..0d483c08456 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -71,7 +71,7 @@ def get_json_params_dict( } -def get_env_for_runner(build_path, repo_path, result_path, work_path): +def get_env_for_runner(check_name, build_path, repo_path, result_path, work_path): binary_path = os.path.join(build_path, "clickhouse") odbc_bridge_path = os.path.join(build_path, "clickhouse-odbc-bridge") library_bridge_path = os.path.join(build_path, "clickhouse-library-bridge") @@ -88,6 +88,9 @@ def get_env_for_runner(build_path, repo_path, result_path, work_path): my_env["CLICKHOUSE_TESTS_JSON_PARAMS_PATH"] = os.path.join(work_path, "params.json") my_env["CLICKHOUSE_TESTS_RUNNER_RESTART_DOCKER"] = "0" + if "analyzer" in check_name.lower(): + my_env["USE_NEW_ANALYZER"] = "1" + return my_env @@ -225,9 +228,7 @@ def main(): else: download_all_deb_packages(check_name, reports_path, build_path) - my_env = get_env_for_runner(build_path, repo_path, result_path, work_path) - if "analyzer" in check_name.lower(): - my_env["USE_NEW_ANALYZER"] = "1" + my_env = get_env_for_runner(check_name, build_path, repo_path, result_path, work_path) json_path = os.path.join(work_path, "params.json") with open(json_path, "w", encoding="utf-8") as json_params: From 14dfebba49543378b80716cffb5aaea7dcc7fbf7 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 14 Jun 2023 13:35:11 +0000 Subject: [PATCH 0165/2047] Fix links in MD --- docs/en/operations/configuration-files.md | 2 +- docs/ru/operations/configuration-files.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index b5d52acca49..71d5885058a 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -56,7 +56,7 @@ Substitutions can also be performed from ZooKeeper. To do this, specify the attr ## Decryption {#decryption} -Elements with text nodes may be encrypted with [encryption codecs](../../sql-reference/statements/create/table.md#encryption-codecs). In this case `` section should be included in configuration file and each element node with encrypted text should have `encryption_codec` attribute with name of codec. +Elements with text nodes may be encrypted with [encryption codecs](../sql-reference/statements/create/table.md#encryption-codecs). In this case `` section should be included in configuration file and each element node with encrypted text should have `encryption_codec` attribute with name of codec. Example: diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index 96512fbbe23..df50d900919 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -87,7 +87,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ## Расшифровка {#decryption} -Элементы с текстовыми узлами могут быть зашифрован с помощью [кодеков шифрования](../../sql-reference/statements/create/table.md#encryption-codecs). В этом случае секция `` должна быть включена в конфигурационный файл и каждый элемент с зашифрованным текстом должен иметь аттрибут `encryption_codec` с именем кодека. +Элементы с текстовыми узлами могут быть зашифрован с помощью [кодеков шифрования](../sql-reference/statements/create/table.md#create-query-encryption-codecs). В этом случае секция `` должна быть включена в конфигурационный файл и каждый элемент с зашифрованным текстом должен иметь аттрибут `encryption_codec` с именем кодека. Пример: From 1230519bec047857d7fb9b1edd6baec1a7be8e6a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 14 Jun 2023 13:38:44 +0000 Subject: [PATCH 0166/2047] Automatic style fix --- tests/ci/integration_test_check.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 0d483c08456..843bbc8b3ee 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -228,7 +228,9 @@ def main(): else: download_all_deb_packages(check_name, reports_path, build_path) - my_env = get_env_for_runner(check_name, build_path, repo_path, result_path, work_path) + my_env = get_env_for_runner( + check_name, build_path, repo_path, result_path, work_path + ) json_path = os.path.join(work_path, "params.json") with open(json_path, "w", encoding="utf-8") as json_params: From 3d64cf4423b9fb4b935786eca392875d3b66c17c Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 14 Jun 2023 15:40:32 +0000 Subject: [PATCH 0167/2047] Add dbms in cmake --- src/Common/Config/CMakeLists.txt | 2 ++ utils/config-processor/CMakeLists.txt | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/Config/CMakeLists.txt b/src/Common/Config/CMakeLists.txt index ec7bdd10196..fdcba5d4a4b 100644 --- a/src/Common/Config/CMakeLists.txt +++ b/src/Common/Config/CMakeLists.txt @@ -15,6 +15,7 @@ target_link_libraries(clickhouse_common_config Poco::XML PRIVATE string_utils + dbms ) add_library(clickhouse_common_config_no_zookeeper_log ${SRCS}) @@ -25,6 +26,7 @@ target_link_libraries(clickhouse_common_config_no_zookeeper_log Poco::XML PRIVATE string_utils + dbms ) if (TARGET ch_contrib::yaml_cpp) diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt index 80c3535ef4e..53b6163ba87 100644 --- a/utils/config-processor/CMakeLists.txt +++ b/utils/config-processor/CMakeLists.txt @@ -1,2 +1,2 @@ clickhouse_add_executable (config-processor config-processor.cpp) -target_link_libraries(config-processor PRIVATE dbms) +target_link_libraries(config-processor PRIVATE clickhouse_common_config_no_zookeeper_log) From a91fc3ddb33865d2db8170ff96e636de293b323a Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 14 Jun 2023 16:44:31 +0000 Subject: [PATCH 0168/2047] Add docs/ add more cases in test --- docs/en/interfaces/formats.md | 3 +- .../operations/settings/settings-formats.md | 5 +++ docs/ru/interfaces/formats.md | 4 +- docs/ru/operations/settings/settings.md | 8 +++- src/Core/Settings.h | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- .../Formats/Impl/CSVRowInputFormat.cpp | 39 +++++++++---------- .../RowInputFormatWithNamesAndTypes.cpp | 4 ++ tests/queries/0_stateless/00301_csv.reference | 10 +++-- tests/queries/0_stateless/00301_csv.sh | 13 ++++--- 11 files changed, 56 insertions(+), 36 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 324930e248f..950692deb77 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -470,6 +470,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_csv_detect_header) - automatically detect header with names and types in CSV format. Default value - `true`. - [input_format_csv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_trailing_empty_lines) - skip trailing empty lines at the end of data. Default value - `false`. - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. +- [input_format_csv_ignore_extra_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_ignore_extra_columns) - ignore extra colums in CSV input. Default value - `false`. ## CSVWithNames {#csvwithnames} @@ -2062,7 +2063,7 @@ Special format for reading Parquet file metadata (https://parquet.apache.org/doc - logical_type - column logical type - compression - compression used for this column - total_uncompressed_size - total uncompressed bytes size of the column, calculated as the sum of total_uncompressed_size of the column from all row groups - - total_compressed_size - total compressed bytes size of the column, calculated as the sum of total_compressed_size of the column from all row groups + - total_compressed_size - total compressed bytes size of the column, calculated as the sum of total_compressed_size of the column from all row groups - space_saved - percent of space saved by compression, calculated as (1 - total_compressed_size/total_uncompressed_size). - encodings - the list of encodings used for this column - row_groups - the list of row groups metadata with the next structure: diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 26501f3f3f6..e721c9408e3 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -931,6 +931,11 @@ Result ```text " string " ``` +### input_format_csv_ignore_extra_columns {#input_format_csv_ignore_extra_columns} + +Ignore extra colums in CSV input. + +Disabled by default. ## Values format settings {#values-format-settings} diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 48a6132170a..8488f4ce55a 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -401,8 +401,8 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR - [output_format_csv_crlf_end_of_line](../operations/settings/settings.md#output_format_csv_crlf_end_of_line) - если установлено значение true, конец строки в формате вывода CSV будет `\r\n` вместо `\n`. Значение по умолчанию - `false`. - [input_format_csv_skip_first_lines](../operations/settings/settings.md#input_format_csv_skip_first_lines) - пропустить указанное количество строк в начале данных. Значение по умолчанию - `0`. - [input_format_csv_detect_header](../operations/settings/settings.md#input_format_csv_detect_header) - обнаружить заголовок с именами и типами в формате CSV. Значение по умолчанию - `true`. -- [input_format_csv_trim_whitespaces](../operations/settings/settings.md#input_format_csv_trim_whitespaces) - удалить пробелы и символы табуляции из строк без кавычек. -Значение по умолчанию - `true`. +- [input_format_csv_trim_whitespaces](../operations/settings/settings.md#input_format_csv_trim_whitespaces) - удалить пробелы и символы табуляции из строк без кавычек. Значение по умолчанию - `true`. +- [input_format_csv_ignore_extra_columns](../operations/settings/settings.md/#input_format_csv_ignore_extra_columns) - игнорировать дополнительные столбцы. Значение по умолчанию - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e3da8302fc8..33d9300f8e1 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1686,7 +1686,7 @@ SELECT * FROM table_with_enum_column_for_csv_insert; ## input_format_csv_detect_header {#input_format_csv_detect_header} Обнаружить заголовок с именами и типами в формате CSV. - + Значение по умолчанию - `true`. ## input_format_csv_skip_first_lines {#input_format_csv_skip_first_lines} @@ -1727,6 +1727,12 @@ echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --in " string " ``` +## input_format_csv_ignore_extra_columns {#input_format_csv_ignore_extra_columns} + +Игнорировать дополнительные столбцы. + +Выключено по умолчанию. + ## output_format_tsv_crlf_end_of_line {#settings-output-format-tsv-crlf-end-of-line} Использовать в качестве разделителя строк для TSV формата CRLF (DOC/Windows стиль) вместо LF (Unix стиль). diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d38f7767252..9582419b98c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -835,7 +835,6 @@ class IColumn; M(Bool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \ M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, -WithNames, -WithNamesAndTypes formats).", IMPORTANT) \ M(Bool, input_format_csv_empty_as_default, true, "Treat empty fields in CSV input as default values.", 0) \ - M(Bool, input_format_csv_ignore_extra_columns, false, "", 0) \ M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ M(Bool, input_format_tsv_enum_as_number, false, "Treat inserted enum values in TSV formats as enum indices.", 0) \ M(Bool, input_format_null_as_default, true, "Initialize null fields with default values if the data type of this field is not nullable and it is supported by the input format", 0) \ @@ -1001,6 +1000,7 @@ class IColumn; M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ + M(Bool, input_format_csv_ignore_extra_columns, false, "Ignore extra colums in CSV input", 0) \ // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 0218d268c51..f29b55f7e73 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -63,7 +63,6 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.delimiter = settings.format_csv_delimiter; format_settings.csv.tuple_delimiter = settings.format_csv_delimiter; format_settings.csv.empty_as_default = settings.input_format_csv_empty_as_default; - format_settings.csv.ignore_extra_columns = settings.input_format_csv_ignore_extra_columns; format_settings.csv.enum_as_number = settings.input_format_csv_enum_as_number; format_settings.csv.null_representation = settings.format_csv_null_representation; format_settings.csv.arrays_as_nested_csv = settings.input_format_csv_arrays_as_nested_csv; @@ -72,6 +71,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.try_detect_header = settings.input_format_csv_detect_header; format_settings.csv.skip_trailing_empty_lines = settings.input_format_csv_skip_trailing_empty_lines; format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; + format_settings.csv.ignore_extra_columns = settings.input_format_csv_ignore_extra_columns; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 3bc53140fe5..38148bda373 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -128,7 +128,6 @@ struct FormatSettings bool allow_single_quotes = true; bool allow_double_quotes = true; bool empty_as_default = false; - bool ignore_extra_columns = false; bool crlf_end_of_line = false; bool enum_as_number = false; bool arrays_as_nested_csv = false; @@ -140,6 +139,7 @@ struct FormatSettings bool try_detect_header = true; bool skip_trailing_empty_lines = false; bool trim_whitespaces = true; + bool ignore_extra_columns = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 0cc5889b732..8aaf8fd3e2f 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -288,6 +288,8 @@ bool CSVFormatReader::readField( const bool at_delimiter = !buf->eof() && *buf->position() == format_settings.csv.delimiter; const bool at_last_column_line_end = is_last_file_column && (buf->eof() || *buf->position() == '\n' || *buf->position() == '\r'); + bool res = false; + /// Note: Tuples are serialized in CSV as separate columns, but with empty_as_default or null_as_default /// only one empty or NULL column will be expected if (format_settings.csv.empty_as_default && (at_delimiter || at_last_column_line_end)) @@ -299,31 +301,28 @@ bool CSVFormatReader::readField( /// they do not contain empty unquoted fields, so this check /// works for tuples as well. column.insertDefault(); - return false; } - - auto skip_all = [&]() - { - if (!is_last_file_column || !format_settings.csv.ignore_extra_columns) - { - return; - } - //std::cout << "skip !!!" << std::endl; - buf->position() = find_first_symbols<'\n'>(buf->position(), buf->buffer().end()); - }; - if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) + else if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) { /// If value is null but type is not nullable then use default value instead. - bool res = SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); - skip_all(); - return res; + res = SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); + } + else + { + /// Read the column normally. + serialization->deserializeTextCSV(column, *buf, format_settings); + res = true; } - /// Read the column normally. - serialization->deserializeTextCSV(column, *buf, format_settings); - - skip_all(); - return true; + if (is_last_file_column && format_settings.csv.ignore_extra_columns) + { + while (checkChar(format_settings.csv.delimiter, *buf)) + { + skipField(); + skipWhitespacesAndTabs(*buf); + } + } + return res; } void CSVFormatReader::skipPrefixBeforeHeader() diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index eaedbbb4a1e..24bf1d0d595 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -212,8 +212,12 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE format_reader->skipRowStartDelimiter(); ext.read_columns.resize(data_types.size()); + //std::cout << "col size " << column_mapping->column_indexes_for_input_fields.size() << std::endl; for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) { + // std::cout << " file_column " << file_column << column_mapping->names_of_columns[file_column] << std::endl; + + const auto & column_index = column_mapping->column_indexes_for_input_fields[file_column]; const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size(); if (column_index) diff --git a/tests/queries/0_stateless/00301_csv.reference b/tests/queries/0_stateless/00301_csv.reference index 61279f3b84a..3dbe3116bea 100644 --- a/tests/queries/0_stateless/00301_csv.reference +++ b/tests/queries/0_stateless/00301_csv.reference @@ -11,7 +11,9 @@ default-eof 1 2019-06-19 2016-01-01 01:02:03 NUL 2016-01-02 01:02:03 Nhello \N \N -Hello world 1 2016-01-01 -Hello world 2 2016-01-02 -Hello world 3 2016-01-03 -Hello world 4 2016-01-04 +Hello 1 String1 +Hello 2 String2 +Hello 3 String3 +Hello 4 String4 +Hello 5 String5 +Hello 6 String6 diff --git a/tests/queries/0_stateless/00301_csv.sh b/tests/queries/0_stateless/00301_csv.sh index e99c39a0f6f..fafe75f6f63 100755 --- a/tests/queries/0_stateless/00301_csv.sh +++ b/tests/queries/0_stateless/00301_csv.sh @@ -39,11 +39,14 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s NULLS LAST"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; -$CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64 DEFAULT 1, d Date DEFAULT '2019-06-19') ENGINE = Memory"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64 DEFAULT 3, d String DEFAULT 'String4') ENGINE = Memory"; -echo 'Hello world, 1, 2016-01-01 -Hello world, 2 ,2016-01-02, -Hello world, 3 ,2016-01-03, 2016-01-13 -Hello world, 4 ,2016-01-04, 2016-01-14, 2016-01-15' | $CLICKHOUSE_CLIENT --input_format_csv_empty_as_default=1 --input_format_csv_ignore_extra_columns=1 --query="INSERT INTO csv FORMAT CSV"; +echo 'Hello, 1, String1 +Hello, 2, String2, +Hello, 3, String3, 2016-01-13 +Hello, 4, , 2016-01-14 +Hello, 5, String5, 2016-01-15, 2016-01-16 +Hello, 6, String6, "line with a +break"' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_csv_empty_as_default=1 --input_format_csv_ignore_extra_columns=1 --query="INSERT INTO csv FORMAT CSV"; $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s, n"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; \ No newline at end of file From 1234fbe6f1adf64161e384eab7820c9ebfc1fe2f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 15 Jun 2023 08:41:37 +0200 Subject: [PATCH 0169/2047] Added test for named collection and fixed issues with glob --- src/Storages/StorageAzureBlob.cpp | 28 ++++++------- src/Storages/StorageAzureBlobCluster.cpp | 6 +-- .../test_cluster.py | 42 +++++++++++++++---- 3 files changed, 49 insertions(+), 27 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 68b21d93ad9..af56efa9ce6 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -889,6 +889,7 @@ StorageAzureBlobSource::Iterator::Iterator( RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { + std::lock_guard lock(next_mutex); if (is_finished) return {}; @@ -906,25 +907,24 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() else { bool need_new_batch = false; - { - std::lock_guard lock(next_mutex); - need_new_batch = !blobs_with_metadata || index >= blobs_with_metadata->size(); - } + need_new_batch = !blobs_with_metadata || index >= blobs_with_metadata->size(); if (need_new_batch) { RelativePathsWithMetadata new_batch; while (new_batch.empty()) { - if (object_storage_iterator->isValid()) { - new_batch = object_storage_iterator->currentBatch(); - object_storage_iterator->nextBatch(); - } - else - { - is_finished = true; - return {}; + if (object_storage_iterator->isValid()) + { + new_batch = object_storage_iterator->currentBatch(); + object_storage_iterator->nextBatch(); + } + else + { + is_finished = true; + return {}; + } } for (auto it = new_batch.begin(); it != new_batch.end();) @@ -952,7 +952,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); const auto & idxs = typeid_cast(*block.getByName("_idx").column); - std::lock_guard lock(next_mutex); blob_path_with_globs.reset(); blob_path_with_globs.emplace(); for (UInt64 idx : idxs.getData()) @@ -968,7 +967,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() if (outer_blobs) outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); - std::lock_guard lock(next_mutex); blobs_with_metadata = std::move(new_batch); for (const auto & [_, info] : *blobs_with_metadata) total_size.fetch_add(info.size_bytes, std::memory_order_relaxed); @@ -976,8 +974,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() } size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - - std::lock_guard lock(next_mutex); return (*blobs_with_metadata)[current_index]; } } diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp index b111262c498..cfdad5c9e59 100644 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ b/src/Storages/StorageAzureBlobCluster.cpp @@ -82,9 +82,9 @@ void StorageAzureBlobCluster::addColumnsStructureToQuery(ASTPtr & query, const S RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const { auto iterator = std::make_shared( - object_storage.get(), configuration.container, configuration.blobs_paths, - std::nullopt, query, virtual_block, context, nullptr); - auto callback = std::make_shared>([iterator]() mutable -> String {return iterator->next().relative_path;}); + object_storage.get(), configuration.container, std::nullopt, + configuration.blob_path, query, virtual_block, context, nullptr); + auto callback = std::make_shared>([iterator]() mutable -> String{ return iterator->next().relative_path; }); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index c42aac35071..2a9ef95776b 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -150,13 +150,13 @@ def test_union_all(cluster): SELECT * FROM ( SELECT * from azureBlobStorage( - 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', - 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', 'auto', 'a Int32, b String') UNION ALL SELECT * from azureBlobStorage( - 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', - 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', 'auto', 'a Int32, b String') ) ORDER BY (a) @@ -168,14 +168,14 @@ def test_union_all(cluster): ( SELECT * from azureBlobStorageCluster( 'simple_cluster', - 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', - 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', 'auto', 'a Int32, b String') UNION ALL SELECT * from azureBlobStorageCluster( 'simple_cluster', - 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', - 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', + 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_parquet_union_all', 'devstoreaccount1', + 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'Parquet', 'auto', 'a Int32, b String') ) ORDER BY (a) @@ -256,3 +256,29 @@ def test_cluster_with_named_collection(cluster): ) assert TSV(pure_azure) == TSV(azure_cluster) + +def test_partition_parallel_readig_withcluster(cluster): + node = cluster.instances["node_0"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + partition_by = "column3" + values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" + filename = "test_tf_{_partition_id}.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + ) + + assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv") + assert "3,2,1\n" == get_azure_file_content("test_tf_1.csv") + assert "78,43,45\n" == get_azure_file_content("test_tf_45.csv") + + azure_cluster = node.query( + """ + SELECT count(*) from azureBlobStorageCluster( + 'simple_cluster', + azure_conf2, container='cont', blob_path='test_tf_*.csv', format='CSV', compression='auto', structure='column1 UInt32, column2 UInt32, column3 UInt32') + """ + ) + + assert azure_cluster == "3\n" \ No newline at end of file From 02cf8a1bade3898196d7a760cb75ae10af3f2ecf Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 15 Jun 2023 14:09:38 +0300 Subject: [PATCH 0170/2047] Update comment StorageFile.cpp --- src/Storages/StorageFile.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 41ddd6b5780..4fbea89baa2 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -100,7 +100,14 @@ void listFilesWithRegexpMatchingImpl( std::vector & result, bool recursive = false); -/// When `{...}` has any `/`s, it must be processed in a different way +/* + * When `{...}` has any `/`s, it must be processed in a different way: + * Basically, a path with globs is processed by listFilesWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, + * listFilesWithFoldedRegexpMatching is in charge from now on. + * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. + * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. + * StorageHDFS.cpp has the same logic. +*/ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const std::string & processed_suffix, const std::string & suffix_with_globs, @@ -111,14 +118,6 @@ void listFilesWithFoldedRegexpMatchingImpl(const std::string & path_for_ls, const size_t next_slash_after_glob_pos, std::vector & result) { - /* - * When `{...}` has any `/`s, it must be processed in a different way: - * Basically, a path with globs is processed by LSWithRegexpMatching. In case it detects multi-dir glob {.../..., .../...}, - * LSWithFoldedRegexpMatching is in charge from now on. - * It works a bit different: it still recursively goes through subdirectories, but does not match every directory to glob. - * Instead, it goes many levels down (until the approximate max_depth is reached) and compares this multi-dir path to a glob. - * StorageHDFS.cpp has the same logic. - */ if (!max_depth) return; From 806176d88e0b4237c16e23aed27179ed93aa17c1 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 15 Jun 2023 11:23:08 +0000 Subject: [PATCH 0171/2047] Add input_format_csv_missing_as_default setting and tests --- docs/en/interfaces/formats.md | 3 ++- .../operations/settings/settings-formats.md | 8 +++++++- docs/ru/interfaces/formats.md | 3 ++- docs/ru/operations/settings/settings.md | 8 +++++++- src/Core/Settings.h | 3 ++- src/Dictionaries/CacheDictionary.cpp | 2 +- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/CSVRowInputFormat.cpp | 18 +++++++++++++++++- .../Formats/Impl/CSVRowInputFormat.h | 1 + .../RowInputFormatWithNamesAndTypes.cpp | 4 ---- tests/queries/0_stateless/00301_csv.reference | 10 ++++++++++ tests/queries/0_stateless/00301_csv.sh | 19 +++++++++++++++++-- 13 files changed, 68 insertions(+), 13 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 950692deb77..e0b0fcfabd5 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -470,7 +470,8 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_csv_detect_header) - automatically detect header with names and types in CSV format. Default value - `true`. - [input_format_csv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_trailing_empty_lines) - skip trailing empty lines at the end of data. Default value - `false`. - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. -- [input_format_csv_ignore_extra_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_ignore_extra_columns) - ignore extra colums in CSV input. Default value - `false`. +- [input_format_csv_ignore_extra_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_ignore_extra_columns) - ignore extra columns in CSV input (if your file has more columns than expected). Default value - `false`. +- [input_format_csv_missing_as_default](/docs/en/operations/settings/settings-formats.md/#input_format_csv_missing_as_default) - treat missing fields in CSV input as default values. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index e721c9408e3..6d9a1fb5160 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -933,7 +933,13 @@ Result ``` ### input_format_csv_ignore_extra_columns {#input_format_csv_ignore_extra_columns} -Ignore extra colums in CSV input. +Ignore extra columns in CSV input (if your file has more columns than expected). + +Disabled by default. + +### input_format_csv_missing_as_default {#input_format_csv_missing_as_default} + +Treat missing fields in CSV input as default values. Disabled by default. diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 8488f4ce55a..7e3bb3f7d26 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -402,7 +402,8 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR - [input_format_csv_skip_first_lines](../operations/settings/settings.md#input_format_csv_skip_first_lines) - пропустить указанное количество строк в начале данных. Значение по умолчанию - `0`. - [input_format_csv_detect_header](../operations/settings/settings.md#input_format_csv_detect_header) - обнаружить заголовок с именами и типами в формате CSV. Значение по умолчанию - `true`. - [input_format_csv_trim_whitespaces](../operations/settings/settings.md#input_format_csv_trim_whitespaces) - удалить пробелы и символы табуляции из строк без кавычек. Значение по умолчанию - `true`. -- [input_format_csv_ignore_extra_columns](../operations/settings/settings.md/#input_format_csv_ignore_extra_columns) - игнорировать дополнительные столбцы. Значение по умолчанию - `false`. +- [input_format_csv_ignore_extra_columns](../operations/settings/settings.md/#input_format_csv_ignore_extra_columns) - игнорировать дополнительные столбцы (если файл содержит больше столбцов чем ожидается). Значение по умолчанию - `false`. +- [input_format_csv_missing_as_default](../operations/settings/settings.md/#input_format_csv_missing_as_default) - рассматривать отсутствующие поля в CSV в качестве значений по умолчанию. Значение по умолчанию - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 33d9300f8e1..61cfc332585 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1729,7 +1729,13 @@ echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --in ## input_format_csv_ignore_extra_columns {#input_format_csv_ignore_extra_columns} -Игнорировать дополнительные столбцы. +Игнорировать дополнительные столбцы (если файл содержит больше столбцов чем ожидается). + +Выключено по умолчанию. + +## input_format_csv_missing_as_default {#input_format_csv_missing_as_default} + +Рассматривать отсутствующие поля в CSV в качестве значений по умолчанию. Выключено по умолчанию. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9582419b98c..ce7c28996e8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1000,7 +1000,8 @@ class IColumn; M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ - M(Bool, input_format_csv_ignore_extra_columns, false, "Ignore extra colums in CSV input", 0) \ + M(Bool, input_format_csv_ignore_extra_columns, false, "Ignore extra columns in CSV input (if your file has more columns than expected)", 0) \ + M(Bool, input_format_csv_missing_as_default, false, "Treat missing fields in CSV input as default values", 0) \ // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index c5c88a9f142..359f7c17436 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -138,7 +138,7 @@ Columns CacheDictionary::getColumns( const Columns & default_values_columns) const { /** - * Flow of getColumsImpl + * Flow of getColumnsImpl * 1. Get fetch result from storage * 2. If all keys are found in storage and not expired * 2.1. If storage returns fetched columns in order of keys then result is returned to client. diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index f29b55f7e73..102b5d7eec0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -72,6 +72,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.skip_trailing_empty_lines = settings.input_format_csv_skip_trailing_empty_lines; format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.ignore_extra_columns = settings.input_format_csv_ignore_extra_columns; + format_settings.csv.missing_as_default = settings.input_format_csv_missing_as_default; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 38148bda373..2b52d88184c 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -140,6 +140,7 @@ struct FormatSettings bool skip_trailing_empty_lines = false; bool trim_whitespaces = true; bool ignore_extra_columns = false; + bool missing_as_default = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 8aaf8fd3e2f..dcc057baef2 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -147,7 +147,18 @@ CSVFormatReader::CSVFormatReader(PeekableReadBuffer & buf_, const FormatSettings void CSVFormatReader::skipFieldDelimiter() { skipWhitespacesAndTabs(*buf); - assertChar(format_settings.csv.delimiter, *buf); + + bool res = checkChar(format_settings.csv.delimiter, *buf); + if (!res && !format_settings.csv.missing_as_default) + { + char err[2] = {format_settings.csv.delimiter, '\0'}; + throwAtAssertionFailed(err, *buf); + } + + if (!res && format_settings.csv.missing_as_default) + { + current_row_has_missing_fields = true; + } } template @@ -187,6 +198,7 @@ void CSVFormatReader::skipRowEndDelimiter() return; skipEndOfLine(*buf); + current_row_has_missing_fields = false; } void CSVFormatReader::skipHeaderRow() @@ -302,6 +314,10 @@ bool CSVFormatReader::readField( /// works for tuples as well. column.insertDefault(); } + else if (current_row_has_missing_fields) + { + column.insertDefault(); + } else if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) { /// If value is null but type is not nullable then use default value instead. diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 0c8099a216c..3958c66bbc6 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -89,6 +89,7 @@ public: protected: PeekableReadBuffer * buf; + bool current_row_has_missing_fields = false; }; class CSVSchemaReader : public FormatWithNamesAndTypesSchemaReader diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index 24bf1d0d595..eaedbbb4a1e 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -212,12 +212,8 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE format_reader->skipRowStartDelimiter(); ext.read_columns.resize(data_types.size()); - //std::cout << "col size " << column_mapping->column_indexes_for_input_fields.size() << std::endl; for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) { - // std::cout << " file_column " << file_column << column_mapping->names_of_columns[file_column] << std::endl; - - const auto & column_index = column_mapping->column_indexes_for_input_fields[file_column]; const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size(); if (column_index) diff --git a/tests/queries/0_stateless/00301_csv.reference b/tests/queries/0_stateless/00301_csv.reference index 3dbe3116bea..fa85fd924e1 100644 --- a/tests/queries/0_stateless/00301_csv.reference +++ b/tests/queries/0_stateless/00301_csv.reference @@ -1,19 +1,29 @@ +=== Test input_format_csv_empty_as_default Hello, world 123 2016-01-01 Hello, "world" 456 2016-01-02 Hello "world" 789 2016-01-03 Hello\n world 100 2016-01-04 default 1 2019-06-19 default-eof 1 2019-06-19 +=== Test datetime 2016-01-01 01:02:03 1 2016-01-02 01:02:03 2 2017-08-15 13:15:01 3 1970-01-02 05:46:39 4 +=== Test nullable datetime 2016-01-01 01:02:03 NUL 2016-01-02 01:02:03 Nhello \N \N +=== Test input_format_csv_ignore_extra_columns Hello 1 String1 Hello 2 String2 Hello 3 String3 Hello 4 String4 Hello 5 String5 Hello 6 String6 +=== Test input_format_csv_missing_as_default +Hello 0 33 \N 55 Default +Hello 0 33 \N 55 Default +Hello 1 2 \N 55 Default +Hello 1 2 3 4 String +Hello 1 2 3 4 String diff --git a/tests/queries/0_stateless/00301_csv.sh b/tests/queries/0_stateless/00301_csv.sh index fafe75f6f63..887a75b0ded 100755 --- a/tests/queries/0_stateless/00301_csv.sh +++ b/tests/queries/0_stateless/00301_csv.sh @@ -4,6 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +echo === Test input_format_csv_empty_as_default $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS csv"; $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64 DEFAULT 1, d Date DEFAULT '2019-06-19') ENGINE = Memory"; @@ -18,6 +19,7 @@ Hello "world", 789 ,2016-01-03 $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY d, s"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; +echo === Test datetime $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (t DateTime('Asia/Istanbul'), s String) ENGINE = Memory"; echo '"2016-01-01 01:02:03","1" @@ -28,7 +30,7 @@ echo '"2016-01-01 01:02:03","1" $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; - +echo === Test nullable datetime $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (t Nullable(DateTime('Asia/Istanbul')), s Nullable(String)) ENGINE = Memory"; echo 'NULL, NULL @@ -39,6 +41,7 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s NULLS LAST"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; +echo === Test input_format_csv_ignore_extra_columns $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64 DEFAULT 3, d String DEFAULT 'String4') ENGINE = Memory"; echo 'Hello, 1, String1 @@ -49,4 +52,16 @@ Hello, 5, String5, 2016-01-15, 2016-01-16 Hello, 6, String6, "line with a break"' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_csv_empty_as_default=1 --input_format_csv_ignore_extra_columns=1 --query="INSERT INTO csv FORMAT CSV"; $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s, n"; -$CLICKHOUSE_CLIENT --query="DROP TABLE csv"; \ No newline at end of file +$CLICKHOUSE_CLIENT --query="DROP TABLE csv"; + + +echo === Test input_format_csv_missing_as_default +$CLICKHOUSE_CLIENT --query="CREATE TABLE csv (f1 String, f2 UInt64, f3 UInt64 Default 33, f4 Nullable(UInt64), f5 Nullable(UInt64) Default 55, f6 String DEFAULT 'Default') ENGINE = Memory"; + +echo 'Hello +Hello, +Hello, 1, 2 +Hello, 1, 2, 3, 4, String +Hello, 1, 2, 3, 4, String,'| $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_csv_empty_as_default=1 --input_format_csv_missing_as_default=1 --query="INSERT INTO csv FORMAT CSV"; +$CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY f1, f2, f3, f4 NULLS FIRST, f5, f6"; +$CLICKHOUSE_CLIENT --query="DROP TABLE csv"; From 0eeee11dc46d462412ad671a7d59006fba59c403 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 15 Jun 2023 12:36:18 +0000 Subject: [PATCH 0172/2047] Style fix, add comment --- .../Formats/Impl/CSVRowInputFormat.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index dcc057baef2..7cd812bc5b0 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -149,15 +149,17 @@ void CSVFormatReader::skipFieldDelimiter() skipWhitespacesAndTabs(*buf); bool res = checkChar(format_settings.csv.delimiter, *buf); - if (!res && !format_settings.csv.missing_as_default) + if (!res) { - char err[2] = {format_settings.csv.delimiter, '\0'}; - throwAtAssertionFailed(err, *buf); - } - - if (!res && format_settings.csv.missing_as_default) - { - current_row_has_missing_fields = true; + if (!format_settings.csv.missing_as_default) + { + char err[2] = {format_settings.csv.delimiter, '\0'}; + throwAtAssertionFailed(err, *buf); + } + else + { + current_row_has_missing_fields = true; + } } } @@ -332,6 +334,7 @@ bool CSVFormatReader::readField( if (is_last_file_column && format_settings.csv.ignore_extra_columns) { + // Skip all fields to next line. while (checkChar(format_settings.csv.delimiter, *buf)) { skipField(); From 9a25958be831d6068597703cef97238ab76ee325 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 15 Jun 2023 13:49:49 +0000 Subject: [PATCH 0173/2047] Add HTTP header filtering --- programs/server/Server.cpp | 2 + programs/server/config.xml | 8 +++ src/Common/HTTPHeaderFilter.cpp | 56 +++++++++++++++++++ src/Common/HTTPHeaderFilter.h | 27 +++++++++ src/Interpreters/Context.cpp | 16 +++++- src/Interpreters/Context.h | 5 ++ src/Storages/StorageS3.cpp | 1 + src/Storages/StorageS3Cluster.cpp | 2 + src/Storages/StorageURL.cpp | 1 + src/Storages/StorageURLCluster.cpp | 1 + tests/config/config.d/forbidden_headers.xml | 6 ++ tests/config/install.sh | 1 + .../02752_forbidden_headers.reference | 0 .../0_stateless/02752_forbidden_headers.sql | 18 ++++++ 14 files changed, 142 insertions(+), 2 deletions(-) create mode 100644 src/Common/HTTPHeaderFilter.cpp create mode 100644 src/Common/HTTPHeaderFilter.h create mode 100644 tests/config/config.d/forbidden_headers.xml create mode 100644 tests/queries/0_stateless/02752_forbidden_headers.reference create mode 100644 tests/queries/0_stateless/02752_forbidden_headers.sql diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d0fc8aca5e8..6490eb6e3f5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -887,6 +887,7 @@ try #endif global_context->setRemoteHostFilter(config()); + global_context->setHTTPHeaderFilter(config()); std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)); fs::path path = path_str; @@ -1184,6 +1185,7 @@ try } global_context->setRemoteHostFilter(*config); + global_context->setHTTPHeaderFilter(*config); global_context->setMaxTableSizeToDrop(server_settings_.max_table_size_to_drop); global_context->setMaxPartitionSizeToDrop(server_settings_.max_partition_size_to_drop); diff --git a/programs/server/config.xml b/programs/server/config.xml index d18b4cb2ac9..f5013104630 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1006,6 +1006,14 @@ --> + + + - - - s3 s3_disk/ @@ -10,7 +9,6 @@ clickhouse 20000 - cache s3_disk @@ -88,13 +86,6 @@ - - -
- s3_cache_small -
-
-
From ba5e26aebf242211db3dbc2f7f987b92dd2c76bd Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 5 Jul 2023 10:49:32 +0000 Subject: [PATCH 0382/2047] need to deal with nulls --- src/Functions/array/arrayIntersect.cpp | 101 +++++++++---------------- 1 file changed, 36 insertions(+), 65 deletions(-) diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 9d0021782ac..22f64428fd1 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include #include @@ -19,10 +21,12 @@ #include #include #include "Common/Exception.h" +#include "Common/logger_useful.h" #include #include #include #include +#include namespace DB @@ -511,11 +515,12 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable map.clear(); bool all_has_nullable = all_nullable; + bool current_has_nullable = false; for (size_t arg_num = 0; arg_num < args; ++arg_num) { const auto & arg = arrays.args[arg_num]; - bool current_has_nullable = false; + current_has_nullable = false; size_t off; // const array has only one row @@ -550,46 +555,19 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable } } - prev_off[arg_num] = off; - if (arg.is_const) - prev_off[arg_num] = 0; - - if (!current_has_nullable) + if (arg_num) + { + prev_off[arg_num] = off; + if (arg.is_const) + prev_off[arg_num] = 0; + } + if(!current_has_nullable) all_has_nullable = false; } - if (all_has_nullable) - { - ++result_offset; - result_data.insertDefault(); - null_map.push_back(1); - } - - for (const auto & pair : map) - { - if (pair.getMapped() == args) - { - ++result_offset; - // if constexpr (is_numeric_column) - // result_data.insertValue(pair.getKey()); - // else if constexpr (std::is_same_v || std::is_same_v) - // result_data.insertData(pair.getKey().data, pair.getKey().size); - // else - // result_data.deserializeAndInsertFromArena(pair.getKey().data); - - // if (all_nullable) - // null_map.push_back(0); - } - } - // result_offsets.getElement(row) = result_offset; - } - - for (size_t row = 0; row < rows; ++row) - { - bool all_has_nullable = all_nullable; for (size_t arg_num = 0; arg_num < 1; ++arg_num) { - bool current_has_nullable = false; + all_has_nullable = all_nullable; const auto & arg = arrays.args[arg_num]; size_t off; // const array has only one row @@ -598,29 +576,23 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable else off = (*arg.offsets)[row]; - prev_off[arg_num] = off; - if (arg.is_const) - prev_off[arg_num] = 0; - // throw Exception(ErrorCodes::LOGICAL_ERROR, "{}", result_offset); - for (size_t res_num = 0; res_num < result_offset; ++res_num) + for (auto i : collections::range(prev_off[arg_num], off)) { - if (arg.null_map && (*arg.null_map)[row]) + typename Map::LookupResult pair = nullptr; + if (arg.null_map && (*arg.null_map)[i]) current_has_nullable = true; - - typename Map::LookupResult pair; - - if constexpr (is_numeric_column) + else if constexpr (is_numeric_column) { - pair = map.find(columns[arg_num]->getElement(res_num)); + pair = map.find(columns[arg_num]->getElement(i)); } else if constexpr (std::is_same_v || std::is_same_v) - pair = map.find(columns[arg_num]->getDataAt(res_num)); + pair = map.find(columns[arg_num]->getDataAt(i)); else { const char * data = nullptr; - pair = map.find(columns[arg_num]->serializeValueIntoArena(res_num, arena, data)); + pair = map.find(columns[arg_num]->serializeValueIntoArena(i, arena, data)); } - + prev_off[arg_num] = off; if (arg.is_const) prev_off[arg_num] = 0; @@ -628,47 +600,46 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable if (!current_has_nullable) all_has_nullable = false; - if (pair->getMapped() == args)//for (const auto & pair : map) + if (pair && pair->getMapped() == args) { + ++pair->getMapped(); + ++result_offset; if constexpr (is_numeric_column) { - if (pair->getKey() == columns[arg_num]->getElement(res_num)) + if (pair->getKey() == columns[arg_num]->getElement(i)) { - // ++result_offset; result_data.insertValue(pair->getKey()); } } else if constexpr (std::is_same_v || std::is_same_v) { - if (pair->getKey() == columns[arg_num]->getDataAt(res_num)) + if (pair->getKey() == columns[arg_num]->getDataAt(i)) { - // ++result_offset; result_data.insertData(pair->getKey().data, pair->getKey().size); } } else { const char * data = nullptr; - if (pair->getKey() == columns[arg_num]->serializeValueIntoArena(res_num, arena, data)) + if (pair->getKey() == columns[arg_num]->serializeValueIntoArena(i, arena, data)) { - // ++result_offset; result_data.deserializeAndInsertFromArena(pair->getKey().data); } } if (all_nullable) null_map.push_back(0); + // std::cerr << "========== " << current_has_nullable << std::endl; + } + if (all_has_nullable) + { + ++result_offset; + result_data.insertDefault(); + null_map.push_back(1); } - } - if (all_has_nullable) - { - ++result_offset; - result_data.insertDefault(); - null_map.push_back(1); } result_offsets.getElement(row) = result_offset; - - } + } } ColumnPtr result_column = std::move(result_data_ptr); if (all_nullable) From 7c3c48c8c44a1c53902dd24d540e25e2634a986b Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 5 Jul 2023 11:20:03 +0000 Subject: [PATCH 0383/2047] Fix use_structure_from_insertion_table_in_table_functions does not work for materialized and alias columns --- src/Interpreters/Context.cpp | 6 +++++- .../0_stateless/02811_insert_schema_inference.reference | 0 .../0_stateless/02811_insert_schema_inference.sql | 9 +++++++++ 3 files changed, 14 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02811_insert_schema_inference.reference create mode 100644 tests/queries/0_stateless/02811_insert_schema_inference.sql diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7482450d529..a0abab349b3 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1519,7 +1519,11 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const uint64_t use_structure_from_insertion_table_in_table_functions = getSettingsRef().use_structure_from_insertion_table_in_table_functions; if (use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint() && hasInsertionTable()) { - const auto & insert_structure = DatabaseCatalog::instance().getTable(getInsertionTable(), shared_from_this())->getInMemoryMetadataPtr()->getColumns(); + const auto & insert_structure = DatabaseCatalog::instance() + .getTable(getInsertionTable(), shared_from_this()) + ->getInMemoryMetadataPtr() + ->getColumns() + .getInsertable(); DB::ColumnsDescription structure_hint; bool use_columns_from_insert_query = true; diff --git a/tests/queries/0_stateless/02811_insert_schema_inference.reference b/tests/queries/0_stateless/02811_insert_schema_inference.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02811_insert_schema_inference.sql b/tests/queries/0_stateless/02811_insert_schema_inference.sql new file mode 100644 index 00000000000..9de710047f7 --- /dev/null +++ b/tests/queries/0_stateless/02811_insert_schema_inference.sql @@ -0,0 +1,9 @@ +drop table if exists test; +create table test +( + n1 UInt32, + n2 UInt32 alias murmurHash3_32(n1), + n3 UInt32 materialized n2 + 1 +)engine=MergeTree order by n1; +insert into test select * from generateRandom() limit 10; +drop table test; From 86014a60a308ec41c7416bdbbfe6b360dcf1617b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 5 Jul 2023 11:42:02 +0000 Subject: [PATCH 0384/2047] Fixed case with spaces before delimiter --- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 1 + tests/queries/0_stateless/00301_csv.sh | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index edbc33fb3c3..9731b4ba465 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -341,6 +341,7 @@ bool CSVFormatReader::readField( if (is_last_file_column && format_settings.csv.ignore_extra_columns) { // Skip all fields to next line. + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); while (checkChar(format_settings.csv.delimiter, *buf)) { skipField(); diff --git a/tests/queries/0_stateless/00301_csv.sh b/tests/queries/0_stateless/00301_csv.sh index dc354433af9..7657745e9f7 100755 --- a/tests/queries/0_stateless/00301_csv.sh +++ b/tests/queries/0_stateless/00301_csv.sh @@ -49,7 +49,7 @@ echo '"Hello", 1, "String1" "Hello", 3, "String3", "2016-01-13" "Hello", 4, , "2016-01-14" "Hello", 5, "String5", "2016-01-15", "2016-01-16" -"Hello", 6, "String6", "line with a +"Hello", 6, "String6" , "line with a break"' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_csv_empty_as_default=1 --input_format_csv_ignore_extra_columns=1 --query="INSERT INTO csv FORMAT CSV"; $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s, n"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; From 665b6d43342f418a9c32bc31d1a969b53766fc96 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 5 Jul 2023 14:50:15 +0200 Subject: [PATCH 0385/2047] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 526e674a154..636b7a9747d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -761,6 +761,7 @@ Rollup RowBinary RowBinaryWithNames RowBinaryWithNamesAndTypes +RowBinaryWithDefaults Runtime SATA SELECTs @@ -2118,6 +2119,7 @@ rowNumberInBlock rowbinary rowbinarywithnames rowbinarywithnamesandtypes +rowbinarywithdefaults rsync rsyslog runnable From 55c526b6e644c9bab04972711efc2512a64cbfdd Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Wed, 5 Jul 2023 10:20:37 -0400 Subject: [PATCH 0386/2047] Update src/AggregateFunctions/UniquesHashSet.h Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- src/AggregateFunctions/UniquesHashSet.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/UniquesHashSet.h b/src/AggregateFunctions/UniquesHashSet.h index 369fce9374f..3e501b29414 100644 --- a/src/AggregateFunctions/UniquesHashSet.h +++ b/src/AggregateFunctions/UniquesHashSet.h @@ -109,7 +109,8 @@ private: inline size_t max_fill() const { return 1ULL << (size_degree - 1); } /// NOLINT inline size_t mask() const { return buf_size() - 1; } - inline size_t place(HashValue x) const { + inline size_t place(HashValue x) const + { if constexpr (std::endian::native == std::endian::little) return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); else From 966f07bd8eb4b2dfdb89e984d35771be56205cc6 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 5 Jul 2023 15:42:49 +0000 Subject: [PATCH 0387/2047] fixed tests and dealt with nulls --- src/Functions/array/arrayIntersect.cpp | 125 +++++++++--------- .../00556_array_intersect.reference | 2 +- 2 files changed, 65 insertions(+), 62 deletions(-) diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 22f64428fd1..46f7af56751 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -565,88 +565,91 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable all_has_nullable = false; } - for (size_t arg_num = 0; arg_num < 1; ++arg_num) + // We have NULL in output only once if it should be there + bool null_added = false; + const auto & arg = arrays.args[0]; + size_t off; + // const array has only one row + if (arg.is_const) + off = (*arg.offsets)[0]; + else + off = (*arg.offsets)[row]; + + for (auto i : collections::range(prev_off[0], off)) { all_has_nullable = all_nullable; - const auto & arg = arrays.args[arg_num]; - size_t off; - // const array has only one row - if (arg.is_const) - off = (*arg.offsets)[0]; - else - off = (*arg.offsets)[row]; + current_has_nullable = false; + typename Map::LookupResult pair = nullptr; - for (auto i : collections::range(prev_off[arg_num], off)) + if (arg.null_map && (*arg.null_map)[i]) { - typename Map::LookupResult pair = nullptr; - if (arg.null_map && (*arg.null_map)[i]) - current_has_nullable = true; - else if constexpr (is_numeric_column) + current_has_nullable = true; + if (null_added) + continue; + } + else if constexpr (is_numeric_column) + { + pair = map.find(columns[0]->getElement(i)); + } + else if constexpr (std::is_same_v || std::is_same_v) + pair = map.find(columns[0]->getDataAt(i)); + else + { + const char * data = nullptr; + pair = map.find(columns[0]->serializeValueIntoArena(i, arena, data)); + } + prev_off[0] = off; + if (arg.is_const) + prev_off[0] = 0; + + if (!current_has_nullable) + all_has_nullable = false; + + if (pair && pair->getMapped() == args) + { + ++pair->getMapped(); + ++result_offset; + if constexpr (is_numeric_column) { - pair = map.find(columns[arg_num]->getElement(i)); + if (pair->getKey() == columns[0]->getElement(i)) + { + result_data.insertValue(pair->getKey()); + } } else if constexpr (std::is_same_v || std::is_same_v) - pair = map.find(columns[arg_num]->getDataAt(i)); + { + if (pair->getKey() == columns[0]->getDataAt(i)) + { + result_data.insertData(pair->getKey().data, pair->getKey().size); + } + } else { const char * data = nullptr; - pair = map.find(columns[arg_num]->serializeValueIntoArena(i, arena, data)); - } - - prev_off[arg_num] = off; - if (arg.is_const) - prev_off[arg_num] = 0; - - if (!current_has_nullable) - all_has_nullable = false; - - if (pair && pair->getMapped() == args) - { - ++pair->getMapped(); - ++result_offset; - if constexpr (is_numeric_column) + if (pair->getKey() == columns[0]->serializeValueIntoArena(i, arena, data)) { - if (pair->getKey() == columns[arg_num]->getElement(i)) - { - result_data.insertValue(pair->getKey()); - } + result_data.deserializeAndInsertFromArena(pair->getKey().data); } - else if constexpr (std::is_same_v || std::is_same_v) - { - if (pair->getKey() == columns[arg_num]->getDataAt(i)) - { - result_data.insertData(pair->getKey().data, pair->getKey().size); - } - } - else - { - const char * data = nullptr; - if (pair->getKey() == columns[arg_num]->serializeValueIntoArena(i, arena, data)) - { - result_data.deserializeAndInsertFromArena(pair->getKey().data); - } - } - if (all_nullable) - null_map.push_back(0); - // std::cerr << "========== " << current_has_nullable << std::endl; - } - if (all_has_nullable) - { - ++result_offset; - result_data.insertDefault(); - null_map.push_back(1); } + if (all_nullable) + null_map.push_back(0); + } + if (all_has_nullable && !null_added) + { + ++result_offset; + result_data.insertDefault(); + null_map.push_back(1); + null_added = true; } - result_offsets.getElement(row) = result_offset; - } + result_offsets.getElement(row) = result_offset; + } ColumnPtr result_column = std::move(result_data_ptr); if (all_nullable) result_column = ColumnNullable::create(result_column, std::move(null_map_column)); return ColumnArray::create(result_column, std::move(result_offsets_ptr)); - } diff --git a/tests/queries/0_stateless/00556_array_intersect.reference b/tests/queries/0_stateless/00556_array_intersect.reference index fad9b2f507d..b9841ef42c4 100644 --- a/tests/queries/0_stateless/00556_array_intersect.reference +++ b/tests/queries/0_stateless/00556_array_intersect.reference @@ -5,7 +5,7 @@ [1] [1] [1] -[NULL,1] +[1,NULL] [1] [1] [[1,1]] From 7c84599764d810b5154b8cb9a997058edbbbb8ca Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 5 Jul 2023 15:46:46 +0000 Subject: [PATCH 0388/2047] removed unused includes --- docs/en/operations/system-tables/parts.md | 2 -- src/Functions/array/arrayIntersect.cpp | 5 ----- 2 files changed, 7 deletions(-) diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index e04c67b1585..9159d1e9284 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -39,8 +39,6 @@ Columns: - `data_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – Total size of uncompressed data in the data part. All the auxiliary files (for example, files with marks) are not included. -- `primary_key_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The amount of memory (in bytes) used by primary key values in the primary.idx/cidx file on disk. - - `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The size of the file with marks. - `secondary_indices_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – Total size of compressed data for secondary indices in the data part. All the auxiliary files (for example, files with marks) are not included. diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 46f7af56751..b2df0263470 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -1,5 +1,3 @@ -#include -#include #include #include #include @@ -20,13 +18,10 @@ #include #include #include -#include "Common/Exception.h" -#include "Common/logger_useful.h" #include #include #include #include -#include namespace DB From 118ea83e219c2a7651852541ca9724705bd1947b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 5 Jul 2023 15:51:16 +0000 Subject: [PATCH 0389/2047] undo unimportant changes --- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 - src/Storages/System/StorageSystemParts.cpp | 3 --- .../0_stateless/02117_show_create_table_system.reference | 1 - 3 files changed, 5 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 4be803fa05e..fd73d802579 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -341,7 +341,6 @@ public: UInt64 getIndexSizeInBytes() const; UInt64 getIndexSizeInAllocatedBytes() const; UInt64 getMarksCount() const; - UInt64 getIndexSizeFromFile() const; UInt64 getBytesOnDisk() const { return bytes_on_disk; } void setBytesOnDisk(UInt64 bytes_on_disk_) { bytes_on_disk = bytes_on_disk_; } diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index e1e8ba1aa00..86ecb336b51 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -57,7 +57,6 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"bytes_on_disk", std::make_shared()}, {"data_compressed_bytes", std::make_shared()}, {"data_uncompressed_bytes", std::make_shared()}, - {"primary_key_size", std::make_shared()}, {"marks_bytes", std::make_shared()}, {"secondary_indices_compressed_bytes", std::make_shared()}, {"secondary_indices_uncompressed_bytes", std::make_shared()}, @@ -169,8 +168,6 @@ void StorageSystemParts::processNextStorage( columns[res_index++]->insert(columns_size.data_compressed); if (columns_mask[src_index++]) columns[res_index++]->insert(columns_size.data_uncompressed); - if (columns_mask[src_index++]) - columns[res_index++]->insert(part->getIndexSizeFromFile()); if (columns_mask[src_index++]) columns[res_index++]->insert(columns_size.marks); if (columns_mask[src_index++]) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index d71e8d4acab..703972279e7 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -466,7 +466,6 @@ CREATE TABLE system.parts `bytes_on_disk` UInt64, `data_compressed_bytes` UInt64, `data_uncompressed_bytes` UInt64, - `primary_key_size` UInt64, `marks_bytes` UInt64, `secondary_indices_compressed_bytes` UInt64, `secondary_indices_uncompressed_bytes` UInt64, From 4e0fed3d60f94a7be1341077b05e8e149c375340 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 5 Jul 2023 15:54:12 +0000 Subject: [PATCH 0390/2047] undo unimportant changes --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 148e9b2e4db..cf2d1b19326 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1,6 +1,5 @@ #include "IMergeTreeDataPart.h" #include "Storages/MergeTree/IDataPartStorage.h" -#include "base/types.h" #include #include @@ -1800,22 +1799,6 @@ MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & di return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, storage.log); } -UInt64 IMergeTreeDataPart::getIndexSizeFromFile() const -{ - auto metadata_snapshot = storage.getInMemoryMetadataPtr(); - if (parent_part) - metadata_snapshot = metadata_snapshot->projections.get(name).metadata; - const auto & pk = metadata_snapshot->getPrimaryKey(); - if (!pk.column_names.empty()) - { - String file = "primary" + getIndexExtension(false); - if (checksums.files.contains("primary" + getIndexExtension(true))) - file = "primary" + getIndexExtension(true); - return getFileSizeOrZero(file); - } - return 0; -} - void IMergeTreeDataPart::checkConsistencyBase() const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); From baee73fd96d1b1974ac7ec637c3b22c4f63a27a4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 5 Jul 2023 18:11:25 +0200 Subject: [PATCH 0391/2047] Make shutdown of replicated tables softer --- programs/server/Server.cpp | 64 +++++--- programs/server/Server.h | 11 +- src/Databases/DatabasesCommon.cpp | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 2 +- src/Storages/IStorage.h | 8 +- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../ReplicatedMergeTreeRestartingThread.cpp | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageProxy.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 137 ++++++++++++------ src/Storages/StorageReplicatedMergeTree.h | 11 +- src/Storages/StorageTableFunction.h | 4 +- .../__init__.py | 1 + .../config/merge_tree_conf.xml | 5 + .../test.py | 74 ++++++++++ 19 files changed, 253 insertions(+), 81 deletions(-) create mode 100644 tests/integration/test_replicated_merge_tree_wait_on_shutdown/__init__.py create mode 100644 tests/integration/test_replicated_merge_tree_wait_on_shutdown/config/merge_tree_conf.xml create mode 100644 tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d2d8a0d07fb..0a311fa4737 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1286,7 +1286,7 @@ try global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config); std::lock_guard lock(servers_lock); - updateServers(*config, server_pool, async_metrics, servers); + updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables); } global_context->updateStorageConfiguration(*config); @@ -1388,10 +1388,15 @@ try } - for (auto & server : servers_to_start_before_tables) { - server.start(); - LOG_INFO(log, "Listening for {}", server.getDescription()); + std::lock_guard lock(servers_lock); + createInterserverServers(config(), interserver_listen_hosts, listen_try, server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ false); + + for (auto & server : servers_to_start_before_tables) + { + server.start(); + LOG_INFO(log, "Listening for {}", server.getDescription()); + } } /// Initialize access storages. @@ -1688,7 +1693,7 @@ try { std::lock_guard lock(servers_lock); - createServers(config(), listen_hosts, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers); + createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers); if (servers.empty()) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No servers started (add valid listen_host and 'tcp_port' or 'http_port' " @@ -1954,7 +1959,6 @@ HTTPContextPtr Server::httpContext() const void Server::createServers( Poco::Util::AbstractConfiguration & config, const Strings & listen_hosts, - const Strings & interserver_listen_hosts, bool listen_try, Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, @@ -2176,6 +2180,23 @@ void Server::createServers( httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); }); } +} + +void Server::createInterserverServers( + Poco::Util::AbstractConfiguration & config, + const Strings & interserver_listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers) +{ + const Settings & settings = global_context->getSettingsRef(); + + Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0); + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(settings.http_receive_timeout); + http_params->setKeepAliveTimeout(keep_alive_timeout); /// Now iterate over interserver_listen_hosts for (const auto & interserver_listen_host : interserver_listen_hosts) @@ -2224,14 +2245,14 @@ void Server::createServers( #endif }); } - } void Server::updateServers( Poco::Util::AbstractConfiguration & config, Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, - std::vector & servers) + std::vector & servers, + std::vector & servers_to_start_before_tables) { Poco::Logger * log = &logger(); @@ -2256,12 +2277,18 @@ void Server::updateServers( std::erase_if(servers, std::bind_front(check_server, " (from one of previous reload)")); Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : this->config(); - + std::vector all_servers; for (auto & server : servers) + all_servers.push_back(&server); + + for (auto & server : servers_to_start_before_tables) + all_servers.push_back(&server); + + for (auto * server : all_servers) { - if (!server.isStopping()) + if (!server->isStopping()) { - std::string port_name = server.getPortName(); + std::string port_name = server->getPortName(); bool has_host = false; bool is_http = false; if (port_name.starts_with("protocols.")) @@ -2299,25 +2326,26 @@ void Server::updateServers( /// NOTE: better to compare using getPortName() over using /// dynamic_cast<> since HTTPServer is also used for prometheus and /// internal replication communications. - is_http = server.getPortName() == "http_port" || server.getPortName() == "https_port"; + is_http = server->getPortName() == "http_port" || server->getPortName() == "https_port"; } if (!has_host) - has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server.getListenHost()) != listen_hosts.end(); + has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server->getListenHost()) != listen_hosts.end(); bool has_port = !config.getString(port_name, "").empty(); bool force_restart = is_http && !isSameConfiguration(previous_config, config, "http_handlers"); if (force_restart) - LOG_TRACE(log, " had been changed, will reload {}", server.getDescription()); + LOG_TRACE(log, " had been changed, will reload {}", server->getDescription()); - if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber() || force_restart) + if (!has_host || !has_port || config.getInt(server->getPortName()) != server->portNumber() || force_restart) { - server.stop(); - LOG_INFO(log, "Stopped listening for {}", server.getDescription()); + server->stop(); + LOG_INFO(log, "Stopped listening for {}", server->getDescription()); } } } - createServers(config, listen_hosts, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); + createServers(config, listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); + createInterserverServers(config, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); std::erase_if(servers, std::bind_front(check_server, "")); } diff --git a/programs/server/Server.h b/programs/server/Server.h index e9ae6d8d937..d13378dcd65 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -102,6 +102,14 @@ private: void createServers( Poco::Util::AbstractConfiguration & config, const Strings & listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers = false); + + void createInterserverServers( + Poco::Util::AbstractConfiguration & config, const Strings & interserver_listen_hosts, bool listen_try, Poco::ThreadPool & server_pool, @@ -113,7 +121,8 @@ private: Poco::Util::AbstractConfiguration & config, Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, - std::vector & servers); + std::vector & servers, + std::vector & servers_to_start_before_tables); }; } diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index bb98e2bd3bb..4ba793d858d 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -292,7 +292,7 @@ void DatabaseWithOwnTablesBase::shutdown() for (const auto & kv : tables_snapshot) { - kv.second->flush(); + kv.second->flushAndPrepareForShutdown(); } for (const auto & kv : tables_snapshot) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 0beb4492aef..84ecb1fc909 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -361,7 +361,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, std::vector> tables_to_drop; for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) { - iterator->table()->flush(); + iterator->table()->flushAndPrepareForShutdown(); tables_to_drop.push_back({iterator->name(), iterator->table()->isDictionary()}); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index b262d88db57..c0d36836444 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -549,15 +549,15 @@ public: /** * If the storage requires some complicated work on destroying, * then you have two virtual methods: - * - flush() + * - flushAndPrepareForShutdown() * - shutdown() * * @see shutdown() - * @see flush() + * @see flushAndPrepareForShutdown() */ void flushAndShutdown() { - flush(); + flushAndPrepareForShutdown(); shutdown(); } @@ -570,7 +570,7 @@ public: /// Called before shutdown() to flush data to underlying storage /// Data in memory need to be persistent - virtual void flush() {} + virtual void flushAndPrepareForShutdown() {} /// Asks table to stop executing some action identified by action_type /// If table does not support such type of lock, and empty lock is returned diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 7424a248491..fc8f599a06e 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -204,7 +204,7 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write else sendPartFromDisk(part, out, client_protocol_version, false, send_projections); - data.addLastSentPart(part->name); + data.addLastSentPart(part->info); } catch (const NetException &) { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index dc24327712c..60c3999f87a 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -119,6 +119,7 @@ struct Settings; M(Bool, detach_not_byte_identical_parts, false, "Do not remove non byte-idential parts for ReplicatedMergeTree, instead detach them (maybe useful for further analysis).", 0) \ M(UInt64, max_replicated_fetches_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ M(UInt64, max_replicated_sends_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ + M(Milliseconds, wait_for_unique_parts_send_before_shutdown_ms, 0, "Before shutdown table will wait for required amount time for unique parts (exist only on current replica) to be fetched by other replicas (0 means disabled).", 0) \ \ /** Check delay of replicas settings. */ \ M(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index d7166b4a3b9..0e381654db0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -330,7 +330,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() void ReplicatedMergeTreeRestartingThread::partialShutdown(bool part_of_full_shutdown) { setReadonly(part_of_full_shutdown); - storage.partialShutdown(); + storage.partialShutdown(part_of_full_shutdown); } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index d021667f771..9c05afd5284 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -682,7 +682,7 @@ void StorageBuffer::startup() } -void StorageBuffer::flush() +void StorageBuffer::flushAndPrepareForShutdown() { if (!flush_handle) return; diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 8f089a4d580..db3cde93be5 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -92,7 +92,7 @@ public: void startup() override; /// Flush all buffers into the subordinate table and stop background thread. - void flush() override; + void flushAndPrepareForShutdown() override; bool optimize( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c46192ab43b..608f65cfeff 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1427,7 +1427,7 @@ ActionLock StorageDistributed::getActionLock(StorageActionBlockType type) return {}; } -void StorageDistributed::flush() +void StorageDistributed::flushAndPrepareForShutdown() { try { diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index f45286341cf..547f61a012b 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -135,7 +135,7 @@ public: void initializeFromDisk(); void shutdown() override; - void flush() override; + void flushAndPrepareForShutdown() override; void drop() override; bool storesDataOnDisk() const override { return data_volume != nullptr; } diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 14b7fc15af2..b243225adb3 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -139,7 +139,7 @@ public: void startup() override { getNested()->startup(); } void shutdown() override { getNested()->shutdown(); } - void flush() override { getNested()->flush(); } + void flushAndPrepareForShutdown() override { getNested()->flushAndPrepareForShutdown(); } ActionLock getActionLock(StorageActionBlockType action_type) override { return getNested()->getActionLock(action_type); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7f282b6c0e6..94727a5495c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -3933,6 +3934,7 @@ void StorageReplicatedMergeTree::addLastSentPart(const MergeTreePartInfo & info) { std::lock_guard lock(last_sent_parts_mutex); last_sent_parts.emplace_back(info); + static constexpr size_t LAST_SENT_PARS_WINDOW_SIZE = 1000; while (last_sent_parts.size() > LAST_SENT_PARS_WINDOW_SIZE) last_sent_parts.pop_front(); } @@ -3950,24 +3952,32 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(si auto zookeeper = getZooKeeper(); - auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper); + auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log); if (unique_parts_set.empty()) { LOG_INFO(log, "Will not wait for unique parts to be fetched because we don't have any unique parts"); return; } + else + { + LOG_INFO(log, "Will wait for {} unique parts to be fetched", unique_parts_set.size()); + } - auto wait_predicate = [&] () -> void + auto wait_predicate = [&] () -> bool { bool all_fetched = true; - for (const auto & part : unique_parts_set) + for (auto it = unique_parts_set.begin(); it != unique_parts_set.end();) { + const auto & part = *it; + bool found = false; - for (const auto & sent_part : last_sent_parts) + for (const auto & sent_part : last_sent_parts | std::views::reverse) { if (sent_part.contains(part)) { + LOG_TRACE(log, "Part {} was fetched by some replica", part.getPartNameForLogs()); found = true; + it = unique_parts_set.erase(it); break; } } @@ -3981,14 +3991,19 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(si }; std::unique_lock lock(last_sent_parts_mutex); - if (!last_sent_parts_cv.wait_for(last_sent_parts_cv, std::chrono::duration_cast(wait_ms), wait_predicate)) - LOG_WARNING(log, "Failed to wait for unqiue parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unqiue_parts_set.size()); + if (!last_sent_parts_cv.wait_for(lock, std::chrono::milliseconds(wait_ms), wait_predicate)) + LOG_WARNING(log, "Failed to wait for unqiue parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); + else + LOG_INFO(log, "Successfuly waited all the parts"); } -std::vector StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_) +std::vector StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_) { - if (zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica_name_ / "is_active")) + if (!zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica_name_ / "is_active")) + { + LOG_INFO(log_, "Our replica is not active, nobody will try to fetch anything"); return {}; + } Strings replicas = zookeeper_->getChildren(fs::path(zookeeper_path_) / "replicas"); Strings our_parts; @@ -3996,40 +4011,54 @@ std::vector StorageReplicatedMergeTree::findReplicaUniquePart for (const String & replica : replicas) { if (!zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica / "is_active")) + { + LOG_TRACE(log_, "Replica {} is not active, skipping", replica); continue; + } Strings parts = zookeeper_->getChildren(fs::path(zookeeper_path_) / "replicas" / replica / "parts"); if (replica == replica_name_) { + LOG_TRACE(log_, "Our replica parts collected {}", replica); our_parts = parts; } else { + LOG_TRACE(log_, "Fetching parts for replica {}", replica); data_parts_on_replicas.emplace_back(format_version_); for (const auto & part : parts) { - if (!data_parts_on_replicas.back().getContainingPart(part).empty()) + if (data_parts_on_replicas.back().getContainingPart(part).empty()) data_parts_on_replicas.back().add(part); } } } - NameSet our_unique_parts; + std::vector our_unique_parts; for (const auto & part : our_parts) { + LOG_TRACE(log_, "Looking for part {}", part); bool found = false; for (const auto & active_parts_set : data_parts_on_replicas) { if (!active_parts_set.getContainingPart(part).empty()) { + LOG_TRACE(log_, "Part {} found", part); found = true; break; } } + if (!found) - our_unique_parts.insert(MergeTreePartInfo::fromPartName(part, format_version)); + { + LOG_TRACE(log_, "Part not {} found", part); + our_unique_parts.emplace_back(MergeTreePartInfo::fromPartName(part, format_version_)); + } } + if (!our_parts.empty() && our_unique_parts.empty()) + LOG_TRACE(log_, "All parts found on replica"); + return our_unique_parts; } @@ -4799,39 +4828,9 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) } -void StorageReplicatedMergeTree::partialShutdown() +void StorageReplicatedMergeTree::flushAndPrepareForShutdown() { - ProfileEvents::increment(ProfileEvents::ReplicaPartialShutdown); - - partial_shutdown_called = true; - partial_shutdown_event.set(); - queue.notifySubscribersOnPartialShutdown(); - replica_is_active_node = nullptr; - - LOG_TRACE(log, "Waiting for threads to finish"); - merge_selecting_task->deactivate(); - queue_updating_task->deactivate(); - mutations_updating_task->deactivate(); - mutations_finalizing_task->deactivate(); - - cleanup_thread.stop(); - async_block_ids_cache.stop(); - part_check_thread.stop(); - - /// Stop queue processing - { - auto fetch_lock = fetcher.blocker.cancel(); - auto merge_lock = merger_mutator.merges_blocker.cancel(); - auto move_lock = parts_mover.moves_blocker.cancel(); - background_operations_assignee.finish(); - } - - LOG_TRACE(log, "Threads finished"); -} - -void StorageReplicatedMergeTree::shutdown() -{ - if (shutdown_called.exchange(true)) + if (shutdown_prepared_called.exchange(true)) return; session_expired_callback_handler.reset(); @@ -4860,6 +4859,58 @@ void StorageReplicatedMergeTree::shutdown() } background_moves_assignee.finish(); +} + +void StorageReplicatedMergeTree::partialShutdown(bool part_of_full_shutdown) +{ + ProfileEvents::increment(ProfileEvents::ReplicaPartialShutdown); + + partial_shutdown_called = true; + partial_shutdown_event.set(); + queue.notifySubscribersOnPartialShutdown(); + if (!part_of_full_shutdown) + { + LOG_DEBUG(log, "Reset active node, replica will be inactive"); + replica_is_active_node = nullptr; + } + else + LOG_DEBUG(log, "Will not reset active node, it will be reset completely during full shutdown"); + + LOG_TRACE(log, "Waiting for threads to finish"); + merge_selecting_task->deactivate(); + queue_updating_task->deactivate(); + mutations_updating_task->deactivate(); + mutations_finalizing_task->deactivate(); + + cleanup_thread.stop(); + async_block_ids_cache.stop(); + part_check_thread.stop(); + + /// Stop queue processing + { + auto fetch_lock = fetcher.blocker.cancel(); + auto merge_lock = merger_mutator.merges_blocker.cancel(); + auto move_lock = parts_mover.moves_blocker.cancel(); + background_operations_assignee.finish(); + } + + LOG_TRACE(log, "Threads finished"); +} + +void StorageReplicatedMergeTree::shutdown() +{ + if (shutdown_called.exchange(true)) + return; + + if (!shutdown_prepared_called.load()) + flushAndPrepareForShutdown(); + + auto settings_ptr = getSettings(); + LOG_DEBUG(log, "Data parts exchange still exists {}", data_parts_exchange_endpoint != nullptr); + waitForUniquePartsToBeFetchedByOtherReplicas(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds()); + + replica_is_active_node = nullptr; + auto data_parts_exchange_ptr = std::atomic_exchange(&data_parts_exchange_endpoint, InterserverIOEndpointPtr{}); if (data_parts_exchange_ptr) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 4661f0a56da..104062def4b 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -113,7 +113,10 @@ public: void startup() override; void shutdown() override; - void partialShutdown(); + + void flushAndPrepareForShutdown() override; + + void partialShutdown(bool part_of_full_shutdown); ~StorageReplicatedMergeTree() override; static String getDefaultZooKeeperPath(const Poco::Util::AbstractConfiguration & config); @@ -453,9 +456,9 @@ private: Poco::Event partial_shutdown_event {false}; /// Poco::Event::EVENT_MANUALRESET std::atomic shutdown_called {false}; + std::atomic shutdown_prepared_called {false}; - static constexpr size_t LAST_SENT_PARS_WINDOW_SIZE = 1000; - std::mutex last_sent_parts_mutex; + mutable std::mutex last_sent_parts_mutex; std::condition_variable last_sent_parts_cv; std::deque last_sent_parts; @@ -711,7 +714,7 @@ private: */ String findReplicaHavingCoveringPart(LogEntry & entry, bool active); String findReplicaHavingCoveringPart(const String & part_name, bool active, String & found_part_name); - static std::vector findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_); + static std::vector findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_); /** Download the specified part from the specified replica. * If `to_detached`, the part is placed in the `detached` directory. diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 26cbe1f0233..5df050d1d0d 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -79,11 +79,11 @@ public: nested->shutdown(); } - void flush() override + void flushAndPrepareForShutdown() override { std::lock_guard lock{nested_mutex}; if (nested) - nested->flush(); + nested->flushAndPrepareForShutdown(); } void drop() override diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/__init__.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/config/merge_tree_conf.xml b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/config/merge_tree_conf.xml new file mode 100644 index 00000000000..8ff3bdf9a2f --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/config/merge_tree_conf.xml @@ -0,0 +1,5 @@ + + + 30000 + + diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py new file mode 100644 index 00000000000..75f0921646e --- /dev/null +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -0,0 +1,74 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager +from multiprocessing.dummy import Pool +import time + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", main_configs=["config/merge_tree_conf.xml"], with_zookeeper=True, stay_alive=True +) + +node2 = cluster.add_instance( + "node2", main_configs=["config/merge_tree_conf.xml"], with_zookeeper=True, stay_alive=True +) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_shutdown_and_wait(start_cluster): + + for i, node in enumerate([node1, node2]): + node.query(f"CREATE TABLE test_table (value UInt64) ENGINE=ReplicatedMergeTree('/test/table', 'r{i}') ORDER BY tuple()") + + node1.query("INSERT INTO test_table VALUES (0)") + node2.query("SYSTEM SYNC REPLICA test_table") + + assert node1.query("SELECT * FROM test_table") == "0\n" + assert node2.query("SELECT * FROM test_table") == "0\n" + + def soft_shutdown(node): + node.stop_clickhouse(kill=False, stop_wait_sec=60) + + p = Pool(50) + pm = PartitionManager() + + pm.partition_instances(node1, node2) + + def insert(value): + node1.query(f"INSERT INTO test_table VALUES ({value})") + + p.map(insert, range(1, 50)) + + # Start shutdown async + waiter = p.apply_async(soft_shutdown, (node1,)) + # to be sure that shutdown started + time.sleep(5) + + # node 2 partitioned and don't see any data + assert node2.query("SELECT * FROM test_table") == "0\n" + + # Restore network + pm.heal_all() + # wait for shutdown to finish + waiter.get() + + node2.query("SYSTEM SYNC REPLICA test_table", timeout=5) + + # check second replica has all data + assert node2.query("SELECT sum(value) FROM test_table") == "1225\n" + # and nothing in queue + assert node2.query("SELECT count() FROM system.replication_queue") == "0\n" + + # It can happend that the second replica is superfast + assert node1.contains_in_log("Successfuly waited all the parts") or node1.contains_in_log("All parts found on replica") From 085576efc9c256d996dacf0f43185146c46eb194 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 5 Jul 2023 16:22:58 +0000 Subject: [PATCH 0392/2047] Automatic style fix --- .../test.py | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index 75f0921646e..e3a2e7a0271 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -9,13 +9,20 @@ import time cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["config/merge_tree_conf.xml"], with_zookeeper=True, stay_alive=True + "node1", + main_configs=["config/merge_tree_conf.xml"], + with_zookeeper=True, + stay_alive=True, ) node2 = cluster.add_instance( - "node2", main_configs=["config/merge_tree_conf.xml"], with_zookeeper=True, stay_alive=True + "node2", + main_configs=["config/merge_tree_conf.xml"], + with_zookeeper=True, + stay_alive=True, ) + @pytest.fixture(scope="module") def start_cluster(): try: @@ -27,9 +34,10 @@ def start_cluster(): def test_shutdown_and_wait(start_cluster): - for i, node in enumerate([node1, node2]): - node.query(f"CREATE TABLE test_table (value UInt64) ENGINE=ReplicatedMergeTree('/test/table', 'r{i}') ORDER BY tuple()") + node.query( + f"CREATE TABLE test_table (value UInt64) ENGINE=ReplicatedMergeTree('/test/table', 'r{i}') ORDER BY tuple()" + ) node1.query("INSERT INTO test_table VALUES (0)") node2.query("SYSTEM SYNC REPLICA test_table") @@ -71,4 +79,6 @@ def test_shutdown_and_wait(start_cluster): assert node2.query("SELECT count() FROM system.replication_queue") == "0\n" # It can happend that the second replica is superfast - assert node1.contains_in_log("Successfuly waited all the parts") or node1.contains_in_log("All parts found on replica") + assert node1.contains_in_log( + "Successfuly waited all the parts" + ) or node1.contains_in_log("All parts found on replica") From 88d3e1723a8a53270c0da62e581217442383cc5c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 5 Jul 2023 18:31:47 +0200 Subject: [PATCH 0393/2047] Fixes and comments --- programs/server/Server.cpp | 16 ++++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 5 ++++ src/Storages/StorageReplicatedMergeTree.h | 30 ++++++++++++++++++--- 3 files changed, 44 insertions(+), 7 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 0a311fa4737..58cf3e5d210 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1390,6 +1390,10 @@ try { std::lock_guard lock(servers_lock); + /// We should start interserver communications before (and more imporant shutdown after) tables. + /// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down. + /// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can + /// communicate with zookeeper, execute merges, etc. createInterserverServers(config(), interserver_listen_hosts, listen_try, server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ false); for (auto & server : servers_to_start_before_tables) @@ -1516,10 +1520,13 @@ try { LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish."); size_t current_connections = 0; - for (auto & server : servers_to_start_before_tables) { - server.stop(); - current_connections += server.currentConnections(); + std::lock_guard lock(servers_lock); + for (auto & server : servers_to_start_before_tables) + { + server.stop(); + current_connections += server.currentConnections(); + } } if (current_connections) @@ -2345,9 +2352,10 @@ void Server::updateServers( } createServers(config, listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); - createInterserverServers(config, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); + createInterserverServers(config, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ true); std::erase_if(servers, std::bind_front(check_server, "")); + std::erase_if(servers_to_start_before_tables, std::bind_front(check_server, "")); } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 94727a5495c..2f165a056a3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4870,6 +4870,11 @@ void StorageReplicatedMergeTree::partialShutdown(bool part_of_full_shutdown) queue.notifySubscribersOnPartialShutdown(); if (!part_of_full_shutdown) { + /// If we are going to completely shutdown table we allow other + /// replicas to fetch parts which are unique for our replica. + /// + /// Replicas try to fetch part only in case the source replica is active, + /// so don't reset handler here. LOG_DEBUG(log, "Reset active node, replica will be inactive"); replica_is_active_node = nullptr; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 104062def4b..baa5af824b4 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -112,11 +112,35 @@ public: bool need_check_structure); void startup() override; + + /// To many shutdown methods.... + /// + /// Partial shutdown called if we loose connection to zookeeper. + /// Table can also recover after partial shutdown and continue + /// to work. This method can be called regularly. + void partialShutdown(bool part_of_full_shutdown); + + /// These two methods are called during final table shutdown (DROP/DETACH/overall server shutdown). + /// The shutdown process is splitted into two methods to make it more soft and fast. In database shutdown() + /// looks like: + /// for (table : tables) + /// table->flushAndPrepareForShutdown() + /// + /// for (table : tables) + /// table->shutdown() + /// + /// So we stop producting all the parts first for all tables (fast operation). And after we can wait in shutdown() + /// for other replicas to download parts. + /// + /// In flushAndPrepareForShutdown we cancel all part-producing operations: + /// merges, fetches, moves and so on. If it wasn't called before shutdown() -- shutdown() will + /// call it (defensive programming). + void flushAndPrepareForShutdown() override; + /// In shutdown we completly terminate table -- remove + /// is_active node and interserver handler. Also optionally + /// wait until other replicas will download some parts from our replica. void shutdown() override; - void flushAndPrepareForShutdown() override; - - void partialShutdown(bool part_of_full_shutdown); ~StorageReplicatedMergeTree() override; static String getDefaultZooKeeperPath(const Poco::Util::AbstractConfiguration & config); From 33f32684d42034db67c77bcf867165c31c4d4354 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 5 Jul 2023 16:34:42 +0000 Subject: [PATCH 0394/2047] fixed style check --- src/Functions/array/arrayIntersect.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index b2df0263470..7a8bde7ab5f 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -556,7 +556,7 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable if (arg.is_const) prev_off[arg_num] = 0; } - if(!current_has_nullable) + if (!current_has_nullable) all_has_nullable = false; } From 2fa45117edfa4b541ad96f056a58e040c0569e4f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 5 Jul 2023 18:38:04 +0200 Subject: [PATCH 0395/2047] Beter --- src/Storages/StorageReplicatedMergeTree.cpp | 5 ++++- src/Storages/StorageReplicatedMergeTree.h | 2 ++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2f165a056a3..e5abf63a72d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3944,13 +3944,16 @@ void StorageReplicatedMergeTree::addLastSentPart(const MergeTreePartInfo & info) void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(size_t wait_ms) { + if (!shutdown_called.load()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Called waitForUniquePartsToBeFetchedByOtherReplicas before shutdown, it's a bug"); + if (wait_ms == 0) { LOG_INFO(log, "Will not wait for unique parts to be fetched by other replicas because wait time is zero"); return; } - auto zookeeper = getZooKeeper(); + auto zookeeper = getZooKeeperIfTableShutDown(); auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log); if (unique_parts_set.empty()) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index baa5af824b4..a1a0717ca64 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -374,6 +374,8 @@ public: return last_sent_parts; } + /// Wait required amount of milliseconds to give other replicas a chance to + /// download unique parts from our replica void waitForUniquePartsToBeFetchedByOtherReplicas(size_t wait_ms); private: From f2d106ffb783b9410cecdfe0e332d660b9fa73fa Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 5 Jul 2023 18:54:33 +0200 Subject: [PATCH 0396/2047] Fix typos --- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.h | 6 +++--- .../test_replicated_merge_tree_wait_on_shutdown/test.py | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e5abf63a72d..114465df496 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3995,9 +3995,9 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(si std::unique_lock lock(last_sent_parts_mutex); if (!last_sent_parts_cv.wait_for(lock, std::chrono::milliseconds(wait_ms), wait_predicate)) - LOG_WARNING(log, "Failed to wait for unqiue parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); + LOG_WARNING(log, "Failed to wait for unique parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); else - LOG_INFO(log, "Successfuly waited all the parts"); + LOG_INFO(log, "Successfully waited all the parts"); } std::vector StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index a1a0717ca64..656e8df6ccb 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -121,7 +121,7 @@ public: void partialShutdown(bool part_of_full_shutdown); /// These two methods are called during final table shutdown (DROP/DETACH/overall server shutdown). - /// The shutdown process is splitted into two methods to make it more soft and fast. In database shutdown() + /// The shutdown process is split into two methods to make it more soft and fast. In database shutdown() /// looks like: /// for (table : tables) /// table->flushAndPrepareForShutdown() @@ -129,14 +129,14 @@ public: /// for (table : tables) /// table->shutdown() /// - /// So we stop producting all the parts first for all tables (fast operation). And after we can wait in shutdown() + /// So we stop producing all the parts first for all tables (fast operation). And after we can wait in shutdown() /// for other replicas to download parts. /// /// In flushAndPrepareForShutdown we cancel all part-producing operations: /// merges, fetches, moves and so on. If it wasn't called before shutdown() -- shutdown() will /// call it (defensive programming). void flushAndPrepareForShutdown() override; - /// In shutdown we completly terminate table -- remove + /// In shutdown we completely terminate table -- remove /// is_active node and interserver handler. Also optionally /// wait until other replicas will download some parts from our replica. void shutdown() override; diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index e3a2e7a0271..a2a4ec92cf7 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -80,5 +80,5 @@ def test_shutdown_and_wait(start_cluster): # It can happend that the second replica is superfast assert node1.contains_in_log( - "Successfuly waited all the parts" + "Successfully waited all the parts" ) or node1.contains_in_log("All parts found on replica") From d11cd0dc30d998324961cef3f649e203a1d8c957 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 5 Jul 2023 17:56:03 +0000 Subject: [PATCH 0397/2047] Fix tests --- src/Processors/Formats/IRowInputFormat.cpp | 1 - src/Storages/HDFS/StorageHDFS.cpp | 7 +++-- src/Storages/StorageAzureBlob.cpp | 7 +++-- src/Storages/StorageFile.cpp | 7 +++-- src/Storages/StorageS3.cpp | 5 ++-- src/Storages/StorageURL.cpp | 32 +++++++++------------- src/Storages/StorageURL.h | 4 +-- src/Storages/prepareReadingFromFormat.cpp | 6 ++-- src/Storages/prepareReadingFromFormat.h | 2 +- 9 files changed, 34 insertions(+), 37 deletions(-) diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 0728aecf61f..8bcf293033b 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -97,7 +97,6 @@ Chunk IRowInputFormat::generate() size_t num_rows = 0; size_t chunk_start_offset = getDataOffsetMaybeCompressed(getReadBuffer()); - try { RowReadExtension info; diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index c662c21e60f..4949874cd74 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -440,7 +440,8 @@ Chunk HDFSSource::generate() size_t chunk_size = input_format->getApproxBytesReadForChunk(); if (!chunk_size) chunk_size = chunk.bytes(); - updateRowsProgressApprox(*this, num_rows, chunk_size, total_files_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + if (chunk_size) + updateRowsProgressApprox(*this, num_rows, chunk_size, total_files_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); } for (const auto & virtual_column : requested_virtual_columns) @@ -602,7 +603,7 @@ private: bool StorageHDFS::supportsSubsetOfColumns() const { - return format_name != "Distributed" && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name); } Pipe StorageHDFS::read( @@ -640,7 +641,7 @@ Pipe StorageHDFS::read( }); } - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, format_name, getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); Pipes pipes; auto this_ptr = std::static_pointer_cast(shared_from_this()); for (size_t i = 0; i < num_streams; ++i) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index e74e141bb16..1512cccf891 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -632,7 +632,7 @@ Pipe StorageAzureBlob::read( query_info.query, virtual_block, local_context, nullptr); } - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, configuration.format, getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); for (size_t i = 0; i < num_streams; ++i) { pipes.emplace_back(std::make_shared( @@ -994,8 +994,9 @@ Chunk StorageAzureBlobSource::generate() size_t chunk_size = reader.getFormat()->getApproxBytesReadForChunk(); if (!chunk_size) chunk_size = chunk.bytes(); - updateRowsProgressApprox( - *this, num_rows, chunk_size, total_objects_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + if (chunk_size) + updateRowsProgressApprox( + *this, num_rows, chunk_size, total_objects_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); } for (const auto & virtual_column : requested_virtual_columns) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 8e6745c14f7..eb5d8fd46b5 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -773,8 +773,9 @@ public: size_t chunk_size = input_format->getApproxBytesReadForChunk(); if (!chunk_size) chunk_size = chunk.bytes(); - updateRowsProgressApprox( - *this, num_rows, chunk_size, total_files_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + if (chunk_size) + updateRowsProgressApprox( + *this, num_rows, chunk_size, total_files_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); } return chunk; } @@ -865,7 +866,7 @@ Pipe StorageFile::read( if (progress_callback) progress_callback(FileProgress(0, total_bytes_to_read)); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, format_name, getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); for (size_t i = 0; i < num_streams; ++i) { diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 6a50d147c67..3eca7a3ef08 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -720,7 +720,8 @@ Chunk StorageS3Source::generate() size_t chunk_size = reader.getFormat()->getApproxBytesReadForChunk(); if (!chunk_size) chunk_size = chunk.bytes(); - updateRowsProgressApprox(*this, num_rows, chunk_size, total_objects_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + if (chunk_size) + updateRowsProgressApprox(*this, num_rows, chunk_size, total_objects_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); } for (const auto & virtual_column : requested_virtual_columns) @@ -1059,7 +1060,7 @@ Pipe StorageS3::read( std::shared_ptr iterator_wrapper = createFileIterator( query_configuration, distributed_processing, local_context, query_info.query, virtual_block); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, query_configuration.format, getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; for (size_t i = 0; i < num_streams; ++i) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 61efff72a49..f0f15e29129 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -290,7 +290,7 @@ StorageURLSource::StorageURLSource( input_format = FormatFactory::instance().getInput( format, *read_buf, - info.format_header, + block_for_format, context, max_block_size, format_settings, @@ -346,8 +346,9 @@ Chunk StorageURLSource::generate() size_t chunk_size = input_format->getApproxBytesReadForChunk(); if (!chunk_size) chunk_size = chunk.bytes(); - updateRowsProgressApprox( - *this, num_rows, chunk_size, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + if (chunk_size) + updateRowsProgressApprox( + *this, num_rows, chunk_size, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); } const String & path{curr_uri.getPath()}; @@ -765,7 +766,7 @@ Pipe IStorageURLBase::read( num_streams = 1; } - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, format_name, getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); Pipes pipes; pipes.reserve(num_streams); @@ -777,7 +778,13 @@ Pipe IStorageURLBase::read( read_from_format_info, iterator_wrapper, getReadMethod(), - getReadPOSTDataCallback(column_names, read_from_format_info.columns_description, query_info, local_context, processed_stage, max_block_size), + getReadPOSTDataCallback( + read_from_format_info.columns_description.getNamesOfPhysical(), + read_from_format_info.columns_description, + query_info, + local_context, + processed_stage, + max_block_size), format_name, format_settings, getName(), @@ -804,19 +811,6 @@ Pipe StorageURLWithFailover::read( size_t max_block_size, size_t /*num_streams*/) { - ColumnsDescription columns_description; - Block block_for_format; - if (supportsSubsetOfColumns()) - { - columns_description = storage_snapshot->getDescriptionForColumns(column_names); - block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); - } - else - { - columns_description = storage_snapshot->metadata->getColumns(); - block_for_format = storage_snapshot->metadata->getSampleBlock(); - } - auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size); auto iterator_wrapper = std::make_shared([&, done = false]() mutable @@ -827,7 +821,7 @@ Pipe StorageURLWithFailover::read( return uri_options; }); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, format_name, getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); auto pipe = Pipe(std::make_shared( read_from_format_info, diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index a6fb6840a22..04da5d177ec 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -47,8 +47,6 @@ public: bool supportsPartitionBy() const override { return true; } - bool supportsSubcolumns() const override { return true; } - NamesAndTypesList getVirtuals() const override; static ColumnsDescription getTableStructureFromData( @@ -110,6 +108,8 @@ protected: bool supportsSubsetOfColumns() const override; + bool supportsSubcolumns() const override { return true; } + bool prefersLargeBlocks() const override; bool parallelizeOutputAfterReading(ContextPtr context) const override; diff --git a/src/Storages/prepareReadingFromFormat.cpp b/src/Storages/prepareReadingFromFormat.cpp index aac5e53c569..7cc31066e31 100644 --- a/src/Storages/prepareReadingFromFormat.cpp +++ b/src/Storages/prepareReadingFromFormat.cpp @@ -4,7 +4,7 @@ namespace DB { -ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, const StorageSnapshotPtr & storage_snapshot, const String & format_name, const NamesAndTypesList & virtuals) +ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, const StorageSnapshotPtr & storage_snapshot, bool supports_subset_of_columns, const NamesAndTypesList & virtuals) { ReadFromFormatInfo info; /// Collect requested virtual columns and remove them from requested columns. @@ -33,9 +33,9 @@ ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, c info.source_header.insert({requested_virtual_column.type->createColumn(), requested_virtual_column.type, requested_virtual_column.name}); /// Set requested columns that should be read from data. - info.requested_columns = storage_snapshot->getColumnsByNames(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns().withExtendedObjects(), columns_to_read); + info.requested_columns = storage_snapshot->getColumnsByNames(GetColumnsOptions(GetColumnsOptions::All).withSubcolumns(), columns_to_read); - if (format_name != "Distributed" && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name)) + if (supports_subset_of_columns) { /// If only virtual columns were requested, just read the smallest column. if (columns_to_read.empty()) diff --git a/src/Storages/prepareReadingFromFormat.h b/src/Storages/prepareReadingFromFormat.h index ac029c215db..c5f3959a550 100644 --- a/src/Storages/prepareReadingFromFormat.h +++ b/src/Storages/prepareReadingFromFormat.h @@ -22,5 +22,5 @@ namespace DB }; /// Get all needed information for reading from data in some input format. - ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, const StorageSnapshotPtr & storage_snapshot, const String & format_name, const NamesAndTypesList & virtuals); + ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, const StorageSnapshotPtr & storage_snapshot, bool supports_subset_of_columns, const NamesAndTypesList & virtuals); } From 2ab6c599a234d31c2f59e1aaa35298c1274390b1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 5 Jul 2023 23:31:44 +0200 Subject: [PATCH 0398/2047] Fix tests --- .../0_stateless/02240_filesystem_query_cache.sql | 15 +++++++++++++++ .../0_stateless/02286_drop_filesystem_cache.sh | 5 ----- .../0_stateless/02344_describe_cache.reference | 2 +- 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02240_filesystem_query_cache.sql b/tests/queries/0_stateless/02240_filesystem_query_cache.sql index 760ec1baa87..02cf54b0caa 100644 --- a/tests/queries/0_stateless/02240_filesystem_query_cache.sql +++ b/tests/queries/0_stateless/02240_filesystem_query_cache.sql @@ -9,6 +9,21 @@ SET filesystem_cache_max_download_size=128; DROP TABLE IF EXISTS test; SYSTEM DROP FILESYSTEM CACHE; +CREATE TABLE test (key UInt32, value String) +Engine=MergeTree() +ORDER BY key +SETTINGS min_bytes_for_wide_part = 10485760, + compress_marks=false, + compress_primary_key=false, + disk = disk( + type = cache, + max_size = '128Mi', + path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', + cache_on_write_operations= 1, + enable_filesystem_query_cache_limit = 1, + do_not_evict_index_and_mark_files = 0, + delayed_cleanup_interval_ms = 100, + disk = 's3_disk'); INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; diff --git a/tests/queries/0_stateless/02286_drop_filesystem_cache.sh b/tests/queries/0_stateless/02286_drop_filesystem_cache.sh index 091bca10bcf..1e1841862e9 100755 --- a/tests/queries/0_stateless/02286_drop_filesystem_cache.sh +++ b/tests/queries/0_stateless/02286_drop_filesystem_cache.sh @@ -67,9 +67,4 @@ for STORAGE_POLICY in 's3_cache' 'local_cache'; do ON data_paths.cache_path = caches.cache_path" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_022862" - - $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_022862 (key UInt32, value String) - Engine=MergeTree() - ORDER BY key - SETTINGS storage_policy='${STORAGE_POLICY}_2', min_bytes_for_wide_part = 10485760" done diff --git a/tests/queries/0_stateless/02344_describe_cache.reference b/tests/queries/0_stateless/02344_describe_cache.reference index 5c3d47d87f6..da84cdabf79 100644 --- a/tests/queries/0_stateless/02344_describe_cache.reference +++ b/tests/queries/0_stateless/02344_describe_cache.reference @@ -1 +1 @@ -134217728 10000000 8388608 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 100 2 0 +134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 100 2 0 From 45db928e4e31aae6a6d7e8e6b35e0a5a3768375c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 02:52:55 +0200 Subject: [PATCH 0399/2047] Fix style --- src/Functions/fromModifiedJulianDay.cpp | 1 - src/Functions/toModifiedJulianDay.cpp | 2 -- 2 files changed, 3 deletions(-) diff --git a/src/Functions/fromModifiedJulianDay.cpp b/src/Functions/fromModifiedJulianDay.cpp index 8736b1fce7f..695d1b7d63c 100644 --- a/src/Functions/fromModifiedJulianDay.cpp +++ b/src/Functions/fromModifiedJulianDay.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_FORMAT_DATETIME; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } diff --git a/src/Functions/toModifiedJulianDay.cpp b/src/Functions/toModifiedJulianDay.cpp index 5b4cd34141c..907c7570ce2 100644 --- a/src/Functions/toModifiedJulianDay.cpp +++ b/src/Functions/toModifiedJulianDay.cpp @@ -17,8 +17,6 @@ namespace DB { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; - extern const int CANNOT_PARSE_DATE; } template From c178a362c573f7212c8f9986f78e78b209713bee Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 6 Jul 2023 02:30:37 +0000 Subject: [PATCH 0400/2047] Fix for new analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 163092f1b7f..34286c266c9 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6238,7 +6238,11 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node, const auto & insertion_table = scope_context->getInsertionTable(); if (!insertion_table.empty()) { - const auto & insert_structure = DatabaseCatalog::instance().getTable(insertion_table, scope_context)->getInMemoryMetadataPtr()->getColumns(); + const auto & insert_structure = DatabaseCatalog::instance() + .getTable(insertion_table, scope_context) + ->getInMemoryMetadataPtr() + ->getColumns() + .getInsertable(); DB::ColumnsDescription structure_hint; bool use_columns_from_insert_query = true; From 546f12dc85fdbbcf3396767917bd9dbbf8522c41 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 6 Jul 2023 05:05:27 +0000 Subject: [PATCH 0401/2047] Fix inserts to MongoDB tables --- src/Storages/StorageMongoDB.cpp | 60 ++++++++++++++++++- .../integration/test_storage_mongodb/test.py | 6 ++ 2 files changed, 63 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 3287e3272e3..45b8aceb058 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -19,6 +19,8 @@ #include #include +#include + namespace DB { @@ -127,9 +129,7 @@ public: for (const auto j : collections::range(0, num_cols)) { - WriteBufferFromOwnString ostr; - data_types[j]->getDefaultSerialization()->serializeText(*columns[j], i, ostr, FormatSettings{}); - document->add(data_names[j], ostr.str()); + insertValueIntoMongoDB(*document, data_names[j], *data_types[j], *columns[j], i); } documents.push_back(std::move(document)); @@ -151,6 +151,60 @@ public: } private: + + void insertValueIntoMongoDB( + Poco::MongoDB::Document & document, + const std::string & name, + const IDataType & data_type, + const IColumn & column, + size_t idx) + { + WhichDataType which(data_type); + + if (which.isArray()) + { + const ColumnArray & column_array = assert_cast(column); + const ColumnArray::Offsets & offsets = column_array.getOffsets(); + + size_t offset = offsets[idx - 1]; + size_t next_offset = offsets[idx]; + + const IColumn & nested_column = column_array.getData(); + + const auto * array_type = assert_cast(&data_type); + const DataTypePtr & nested_type = array_type->getNestedType(); + + Poco::MongoDB::Array::Ptr array = new Poco::MongoDB::Array(); + for (size_t i = 0; i + offset < next_offset; ++i) + { + insertValueIntoMongoDB(*array, Poco::NumberFormatter::format(i), *nested_type, nested_column, i + offset); + } + + document.add(name, array); + return; + } + + /// MongoDB does not support UInt64 type, so just cast it to Int64 + if (which.isNativeUInt()) + document.add(name, static_cast(column.getUInt(idx))); + else if (which.isNativeInt()) + document.add(name, static_cast(column.getInt(idx))); + else if (which.isFloat32()) + document.add(name, static_cast(column.getFloat32(idx))); + else if (which.isFloat64()) + document.add(name, static_cast(column.getFloat64(idx))); + else if (which.isDate()) + document.add(name, Poco::Timestamp(DateLUT::instance().fromDayNum(DayNum(column.getUInt(idx))) * 1000000)); + else if (which.isDateTime()) + document.add(name, Poco::Timestamp(column.getUInt(idx) * 1000000)); + else + { + WriteBufferFromOwnString ostr; + data_type.getDefaultSerialization()->serializeText(column, idx, ostr, FormatSettings{}); + document.add(name, ostr.str()); + } + } + String collection_name; String db_name; StorageMetadataPtr metadata_snapshot; diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 6ce71fb91fa..0abaa7a8214 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -244,6 +244,12 @@ def test_arrays(started_cluster): == "[]\n" ) + # Test INSERT SELECT + node.query("INSERT INTO arrays_mongo_table SELECT * FROM arrays_mongo_table") + + assert node.query("SELECT COUNT() FROM arrays_mongo_table") == "200\n" + assert node.query("SELECT COUNT(DISTINCT *) FROM arrays_mongo_table") == "100\n" + node.query("DROP TABLE arrays_mongo_table") arrays_mongo_table.drop() From 24b5c9c204dcc0f3c181d13528d46d012dae86c9 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 6 Jul 2023 06:05:43 +0000 Subject: [PATCH 0402/2047] Use one setting input_format_csv_allow_variable_number_of_colums and code in RowInput --- docs/en/interfaces/formats.md | 3 +- .../operations/settings/settings-formats.md | 10 +--- docs/ru/interfaces/formats.md | 3 +- docs/ru/operations/settings/settings.md | 10 +--- src/Core/Settings.h | 3 +- src/Formats/FormatFactory.cpp | 3 +- src/Formats/FormatSettings.h | 3 +- .../Formats/Impl/CSVRowInputFormat.cpp | 58 ++++++------------- .../Formats/Impl/CSVRowInputFormat.h | 6 +- .../RowInputFormatWithNamesAndTypes.cpp | 23 ++++++++ .../Formats/RowInputFormatWithNamesAndTypes.h | 4 ++ 11 files changed, 58 insertions(+), 68 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 79790cef5b2..34f9abb91d4 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -471,8 +471,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_trailing_empty_lines) - skip trailing empty lines at the end of data. Default value - `false`. - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. -- [input_format_csv_ignore_extra_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_ignore_extra_columns) - ignore extra columns in CSV input (if file has more columns than expected). Default value - `false`. -- [input_format_csv_missing_as_default](/docs/en/operations/settings/settings-formats.md/#input_format_csv_missing_as_default) - treat missing fields in CSV input as default values. Default value - `false`. +- [input_format_csv_allow_variable_number_of_colums](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_colums) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 6b05f41666c..43e410ceee8 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -931,15 +931,9 @@ Result ```text " string " ``` -### input_format_csv_ignore_extra_columns {#input_format_csv_ignore_extra_columns} +### input_format_csv_allow_variable_number_of_colums {#input_format_csv_allow_variable_number_of_colums} -Ignore extra columns in CSV input (if file has more columns than expected). - -Disabled by default. - -### input_format_csv_missing_as_default {#input_format_csv_missing_as_default} - -Treat missing fields in CSV input as default values. +ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Disabled by default. diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 7e3bb3f7d26..e7c57fff749 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -402,8 +402,7 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR - [input_format_csv_skip_first_lines](../operations/settings/settings.md#input_format_csv_skip_first_lines) - пропустить указанное количество строк в начале данных. Значение по умолчанию - `0`. - [input_format_csv_detect_header](../operations/settings/settings.md#input_format_csv_detect_header) - обнаружить заголовок с именами и типами в формате CSV. Значение по умолчанию - `true`. - [input_format_csv_trim_whitespaces](../operations/settings/settings.md#input_format_csv_trim_whitespaces) - удалить пробелы и символы табуляции из строк без кавычек. Значение по умолчанию - `true`. -- [input_format_csv_ignore_extra_columns](../operations/settings/settings.md/#input_format_csv_ignore_extra_columns) - игнорировать дополнительные столбцы (если файл содержит больше столбцов чем ожидается). Значение по умолчанию - `false`. -- [input_format_csv_missing_as_default](../operations/settings/settings.md/#input_format_csv_missing_as_default) - рассматривать отсутствующие поля в CSV в качестве значений по умолчанию. Значение по умолчанию - `false`. +- [input_format_csv_allow_variable_number_of_colums](../operations/settings/settings.md/#input_format_csv_allow_variable_number_of_colums) - игнорировать дополнительные столбцы (если файл содержит больше столбцов чем ожидается) и рассматривать отсутствующие поля в CSV в качестве значений по умолчанию. Значение по умолчанию - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e679ce6abe1..ddc101c6991 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1727,15 +1727,9 @@ echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --in " string " ``` -## input_format_csv_ignore_extra_columns {#input_format_csv_ignore_extra_columns} +## input_format_csv_allow_variable_number_of_colums {#input_format_csv_allow_variable_number_of_colums} -Игнорировать дополнительные столбцы (если файл содержит больше столбцов чем ожидается). - -Выключено по умолчанию. - -## input_format_csv_missing_as_default {#input_format_csv_missing_as_default} - -Рассматривать отсутствующие поля в CSV в качестве значений по умолчанию. +Игнорировать дополнительные столбцы (если файл содержит больше столбцов чем ожидается) и рассматривать отсутствующие поля в CSV в качестве значений по умолчанию. Выключено по умолчанию. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 044b3c34dc2..df2a916b7cf 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1009,8 +1009,7 @@ class IColumn; M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ - M(Bool, input_format_csv_ignore_extra_columns, false, "Ignore extra columns in CSV input (if file has more columns than expected)", 0) \ - M(Bool, input_format_csv_missing_as_default, false, "Treat missing fields in CSV input as default values", 0) \ + M(Bool, input_format_csv_allow_variable_number_of_colums, false, "Ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values", 0) \ // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 04b095a92d6..af9823dde73 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -72,8 +72,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.skip_trailing_empty_lines = settings.input_format_csv_skip_trailing_empty_lines; format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; - format_settings.csv.ignore_extra_columns = settings.input_format_csv_ignore_extra_columns; - format_settings.csv.missing_as_default = settings.input_format_csv_missing_as_default; + format_settings.csv.allow_variable_number_of_colums = settings.input_format_csv_allow_variable_number_of_colums; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 4bdc9077a0b..653578f8496 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -140,8 +140,7 @@ struct FormatSettings bool skip_trailing_empty_lines = false; bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; - bool ignore_extra_columns = false; - bool missing_as_default = false; + bool allow_variable_number_of_colums = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 9731b4ba465..57e05ae7cd3 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -155,18 +155,7 @@ CSVFormatReader::CSVFormatReader(PeekableReadBuffer & buf_, const FormatSettings void CSVFormatReader::skipFieldDelimiter() { skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); - - bool res = checkChar(format_settings.csv.delimiter, *buf); - if (res) - return; - - if (!format_settings.csv.missing_as_default) - { - char err[2] = {format_settings.csv.delimiter, '\0'}; - throwAtAssertionFailed(err, *buf); - } - else - current_row_has_missing_fields = true; + assertChar(format_settings.csv.delimiter, *buf); } template @@ -206,7 +195,6 @@ void CSVFormatReader::skipRowEndDelimiter() return; skipEndOfLine(*buf); - current_row_has_missing_fields = false; } void CSVFormatReader::skipHeaderRow() @@ -295,6 +283,11 @@ bool CSVFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) return true; } +bool CSVFormatReader::allowVariableNumberOfColumns() +{ + return format_settings.csv.allow_variable_number_of_colums; +} + bool CSVFormatReader::readField( IColumn & column, const DataTypePtr & type, @@ -308,8 +301,6 @@ bool CSVFormatReader::readField( const bool at_delimiter = !buf->eof() && *buf->position() == format_settings.csv.delimiter; const bool at_last_column_line_end = is_last_file_column && (buf->eof() || *buf->position() == '\n' || *buf->position() == '\r'); - bool res = false; - /// Note: Tuples are serialized in CSV as separate columns, but with empty_as_default or null_as_default /// only one empty or NULL column will be expected if (format_settings.csv.empty_as_default && (at_delimiter || at_last_column_line_end)) @@ -321,34 +312,18 @@ bool CSVFormatReader::readField( /// they do not contain empty unquoted fields, so this check /// works for tuples as well. column.insertDefault(); - } - else if (current_row_has_missing_fields) - { - column.insertDefault(); - } - else if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) - { - /// If value is null but type is not nullable then use default value instead. - res = SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); - } - else - { - /// Read the column normally. - serialization->deserializeTextCSV(column, *buf, format_settings); - res = true; + return false; } - if (is_last_file_column && format_settings.csv.ignore_extra_columns) + if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) { - // Skip all fields to next line. - skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); - while (checkChar(format_settings.csv.delimiter, *buf)) - { - skipField(); - skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); - } + /// If value is null but type is not nullable then use default value instead. + return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); } - return res; + + /// Read the column normally. + serialization->deserializeTextCSV(column, *buf, format_settings); + return true; } void CSVFormatReader::skipPrefixBeforeHeader() @@ -377,6 +352,11 @@ bool CSVFormatReader::checkForSuffix() return false; } +bool CSVFormatReader::checkForEndOfRow() +{ + return buf->eof() || *buf->position() == '\n' || *buf->position() == '\r'; +} + CSVSchemaReader::CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesSchemaReader( buf, diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 82e03c453e7..8ccf04feed3 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -69,6 +69,9 @@ public: void skipRowEndDelimiter() override; void skipPrefixBeforeHeader() override; + bool checkForEndOfRow() override; + bool allowVariableNumberOfColumns() override; + std::vector readNames() override { return readHeaderRow(); } std::vector readTypes() override { return readHeaderRow(); } std::vector readHeaderRow() { return readRowImpl(); } @@ -89,9 +92,6 @@ public: protected: PeekableReadBuffer * buf; - -private: - bool current_row_has_missing_fields = false; }; class CSVSchemaReader : public FormatWithNamesAndTypesSchemaReader diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index eaedbbb4a1e..fb49779e0af 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -227,7 +227,30 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE format_reader->skipField(file_column); if (!is_last_file_column) + { + if (format_reader->allowVariableNumberOfColumns() && format_reader->checkForEndOfRow()) + { + ++file_column; + while (file_column < column_mapping->column_indexes_for_input_fields.size()) + { + const auto & rem_column_index = column_mapping->column_indexes_for_input_fields[file_column]; + columns[*rem_column_index]->insertDefault(); + ++file_column; + } + } + else + format_reader->skipFieldDelimiter(); + } + } + + if (format_reader->allowVariableNumberOfColumns() && !format_reader->checkForEndOfRow()) + { + do + { format_reader->skipFieldDelimiter(); + format_reader->skipField(1); + } + while (!format_reader->checkForEndOfRow()); } format_reader->skipRowEndDelimiter(); diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 5648acd392d..b5103d3db39 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -119,6 +119,10 @@ public: /// Check suffix. virtual bool checkForSuffix() { return in->eof(); } + virtual bool checkForEndOfRow() { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method checkForEndOfRow is not implemented"); } + + virtual bool allowVariableNumberOfColumns() { return false; } + const FormatSettings & getFormatSettings() const { return format_settings; } virtual void setReadBuffer(ReadBuffer & in_) { in = &in_; } From d86ceef663cd0d3fcd8532ae63539e85bc4b210b Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 6 Jul 2023 14:14:48 +0800 Subject: [PATCH 0403/2047] Implement log file names rendering --- src/Loggers/Loggers.cpp | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 0c3a7bd615d..1e169190ca4 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -34,6 +34,16 @@ static std::string createDirectory(const std::string & file) return path; } +static std::string renderFileNameTemplate(time_t now, const std::string & file_path) +{ + fs::path path{file_path}; + std::tm buf; + localtime_r(&now, &buf); + std::stringstream ss; + ss << std::put_time(&buf, file_path.c_str()); + return path.replace_filename(ss.str()); +} + #ifndef WITHOUT_TEXT_LOG void Loggers::setTextLog(std::shared_ptr log, int max_priority) { @@ -68,9 +78,12 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log /// The maximum (the most verbose) of those will be used as default for Poco loggers int max_log_level = 0; - const auto log_path = config.getString("logger.log", ""); - if (!log_path.empty()) + time_t now = std::time({}); + + const auto log_path_prop = config.getString("logger.log", ""); + if (!log_path_prop.empty()) { + const auto log_path = renderFileNameTemplate(now, log_path_prop); createDirectory(log_path); std::string ext; @@ -109,9 +122,10 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log split->addChannel(log, "log"); } - const auto errorlog_path = config.getString("logger.errorlog", ""); - if (!errorlog_path.empty()) + const auto errorlog_path_prop = config.getString("logger.errorlog", ""); + if (!errorlog_path_prop.empty()) { + const auto errorlog_path = renderFileNameTemplate(now, errorlog_path_prop); createDirectory(errorlog_path); // NOTE: we don't use notice & critical in the code, so in practice error log collects fatal & error & warning. From 479efaa79acd23e72fb06413fd84d4b7091bd019 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 6 Jul 2023 14:16:18 +0800 Subject: [PATCH 0404/2047] Add clickhouse_log_file and clickhouse_error_log_file args to add_instance() --- tests/integration/helpers/cluster.py | 35 +++++++++++++++++----------- 1 file changed, 21 insertions(+), 14 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 21398790be3..5b583b865de 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -64,6 +64,13 @@ DEFAULT_ENV_NAME = ".env" SANITIZER_SIGN = "==================" +CLICKHOUSE_START_COMMAND = ( + "clickhouse server --config-file=/etc/clickhouse-server/{main_config_file}" +) + +CLICKHOUSE_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.log" + +CLICKHOUSE_ERROR_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.err.log" # to create docker-compose env file def _create_env_file(path, variables): @@ -1497,6 +1504,8 @@ class ClickHouseCluster: with_postgres=False, with_postgres_cluster=False, with_postgresql_java_client=False, + clickhouse_log_file=CLICKHOUSE_LOG_FILE, + clickhouse_error_log_file=CLICKHOUSE_ERROR_LOG_FILE, with_hdfs=False, with_kerberized_hdfs=False, with_mongo=False, @@ -1563,6 +1572,13 @@ class ClickHouseCluster: "LLVM_PROFILE_FILE" ] = "/var/lib/clickhouse/server_%h_%p_%m.profraw" + clickhouse_start_command = CLICKHOUSE_START_COMMAND + if clickhouse_log_file: + clickhouse_start_command += " --log-file=" + clickhouse_log_file + if clickhouse_error_log_file: + clickhouse_start_command += " --errorlog-file=" + clickhouse_error_log_file + logging.debug(f"clickhouse_start_command: {clickhouse_start_command}") + instance = ClickHouseInstance( cluster=self, base_path=self.base_dir, @@ -1592,10 +1608,10 @@ class ClickHouseCluster: with_redis=with_redis, with_minio=with_minio, with_azurite=with_azurite, - with_cassandra=with_cassandra, with_jdbc_bridge=with_jdbc_bridge, with_hive=with_hive, with_coredns=with_coredns, + with_cassandra=with_cassandra, server_bin_path=self.server_bin_path, odbc_bridge_bin_path=self.odbc_bridge_bin_path, library_bridge_bin_path=self.library_bridge_bin_path, @@ -1604,6 +1620,10 @@ class ClickHouseCluster: with_postgres=with_postgres, with_postgres_cluster=with_postgres_cluster, with_postgresql_java_client=with_postgresql_java_client, + clickhouse_start_command=clickhouse_start_command, + main_config_name=main_config_name, + users_config_name=users_config_name, + copy_common_configs=copy_common_configs, hostname=hostname, env_variables=env_variables, image=image, @@ -1612,9 +1632,6 @@ class ClickHouseCluster: ipv4_address=ipv4_address, ipv6_address=ipv6_address, with_installed_binary=with_installed_binary, - main_config_name=main_config_name, - users_config_name=users_config_name, - copy_common_configs=copy_common_configs, external_dirs=external_dirs, tmpfs=tmpfs or [], config_root_name=config_root_name, @@ -3046,16 +3063,6 @@ class ClickHouseCluster: subprocess_check_call(self.base_zookeeper_cmd + ["start", n]) -CLICKHOUSE_START_COMMAND = ( - "clickhouse server --config-file=/etc/clickhouse-server/{main_config_file}" - " --log-file=/var/log/clickhouse-server/clickhouse-server.log " - " --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" -) - -CLICKHOUSE_STAY_ALIVE_COMMAND = "bash -c \"trap 'pkill tail' INT TERM; {} --daemon; coproc tail -f /dev/null; wait $$!\"".format( - CLICKHOUSE_START_COMMAND -) - DOCKER_COMPOSE_TEMPLATE = """ version: '2.3' services: From fef71ab0b8759f7a659c4bb8c1be03a89df92f79 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 6 Jul 2023 14:16:46 +0800 Subject: [PATCH 0405/2047] Add inegration test --- .../__init__.py | 58 +++++++++++++++++++ .../configs/config-file-template.xml | 6 ++ .../test.py | 0 3 files changed, 64 insertions(+) create mode 100644 tests/integration/test_render_log_file_name_templates/__init__.py create mode 100644 tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml create mode 100644 tests/integration/test_render_log_file_name_templates/test.py diff --git a/tests/integration/test_render_log_file_name_templates/__init__.py b/tests/integration/test_render_log_file_name_templates/__init__.py new file mode 100644 index 00000000000..9fa87056d2c --- /dev/null +++ b/tests/integration/test_render_log_file_name_templates/__init__.py @@ -0,0 +1,58 @@ +import pytest +import logging +from helpers.cluster import ClickHouseCluster +from datetime import datetime + + +log_dir = "/var/log/clickhouse-server/" +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def started_cluster(): + cluster.add_instance( + "file-names-from-config", + main_configs=["configs/config-file-template.xml"], + clickhouse_log_file=None, + clickhouse_error_log_file=None, + ) + cluster.add_instance( + "file-names-from-params", + clickhouse_log_file=log_dir + "clickhouse-server-%Y-%m.log", + clickhouse_error_log_file=log_dir + "clickhouse-server-%Y-%m.err.log", + ) + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_check_file_names(started_cluster): + now = datetime.now() + log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" + ) + err_log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" + ) + logging.debug(f"log_file {log_file} err_log_file {err_log_file}") + + for name, instance in started_cluster.instances.items(): + files = instance.exec_in_container( + ["bash", "-c", f"ls -lh {log_dir}"], nothrow=True + ) + + logging.debug(f"check instance '{name}': {log_dir} contains: {files}") + + assert ( + instance.exec_in_container(["bash", "-c", f"ls {log_file}"], nothrow=True) + == log_file + "\n" + ) + + assert ( + instance.exec_in_container( + ["bash", "-c", f"ls {err_log_file}"], nothrow=True + ) + == err_log_file + "\n" + ) diff --git a/tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml b/tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml new file mode 100644 index 00000000000..ba408eb9823 --- /dev/null +++ b/tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml @@ -0,0 +1,6 @@ + + + /var/log/clickhouse-server/clickhouse-server-%Y-%m.log + /var/log/clickhouse-server/clickhouse-server-%Y-%m.err.log + + diff --git a/tests/integration/test_render_log_file_name_templates/test.py b/tests/integration/test_render_log_file_name_templates/test.py new file mode 100644 index 00000000000..e69de29bb2d From b9fffacc653fb9175af03cbb8f53766b0272ddbc Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 6 Jul 2023 06:31:09 +0000 Subject: [PATCH 0406/2047] Fix build --- src/Storages/StorageMongoDB.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 45b8aceb058..21543541f36 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -186,9 +186,9 @@ private: /// MongoDB does not support UInt64 type, so just cast it to Int64 if (which.isNativeUInt()) - document.add(name, static_cast(column.getUInt(idx))); + document.add(name, static_cast(column.getUInt(idx))); else if (which.isNativeInt()) - document.add(name, static_cast(column.getInt(idx))); + document.add(name, static_cast(column.getInt(idx))); else if (which.isFloat32()) document.add(name, static_cast(column.getFloat32(idx))); else if (which.isFloat64()) From 32f5a7830229b53df80f9e788b860066a4a86947 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 6 Jul 2023 07:32:46 +0000 Subject: [PATCH 0407/2047] Fix setting name --- docs/en/interfaces/formats.md | 2 +- docs/en/operations/settings/settings-formats.md | 2 +- docs/ru/interfaces/formats.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- src/Core/Settings.h | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 2 +- tests/queries/0_stateless/00301_csv.reference | 4 ++-- tests/queries/0_stateless/00301_csv.sh | 8 ++++---- 10 files changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 34f9abb91d4..ed2f010a632 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -471,7 +471,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_trailing_empty_lines) - skip trailing empty lines at the end of data. Default value - `false`. - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. -- [input_format_csv_allow_variable_number_of_colums](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_colums) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. +- [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 43e410ceee8..3eea5ef4ad9 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -931,7 +931,7 @@ Result ```text " string " ``` -### input_format_csv_allow_variable_number_of_colums {#input_format_csv_allow_variable_number_of_colums} +### input_format_csv_allow_variable_number_of_columns {#input_format_csv_allow_variable_number_of_columns} ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index e7c57fff749..e232b63f049 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -402,7 +402,7 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR - [input_format_csv_skip_first_lines](../operations/settings/settings.md#input_format_csv_skip_first_lines) - пропустить указанное количество строк в начале данных. Значение по умолчанию - `0`. - [input_format_csv_detect_header](../operations/settings/settings.md#input_format_csv_detect_header) - обнаружить заголовок с именами и типами в формате CSV. Значение по умолчанию - `true`. - [input_format_csv_trim_whitespaces](../operations/settings/settings.md#input_format_csv_trim_whitespaces) - удалить пробелы и символы табуляции из строк без кавычек. Значение по умолчанию - `true`. -- [input_format_csv_allow_variable_number_of_colums](../operations/settings/settings.md/#input_format_csv_allow_variable_number_of_colums) - игнорировать дополнительные столбцы (если файл содержит больше столбцов чем ожидается) и рассматривать отсутствующие поля в CSV в качестве значений по умолчанию. Значение по умолчанию - `false`. +- [input_format_csv_allow_variable_number_of_columns](../operations/settings/settings.md/#input_format_csv_allow_variable_number_of_columns) - игнорировать дополнительные столбцы (если файл содержит больше столбцов чем ожидается) и рассматривать отсутствующие поля в CSV в качестве значений по умолчанию. Значение по умолчанию - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index ddc101c6991..42e21f6140b 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1727,7 +1727,7 @@ echo ' string ' | ./clickhouse local -q "select * from table FORMAT CSV" --in " string " ``` -## input_format_csv_allow_variable_number_of_colums {#input_format_csv_allow_variable_number_of_colums} +## input_format_csv_allow_variable_number_of_columns {#input_format_csv_allow_variable_number_of_columns} Игнорировать дополнительные столбцы (если файл содержит больше столбцов чем ожидается) и рассматривать отсутствующие поля в CSV в качестве значений по умолчанию. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index df2a916b7cf..7f8a52c69fa 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1009,7 +1009,7 @@ class IColumn; M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ - M(Bool, input_format_csv_allow_variable_number_of_colums, false, "Ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values", 0) \ + M(Bool, input_format_csv_allow_variable_number_of_columns, false, "Ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values", 0) \ // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index af9823dde73..182abc84ffe 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -72,7 +72,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.skip_trailing_empty_lines = settings.input_format_csv_skip_trailing_empty_lines; format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; - format_settings.csv.allow_variable_number_of_colums = settings.input_format_csv_allow_variable_number_of_colums; + format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 653578f8496..dd4608227d0 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -140,7 +140,7 @@ struct FormatSettings bool skip_trailing_empty_lines = false; bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; - bool allow_variable_number_of_colums = false; + bool allow_variable_number_of_columns = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 57e05ae7cd3..60f1cbe1f80 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -285,7 +285,7 @@ bool CSVFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) bool CSVFormatReader::allowVariableNumberOfColumns() { - return format_settings.csv.allow_variable_number_of_colums; + return format_settings.csv.allow_variable_number_of_columns; } bool CSVFormatReader::readField( diff --git a/tests/queries/0_stateless/00301_csv.reference b/tests/queries/0_stateless/00301_csv.reference index 804ccf0c713..ec8c5f2b371 100644 --- a/tests/queries/0_stateless/00301_csv.reference +++ b/tests/queries/0_stateless/00301_csv.reference @@ -14,14 +14,14 @@ default-eof 1 2019-06-19 2016-01-01 01:02:03 NUL 2016-01-02 01:02:03 Nhello \N \N -=== Test input_format_csv_ignore_extra_columns +=== Test ignore extra columns Hello 1 String1 Hello 2 String2 Hello 3 String3 Hello 4 String4 Hello 5 String5 Hello 6 String6 -=== Test input_format_csv_missing_as_default +=== Test missing as default 0 0 33 \N 55 Default 0 0 33 \N 55 Default Hello 0 0 33 \N 55 Default diff --git a/tests/queries/0_stateless/00301_csv.sh b/tests/queries/0_stateless/00301_csv.sh index 7657745e9f7..776bd39fc03 100755 --- a/tests/queries/0_stateless/00301_csv.sh +++ b/tests/queries/0_stateless/00301_csv.sh @@ -41,7 +41,7 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s NULLS LAST"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; -echo === Test input_format_csv_ignore_extra_columns +echo === Test ignore extra columns $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64 DEFAULT 3, d String DEFAULT 'String4') ENGINE = Memory"; echo '"Hello", 1, "String1" @@ -50,12 +50,12 @@ echo '"Hello", 1, "String1" "Hello", 4, , "2016-01-14" "Hello", 5, "String5", "2016-01-15", "2016-01-16" "Hello", 6, "String6" , "line with a -break"' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_csv_empty_as_default=1 --input_format_csv_ignore_extra_columns=1 --query="INSERT INTO csv FORMAT CSV"; +break"' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_csv_empty_as_default=1 --input_format_csv_allow_variable_number_of_columns=1 --query="INSERT INTO csv FORMAT CSV"; $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s, n"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; -echo === Test input_format_csv_missing_as_default +echo === Test missing as default $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (f1 String, f2 UInt64, f3 UInt256, f4 UInt64 Default 33, f5 Nullable(UInt64), f6 Nullable(UInt64) Default 55, f7 String DEFAULT 'Default') ENGINE = Memory"; echo ' @@ -65,6 +65,6 @@ echo ' "Hello", 1, 3, 2 "Hello",1,4,2,3,4,"String" "Hello", 1, 4, 2, 3, 4, "String" -"Hello", 1, 5, 2, 3, 4, "String",'| $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_csv_missing_as_default=1 --query="INSERT INTO csv FORMAT CSV"; +"Hello", 1, 5, 2, 3, 4, "String",'| $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_csv_allow_variable_number_of_columns=1 --query="INSERT INTO csv FORMAT CSV"; $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY f1, f2, f3, f4, f5 NULLS FIRST, f6, f7"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; From 2d3a148ffd48d683276859b29cf952a985b6fb5e Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 6 Jul 2023 10:56:07 +0200 Subject: [PATCH 0408/2047] Added option to check if container exists before attempting to read/create it, added this flag for all table functions, only used in azureBlobStorage --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Storages/StorageAzureBlob.cpp | 72 +++++++++---------- src/Storages/StorageAzureBlob.h | 2 +- src/Storages/getStructureOfRemoteTable.cpp | 2 +- src/TableFunctions/Hive/TableFunctionHive.cpp | 5 +- src/TableFunctions/Hive/TableFunctionHive.h | 4 +- src/TableFunctions/ITableFunction.cpp | 8 +-- src/TableFunctions/ITableFunction.h | 4 +- src/TableFunctions/ITableFunctionDataLake.h | 5 +- src/TableFunctions/ITableFunctionFileLike.cpp | 2 +- src/TableFunctions/ITableFunctionFileLike.h | 2 +- src/TableFunctions/ITableFunctionXDBC.cpp | 6 +- src/TableFunctions/ITableFunctionXDBC.h | 4 +- .../TableFunctionAzureBlobStorage.cpp | 8 +-- .../TableFunctionAzureBlobStorage.h | 5 +- .../TableFunctionDictionary.cpp | 6 +- src/TableFunctions/TableFunctionDictionary.h | 4 +- .../TableFunctionExecutable.cpp | 6 +- src/TableFunctions/TableFunctionExecutable.h | 4 +- src/TableFunctions/TableFunctionExplain.cpp | 6 +- src/TableFunctions/TableFunctionExplain.h | 4 +- src/TableFunctions/TableFunctionFile.cpp | 2 +- src/TableFunctions/TableFunctionFile.h | 2 +- src/TableFunctions/TableFunctionFormat.cpp | 6 +- src/TableFunctions/TableFunctionFormat.h | 4 +- .../TableFunctionGenerateRandom.cpp | 6 +- .../TableFunctionGenerateRandom.h | 4 +- src/TableFunctions/TableFunctionHDFS.cpp | 2 +- src/TableFunctions/TableFunctionHDFS.h | 2 +- src/TableFunctions/TableFunctionInput.cpp | 6 +- src/TableFunctions/TableFunctionInput.h | 4 +- .../TableFunctionMeiliSearch.cpp | 4 +- src/TableFunctions/TableFunctionMeiliSearch.h | 4 +- src/TableFunctions/TableFunctionMerge.cpp | 6 +- src/TableFunctions/TableFunctionMerge.h | 4 +- src/TableFunctions/TableFunctionMongoDB.cpp | 6 +- src/TableFunctions/TableFunctionMongoDB.h | 4 +- src/TableFunctions/TableFunctionMySQL.cpp | 5 +- src/TableFunctions/TableFunctionMySQL.h | 4 +- src/TableFunctions/TableFunctionNull.cpp | 4 +- src/TableFunctions/TableFunctionNull.h | 4 +- src/TableFunctions/TableFunctionNumbers.cpp | 4 +- src/TableFunctions/TableFunctionNumbers.h | 4 +- .../TableFunctionPostgreSQL.cpp | 4 +- src/TableFunctions/TableFunctionPostgreSQL.h | 4 +- src/TableFunctions/TableFunctionRedis.cpp | 6 +- src/TableFunctions/TableFunctionRedis.h | 4 +- src/TableFunctions/TableFunctionRemote.cpp | 6 +- src/TableFunctions/TableFunctionRemote.h | 4 +- src/TableFunctions/TableFunctionS3.cpp | 4 +- src/TableFunctions/TableFunctionS3.h | 5 +- src/TableFunctions/TableFunctionS3Cluster.cpp | 2 +- src/TableFunctions/TableFunctionS3Cluster.h | 3 +- src/TableFunctions/TableFunctionSQLite.cpp | 4 +- src/TableFunctions/TableFunctionSQLite.h | 4 +- src/TableFunctions/TableFunctionURL.cpp | 2 +- src/TableFunctions/TableFunctionURL.h | 2 +- .../TableFunctionURLCluster.cpp | 2 +- src/TableFunctions/TableFunctionValues.cpp | 6 +- src/TableFunctions/TableFunctionValues.h | 4 +- src/TableFunctions/TableFunctionView.cpp | 6 +- src/TableFunctions/TableFunctionView.h | 4 +- .../TableFunctionViewIfPermitted.cpp | 8 +-- .../TableFunctionViewIfPermitted.h | 4 +- src/TableFunctions/TableFunctionZeros.cpp | 4 +- src/TableFunctions/TableFunctionZeros.h | 4 +- 67 files changed, 176 insertions(+), 174 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d0bb3dd389f..35a399b3ce7 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -764,7 +764,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti /// Table function without columns list. auto table_function_ast = create.as_table_function->ptr(); auto table_function = TableFunctionFactory::instance().get(table_function_ast, getContext()); - properties.columns = table_function->getActualTableStructure(getContext()); + properties.columns = table_function->getActualTableStructure(getContext(), /*is_insert_query*/ true); } else if (create.is_dictionary) { diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 0b9eea86b46..26d9e5254f3 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -96,7 +96,7 @@ BlockIO InterpreterDescribeQuery::execute() else if (table_expression.table_function) { TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, getContext()); - auto table_function_column_descriptions = table_function_ptr->getActualTableStructure(getContext()); + auto table_function_column_descriptions = table_function_ptr->getActualTableStructure(getContext(), /*is_insert_query*/ true); for (const auto & table_function_column_description : table_function_column_descriptions) columns.emplace_back(table_function_column_description); } diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 2485f032808..c2c7700a62f 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -257,7 +257,7 @@ void registerStorageAzureBlob(StorageFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); auto configuration = StorageAzureBlob::getConfiguration(engine_args, args.getLocalContext()); - auto client = StorageAzureBlob::createClient(configuration); + auto client = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current // session and user are ignored. @@ -309,35 +309,41 @@ void registerStorageAzureBlob(StorageFactory & factory) }); } -AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration configuration) +static bool containerExists(std::unique_ptr &blob_service_client, std::string container_name) +{ + Azure::Storage::Blobs::ListBlobContainersOptions options; + options.Prefix = container_name; + options.PageSizeHint = 1; + + auto containers_list_response = blob_service_client->ListBlobContainers(options); + auto containers_list = containers_list_response.BlobContainers; + + for (const auto & container : containers_list) + { + if (container_name == container.Name) + return true; + } + return false; +} + +AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration configuration, bool is_read_only) { AzureClientPtr result; if (configuration.is_connection_string) { std::unique_ptr blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(configuration.connection_url)); - - Azure::Storage::Blobs::ListBlobContainersOptions options; - options.Prefix = configuration.container; - options.PageSizeHint = 1; - - auto containers_list_response = blob_service_client->ListBlobContainers(options); - auto containers_list = containers_list_response.BlobContainers; - - bool container_exists = false; - for (const auto & container : containers_list) - { - if (configuration.container == container.Name) - { - container_exists = true; - break; - } - } - result = std::make_unique(BlobContainerClient::CreateFromConnectionString(configuration.connection_url, configuration.container)); + bool container_exists = containerExists(blob_service_client,configuration.container); if (!container_exists) { + if (is_read_only) + throw Exception( + ErrorCodes::DATABASE_ACCESS_DENIED, + "AzureBlobStorage container does not exist '{}'", + configuration.blob_path); + result->CreateIfNotExists(); } } @@ -360,22 +366,7 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co blob_service_client = std::make_unique(configuration.connection_url); } - Azure::Storage::Blobs::ListBlobContainersOptions options; - options.Prefix = configuration.container; - options.PageSizeHint = 1; - - auto containers_list_response = blob_service_client->ListBlobContainers(options); - auto containers_list = containers_list_response.BlobContainers; - - bool container_exists = false; - for (const auto & container : containers_list) - { - if (configuration.container == container.Name) - { - container_exists = true; - break; - } - } + bool container_exists = containerExists(blob_service_client,configuration.container); if (container_exists) { @@ -398,6 +389,11 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co } else { + if (is_read_only) + throw Exception( + ErrorCodes::DATABASE_ACCESS_DENIED, + "AzureBlobStorage container does not exist '{}'", + configuration.blob_path); result = std::make_unique(blob_service_client->CreateBlobContainer(configuration.container).Value); } } @@ -470,7 +466,7 @@ void StorageAzureBlob::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont { throw Exception( ErrorCodes::DATABASE_ACCESS_DENIED, - "S3 key '{}' contains globs, so the table is in readonly mode", + "AzureBlobStorage key '{}' contains globs, so the table is in readonly mode", configuration.blob_path); } @@ -1259,7 +1255,7 @@ ColumnsDescription StorageAzureBlob::getTableStructureFromData( return nullptr; } - /// S3 file iterator could get new keys after new iteration, check them in schema cache. + ///AzureBlobStorage file iterator could get new keys after new iteration, check them in schema cache. if (ctx->getSettingsRef().schema_inference_use_cache_for_azure && read_keys.size() > prev_read_keys_size) { columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), configuration, format_settings, ctx); diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 3d6b0c64998..9c5541dac38 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -65,7 +65,7 @@ public: ASTPtr partition_by_); static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context); - static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration); + static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration, bool is_read_only); static AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context); diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index e5fc01be9f4..604da0df567 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -39,7 +39,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( if (shard_info.isLocal()) { TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_func_ptr, context); - return table_function_ptr->getActualTableStructure(context); + return table_function_ptr->getActualTableStructure(context, /*is_insert_query*/ true); } auto table_func_name = queryToString(table_func_ptr); diff --git a/src/TableFunctions/Hive/TableFunctionHive.cpp b/src/TableFunctions/Hive/TableFunctionHive.cpp index fb7635181dc..ebebee13092 100644 --- a/src/TableFunctions/Hive/TableFunctionHive.cpp +++ b/src/TableFunctions/Hive/TableFunctionHive.cpp @@ -49,13 +49,14 @@ namespace DB actual_columns = parseColumnsListFromString(table_structure, context_); } - ColumnsDescription TableFunctionHive::getActualTableStructure(ContextPtr /*context_*/) const { return actual_columns; } + ColumnsDescription TableFunctionHive::getActualTableStructure(ContextPtr /*context_*/, bool /*is_insert_query*/) const { return actual_columns; } StoragePtr TableFunctionHive::executeImpl( const ASTPtr & /*ast_function_*/, ContextPtr context_, const std::string & table_name_, - ColumnsDescription /*cached_columns_*/) const + ColumnsDescription /*cached_columns_*/, + bool /*is_insert_query*/) const { const Settings & settings = context_->getSettings(); ParserExpression partition_by_parser; diff --git a/src/TableFunctions/Hive/TableFunctionHive.h b/src/TableFunctions/Hive/TableFunctionHive.h index ec09a87a876..5e48be46ce1 100644 --- a/src/TableFunctions/Hive/TableFunctionHive.h +++ b/src/TableFunctions/Hive/TableFunctionHive.h @@ -17,10 +17,10 @@ public: bool hasStaticStructure() const override { return true; } StoragePtr executeImpl( - const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return storage_type_name; } - ColumnsDescription getActualTableStructure(ContextPtr) const override; + ColumnsDescription getActualTableStructure(ContextPtr, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function_, ContextPtr context_) override; private: diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index df19e0ebad3..137e1dc27fe 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -34,15 +34,15 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, ContextPtr conte auto context_to_use = use_global_context ? context->getGlobalContext() : context; if (cached_columns.empty()) - return executeImpl(ast_function, context, table_name, std::move(cached_columns)); + return executeImpl(ast_function, context, table_name, std::move(cached_columns), is_insert_query); - if (hasStaticStructure() && cached_columns == getActualTableStructure(context)) - return executeImpl(ast_function, context_to_use, table_name, std::move(cached_columns)); + if (hasStaticStructure() && cached_columns == getActualTableStructure(context,is_insert_query)) + return executeImpl(ast_function, context_to_use, table_name, std::move(cached_columns), is_insert_query); auto this_table_function = shared_from_this(); auto get_storage = [=]() -> StoragePtr { - return this_table_function->executeImpl(ast_function, context_to_use, table_name, cached_columns); + return this_table_function->executeImpl(ast_function, context_to_use, table_name, cached_columns, is_insert_query); }; /// It will request actual table structure and create underlying storage lazily diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index fe71005cb9c..028bbb99c15 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -58,7 +58,7 @@ public: virtual void parseArguments(const ASTPtr & /*ast_function*/, ContextPtr /*context*/) {} /// Returns actual table structure probably requested from remote server, may fail - virtual ColumnsDescription getActualTableStructure(ContextPtr /*context*/) const = 0; + virtual ColumnsDescription getActualTableStructure(ContextPtr /*context*/, bool is_insert_query) const = 0; /// Check if table function needs a structure hint from SELECT query in case of /// INSERT INTO FUNCTION ... SELECT ... and INSERT INTO ... SELECT ... FROM table_function(...) @@ -89,7 +89,7 @@ protected: private: virtual StoragePtr executeImpl( - const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const = 0; + const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const = 0; virtual const char * getStorageTypeName() const = 0; }; diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index f87838cfb56..6d50e9138ff 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -26,7 +26,8 @@ protected: const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, - ColumnsDescription /*cached_columns*/) const override + ColumnsDescription /*cached_columns*/, + bool /*is_insert_query*/) const override { ColumnsDescription columns; if (TableFunction::configuration.structure != "auto") @@ -42,7 +43,7 @@ protected: const char * getStorageTypeName() const override { return Storage::name; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override + ColumnsDescription getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const override { if (TableFunction::configuration.structure == "auto") { diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index a60ab70d570..487826dc363 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -110,7 +110,7 @@ void ITableFunctionFileLike::addColumnsStructureToArguments(ASTs & args, const S } } -StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const { ColumnsDescription columns; if (structure != "auto") diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 8300cc27591..49a144c17fc 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -48,7 +48,7 @@ protected: ColumnsDescription structure_hint; private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; virtual StoragePtr getStorage( const String & source, const String & format, const ColumnsDescription & columns, ContextPtr global_context, diff --git a/src/TableFunctions/ITableFunctionXDBC.cpp b/src/TableFunctions/ITableFunctionXDBC.cpp index 1fb0f392e33..59702259b35 100644 --- a/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/src/TableFunctions/ITableFunctionXDBC.cpp @@ -61,7 +61,7 @@ void ITableFunctionXDBC::startBridgeIfNot(ContextPtr context) const } } -ColumnsDescription ITableFunctionXDBC::getActualTableStructure(ContextPtr context) const +ColumnsDescription ITableFunctionXDBC::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { startBridgeIfNot(context); @@ -92,10 +92,10 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(ContextPtr contex return ColumnsDescription{columns}; } -StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { startBridgeIfNot(context); - auto columns = getActualTableStructure(context); + auto columns = getActualTableStructure(context, is_insert_query); auto result = std::make_shared( StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, columns, ConstraintsDescription{}, String{}, context, helper); result->startup(); diff --git a/src/TableFunctions/ITableFunctionXDBC.h b/src/TableFunctions/ITableFunctionXDBC.h index 984a6a1957f..da0fa83033b 100644 --- a/src/TableFunctions/ITableFunctionXDBC.h +++ b/src/TableFunctions/ITableFunctionXDBC.h @@ -16,7 +16,7 @@ namespace DB class ITableFunctionXDBC : public ITableFunction { private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; /* A factory method to create bridge helper, that will assist in remote interaction */ virtual BridgeHelperPtr createBridgeHelper(ContextPtr context, @@ -24,7 +24,7 @@ private: const std::string & connection_string_, bool use_connection_pooling_) const = 0; - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index d51942d133d..278f602d5ff 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -193,12 +193,12 @@ void TableFunctionAzureBlobStorage::parseArguments(const ASTPtr & ast_function, configuration = parseArgumentsImpl(args, context); } -ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(ContextPtr context, bool is_insert_query) const { if (configuration.structure == "auto") { context->checkAccess(getSourceAccessType()); - auto client = StorageAzureBlob::createClient(configuration); + auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); auto settings = StorageAzureBlob::createSettings(context); auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings)); @@ -213,9 +213,9 @@ bool TableFunctionAzureBlobStorage::supportsReadingSubsetOfColumns() return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); } -StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { - auto client = StorageAzureBlob::createClient(configuration); + auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); auto settings = StorageAzureBlob::createSettings(context); ColumnsDescription columns; diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.h b/src/TableFunctions/TableFunctionAzureBlobStorage.h index 0ac3f9771c7..e4fcd4dda31 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.h +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.h @@ -54,11 +54,12 @@ protected: const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, - ColumnsDescription cached_columns) const override; + ColumnsDescription cached_columns, + bool is_insert_query) const override; const char * getStorageTypeName() const override { return "Azure"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; mutable StorageAzureBlob::Configuration configuration; diff --git a/src/TableFunctions/TableFunctionDictionary.cpp b/src/TableFunctions/TableFunctionDictionary.cpp index 90db9550a72..f0060acb411 100644 --- a/src/TableFunctions/TableFunctionDictionary.cpp +++ b/src/TableFunctions/TableFunctionDictionary.cpp @@ -43,7 +43,7 @@ void TableFunctionDictionary::parseArguments(const ASTPtr & ast_function, Contex dictionary_name = checkAndGetLiteralArgument(args[0], "dictionary_name"); } -ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { const ExternalDictionariesLoader & external_loader = context->getExternalDictionariesLoader(); std::string resolved_name = external_loader.resolveDictionaryName(dictionary_name, context->getCurrentDatabase()); @@ -76,10 +76,10 @@ ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr c } StoragePtr TableFunctionDictionary::executeImpl( - const ASTPtr &, ContextPtr context, const std::string & table_name, ColumnsDescription) const + const ASTPtr &, ContextPtr context, const std::string & table_name, ColumnsDescription, bool is_insert_query) const { StorageID dict_id(getDatabaseName(), table_name); - auto dictionary_table_structure = getActualTableStructure(context); + auto dictionary_table_structure = getActualTableStructure(context, is_insert_query); auto result = std::make_shared( dict_id, dictionary_name, std::move(dictionary_table_structure), String{}, StorageDictionary::Location::Custom, context); diff --git a/src/TableFunctions/TableFunctionDictionary.h b/src/TableFunctions/TableFunctionDictionary.h index cc184a32a17..d0beb292fe1 100644 --- a/src/TableFunctions/TableFunctionDictionary.h +++ b/src/TableFunctions/TableFunctionDictionary.h @@ -18,9 +18,9 @@ public: void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "Dictionary"; } diff --git a/src/TableFunctions/TableFunctionExecutable.cpp b/src/TableFunctions/TableFunctionExecutable.cpp index 66e7ce7592c..5a64a988156 100644 --- a/src/TableFunctions/TableFunctionExecutable.cpp +++ b/src/TableFunctions/TableFunctionExecutable.cpp @@ -120,12 +120,12 @@ void TableFunctionExecutable::parseArguments(const ASTPtr & ast_function, Contex } } -ColumnsDescription TableFunctionExecutable::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionExecutable::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { return parseColumnsListFromString(structure, context); } -StoragePtr TableFunctionExecutable::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionExecutable::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { auto storage_id = StorageID(getDatabaseName(), table_name); auto global_context = context->getGlobalContext(); @@ -135,7 +135,7 @@ StoragePtr TableFunctionExecutable::executeImpl(const ASTPtr & /*ast_function*/, if (settings_query != nullptr) settings.applyChanges(settings_query->as()->changes); - auto storage = std::make_shared(storage_id, format, settings, input_queries, getActualTableStructure(context), ConstraintsDescription{}); + auto storage = std::make_shared(storage_id, format, settings, input_queries, getActualTableStructure(context, is_insert_query), ConstraintsDescription{}); storage->startup(); return storage; } diff --git a/src/TableFunctions/TableFunctionExecutable.h b/src/TableFunctions/TableFunctionExecutable.h index 2d9f86e14e6..aa595312fe4 100644 --- a/src/TableFunctions/TableFunctionExecutable.h +++ b/src/TableFunctions/TableFunctionExecutable.h @@ -24,11 +24,11 @@ public: bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "Executable"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; std::vector skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override; diff --git a/src/TableFunctions/TableFunctionExplain.cpp b/src/TableFunctions/TableFunctionExplain.cpp index aae93c8b596..f127979d92a 100644 --- a/src/TableFunctions/TableFunctionExplain.cpp +++ b/src/TableFunctions/TableFunctionExplain.cpp @@ -91,7 +91,7 @@ void TableFunctionExplain::parseArguments(const ASTPtr & ast_function, ContextPt query = std::move(explain_query); } -ColumnsDescription TableFunctionExplain::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionExplain::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { Block sample_block = getInterpreter(context).getSampleBlock(query->as()->getKind()); ColumnsDescription columns_description; @@ -123,7 +123,7 @@ static Block executeMonoBlock(QueryPipeline & pipeline) } StoragePtr TableFunctionExplain::executeImpl( - const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const + const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { /// To support settings inside explain subquery. auto mutable_context = Context::createCopy(context); @@ -132,7 +132,7 @@ StoragePtr TableFunctionExplain::executeImpl( Block block = executeMonoBlock(blockio.pipeline); StorageID storage_id(getDatabaseName(), table_name); - auto storage = std::make_shared(storage_id, getActualTableStructure(context), std::move(block)); + auto storage = std::make_shared(storage_id, getActualTableStructure(context, is_insert_query), std::move(block)); storage->startup(); return storage; } diff --git a/src/TableFunctions/TableFunctionExplain.h b/src/TableFunctions/TableFunctionExplain.h index 99d3e52ee68..2eb7e35d0b5 100644 --- a/src/TableFunctions/TableFunctionExplain.h +++ b/src/TableFunctions/TableFunctionExplain.h @@ -17,7 +17,7 @@ public: std::string getName() const override { return name; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "Explain"; } @@ -25,7 +25,7 @@ private: void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; InterpreterExplainQuery getInterpreter(ContextPtr context) const; diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 0e49f26db40..806ea332636 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -83,7 +83,7 @@ StoragePtr TableFunctionFile::getStorage(const String & source, return std::make_shared(source, global_context->getUserFilesPath(), args); } -ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { if (structure == "auto") { diff --git a/src/TableFunctions/TableFunctionFile.h b/src/TableFunctions/TableFunctionFile.h index 797948cad03..439ae87b4ae 100644 --- a/src/TableFunctions/TableFunctionFile.h +++ b/src/TableFunctions/TableFunctionFile.h @@ -20,7 +20,7 @@ public: return name; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override { diff --git a/src/TableFunctions/TableFunctionFormat.cpp b/src/TableFunctions/TableFunctionFormat.cpp index f5aff4bd098..8d788a7974e 100644 --- a/src/TableFunctions/TableFunctionFormat.cpp +++ b/src/TableFunctions/TableFunctionFormat.cpp @@ -51,7 +51,7 @@ void TableFunctionFormat::parseArguments(const ASTPtr & ast_function, ContextPtr structure = checkAndGetLiteralArgument(args[1], "structure"); } -ColumnsDescription TableFunctionFormat::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionFormat::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { if (structure == "auto") { @@ -87,9 +87,9 @@ Block TableFunctionFormat::parseData(ColumnsDescription columns, ContextPtr cont return concatenateBlocks(blocks); } -StoragePtr TableFunctionFormat::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionFormat::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { - auto columns = getActualTableStructure(context); + auto columns = getActualTableStructure(context, is_insert_query); Block res_block = parseData(columns, context); auto res = std::make_shared(StorageID(getDatabaseName(), table_name), columns, res_block); res->startup(); diff --git a/src/TableFunctions/TableFunctionFormat.h b/src/TableFunctions/TableFunctionFormat.h index d64ab14cb64..e20e8b6ea4b 100644 --- a/src/TableFunctions/TableFunctionFormat.h +++ b/src/TableFunctions/TableFunctionFormat.h @@ -18,10 +18,10 @@ public: bool hasStaticStructure() const override { return false; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "Values"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; Block parseData(ColumnsDescription columns, ContextPtr context) const; diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 08059796660..c6a9154cc66 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -97,7 +97,7 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co } } -ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { if (structure == "auto") { @@ -113,9 +113,9 @@ ColumnsDescription TableFunctionGenerateRandom::getActualTableStructure(ContextP return parseColumnsListFromString(structure, context); } -StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { - ColumnsDescription columns = getActualTableStructure(context); + ColumnsDescription columns = getActualTableStructure(context, is_insert_query); auto res = std::make_shared( StorageID(getDatabaseName(), table_name), columns, String{}, max_array_length, max_string_length, random_seed); res->startup(); diff --git a/src/TableFunctions/TableFunctionGenerateRandom.h b/src/TableFunctions/TableFunctionGenerateRandom.h index 584d65311f4..a5d11ce0af6 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.h +++ b/src/TableFunctions/TableFunctionGenerateRandom.h @@ -19,10 +19,10 @@ public: void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "GenerateRandom"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; String structure = "auto"; diff --git a/src/TableFunctions/TableFunctionHDFS.cpp b/src/TableFunctions/TableFunctionHDFS.cpp index 7aab55b48c9..a8e2108fda8 100644 --- a/src/TableFunctions/TableFunctionHDFS.cpp +++ b/src/TableFunctions/TableFunctionHDFS.cpp @@ -28,7 +28,7 @@ StoragePtr TableFunctionHDFS::getStorage( compression_method_); } -ColumnsDescription TableFunctionHDFS::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionHDFS::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { if (structure == "auto") { diff --git a/src/TableFunctions/TableFunctionHDFS.h b/src/TableFunctions/TableFunctionHDFS.h index eec0a05fe8d..a7eb5daa440 100644 --- a/src/TableFunctions/TableFunctionHDFS.h +++ b/src/TableFunctions/TableFunctionHDFS.h @@ -34,7 +34,7 @@ public: return signature; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override { diff --git a/src/TableFunctions/TableFunctionInput.cpp b/src/TableFunctions/TableFunctionInput.cpp index 4941241acae..658a55c6fc4 100644 --- a/src/TableFunctions/TableFunctionInput.cpp +++ b/src/TableFunctions/TableFunctionInput.cpp @@ -43,7 +43,7 @@ void TableFunctionInput::parseArguments(const ASTPtr & ast_function, ContextPtr structure = checkAndGetLiteralArgument(evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context), "structure"); } -ColumnsDescription TableFunctionInput::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionInput::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { if (structure == "auto") { @@ -58,9 +58,9 @@ ColumnsDescription TableFunctionInput::getActualTableStructure(ContextPtr contex return parseColumnsListFromString(structure, context); } -StoragePtr TableFunctionInput::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionInput::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { - auto storage = std::make_shared(StorageID(getDatabaseName(), table_name), getActualTableStructure(context)); + auto storage = std::make_shared(StorageID(getDatabaseName(), table_name), getActualTableStructure(context, is_insert_query)); storage->startup(); return storage; } diff --git a/src/TableFunctions/TableFunctionInput.h b/src/TableFunctions/TableFunctionInput.h index 8e7b34cb829..3164ce43eef 100644 --- a/src/TableFunctions/TableFunctionInput.h +++ b/src/TableFunctions/TableFunctionInput.h @@ -20,10 +20,10 @@ public: void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "Input"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; String structure; diff --git a/src/TableFunctions/TableFunctionMeiliSearch.cpp b/src/TableFunctions/TableFunctionMeiliSearch.cpp index 41ae5bb9ee2..01840a80262 100644 --- a/src/TableFunctions/TableFunctionMeiliSearch.cpp +++ b/src/TableFunctions/TableFunctionMeiliSearch.cpp @@ -8,13 +8,13 @@ namespace DB { StoragePtr TableFunctionMeiliSearch::executeImpl( - const ASTPtr & /* ast_function */, ContextPtr /*context*/, const String & table_name, ColumnsDescription /*cached_columns*/) const + const ASTPtr & /* ast_function */, ContextPtr /*context*/, const String & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const { return std::make_shared( StorageID(getDatabaseName(), table_name), configuration.value(), ColumnsDescription{}, ConstraintsDescription{}, String{}); } -ColumnsDescription TableFunctionMeiliSearch::getActualTableStructure(ContextPtr /* context */) const +ColumnsDescription TableFunctionMeiliSearch::getActualTableStructure(ContextPtr /* context */, bool /*is_insert_query*/) const { return StorageMeiliSearch::getTableStructureFromData(configuration.value()); } diff --git a/src/TableFunctions/TableFunctionMeiliSearch.h b/src/TableFunctions/TableFunctionMeiliSearch.h index 86be944ab12..a127809a9c5 100644 --- a/src/TableFunctions/TableFunctionMeiliSearch.h +++ b/src/TableFunctions/TableFunctionMeiliSearch.h @@ -13,11 +13,11 @@ public: private: StoragePtr executeImpl( - const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns) const override; + const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "meilisearch"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; std::optional configuration; diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 586cee54085..599953a1add 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -118,7 +118,7 @@ const TableFunctionMerge::DBToTableSetMap & TableFunctionMerge::getSourceDatabas return *source_databases_and_tables; } -ColumnsDescription TableFunctionMerge::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionMerge::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { for (const auto & db_with_tables : getSourceDatabasesAndTables(context)) { @@ -134,11 +134,11 @@ ColumnsDescription TableFunctionMerge::getActualTableStructure(ContextPtr contex } -StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { auto res = std::make_shared( StorageID(getDatabaseName(), table_name), - getActualTableStructure(context), + getActualTableStructure(context, is_insert_query), String{}, source_database_name_or_regexp, database_is_regexp, diff --git a/src/TableFunctions/TableFunctionMerge.h b/src/TableFunctions/TableFunctionMerge.h index 3439056deda..8cc5119978a 100644 --- a/src/TableFunctions/TableFunctionMerge.h +++ b/src/TableFunctions/TableFunctionMerge.h @@ -17,13 +17,13 @@ public: std::string getName() const override { return name; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "Merge"; } using TableSet = std::set; using DBToTableSetMap = std::map; const DBToTableSetMap & getSourceDatabasesAndTables(ContextPtr context) const; - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; std::vector skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; static TableSet getMatchedTablesWithAccess(const String & database_name, const String & table_regexp, const ContextPtr & context); diff --git a/src/TableFunctions/TableFunctionMongoDB.cpp b/src/TableFunctions/TableFunctionMongoDB.cpp index 31dd64f8254..5c7c1d98cdf 100644 --- a/src/TableFunctions/TableFunctionMongoDB.cpp +++ b/src/TableFunctions/TableFunctionMongoDB.cpp @@ -27,9 +27,9 @@ namespace ErrorCodes StoragePtr TableFunctionMongoDB::executeImpl(const ASTPtr & /*ast_function*/, - ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/) const + ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { - auto columns = getActualTableStructure(context); + auto columns = getActualTableStructure(context, is_insert_query); auto storage = std::make_shared( StorageID(configuration->database, table_name), configuration->host, @@ -46,7 +46,7 @@ StoragePtr TableFunctionMongoDB::executeImpl(const ASTPtr & /*ast_function*/, return storage; } -ColumnsDescription TableFunctionMongoDB::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionMongoDB::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { return parseColumnsListFromString(structure, context); } diff --git a/src/TableFunctions/TableFunctionMongoDB.h b/src/TableFunctions/TableFunctionMongoDB.h index b5033b2d654..c2c15cabe5a 100644 --- a/src/TableFunctions/TableFunctionMongoDB.h +++ b/src/TableFunctions/TableFunctionMongoDB.h @@ -17,11 +17,11 @@ public: private: StoragePtr executeImpl( const ASTPtr & ast_function, ContextPtr context, - const std::string & table_name, ColumnsDescription cached_columns) const override; + const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "MongoDB"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; std::optional configuration; diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index 1705dfcbfa5..03bd2264551 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -57,7 +57,7 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, ContextPtr pool.emplace(createMySQLPoolWithFailover(*configuration, mysql_settings)); } -ColumnsDescription TableFunctionMySQL::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionMySQL::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { return StorageMySQL::getTableStructureFromData(*pool, configuration->database, configuration->table, context); } @@ -66,7 +66,8 @@ StoragePtr TableFunctionMySQL::executeImpl( const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, - ColumnsDescription /*cached_columns*/) const + ColumnsDescription /*cached_columns*/, + bool /*is_insert_query*/) const { auto res = std::make_shared( StorageID(getDatabaseName(), table_name), diff --git a/src/TableFunctions/TableFunctionMySQL.h b/src/TableFunctions/TableFunctionMySQL.h index 5a230530bc4..04f619f5f4b 100644 --- a/src/TableFunctions/TableFunctionMySQL.h +++ b/src/TableFunctions/TableFunctionMySQL.h @@ -23,10 +23,10 @@ public: return name; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "MySQL"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; mutable std::optional pool; diff --git a/src/TableFunctions/TableFunctionNull.cpp b/src/TableFunctions/TableFunctionNull.cpp index d25b9e15aa7..57911e16d4b 100644 --- a/src/TableFunctions/TableFunctionNull.cpp +++ b/src/TableFunctions/TableFunctionNull.cpp @@ -32,14 +32,14 @@ void TableFunctionNull::parseArguments(const ASTPtr & ast_function, ContextPtr c structure = checkAndGetLiteralArgument(evaluateConstantExpressionOrIdentifierAsLiteral(arguments[0], context), "structure"); } -ColumnsDescription TableFunctionNull::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionNull::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { if (structure != "auto") return parseColumnsListFromString(structure, context); return default_structure; } -StoragePtr TableFunctionNull::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionNull::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const { ColumnsDescription columns; if (structure != "auto") diff --git a/src/TableFunctions/TableFunctionNull.h b/src/TableFunctions/TableFunctionNull.h index 4fece9e6da9..e80552d4cff 100644 --- a/src/TableFunctions/TableFunctionNull.h +++ b/src/TableFunctions/TableFunctionNull.h @@ -23,11 +23,11 @@ public: void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "Null"; } void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; String structure = "auto"; ColumnsDescription structure_hint; diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index ba7a4dc4b36..d6cf50bc7d6 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -23,14 +23,14 @@ namespace ErrorCodes template -ColumnsDescription TableFunctionNumbers::getActualTableStructure(ContextPtr /*context*/) const +ColumnsDescription TableFunctionNumbers::getActualTableStructure(ContextPtr /*context*/, bool /*is_insert_query*/) const { /// NOTE: https://bugs.llvm.org/show_bug.cgi?id=47418 return ColumnsDescription{{{"number", std::make_shared()}}}; } template -StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const { if (const auto * function = ast_function->as()) { diff --git a/src/TableFunctions/TableFunctionNumbers.h b/src/TableFunctions/TableFunctionNumbers.h index 0a2f3eb863e..e380f40f7b2 100644 --- a/src/TableFunctions/TableFunctionNumbers.h +++ b/src/TableFunctions/TableFunctionNumbers.h @@ -19,12 +19,12 @@ public: std::string getName() const override { return name; } bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "SystemNumbers"; } UInt64 evaluateArgument(ContextPtr context, ASTPtr & argument) const; - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; }; diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index 2b17a1b2c2b..322e0df7c15 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, - ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const + ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const { auto result = std::make_shared( StorageID(getDatabaseName(), table_name), @@ -38,7 +38,7 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, } -ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { return StoragePostgreSQL::getTableStructureFromData(connection_pool, configuration->table, configuration->schema, context); } diff --git a/src/TableFunctions/TableFunctionPostgreSQL.h b/src/TableFunctions/TableFunctionPostgreSQL.h index 9f10e1c180e..f7d77567dd4 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.h +++ b/src/TableFunctions/TableFunctionPostgreSQL.h @@ -20,11 +20,11 @@ public: private: StoragePtr executeImpl( const ASTPtr & ast_function, ContextPtr context, - const std::string & table_name, ColumnsDescription cached_columns) const override; + const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "PostgreSQL"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; postgres::PoolWithFailoverPtr connection_pool; diff --git a/src/TableFunctions/TableFunctionRedis.cpp b/src/TableFunctions/TableFunctionRedis.cpp index ec659ae61e0..0b7433845b4 100644 --- a/src/TableFunctions/TableFunctionRedis.cpp +++ b/src/TableFunctions/TableFunctionRedis.cpp @@ -25,9 +25,9 @@ namespace ErrorCodes } StoragePtr TableFunctionRedis::executeImpl( - const ASTPtr & /*ast_function*/, ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/) const + const ASTPtr & /*ast_function*/, ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { - auto columns = getActualTableStructure(context); + auto columns = getActualTableStructure(context, is_insert_query); StorageInMemoryMetadata metadata; metadata.setColumns(columns); @@ -39,7 +39,7 @@ StoragePtr TableFunctionRedis::executeImpl( return storage; } -ColumnsDescription TableFunctionRedis::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionRedis::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { return parseColumnsListFromString(structure, context); } diff --git a/src/TableFunctions/TableFunctionRedis.h b/src/TableFunctions/TableFunctionRedis.h index b985a89e3d7..a7fc0df0a15 100644 --- a/src/TableFunctions/TableFunctionRedis.h +++ b/src/TableFunctions/TableFunctionRedis.h @@ -19,11 +19,11 @@ public: private: StoragePtr executeImpl( const ASTPtr & ast_function, ContextPtr context, - const String & table_name, ColumnsDescription cached_columns) const override; + const String & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "Redis"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; RedisConfiguration configuration; diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 4143014a7b3..72b5fb9b40e 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -276,12 +276,12 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr remote_table_id.table_name = table; } -StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const +StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const { /// StorageDistributed supports mismatching structure of remote table, so we can use outdated structure for CREATE ... AS remote(...) /// without additional conversion in StorageTableFunctionProxy if (cached_columns.empty()) - cached_columns = getActualTableStructure(context); + cached_columns = getActualTableStructure(context, is_insert_query); assert(cluster); StoragePtr res = remote_table_function_ptr @@ -318,7 +318,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, Con return res; } -ColumnsDescription TableFunctionRemote::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionRemote::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { assert(cluster); return getStructureOfRemoteTable(*cluster, remote_table_id, context, remote_table_function_ptr); diff --git a/src/TableFunctions/TableFunctionRemote.h b/src/TableFunctions/TableFunctionRemote.h index 32039d1e6a8..0f75bf2b854 100644 --- a/src/TableFunctions/TableFunctionRemote.h +++ b/src/TableFunctions/TableFunctionRemote.h @@ -22,13 +22,13 @@ public: std::string getName() const override { return name; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; bool needStructureConversion() const override { return false; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "Distributed"; } void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index c8cc0cddd30..d1955e5028b 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -292,7 +292,7 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String & } } -ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { if (configuration.structure == "auto") { @@ -309,7 +309,7 @@ bool TableFunctionS3::supportsReadingSubsetOfColumns() return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); } -StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const { S3::URI s3_uri (configuration.url); diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index c983bec9bf4..4a469923b14 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -64,11 +64,12 @@ protected: const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, - ColumnsDescription cached_columns) const override; + ColumnsDescription cached_columns, + bool is_insert_query) const override; const char * getStorageTypeName() const override { return "S3"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; mutable StorageS3::Configuration configuration; diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 1d93132c411..ce96f7f580b 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -17,7 +17,7 @@ namespace DB StoragePtr TableFunctionS3Cluster::executeImpl( const ASTPtr & /*function*/, ContextPtr context, - const std::string & table_name, ColumnsDescription /*cached_columns*/) const + const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const { StoragePtr storage; ColumnsDescription columns; diff --git a/src/TableFunctions/TableFunctionS3Cluster.h b/src/TableFunctions/TableFunctionS3Cluster.h index 459ff144f02..4fe25079cf4 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.h +++ b/src/TableFunctions/TableFunctionS3Cluster.h @@ -52,7 +52,8 @@ protected: const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, - ColumnsDescription cached_columns) const override; + ColumnsDescription cached_columns, + bool is_insert_query) const override; const char * getStorageTypeName() const override { return "S3Cluster"; } }; diff --git a/src/TableFunctions/TableFunctionSQLite.cpp b/src/TableFunctions/TableFunctionSQLite.cpp index a9831363bd9..27e6fcf1fd1 100644 --- a/src/TableFunctions/TableFunctionSQLite.cpp +++ b/src/TableFunctions/TableFunctionSQLite.cpp @@ -29,7 +29,7 @@ namespace ErrorCodes StoragePtr TableFunctionSQLite::executeImpl(const ASTPtr & /*ast_function*/, - ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/) const + ContextPtr context, const String & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const { auto storage = std::make_shared(StorageID(getDatabaseName(), table_name), sqlite_db, @@ -42,7 +42,7 @@ StoragePtr TableFunctionSQLite::executeImpl(const ASTPtr & /*ast_function*/, } -ColumnsDescription TableFunctionSQLite::getActualTableStructure(ContextPtr /* context */) const +ColumnsDescription TableFunctionSQLite::getActualTableStructure(ContextPtr /* context */, bool /*is_insert_query*/) const { return StorageSQLite::getTableStructureFromData(sqlite_db, remote_table_name); } diff --git a/src/TableFunctions/TableFunctionSQLite.h b/src/TableFunctions/TableFunctionSQLite.h index fded5646b39..74318f058a9 100644 --- a/src/TableFunctions/TableFunctionSQLite.h +++ b/src/TableFunctions/TableFunctionSQLite.h @@ -18,11 +18,11 @@ public: private: StoragePtr executeImpl( const ASTPtr & ast_function, ContextPtr context, - const std::string & table_name, ColumnsDescription cached_columns) const override; + const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "SQLite"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; String database_path, remote_table_name; diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 4ed204a2af3..8d5a023fc3b 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -118,7 +118,7 @@ StoragePtr TableFunctionURL::getStorage( configuration.http_method); } -ColumnsDescription TableFunctionURL::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionURL::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { if (structure == "auto") { diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index 021eb71df53..5e58a36dde9 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -32,7 +32,7 @@ public: return signature; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; static void addColumnsStructureToArguments(ASTs & args, const String & desired_structure, const ContextPtr & context); diff --git a/src/TableFunctions/TableFunctionURLCluster.cpp b/src/TableFunctions/TableFunctionURLCluster.cpp index c94943db758..a2949278155 100644 --- a/src/TableFunctions/TableFunctionURLCluster.cpp +++ b/src/TableFunctions/TableFunctionURLCluster.cpp @@ -38,7 +38,7 @@ StoragePtr TableFunctionURLCluster::getStorage( format, compression_method, StorageID(getDatabaseName(), table_name), - getActualTableStructure(context), + getActualTableStructure(context, /* is_insert_query */ true), ConstraintsDescription{}, configuration, structure != "auto"); diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index cf0e20c624c..42a19874704 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -119,14 +119,14 @@ void TableFunctionValues::parseArguments(const ASTPtr & ast_function, ContextPtr structure = ColumnsDescription(names_and_types); } -ColumnsDescription TableFunctionValues::getActualTableStructure(ContextPtr /*context*/) const +ColumnsDescription TableFunctionValues::getActualTableStructure(ContextPtr /*context*/, bool /*is_insert_query*/) const { return structure; } -StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { - auto columns = getActualTableStructure(context); + auto columns = getActualTableStructure(context, is_insert_query); Block sample_block; for (const auto & name_type : columns.getOrdinary()) diff --git a/src/TableFunctions/TableFunctionValues.h b/src/TableFunctions/TableFunctionValues.h index 61ce5158086..7c87bff835e 100644 --- a/src/TableFunctions/TableFunctionValues.h +++ b/src/TableFunctions/TableFunctionValues.h @@ -14,10 +14,10 @@ public: std::string getName() const override { return name; } bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "Values"; } - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; static DataTypes getTypesFromArgument(const ASTPtr & arg, ContextPtr context); diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index 6b50e7e0611..2a50fb2d006 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -41,7 +41,7 @@ void TableFunctionView::parseArguments(const ASTPtr & ast_function, ContextPtr / throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' requires a query argument.", getName()); } -ColumnsDescription TableFunctionView::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionView::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const { assert(create.select); assert(create.children.size() == 1); @@ -58,9 +58,9 @@ ColumnsDescription TableFunctionView::getActualTableStructure(ContextPtr context } StoragePtr TableFunctionView::executeImpl( - const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const + const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { - auto columns = getActualTableStructure(context); + auto columns = getActualTableStructure(context, is_insert_query); auto res = std::make_shared(StorageID(getDatabaseName(), table_name), create, columns, ""); res->startup(); return res; diff --git a/src/TableFunctions/TableFunctionView.h b/src/TableFunctions/TableFunctionView.h index bbf072655ed..c679a1f315d 100644 --- a/src/TableFunctions/TableFunctionView.h +++ b/src/TableFunctions/TableFunctionView.h @@ -21,7 +21,7 @@ public: const ASTSelectWithUnionQuery & getSelectQuery() const; private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "View"; } @@ -29,7 +29,7 @@ private: void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; ASTCreateQuery create; }; diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.cpp b/src/TableFunctions/TableFunctionViewIfPermitted.cpp index 12762e24f7e..d7944df1b28 100644 --- a/src/TableFunctions/TableFunctionViewIfPermitted.cpp +++ b/src/TableFunctions/TableFunctionViewIfPermitted.cpp @@ -55,16 +55,16 @@ void TableFunctionViewIfPermitted::parseArguments(const ASTPtr & ast_function, C else_table_function = TableFunctionFactory::instance().get(else_ast, context); } -ColumnsDescription TableFunctionViewIfPermitted::getActualTableStructure(ContextPtr context) const +ColumnsDescription TableFunctionViewIfPermitted::getActualTableStructure(ContextPtr context, bool is_insert_query) const { - return else_table_function->getActualTableStructure(context); + return else_table_function->getActualTableStructure(context, is_insert_query); } StoragePtr TableFunctionViewIfPermitted::executeImpl( - const ASTPtr & /* ast_function */, ContextPtr context, const std::string & table_name, ColumnsDescription /* cached_columns */) const + const ASTPtr & /* ast_function */, ContextPtr context, const std::string & table_name, ColumnsDescription /* cached_columns */, bool is_insert_query) const { StoragePtr storage; - auto columns = getActualTableStructure(context); + auto columns = getActualTableStructure(context, is_insert_query); if (isPermitted(context, columns)) { diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.h b/src/TableFunctions/TableFunctionViewIfPermitted.h index 9fdb34f30ab..bee4e15bfa5 100644 --- a/src/TableFunctions/TableFunctionViewIfPermitted.h +++ b/src/TableFunctions/TableFunctionViewIfPermitted.h @@ -20,7 +20,7 @@ public: const ASTSelectWithUnionQuery & getSelectQuery() const; private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "ViewIfPermitted"; } @@ -28,7 +28,7 @@ private: void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; bool isPermitted(const ContextPtr & context, const ColumnsDescription & else_columns) const; diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index 3c487362e1f..eb93626590e 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -20,14 +20,14 @@ extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; template -ColumnsDescription TableFunctionZeros::getActualTableStructure(ContextPtr /*context*/) const +ColumnsDescription TableFunctionZeros::getActualTableStructure(ContextPtr /*context*/, bool /*is_insert_query*/) const { /// NOTE: https://bugs.llvm.org/show_bug.cgi?id=47418 return ColumnsDescription{{{"zero", std::make_shared()}}}; } template -StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const { if (const auto * function = ast_function->as()) { diff --git a/src/TableFunctions/TableFunctionZeros.h b/src/TableFunctions/TableFunctionZeros.h index eef1577673e..07d523ee37c 100644 --- a/src/TableFunctions/TableFunctionZeros.h +++ b/src/TableFunctions/TableFunctionZeros.h @@ -19,12 +19,12 @@ public: std::string getName() const override { return name; } bool hasStaticStructure() const override { return true; } private: - StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const override; const char * getStorageTypeName() const override { return "SystemZeros"; } UInt64 evaluateArgument(ContextPtr context, ASTPtr & argument) const; - ColumnsDescription getActualTableStructure(ContextPtr context) const override; + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; }; From 56e45ba384e69ecabcaf1f147ef29a9b2e2dc872 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 6 Jul 2023 11:17:04 +0200 Subject: [PATCH 0409/2047] Added test for select query when container does not exist --- src/Storages/StorageAzureBlob.cpp | 4 ++-- tests/integration/test_storage_azure_blob_storage/test.py | 6 ++++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index c2c7700a62f..d0c51423376 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -342,7 +342,7 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co throw Exception( ErrorCodes::DATABASE_ACCESS_DENIED, "AzureBlobStorage container does not exist '{}'", - configuration.blob_path); + configuration.container); result->CreateIfNotExists(); } @@ -393,7 +393,7 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co throw Exception( ErrorCodes::DATABASE_ACCESS_DENIED, "AzureBlobStorage container does not exist '{}'", - configuration.blob_path); + configuration.container); result = std::make_unique(blob_service_client->CreateBlobContainer(configuration.container).Value); } } diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 6089466ff5d..87ed6e34116 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -611,3 +611,9 @@ def test_filter_using_file(cluster): query = f"select count(*) from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_partition_tf_*.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') WHERE _file='test_partition_tf_3.csv'" assert azure_query(node, query) == "1\n" + +def test_read_from_not_existing_container(cluster): + node = cluster.instances["node"] + query = f"select * from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont_not_exists', 'test_table.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto')" + expected_err_msg = "container does not exist" + assert expected_err_msg in node.query_and_get_error(query) From 69691fd28cebe0d6304bfd7b2fc3607656e816d1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 6 Jul 2023 09:38:16 +0000 Subject: [PATCH 0410/2047] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 87ed6e34116..4042e41c054 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -612,6 +612,7 @@ def test_filter_using_file(cluster): query = f"select count(*) from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_partition_tf_*.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') WHERE _file='test_partition_tf_3.csv'" assert azure_query(node, query) == "1\n" + def test_read_from_not_existing_container(cluster): node = cluster.instances["node"] query = f"select * from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont_not_exists', 'test_table.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto')" From 57e9cbfe58b8f21477c83a15adae4a981ee20a24 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 6 Jul 2023 11:47:54 +0200 Subject: [PATCH 0411/2047] Add clickhouse-keeper-client to the packages --- packages/clickhouse-server.yaml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/packages/clickhouse-server.yaml b/packages/clickhouse-server.yaml index 018e88ef828..66299fddd4a 100644 --- a/packages/clickhouse-server.yaml +++ b/packages/clickhouse-server.yaml @@ -55,6 +55,9 @@ contents: - src: clickhouse dst: /usr/bin/clickhouse-keeper type: symlink +- src: clickhouse + dst: /usr/bin/clickhouse-keeper-client + type: symlink - src: root/usr/bin/clickhouse-report dst: /usr/bin/clickhouse-report - src: root/usr/bin/clickhouse-server From 86fc70223693db8aac9edfa7c85e7e80286042ec Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 6 Jul 2023 15:14:18 +0300 Subject: [PATCH 0412/2047] Add skipWhitespacesAndTabs() Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 60f1cbe1f80..79ce2549b4d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -354,6 +354,7 @@ bool CSVFormatReader::checkForSuffix() bool CSVFormatReader::checkForEndOfRow() { + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); return buf->eof() || *buf->position() == '\n' || *buf->position() == '\r'; } From 28332076054cc77660a4dbc3e13dcea1999a6342 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 6 Jul 2023 13:09:49 +0000 Subject: [PATCH 0413/2047] Edit tests to test last commit --- tests/queries/0_stateless/00301_csv.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00301_csv.sh b/tests/queries/0_stateless/00301_csv.sh index 776bd39fc03..80053c99a17 100755 --- a/tests/queries/0_stateless/00301_csv.sh +++ b/tests/queries/0_stateless/00301_csv.sh @@ -44,7 +44,7 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; echo === Test ignore extra columns $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64 DEFAULT 3, d String DEFAULT 'String4') ENGINE = Memory"; -echo '"Hello", 1, "String1" +echo '"Hello", 1, "String1" "Hello", 2, "String2", "Hello", 3, "String3", "2016-01-13" "Hello", 4, , "2016-01-14" From dee71d2e2f8cdd6be4a82f26e7af9b8a75453091 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 6 Jul 2023 13:16:31 +0000 Subject: [PATCH 0414/2047] Add first version of hasSubsequence() --- src/Functions/HasSubsequenceImpl.h | 131 ++++++++++++++++++ src/Functions/hasSubsequence.cpp | 29 ++++ .../hasSubsequenceCaseInsensitive.cpp | 28 ++++ src/Functions/like.cpp | 1 - .../02809_has_subsequence.reference | 16 +++ .../0_stateless/02809_has_subsequence.sql | 19 +++ 6 files changed, 223 insertions(+), 1 deletion(-) create mode 100644 src/Functions/HasSubsequenceImpl.h create mode 100644 src/Functions/hasSubsequence.cpp create mode 100644 src/Functions/hasSubsequenceCaseInsensitive.cpp create mode 100644 tests/queries/0_stateless/02809_has_subsequence.reference create mode 100644 tests/queries/0_stateless/02809_has_subsequence.sql diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h new file mode 100644 index 00000000000..3a29ef68b0b --- /dev/null +++ b/src/Functions/HasSubsequenceImpl.h @@ -0,0 +1,131 @@ +#pragma once + + +namespace DB +{ +namespace +{ + +template +struct HasSubsequenceImpl +{ + using ResultType = UInt8; + + static constexpr bool use_default_implementation_for_constants = false; + static constexpr bool supports_start_pos = false; + static constexpr auto name = Name::name; + + static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {};} + + /// Find one substring in many strings. + static void vectorConstant( + const ColumnString::Chars & /*haystack_data*/, + const ColumnString::Offsets & /*haystack_offsets*/, + const std::string & /*needle*/, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + [[maybe_unused]] ColumnUInt8 * /*res_null*/) + { + size_t size = res.size(); + for (size_t i = 0; i < size; ++i) + { + res[i] = 0; + } + } + + /// Search each time for a different single substring inside each time different string. + static void vectorVector( + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const ColumnString::Chars & needle_data, + const ColumnString::Offsets & needle_offsets, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + ColumnUInt8 * /*res_null*/) + { + ColumnString::Offset prev_haystack_offset = 0; + ColumnString::Offset prev_needle_offset = 0; + + size_t size = haystack_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + + if (0 == needle_size) + { + res[i] = 1; + } + else + { + const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); + const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); + res[i] = impl(haystack, haystack_size, needle, needle_size); + } + + prev_haystack_offset = haystack_offsets[i]; + prev_needle_offset = needle_offsets[i]; + } + } + + /// Find many substrings in single string. + static void constantVector( + const String & /*haystack*/, + const ColumnString::Chars & /*needle_data*/, + const ColumnString::Offsets & needle_offsets, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + ColumnUInt8 * /*res_null*/) + { + size_t size = needle_offsets.size(); + + for (size_t i = 0; i < size; ++i) + { + res[i] = 0; + } + } + + static UInt8 impl(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) + { + size_t j = 0; + for (size_t i = 0; (i < haystack_size) && (j < needle_size); i++) + if (needle[j] == haystack[i]) + ++j; + return j == needle_size; + } + + static void constantConstant( + std::string haystack, + std::string needle, + const ColumnPtr & /*start_pos*/, + PaddedPODArray & res, + ColumnUInt8 * /*res_null*/) + { + size_t size = res.size(); + Impl::toLowerIfNeed(haystack); + Impl::toLowerIfNeed(needle); + + UInt8 result = impl(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + + for (size_t i = 0; i < size; ++i) + { + res[i] = result; + } + } + template + static void vectorFixedConstant(Args &&...) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); + } + + template + static void vectorFixedVector(Args &&...) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); + } +}; + +} + +} diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp new file mode 100644 index 00000000000..da2aaddcf50 --- /dev/null +++ b/src/Functions/hasSubsequence.cpp @@ -0,0 +1,29 @@ +#include +#include +#include + + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseSensitiveASCII +{ + static void toLowerIfNeed(std::string & /*s*/) { } +}; + +struct NameHasSubsequence +{ + static constexpr auto name = "hasSubsequence"; +}; + +using FunctionHasSubsequence = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequence) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp new file mode 100644 index 00000000000..f5c13a7cf8c --- /dev/null +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -0,0 +1,28 @@ +#include +#include +#include + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseInsensitiveASCII +{ + static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } +}; + +struct NameHasSubsequenceCaseInsensitive +{ + static constexpr auto name = "hasSubsequenceCaseInsensitive"; +}; + +using FunctionHasSubsequenceCaseInsensitive = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequenceCaseInsensitive) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/like.cpp b/src/Functions/like.cpp index 3a3345051d4..5a86e37a92d 100644 --- a/src/Functions/like.cpp +++ b/src/Functions/like.cpp @@ -1,4 +1,3 @@ -#include "FunctionsStringSearch.h" #include "FunctionFactory.h" #include "like.h" diff --git a/tests/queries/0_stateless/02809_has_subsequence.reference b/tests/queries/0_stateless/02809_has_subsequence.reference new file mode 100644 index 00000000000..827caa105d0 --- /dev/null +++ b/tests/queries/0_stateless/02809_has_subsequence.reference @@ -0,0 +1,16 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +0 +0 +0 +1 +1 +1 +0 \ No newline at end of file diff --git a/tests/queries/0_stateless/02809_has_subsequence.sql b/tests/queries/0_stateless/02809_has_subsequence.sql new file mode 100644 index 00000000000..63ffb49dc54 --- /dev/null +++ b/tests/queries/0_stateless/02809_has_subsequence.sql @@ -0,0 +1,19 @@ +select hasSubsequence('garbage', ''); +select hasSubsequence('garbage', 'g'); +select hasSubsequence('garbage', 'a'); +select hasSubsequence('garbage', 'e'); +select hasSubsequence('garbage', 'gr'); +select hasSubsequence('garbage', 'ab'); +select hasSubsequence('garbage', 'be'); +select hasSubsequence('garbage', 'arg'); +select hasSubsequence('garbage', 'garbage'); + +select hasSubsequence('garbage', 'garbage1'); +select hasSubsequence('garbage', 'arbw'); +select hasSubsequence('garbage', 'ARG'); + +select hasSubsequenceCaseInsensitive('garbage', 'ARG'); + +select hasSubsequence(materialize('garbage'), materialize('')); +select hasSubsequence(materialize('garbage'), materialize('arg')); +select hasSubsequence(materialize('garbage'), materialize('garbage1')); \ No newline at end of file From 810d1ee0694cc769170f4b08c58aa4c2c5b0807a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 6 Jul 2023 13:48:57 +0000 Subject: [PATCH 0415/2047] Fix tests --- src/Processors/Formats/IRowInputFormat.h | 2 +- .../Formats/Impl/ArrowBlockInputFormat.h | 2 +- .../Impl/JSONColumnsBlockInputFormatBase.h | 2 +- src/Processors/Formats/Impl/NativeFormat.cpp | 2 +- .../Formats/Impl/ORCBlockInputFormat.h | 2 +- .../Formats/Impl/ParallelParsingInputFormat.h | 2 +- .../Formats/Impl/ParquetBlockInputFormat.h | 2 +- .../Formats/Impl/ValuesBlockInputFormat.h | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 22 +++++++++++++------ 9 files changed, 23 insertions(+), 15 deletions(-) diff --git a/src/Processors/Formats/IRowInputFormat.h b/src/Processors/Formats/IRowInputFormat.h index b7b1b0b29a6..00888cfa5e9 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -85,7 +85,7 @@ private: size_t num_errors = 0; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; }; } diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index df77994c3d5..2db8bd6c59c 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -50,7 +50,7 @@ private: int record_batch_current = 0; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; const FormatSettings format_settings; diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h index 5ab20c796ea..bb52e2aa516 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h @@ -67,7 +67,7 @@ protected: Serializations serializations; std::unique_ptr reader; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; }; diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index f8c9a39eedf..65ea87479a3 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -66,7 +66,7 @@ private: std::unique_ptr reader; Block header; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; }; class NativeOutputFormat final : public IOutputFormat diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 98561e72e61..7097ea3ac08 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -52,7 +52,7 @@ private: std::vector include_indices; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; const FormatSettings format_settings; const std::unordered_set & skip_stripes; diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index 4495680f5b2..f61dc3fbc78 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -202,7 +202,7 @@ private: const size_t max_block_size; BlockMissingValues last_block_missing_values; - size_t last_approx_bytes_read_for_chunk; + size_t last_approx_bytes_read_for_chunk = 0; /// Non-atomic because it is used in one thread. std::optional next_block_in_current_unit; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index a14c51f8b9f..dc14edf2099 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -273,7 +273,7 @@ private: std::unique_ptr pool; BlockMissingValues previous_block_missing_values; - size_t previous_approx_bytes_read_for_chunk; + size_t previous_approx_bytes_read_for_chunk = 0; std::exception_ptr background_exception = nullptr; std::atomic is_stopped{0}; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index d540a24fa70..8f8d44ec088 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -96,7 +96,7 @@ private: Serializations serializations; BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; + size_t approx_bytes_read_for_chunk = 0; }; class ValuesSchemaReader : public IRowSchemaReader diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index b9be01cf2ae..e583d2e30b7 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -301,18 +301,26 @@ public: StorageHDFS::PathWithInfo next() { - size_t current_index = index.fetch_add(1); - if (current_index >= uris.size()) - return {"", {}}; + String uri; + hdfsFileInfo * hdfs_info; + do + { + size_t current_index = index.fetch_add(1); + if (current_index >= uris.size()) + return {"", {}}; + + uri = uris[current_index]; + auto path_and_uri = getPathFromUriAndUriWithoutPath(uri); + hdfs_info = hdfsGetPathInfo(fs.get(), path_and_uri.first.c_str()); + } + /// Skip non-existed files. + while (String(hdfsGetLastError()).find("FileNotFoundException") != std::string::npos); - auto uri = uris[current_index]; - auto path_and_uri = getPathFromUriAndUriWithoutPath(uri); - auto * hdfs_info = hdfsGetPathInfo(fs.get(), path_and_uri.first.c_str()); std::optional info; if (hdfs_info) { info = StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}; - if (file_progress_callback && hdfs_info) + if (file_progress_callback) file_progress_callback(FileProgress(0, hdfs_info->mSize)); } From 67e2dee7e2ea926d6a0a6ab35b31b2515f518426 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 6 Jul 2023 14:29:58 +0000 Subject: [PATCH 0416/2047] Allow SETTINGS before FORMAT in DESCRIBE TABLE query --- src/Parsers/ParserDescribeTableQuery.cpp | 20 +++++++++++++++---- src/Parsers/ParserQueryWithOutput.cpp | 2 +- src/Parsers/ParserTablePropertiesQuery.cpp | 2 -- src/Storages/StorageDistributed.cpp | 1 - src/Storages/getStructureOfRemoteTable.cpp | 1 - .../02789_describe_table_settings.reference | 10 ++++++++++ .../02789_describe_table_settings.sql | 3 +++ 7 files changed, 30 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02789_describe_table_settings.reference create mode 100644 tests/queries/0_stateless/02789_describe_table_settings.sql diff --git a/src/Parsers/ParserDescribeTableQuery.cpp b/src/Parsers/ParserDescribeTableQuery.cpp index ad6d2c5bcc6..fcfc4799dbe 100644 --- a/src/Parsers/ParserDescribeTableQuery.cpp +++ b/src/Parsers/ParserDescribeTableQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -16,8 +17,10 @@ bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ex ParserKeyword s_describe("DESCRIBE"); ParserKeyword s_desc("DESC"); ParserKeyword s_table("TABLE"); + ParserKeyword s_settings("SETTINGS"); ParserToken s_dot(TokenType::Dot); ParserIdentifier name_p; + ParserSetQuery parser_settings(true); ASTPtr database; ASTPtr table; @@ -29,12 +32,21 @@ bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ex s_table.ignore(pos, expected); - ASTPtr table_expression; - if (!ParserTableExpression().parse(pos, table_expression, expected)) + if (!ParserTableExpression().parse(pos, query->table_expression, expected)) return false; - query->children.push_back(std::move(table_expression)); - query->table_expression = query->children.back(); + /// For compatibility with SELECTs, where SETTINGS can be in front of FORMAT + ASTPtr settings; + if (s_settings.ignore(pos, expected)) + { + if (!parser_settings.parse(pos, query->settings_ast, expected)) + return false; + } + + query->children.push_back(query->table_expression); + + if (query->settings_ast) + query->children.push_back(query->settings_ast); node = query; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 6796f4528c4..5dc713ca8c6 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -150,7 +150,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec // SETTINGS key1 = value1, key2 = value2, ... ParserKeyword s_settings("SETTINGS"); - if (s_settings.ignore(pos, expected)) + if (!query_with_output.settings_ast && s_settings.ignore(pos, expected)) { ParserSetQuery parser_settings(true); if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) diff --git a/src/Parsers/ParserTablePropertiesQuery.cpp b/src/Parsers/ParserTablePropertiesQuery.cpp index b73ce8de359..94f264fcc89 100644 --- a/src/Parsers/ParserTablePropertiesQuery.cpp +++ b/src/Parsers/ParserTablePropertiesQuery.cpp @@ -14,8 +14,6 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & { ParserKeyword s_exists("EXISTS"); ParserKeyword s_temporary("TEMPORARY"); - ParserKeyword s_describe("DESCRIBE"); - ParserKeyword s_desc("DESC"); ParserKeyword s_show("SHOW"); ParserKeyword s_create("CREATE"); ParserKeyword s_database("DATABASE"); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b91ad0b963a..b6359bbb251 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -60,7 +60,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index e5fc01be9f4..ec8f27feeda 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include diff --git a/tests/queries/0_stateless/02789_describe_table_settings.reference b/tests/queries/0_stateless/02789_describe_table_settings.reference new file mode 100644 index 00000000000..c2bf9219f4d --- /dev/null +++ b/tests/queries/0_stateless/02789_describe_table_settings.reference @@ -0,0 +1,10 @@ +"id","Nullable(Int64)","","","","","" +"age","LowCardinality(UInt8)","","","","","" +"name","Nullable(String)","","","","","" +"status","Nullable(String)","","","","","" +"hobbies","Array(Nullable(String))","","","","","" +"id","Nullable(Int64)","","","","","" +"age","LowCardinality(UInt8)","","","","","" +"name","Nullable(String)","","","","","" +"status","Nullable(String)","","","","","" +"hobbies","Array(Nullable(String))","","","","","" diff --git a/tests/queries/0_stateless/02789_describe_table_settings.sql b/tests/queries/0_stateless/02789_describe_table_settings.sql new file mode 100644 index 00000000000..64b5b21fea8 --- /dev/null +++ b/tests/queries/0_stateless/02789_describe_table_settings.sql @@ -0,0 +1,3 @@ +DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}') SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)', allow_suspicious_low_cardinality_types=1 FORMAT CSV; +DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}') FORMAT CSV SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)', allow_suspicious_low_cardinality_types=1; +DESC format(JSONEachRow, '{"id" : 1, "age" : 25, "name" : "Josh", "status" : null, "hobbies" : ["football", "cooking"]}') FORMAT CSV SETTINGS schema_inference_hints = 'age LowCardinality(UInt8), status Nullable(String)', allow_suspicious_low_cardinality_types=1 SETTINGS max_threads=0; -- { clientError SYNTAX_ERROR } From 9a295eca46fea2c88d1c1767fc4625b31c999572 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Jul 2023 14:28:50 +0000 Subject: [PATCH 0417/2047] Incorporate review feedback --- docs/en/sql-reference/statements/show.md | 24 ++--- .../InterpreterShowIndexesQuery.cpp | 4 +- .../0_stateless/02724_show_indexes.reference | 88 +++++++++---------- 3 files changed, 58 insertions(+), 58 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 1a1e4dbd2c7..1c399d2072b 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -289,18 +289,18 @@ The statement produces a result table with the following structure: - table - The name of the table. (String) - non_unique - Always `1` as ClickHouse does not support uniqueness constraints. (UInt8) - key_name - The name of the index, `PRIMARY` if the index is a primary key index. (String) -- column_name - For a primary key index, the name of the column. For a data skipping index: '' (empty string), see field "expression". (String) - seq_in_index - For a primary key index, the position of the column starting from `1`. For a data skipping index: always `1`. (UInt8) +- column_name - For a primary key index, the name of the column. For a data skipping index: `''` (empty string), see field "expression". (String) - collation - The sorting of the column in the index: `A` if ascending, `D` if descending, `NULL` if unsorted. (Nullable(String)) - cardinality - An estimation of the index cardinality (number of unique values in the index). Currently always 0. (UInt64) - sub_part - Always `NULL` because ClickHouse does not support index prefixes like MySQL. (Nullable(String)) - packed - Always `NULL` because ClickHouse does not support packed indexes (like MySQL). (Nullable(String)) - null - Currently unused - index_type - The index type, e.g. `PRIMARY`, `MINMAX`, `BLOOM_FILTER` etc. (String) -- comment - Additional information about the index, currently always `` (empty string). (String) -- index_comment - `` (empty string) because indexes in ClickHouse cannot have a `COMMENT` field (like in MySQL). (String) +- comment - Additional information about the index, currently always `''` (empty string). (String) +- index_comment - `''` (empty string) because indexes in ClickHouse cannot have a `COMMENT` field (like in MySQL). (String) - visible - If the index is visible to the optimizer, always `YES`. (String) -- expression - For a data skipping index, the index expression. For a primary key index: '' (empty string). (String) +- expression - For a data skipping index, the index expression. For a primary key index: `''` (empty string). (String) **Examples** @@ -313,14 +313,14 @@ SHOW INDEX FROM 'tbl' Result: ``` text -┌─table─┬─non_unique─┬─key_name─┬─column_name─┬─seq_in_index─┬─collation─┬─cardinality─┬─sub_part─┬─packed─┬─null─┬─index_type───┬─comment─┬─index_comment─┬─visible─┬─expression─┐ -│ tbl │ 1 │ blf_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ BLOOM_FILTER │ │ │ YES │ d, b │ -│ tbl │ 1 │ mm1_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ a, c, d │ -│ tbl │ 1 │ mm2_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ c, d, e │ -│ tbl │ 1 │ PRIMARY │ c │ 1 │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ -│ tbl │ 1 │ PRIMARY │ a │ 2 │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ -│ tbl │ 1 │ set_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ SET │ │ │ YES │ e │ -└───────┴────────────┴──────────┴─────────────┴──────────────┴───────────┴─────────────┴──────────┴────────┴──────┴──────────────┴─────────┴───────────────┴─────────┴────────────┘ +┌─table─┬─non_unique─┬─key_name─┬─seq_in_index─┬─column_name─┬─collation─┬─cardinality─┬─sub_part─┬─packed─┬─null─┬─index_type───┬─comment─┬─index_comment─┬─visible─┬─expression─┐ +│ tbl │ 1 │ blf_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ BLOOM_FILTER │ │ │ YES │ d, b │ +│ tbl │ 1 │ mm1_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ a, c, d │ +│ tbl │ 1 │ mm2_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ c, d, e │ +│ tbl │ 1 │ PRIMARY │ 1 │ c │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ +│ tbl │ 1 │ PRIMARY │ 2 │ a │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │ +│ tbl │ 1 │ set_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ SET │ │ │ YES │ e │ +└───────┴────────────┴──────────┴──────────────┴─────────────┴───────────┴─────────────┴──────────┴────────┴──────┴──────────────┴─────────┴───────────────┴─────────┴────────────┘ ``` **See also** diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index 35f32a79310..149420006fb 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -42,8 +42,8 @@ FROM ( name AS table, 1 AS non_unique, 'PRIMARY' AS key_name, - arrayJoin(splitByString(', ', primary_key)) AS column_name, row_number() over (order by column_name) AS seq_in_index, + arrayJoin(splitByString(', ', primary_key)) AS column_name, 'A' AS collation, 0 AS cardinality, NULL AS sub_part, @@ -63,8 +63,8 @@ FROM ( table AS table, 1 AS non_unique, name AS key_name, - '' AS column_name, 1 AS seq_in_index, + '' AS column_name, NULL AS collation, 0 AS cardinality, NULL AS sub_part, diff --git a/tests/queries/0_stateless/02724_show_indexes.reference b/tests/queries/0_stateless/02724_show_indexes.reference index cee0598d625..e41f2521f5c 100644 --- a/tests/queries/0_stateless/02724_show_indexes.reference +++ b/tests/queries/0_stateless/02724_show_indexes.reference @@ -1,51 +1,51 @@ --- Aliases of SHOW INDEX -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- EXTENDED -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- WHERE --- Check with weird table names -$4@^7 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES -NULL 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES -\' 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES -\' 1 PRIMARY c 1 A 0 \N \N \N PRIMARY YES +$4@^7 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +NULL 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +\' 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +\' 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES --- Original table -tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY c 2 A 0 \N \N \N PRIMARY YES -tbl 1 set_idx 1 \N 0 \N \N \N SET YES e +tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- Equally named table in other database -tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES +tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES --- Short form -tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b -tbl 1 PRIMARY a 1 A 0 \N \N \N PRIMARY YES +tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b +tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES From 9ae0dc730c586a37f8fdbbd880267ec11c2c8e51 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Thu, 6 Jul 2023 17:15:26 +0200 Subject: [PATCH 0418/2047] Review fixes + enable in ci --- docker/test/upgrade/run.sh | 2 + .../ReplicatedMergeTreeRestartingThread.cpp | 7 +- .../ReplicatedMergeTreeRestartingThread.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 99 +++++++++---------- src/Storages/StorageReplicatedMergeTree.h | 14 ++- ...le_wait_for_shutdown_replicated_tables.xml | 5 + tests/config/install.sh | 1 + 7 files changed, 66 insertions(+), 63 deletions(-) create mode 100644 tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 82a88272df9..07e6e7dd0ec 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -61,6 +61,7 @@ configure # it contains some new settings, but we can safely remove it rm /etc/clickhouse-server/config.d/merge_tree.xml +rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml rm /etc/clickhouse-server/users.d/nonconst_timezone.xml start @@ -90,6 +91,7 @@ configure # it contains some new settings, but we can safely remove it rm /etc/clickhouse-server/config.d/merge_tree.xml +rm /etc/clickhouse-server/config.d/enable_wait_for_shutdown_replicated_tables.xml rm /etc/clickhouse-server/users.d/nonconst_timezone.xml start diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 0e381654db0..e43cc879e93 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -329,8 +329,8 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() void ReplicatedMergeTreeRestartingThread::partialShutdown(bool part_of_full_shutdown) { - setReadonly(part_of_full_shutdown); - storage.partialShutdown(part_of_full_shutdown); + setReadonly(/* on_shutdown = */ part_of_full_shutdown); + storage.partialShutdown(); } @@ -341,8 +341,7 @@ void ReplicatedMergeTreeRestartingThread::shutdown(bool part_of_full_shutdown) task->deactivate(); LOG_TRACE(log, "Restarting thread finished"); - /// Stop other tasks. - partialShutdown(part_of_full_shutdown); + setReadonly(part_of_full_shutdown); } void ReplicatedMergeTreeRestartingThread::setReadonly(bool on_shutdown) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 9e99baab4c3..01a877a07e5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -36,6 +36,7 @@ public: void shutdown(bool part_of_full_shutdown); void run(); + private: StorageReplicatedMergeTree & storage; String log_name; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 114465df496..88bd788b9ef 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3942,17 +3942,26 @@ void StorageReplicatedMergeTree::addLastSentPart(const MergeTreePartInfo & info) last_sent_parts_cv.notify_all(); } -void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(size_t wait_ms) +void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(StorageReplicatedMergeTree::ShutdownDeadline shutdown_deadline_) { if (!shutdown_called.load()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Called waitForUniquePartsToBeFetchedByOtherReplicas before shutdown, it's a bug"); + auto settings_ptr = getSettings(); + + auto wait_ms = settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds(); if (wait_ms == 0) { LOG_INFO(log, "Will not wait for unique parts to be fetched by other replicas because wait time is zero"); return; } + if (shutdown_deadline_ <= std::chrono::system_clock::now()) + { + LOG_INFO(log, "Will not wait for unique parts to be fetched by other replicas because shutdown_deadline already passed"); + return; + } + auto zookeeper = getZooKeeperIfTableShutDown(); auto unique_parts_set = findReplicaUniqueParts(replica_name, zookeeper_path, format_version, zookeeper, log); @@ -3968,7 +3977,6 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(si auto wait_predicate = [&] () -> bool { - bool all_fetched = true; for (auto it = unique_parts_set.begin(); it != unique_parts_set.end();) { const auto & part = *it; @@ -3985,22 +3993,19 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(si } } if (!found) - { - all_fetched = false; break; - } } - return all_fetched; + return unique_parts_set.empty(); }; std::unique_lock lock(last_sent_parts_mutex); - if (!last_sent_parts_cv.wait_for(lock, std::chrono::milliseconds(wait_ms), wait_predicate)) + if (!last_sent_parts_cv.wait_until(lock, shutdown_deadline_, wait_predicate)) LOG_WARNING(log, "Failed to wait for unique parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); else LOG_INFO(log, "Successfully waited all the parts"); } -std::vector StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_) +std::set StorageReplicatedMergeTree::findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_) { if (!zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica_name_ / "is_active")) { @@ -4027,26 +4032,25 @@ std::vector StorageReplicatedMergeTree::findReplicaUniquePart } else { - LOG_TRACE(log_, "Fetching parts for replica {}", replica); - data_parts_on_replicas.emplace_back(format_version_); - for (const auto & part : parts) - { - if (data_parts_on_replicas.back().getContainingPart(part).empty()) - data_parts_on_replicas.back().add(part); - } + LOG_TRACE(log_, "Fetching parts for replica {}: [{}]", replica, fmt::join(parts, ", ")); + data_parts_on_replicas.emplace_back(format_version_, parts); } } - std::vector our_unique_parts; + if (data_parts_on_replicas.empty()) + { + LOG_TRACE(log_, "Has no active replicas, will no try to wait for fetch"); + return {}; + } + + std::set our_unique_parts; for (const auto & part : our_parts) { - LOG_TRACE(log_, "Looking for part {}", part); bool found = false; for (const auto & active_parts_set : data_parts_on_replicas) { if (!active_parts_set.getContainingPart(part).empty()) { - LOG_TRACE(log_, "Part {} found", part); found = true; break; } @@ -4054,8 +4058,8 @@ std::vector StorageReplicatedMergeTree::findReplicaUniquePart if (!found) { - LOG_TRACE(log_, "Part not {} found", part); - our_unique_parts.emplace_back(MergeTreePartInfo::fromPartName(part, format_version_)); + LOG_TRACE(log_, "Part not {} found on other replicas", part); + our_unique_parts.emplace(MergeTreePartInfo::fromPartName(part, format_version_)); } } @@ -4836,9 +4840,7 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() if (shutdown_prepared_called.exchange(true)) return; - session_expired_callback_handler.reset(); - stopOutdatedDataPartsLoadingTask(); - + auto settings_ptr = getSettings(); /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP. fetcher.blocker.cancelForever(); merger_mutator.merges_blocker.cancelForever(); @@ -4850,39 +4852,17 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() attach_thread->shutdown(); restarting_thread.shutdown(/* part_of_full_shutdown */true); - background_operations_assignee.finish(); - part_moves_between_shards_orchestrator.shutdown(); - - { - auto lock = queue.lockQueue(); - /// Cancel logs pulling after background task were cancelled. It's still - /// required because we can trigger pullLogsToQueue during manual OPTIMIZE, - /// MUTATE, etc. query. - queue.pull_log_blocker.cancelForever(); - } - background_moves_assignee.finish(); - + shutdown_deadline.emplace(std::chrono::system_clock::now() + std::chrono::milliseconds(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds())); } -void StorageReplicatedMergeTree::partialShutdown(bool part_of_full_shutdown) +void StorageReplicatedMergeTree::partialShutdown() { ProfileEvents::increment(ProfileEvents::ReplicaPartialShutdown); partial_shutdown_called = true; partial_shutdown_event.set(); queue.notifySubscribersOnPartialShutdown(); - if (!part_of_full_shutdown) - { - /// If we are going to completely shutdown table we allow other - /// replicas to fetch parts which are unique for our replica. - /// - /// Replicas try to fetch part only in case the source replica is active, - /// so don't reset handler here. - LOG_DEBUG(log, "Reset active node, replica will be inactive"); - replica_is_active_node = nullptr; - } - else - LOG_DEBUG(log, "Will not reset active node, it will be reset completely during full shutdown"); + replica_is_active_node = nullptr; LOG_TRACE(log, "Waiting for threads to finish"); merge_selecting_task->deactivate(); @@ -4914,10 +4894,27 @@ void StorageReplicatedMergeTree::shutdown() flushAndPrepareForShutdown(); auto settings_ptr = getSettings(); - LOG_DEBUG(log, "Data parts exchange still exists {}", data_parts_exchange_endpoint != nullptr); - waitForUniquePartsToBeFetchedByOtherReplicas(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds()); + if (!shutdown_deadline.has_value()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Shutdown deadline is not set in shutdown"); - replica_is_active_node = nullptr; + waitForUniquePartsToBeFetchedByOtherReplicas(*shutdown_deadline); + + session_expired_callback_handler.reset(); + stopOutdatedDataPartsLoadingTask(); + + partialShutdown(); + + part_moves_between_shards_orchestrator.shutdown(); + background_operations_assignee.finish(); + + { + auto lock = queue.lockQueue(); + /// Cancel logs pulling after background task were cancelled. It's still + /// required because we can trigger pullLogsToQueue during manual OPTIMIZE, + /// MUTATE, etc. query. + queue.pull_log_blocker.cancelForever(); + } + background_moves_assignee.finish(); auto data_parts_exchange_ptr = std::atomic_exchange(&data_parts_exchange_endpoint, InterserverIOEndpointPtr{}); if (data_parts_exchange_ptr) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 656e8df6ccb..811a8524064 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -118,7 +118,7 @@ public: /// Partial shutdown called if we loose connection to zookeeper. /// Table can also recover after partial shutdown and continue /// to work. This method can be called regularly. - void partialShutdown(bool part_of_full_shutdown); + void partialShutdown(); /// These two methods are called during final table shutdown (DROP/DETACH/overall server shutdown). /// The shutdown process is split into two methods to make it more soft and fast. In database shutdown() @@ -368,15 +368,11 @@ public: ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const; void addLastSentPart(const MergeTreePartInfo & info); - std::deque getLastSentParts() const - { - std::lock_guard lock(last_sent_parts_mutex); - return last_sent_parts; - } /// Wait required amount of milliseconds to give other replicas a chance to /// download unique parts from our replica - void waitForUniquePartsToBeFetchedByOtherReplicas(size_t wait_ms); + using ShutdownDeadline = std::chrono::time_point; + void waitForUniquePartsToBeFetchedByOtherReplicas(ShutdownDeadline shutdown_deadline); private: std::atomic_bool are_restoring_replica {false}; @@ -483,6 +479,8 @@ private: std::atomic shutdown_called {false}; std::atomic shutdown_prepared_called {false}; + std::optional shutdown_deadline; + mutable std::mutex last_sent_parts_mutex; std::condition_variable last_sent_parts_cv; @@ -740,7 +738,7 @@ private: */ String findReplicaHavingCoveringPart(LogEntry & entry, bool active); String findReplicaHavingCoveringPart(const String & part_name, bool active, String & found_part_name); - static std::vector findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_); + static std::set findReplicaUniqueParts(const String & replica_name_, const String & zookeeper_path_, MergeTreeDataFormatVersion format_version_, zkutil::ZooKeeper::Ptr zookeeper_, Poco::Logger * log_); /** Download the specified part from the specified replica. * If `to_detached`, the part is placed in the `detached` directory. diff --git a/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml b/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml new file mode 100644 index 00000000000..b23dbdc2607 --- /dev/null +++ b/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml @@ -0,0 +1,5 @@ + + + 1000 + + diff --git a/tests/config/install.sh b/tests/config/install.sh index 50f2627d37c..33d5c99202e 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -57,6 +57,7 @@ ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/reverse_dns_query_function.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/compressed_marks_and_index.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/disable_s3_env_credentials.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/enable_wait_for_shutdown_replicated_tables.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/backups.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. From da61a8c509e2d43275f0c8f06dfd065f2874f79f Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Thu, 6 Jul 2023 17:16:00 +0200 Subject: [PATCH 0419/2047] Fix --- .../config.d/enable_wait_for_shutdown_replicated_tables.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml b/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml index b23dbdc2607..504841296a8 100644 --- a/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml +++ b/tests/config/config.d/enable_wait_for_shutdown_replicated_tables.xml @@ -1,5 +1,5 @@ - 1000 + 3000 From 8cdb181c3909802ae3bdd48fd118358b50ef027d Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Thu, 6 Jul 2023 17:52:31 +0200 Subject: [PATCH 0420/2047] Reduce logging level --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 88bd788b9ef..0f5a52b275c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4000,7 +4000,7 @@ void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(St std::unique_lock lock(last_sent_parts_mutex); if (!last_sent_parts_cv.wait_until(lock, shutdown_deadline_, wait_predicate)) - LOG_WARNING(log, "Failed to wait for unique parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); + LOG_INFO(log, "Failed to wait for unique parts to be fetched in {} ms, {} parts can be left on this replica", wait_ms, unique_parts_set.size()); else LOG_INFO(log, "Successfully waited all the parts"); } From 8d634c992bda74095befdf6b47012cbe17acceae Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 6 Jul 2023 17:47:01 +0000 Subject: [PATCH 0421/2047] Fix tests --- src/Storages/StorageURL.cpp | 2 +- src/Storages/StorageURL.h | 4 ++-- src/Storages/prepareReadingFromFormat.cpp | 13 +++++++++++-- 3 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index f0f15e29129..bb103c0d947 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -827,7 +827,7 @@ Pipe StorageURLWithFailover::read( read_from_format_info, iterator_wrapper, getReadMethod(), - getReadPOSTDataCallback(column_names, read_from_format_info.columns_description, query_info, local_context, processed_stage, max_block_size), + getReadPOSTDataCallback(read_from_format_info.columns_description.getNamesOfPhysical(), read_from_format_info.columns_description, query_info, local_context, processed_stage, max_block_size), format_name, format_settings, getName(), diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 04da5d177ec..c77eb37601c 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -108,8 +108,6 @@ protected: bool supportsSubsetOfColumns() const override; - bool supportsSubcolumns() const override { return true; } - bool prefersLargeBlocks() const override; bool parallelizeOutputAfterReading(ContextPtr context) const override; @@ -278,6 +276,8 @@ public: return storage_snapshot->metadata->getSampleBlock(); } + bool supportsSubcolumns() const override { return true; } + static FormatSettings getFormatSettingsFromArgs(const StorageFactory::Arguments & args); struct Configuration : public StatelessTableEngineConfiguration diff --git a/src/Storages/prepareReadingFromFormat.cpp b/src/Storages/prepareReadingFromFormat.cpp index 7cc31066e31..6be4213ec6b 100644 --- a/src/Storages/prepareReadingFromFormat.cpp +++ b/src/Storages/prepareReadingFromFormat.cpp @@ -48,9 +48,18 @@ ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, c else { std::unordered_set columns_to_read_set; + /// Save original order of columns. + std::vector new_columns_to_read; for (const auto & column_to_read : info.requested_columns) - columns_to_read_set.insert(column_to_read.getNameInStorage()); - columns_to_read = Strings(columns_to_read_set.begin(), columns_to_read_set.end()); + { + auto name = column_to_read.getNameInStorage(); + if (!columns_to_read_set.contains(name)) + { + columns_to_read_set.insert(name); + new_columns_to_read.push_back(name); + } + } + columns_to_read = std::move(new_columns_to_read); } info.columns_description = storage_snapshot->getDescriptionForColumns(columns_to_read); } From 7255c35edcefe03a39ad7bcf460d9dca5670ca3b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 6 Jul 2023 19:43:37 +0000 Subject: [PATCH 0422/2047] Add more tests --- .../functions/string-search-functions.md | 50 +++++++++++++ .../functions/string-search-functions.md | 52 +++++++++++++ src/Functions/HasSubsequenceImpl.h | 74 ++++++++++++------- src/Functions/hasSubsequence.cpp | 2 +- .../hasSubsequenceCaseInsensitive.cpp | 2 +- .../hasSubsequenceCaseInsensitiveUTF8.cpp | 28 +++++++ src/Functions/hasSubsequenceUTF8.cpp | 29 ++++++++ .../02809_has_subsequence.reference | 13 +++- .../0_stateless/02809_has_subsequence.sql | 20 ++++- 9 files changed, 237 insertions(+), 33 deletions(-) create mode 100644 src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp create mode 100644 src/Functions/hasSubsequenceUTF8.cpp diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 3d8f89f7295..04ad6474310 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -631,3 +631,53 @@ Result: │ 100 │ 200 │ 100-200 │ 100 │ └──────────────────────────────────────────────┴──────────────────────────────────────────────┴──────────────────────────────────────────────┴───────────────────────────────────────────┘ ``` + +## hasSubsequence + +Returns 1 if needle is a subsequence of haystack, or 0 otherwise. +A subsequence of a string is a sequence that can be derived from the given string by deleting zero or more elements without changing the order of the remaining elements. + + +**Syntax** + +``` sql +hasSubsequence(haystack, needle) +``` + +**Arguments** + +- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal). + +**Returned values** + +- 1, if needle is a subsequence of haystack. +- 0, otherwise. + +Type: `UInt8`. + +**Examples** + +``` sql +SELECT hasSubsequence('garbage', 'arg') ; +``` + +Result: + +``` text +┌─hasSubsequence('garbage', 'arg')─┐ +│ 1 │ +└──────────────────────────────────┘ +``` + +## hasSubsequenceCaseInsensitive + +Like [hasSubsequence](#hasSubsequence) but searches case-insensitively. + +## hasSubsequenceUTF8 + +Like [hasSubsequence](#hasSubsequence) but assumes `haystack` and `needle` are UTF-8 encoded strings. + +## hasSubsequenceCaseInsensitiveUTF8 + +Like [hasSubsequenceUTF8](#hasSubsequenceUTF8) but searches case-insensitively. \ No newline at end of file diff --git a/docs/ru/sql-reference/functions/string-search-functions.md b/docs/ru/sql-reference/functions/string-search-functions.md index ea4f90d4f66..21989e882b6 100644 --- a/docs/ru/sql-reference/functions/string-search-functions.md +++ b/docs/ru/sql-reference/functions/string-search-functions.md @@ -801,3 +801,55 @@ SELECT countSubstringsCaseInsensitiveUTF8('аБв__АбВ__абв', 'Абв'); │ 3 │ └────────────────────────────────────────────────────────────┘ ``` + +## hasSubsequence(haystack, needle) {#hasSubsequence} + +Возвращает 1 если needle является подпоследовательностью haystack, иначе 0. + + +**Синтаксис** + +``` sql +hasSubsequence(haystack, needle) +``` + +**Аргументы** + +- `haystack` — строка, по которой выполняется поиск. [Строка](../syntax.md#syntax-string-literal). +- `needle` — подстрока, которую необходимо найти. [Строка](../syntax.md#syntax-string-literal). + +**Возвращаемые значения** + +- 1, если +- 0, если подстрока не найдена. + +Тип: `UInt8`. + +**Примеры** + +Запрос: + +``` sql +SELECT hasSubsequence('garbage', 'arg') ; +``` + +Результат: + +``` text +┌─hasSubsequence('garbage', 'arg')─┐ +│ 1 │ +└──────────────────────────────────┘ +``` + + +## hasSubsequenceCaseInsensitive + +Такая же, как и [hasSubsequence](#hasSubsequence), но работает без учета регистра. + +## hasSubsequenceUTF8 + +Такая же, как и [hasSubsequence](#hasSubsequence) при допущении что `haystack` и `needle` содержат набор кодовых точек, представляющий текст в кодировке UTF-8. + +## hasSubsequenceCaseInsensitiveUTF8 + +Такая же, как и [hasSubsequenceUTF8](#hasSubsequenceUTF8), но работает без учета регистра. diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h index 3a29ef68b0b..bcb8e8e99e6 100644 --- a/src/Functions/HasSubsequenceImpl.h +++ b/src/Functions/HasSubsequenceImpl.h @@ -1,11 +1,8 @@ #pragma once - - namespace DB { namespace { - template struct HasSubsequenceImpl { @@ -17,23 +14,31 @@ struct HasSubsequenceImpl static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {};} - /// Find one substring in many strings. static void vectorConstant( - const ColumnString::Chars & /*haystack_data*/, - const ColumnString::Offsets & /*haystack_offsets*/, - const std::string & /*needle*/, + const ColumnString::Chars & haystack_data, + const ColumnString::Offsets & haystack_offsets, + const String & needle, const ColumnPtr & /*start_pos*/, PaddedPODArray & res, [[maybe_unused]] ColumnUInt8 * /*res_null*/) { - size_t size = res.size(); - for (size_t i = 0; i < size; ++i) + if (needle.empty()) { - res[i] = 0; + for (auto & r : res) + r = 1; + return; + } + + ColumnString::Offset prev_haystack_offset = 0; + for (size_t i = 0; i < haystack_offsets.size(); ++i) + { + size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); + res[i] = hasSubsequence(haystack, haystack_size, needle.c_str(), needle.size()); + prev_haystack_offset = haystack_offsets[i]; } } - /// Search each time for a different single substring inside each time different string. static void vectorVector( const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, @@ -61,7 +66,7 @@ struct HasSubsequenceImpl { const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); - res[i] = impl(haystack, haystack_size, needle, needle_size); + res[i] = hasSubsequence(haystack, haystack_size, needle, needle_size); } prev_haystack_offset = haystack_offsets[i]; @@ -69,35 +74,38 @@ struct HasSubsequenceImpl } } - /// Find many substrings in single string. static void constantVector( - const String & /*haystack*/, - const ColumnString::Chars & /*needle_data*/, + const String & haystack, + const ColumnString::Chars & needle_data, const ColumnString::Offsets & needle_offsets, const ColumnPtr & /*start_pos*/, PaddedPODArray & res, ColumnUInt8 * /*res_null*/) { + ColumnString::Offset prev_needle_offset = 0; + size_t size = needle_offsets.size(); for (size_t i = 0; i < size; ++i) { - res[i] = 0; + size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + + if (0 == needle_size) + { + res[i] = 1; + } + else + { + const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); + res[i] = hasSubsequence(haystack.c_str(), haystack.size(), needle, needle_size); + } + prev_needle_offset = needle_offsets[i]; } } - static UInt8 impl(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) - { - size_t j = 0; - for (size_t i = 0; (i < haystack_size) && (j < needle_size); i++) - if (needle[j] == haystack[i]) - ++j; - return j == needle_size; - } - static void constantConstant( - std::string haystack, - std::string needle, + String haystack, + String needle, const ColumnPtr & /*start_pos*/, PaddedPODArray & res, ColumnUInt8 * /*res_null*/) @@ -106,13 +114,23 @@ struct HasSubsequenceImpl Impl::toLowerIfNeed(haystack); Impl::toLowerIfNeed(needle); - UInt8 result = impl(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + UInt8 result = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); for (size_t i = 0; i < size; ++i) { res[i] = result; } } + + static UInt8 hasSubsequence(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) + { + size_t j = 0; + for (size_t i = 0; (i < haystack_size) && (j < needle_size); i++) + if (needle[j] == haystack[i]) + ++j; + return j == needle_size; + } + template static void vectorFixedConstant(Args &&...) { diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp index da2aaddcf50..bb1f295cee4 100644 --- a/src/Functions/hasSubsequence.cpp +++ b/src/Functions/hasSubsequence.cpp @@ -10,7 +10,7 @@ namespace struct HasSubsequenceCaseSensitiveASCII { - static void toLowerIfNeed(std::string & /*s*/) { } + static void toLowerIfNeed(String & /*s*/) { } }; struct NameHasSubsequence diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp index f5c13a7cf8c..fe50ada9be9 100644 --- a/src/Functions/hasSubsequenceCaseInsensitive.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -9,7 +9,7 @@ namespace struct HasSubsequenceCaseInsensitiveASCII { - static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } + static void toLowerIfNeed(String & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } }; struct NameHasSubsequenceCaseInsensitive diff --git a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp new file mode 100644 index 00000000000..2908c284a25 --- /dev/null +++ b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp @@ -0,0 +1,28 @@ +#include +#include +#include + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseInsensitiveUTF8 +{ + static void toLowerIfNeed(String & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } +}; + +struct NameHasSubsequenceCaseInsensitiveUTF8 +{ + static constexpr auto name = "hasSubsequenceCaseInsensitiveUTF8"; +}; + +using FunctionHasSubsequenceCaseInsensitiveUTF8 = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequenceCaseInsensitiveUTF8) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/hasSubsequenceUTF8.cpp b/src/Functions/hasSubsequenceUTF8.cpp new file mode 100644 index 00000000000..c0811de6575 --- /dev/null +++ b/src/Functions/hasSubsequenceUTF8.cpp @@ -0,0 +1,29 @@ +#include +#include +#include + + +namespace DB +{ +namespace +{ + +struct HasSubsequenceCaseSensitiveUTF8 +{ + static void toLowerIfNeed(String & /*s*/) { } +}; + +struct NameHasSubsequenceUTF8 +{ + static constexpr auto name = "hasSubsequenceUTF8"; +}; + +using FunctionHasSubsequenceUTF8 = FunctionsStringSearch>; +} + +REGISTER_FUNCTION(hasSubsequenceUTF8) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/tests/queries/0_stateless/02809_has_subsequence.reference b/tests/queries/0_stateless/02809_has_subsequence.reference index 827caa105d0..d12c0ba9fb3 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.reference +++ b/tests/queries/0_stateless/02809_has_subsequence.reference @@ -1,3 +1,4 @@ +hasSubsequence / const / const 1 1 1 @@ -10,7 +11,17 @@ 0 0 0 +hasSubsequence / const / string 1 1 +0 +hasSubsequence / string / const +1 +1 +0 +hasSubsequence / string / string +1 +1 +0 +hasSubsequenceCaseInsensitive / const / const 1 -0 \ No newline at end of file diff --git a/tests/queries/0_stateless/02809_has_subsequence.sql b/tests/queries/0_stateless/02809_has_subsequence.sql index 63ffb49dc54..64f3fd8dc77 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.sql +++ b/tests/queries/0_stateless/02809_has_subsequence.sql @@ -1,3 +1,4 @@ +select 'hasSubsequence / const / const'; select hasSubsequence('garbage', ''); select hasSubsequence('garbage', 'g'); select hasSubsequence('garbage', 'a'); @@ -12,8 +13,23 @@ select hasSubsequence('garbage', 'garbage1'); select hasSubsequence('garbage', 'arbw'); select hasSubsequence('garbage', 'ARG'); -select hasSubsequenceCaseInsensitive('garbage', 'ARG'); +select 'hasSubsequence / const / string'; +select hasSubsequence('garbage', materialize('')); +select hasSubsequence('garbage', materialize('arg')); +select hasSubsequence('garbage', materialize('arbw')); + +select 'hasSubsequence / string / const'; +select hasSubsequence(materialize('garbage'), ''); +select hasSubsequence(materialize('garbage'), 'arg'); +select hasSubsequence(materialize('garbage'), 'arbw'); + +select 'hasSubsequence / string / string'; select hasSubsequence(materialize('garbage'), materialize('')); select hasSubsequence(materialize('garbage'), materialize('arg')); -select hasSubsequence(materialize('garbage'), materialize('garbage1')); \ No newline at end of file +select hasSubsequence(materialize('garbage'), materialize('garbage1')); + +select 'hasSubsequenceCaseInsensitive / const / const'; + +select hasSubsequenceCaseInsensitive('garbage', 'ARG'); + From c35294317dbff31b8ff8b48f6256162d6d5dc02e Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jul 2023 15:06:54 +0000 Subject: [PATCH 0423/2047] Remove parts in order for object storage always --- src/Storages/MergeTree/MergeTreeData.cpp | 34 +++++++++++++++--------- 1 file changed, 22 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fa9bfd38a23..0ef71895999 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2137,20 +2137,20 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force) /// Please don't use "zero-copy replication" (a non-production feature) in production. /// It is not ready for production usage. Don't use it. - bool need_remove_parts_in_order = supportsReplication() && getSettings()->allow_remote_fs_zero_copy_replication; + /// It also is disabled for any object storage, because it can lead to race conditions on blob removal. + /// (see comment at `clearPartsFromFilesystemImpl`). + bool need_remove_parts_in_order = false; - if (need_remove_parts_in_order) + if (supportsReplication()) { - bool has_zero_copy_disk = false; for (const auto & disk : getDisks()) { - if (disk->supportZeroCopyReplication()) + if (disk->isRemote()) { - has_zero_copy_disk = true; + need_remove_parts_in_order = true; break; } } - need_remove_parts_in_order = has_zero_copy_disk; } std::vector parts_to_delete; @@ -2394,18 +2394,28 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t std::mutex part_names_mutex; auto runner = threadPoolCallbackRunner(getPartsCleaningThreadPool().get(), "PartsCleaning"); - /// This flag disallow straightforward concurrent parts removal. It's required only in case - /// when we have parts on zero-copy disk + at least some of them were mutated. + /** Straightforward concurrent parts removal can be applied for the case + * when we have parts on object storage disk + at least some of them were mutated + * (thus, can contains hardlinks to files in the previous parts). + * If we are deleting parts that contains hardlinks to the same file we may face into race condition + * and delete only local metadata files, but not the blobs on object storage. + * Given that, we remove in parallel only "independent" parts that don't have such hardlinks. + * Note that it also may be applicable for the regular MergeTree, fixed only for Replicated. + * + * To avoid this we need to fix race conditions on parts and blob removal. + */ bool remove_parts_in_order = false; - if (settings->allow_remote_fs_zero_copy_replication && dynamic_cast(this) != nullptr) + if (dynamic_cast(this) != nullptr) { remove_parts_in_order = std::any_of( parts_to_remove.begin(), parts_to_remove.end(), - [] (const auto & data_part) { return data_part->isStoredOnRemoteDiskWithZeroCopySupport() && data_part->info.getMutationVersion() > 0; } + [] (const auto & data_part) + { + return data_part->isStoredOnRemoteDisk() && data_part->info.getMutationVersion() > 0; + } ); } - if (!remove_parts_in_order) { /// NOTE: Under heavy system load you may get "Cannot schedule a task" from ThreadPool. @@ -2441,7 +2451,7 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t /// NOTE: Under heavy system load you may get "Cannot schedule a task" from ThreadPool. LOG_DEBUG( - log, "Removing {} parts from filesystem (concurrently): Parts: [{}]", parts_to_remove.size(), fmt::join(parts_to_remove, ", ")); + log, "Removing {} parts from filesystem (concurrently in order): Parts: [{}]", parts_to_remove.size(), fmt::join(parts_to_remove, ", ")); /// We have "zero copy replication" parts and we are going to remove them in parallel. /// The problem is that all parts in a mutation chain must be removed sequentially to avoid "key does not exits" issues. From 8b6376005a730b9ae461d3fe93a55e51cd494181 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 20 Apr 2023 13:26:02 +0000 Subject: [PATCH 0424/2047] "reconfig" support for CH Keeper --- base/base/find_symbols.h | 4 +- base/base/move_extend.h | 9 + contrib/NuRaft | 2 +- programs/keeper/CMakeLists.txt | 2 + src/Common/ProfileEvents.cpp | 2 + src/Common/ZooKeeper/IKeeper.cpp | 1 + src/Common/ZooKeeper/IKeeper.h | 41 +++- src/Common/ZooKeeper/TestKeeper.cpp | 56 ++++- src/Common/ZooKeeper/TestKeeper.h | 7 + src/Common/ZooKeeper/ZooKeeper.cpp | 32 ++- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 44 +++- src/Common/ZooKeeper/ZooKeeperCommon.h | 29 +++ src/Common/ZooKeeper/ZooKeeperConstants.cpp | 50 +--- src/Common/ZooKeeper/ZooKeeperConstants.h | 2 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 28 ++- src/Common/ZooKeeper/ZooKeeperImpl.h | 7 + src/Coordination/KeeperConstants.h | 9 +- src/Coordination/KeeperContext.h | 10 +- src/Coordination/KeeperDispatcher.cpp | 103 +++++--- src/Coordination/KeeperDispatcher.h | 16 +- src/Coordination/KeeperReconfiguration.cpp | 92 +++++++ src/Coordination/KeeperReconfiguration.h | 10 + src/Coordination/KeeperServer.cpp | 232 ++++++++---------- src/Coordination/KeeperServer.h | 23 +- src/Coordination/KeeperStateMachine.cpp | 124 ++++++++-- src/Coordination/KeeperStateMachine.h | 16 +- src/Coordination/KeeperStateManager.cpp | 15 +- src/Coordination/KeeperStateManager.h | 32 +-- src/Coordination/KeeperStorage.cpp | 20 +- src/Coordination/RaftServerConfig.cpp | 96 ++++++++ src/Coordination/RaftServerConfig.h | 78 ++++++ src/Coordination/tests/gtest_coordination.cpp | 51 +++- src/Interpreters/ZooKeeperLog.cpp | 1 + src/Storages/DataLakes/HudiMetadataParser.cpp | 3 +- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 6 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 8 +- src/Storages/StorageReplicatedMergeTree.cpp | 16 +- tests/integration/helpers/keeper_utils.py | 33 ++- .../test_keeper_nodes_move/test.py | 5 - .../test_keeper_reconfig_add/__init__.py | 0 .../configs/keeper1.xml | 20 ++ .../configs/keeper2.xml | 21 ++ .../configs/keeper3.xml | 22 ++ .../test_keeper_reconfig_add/test.py | 155 ++++++++++++ .../test_keeper_reconfig_remove/__init__.py | 0 .../configs/keeper1.xml | 37 +++ .../configs/keeper2.xml | 37 +++ .../configs/keeper3.xml | 37 +++ .../test_keeper_reconfig_remove/test.py | 145 +++++++++++ .../__init__.py | 0 .../configs/keeper1.xml | 47 ++++ .../configs/keeper2.xml | 47 ++++ .../configs/keeper3.xml | 47 ++++ .../configs/keeper4.xml | 47 ++++ .../configs/keeper5.xml | 47 ++++ .../test_keeper_reconfig_remove_many/test.py | 149 +++++++++++ .../__init__.py | 0 .../configs/keeper1.xml | 35 +++ .../configs/keeper2.xml | 35 +++ .../configs/keeper3.xml | 35 +++ .../configs/keeper4.xml | 21 ++ .../test.py | 127 ++++++++++ .../__init__.py | 0 .../configs/keeper1.xml | 35 +++ .../configs/keeper2.xml | 35 +++ .../configs/keeper3.xml | 35 +++ .../configs/keeper4.xml | 21 ++ .../test.py | 120 +++++++++ utils/keeper-data-dumper/main.cpp | 2 +- 70 files changed, 2309 insertions(+), 367 deletions(-) create mode 100644 base/base/move_extend.h create mode 100644 src/Coordination/KeeperReconfiguration.cpp create mode 100644 src/Coordination/KeeperReconfiguration.h create mode 100644 src/Coordination/RaftServerConfig.cpp create mode 100644 src/Coordination/RaftServerConfig.h create mode 100644 tests/integration/test_keeper_reconfig_add/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_add/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_add/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_add/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_add/test.py create mode 100644 tests/integration/test_keeper_reconfig_remove/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_remove/test.py create mode 100644 tests/integration/test_keeper_reconfig_remove_many/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml create mode 100644 tests/integration/test_keeper_reconfig_remove_many/test.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader/test.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml create mode 100644 tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py diff --git a/base/base/find_symbols.h b/base/base/find_symbols.h index 83232669c04..f7d24ccfc11 100644 --- a/base/base/find_symbols.h +++ b/base/base/find_symbols.h @@ -448,7 +448,7 @@ inline char * find_last_not_symbols_or_null(char * begin, char * end) /// See https://github.com/boostorg/algorithm/issues/63 /// And https://bugs.llvm.org/show_bug.cgi?id=41141 template -inline void splitInto(To & to, const std::string & what, bool token_compress = false) +inline To& splitInto(To & to, std::string_view what, bool token_compress = false) { const char * pos = what.data(); const char * end = pos + what.size(); @@ -464,4 +464,6 @@ inline void splitInto(To & to, const std::string & what, bool token_compress = f else pos = delimiter_or_end; } + + return to; } diff --git a/base/base/move_extend.h b/base/base/move_extend.h new file mode 100644 index 00000000000..6e5b16e037c --- /dev/null +++ b/base/base/move_extend.h @@ -0,0 +1,9 @@ +#pragma once + +/// Extend @p to by moving elements from @p from to @p to end +/// @return @p to iterator to first of moved elements. +template +typename To::iterator moveExtend(To & to, From && from) +{ + return to.insert(to.end(), std::make_move_iterator(from.begin()), std::make_move_iterator(from.end())); +} diff --git a/contrib/NuRaft b/contrib/NuRaft index 491eaf592d9..eb1572129c7 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 491eaf592d950e0e37accbe8b3f217e068c9fecf +Subproject commit eb1572129c71beb2156dcdaadc3fb136954aed96 diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 18bdc8f317c..20cab03dec2 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -34,6 +34,8 @@ add_dependencies(clickhouse-keeper-lib clickhouse_keeper_configs) if (BUILD_STANDALONE_KEEPER) # Straight list of all required sources set(CLICKHOUSE_KEEPER_STANDALONE_SOURCES + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperReconfiguration.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/RaftServerConfig.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/ACLMap.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Changelog.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/CoordinationSettings.cpp diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0838e0366df..8e3ec4f9e65 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -125,6 +125,7 @@ M(ZooKeeperMulti, "Number of 'multi' requests to ZooKeeper (compound transactions).") \ M(ZooKeeperCheck, "Number of 'check' requests to ZooKeeper. Usually they don't make sense in isolation, only as part of a complex transaction.") \ M(ZooKeeperSync, "Number of 'sync' requests to ZooKeeper. These requests are rarely needed or usable.") \ + M(ZooKeeperReconfig, "Number of 'reconfig' requests to ZooKeeper.") \ M(ZooKeeperClose, "Number of times connection with ZooKeeper has been closed voluntary.") \ M(ZooKeeperWatchResponse, "Number of times watch notification has been received from ZooKeeper.") \ M(ZooKeeperUserExceptions, "Number of exceptions while working with ZooKeeper related to the data (no node, bad version or similar).") \ @@ -499,6 +500,7 @@ The server successfully detected this situation and will download merged part fr M(KeeperCreateRequest, "Number of create requests")\ M(KeeperRemoveRequest, "Number of remove requests")\ M(KeeperSetRequest, "Number of set requests")\ + M(KeeperReconfigRequest, "Number of reconfig requests")\ M(KeeperCheckRequest, "Number of check requests")\ M(KeeperMultiRequest, "Number of multi requests")\ M(KeeperMultiReadRequest, "Number of multi read requests")\ diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index f0a07241735..50160279506 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -110,6 +110,7 @@ const char * errorMessage(Error code) case Error::ZCLOSING: return "ZooKeeper is closing"; case Error::ZNOTHING: return "(not error) no server responses to process"; case Error::ZSESSIONMOVED: return "Session moved to another server, so operation is ignored"; + case Error::ZRECONFIGINPROGRESS: return "Another reconfiguration is progress"; } UNREACHABLE(); diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 2703c1079c0..20ce2a748e6 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -82,6 +82,7 @@ enum class Error : int32_t ZOPERATIONTIMEOUT = -7, /// Operation timeout ZBADARGUMENTS = -8, /// Invalid arguments ZINVALIDSTATE = -9, /// Invalid zhandle state + ZRECONFIGINPROGRESS = -14, /// Another reconfig is running /** API errors. * This is never thrown by the server, it shouldn't be used other than @@ -350,6 +351,29 @@ struct SyncResponse : virtual Response size_t bytesSize() const override { return path.size(); } }; +struct ReconfigRequest : virtual Request +{ + String joining; + String leaving; + String new_members; + int32_t version; + + String getPath() const final { return keeper_config_path; } + + size_t bytesSize() const final + { + return joining.size() + leaving.size() + new_members.size() + sizeof(version); + } +}; + +struct ReconfigResponse : virtual Response +{ + String value; + Stat stat; + + size_t bytesSize() const override { return value.size() + sizeof(stat); } +}; + struct MultiRequest : virtual Request { Requests requests; @@ -395,9 +419,9 @@ using SetCallback = std::function; using ListCallback = std::function; using CheckCallback = std::function; using SyncCallback = std::function; +using ReconfigCallback = std::function; using MultiCallback = std::function; - /// For watches. enum State { @@ -526,6 +550,13 @@ public: const String & path, SyncCallback callback) = 0; + virtual void reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) = 0; + virtual void multi( const Requests & requests, MultiCallback callback) = 0; @@ -539,3 +570,11 @@ public: }; } + +template <> struct fmt::formatter : fmt::formatter +{ + constexpr auto format(Coordination::Error code, auto& ctx) + { + return formatter::format(Coordination::errorMessage(code), ctx); + } +}; diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index fe4cb83c78a..87c87c4fc92 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -3,12 +3,8 @@ #include #include #include - -#include -#include #include - namespace Coordination { @@ -147,6 +143,14 @@ struct TestKeeperSyncRequest final : SyncRequest, TestKeeperRequest std::pair process(TestKeeper::Container & container, int64_t zxid) const override; }; +struct TestKeeperReconfigRequest final : ReconfigRequest, TestKeeperRequest +{ + TestKeeperReconfigRequest() = default; + explicit TestKeeperReconfigRequest(const ReconfigRequest & base) : ReconfigRequest(base) {} + ResponsePtr createResponse() const override; + std::pair process(TestKeeper::Container & container, int64_t zxid) const override; +}; + struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest { explicit TestKeeperMultiRequest(const Requests & generic_requests) @@ -226,15 +230,7 @@ std::pair TestKeeperCreateRequest::process(TestKeeper::Contai std::string path_created = path; if (is_sequential) - { - auto seq_num = it->second.seq_num; - - std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - seq_num_str.exceptions(std::ios::failbit); - seq_num_str << std::setw(10) << std::setfill('0') << seq_num; - - path_created += seq_num_str.str(); - } + path_created += fmt::format("{:0>10}", it->second.seq_num); /// Increment sequential number even if node is not sequential ++it->second.seq_num; @@ -446,6 +442,17 @@ std::pair TestKeeperSyncRequest::process(TestKeeper::Containe return { std::make_shared(std::move(response)), {} }; } +std::pair TestKeeperReconfigRequest::process(TestKeeper::Container &, int64_t) const +{ + // In TestKeeper we assume data is stored on one server, so this is a dummy implementation to + // satisfy IKeeper interface. + // We can't even check the validity of input data, neither can we create the /keeper/config znode + // as we don't know the id of current "server". + ReconfigResponse response; + response.error = Error::ZOK; + return { std::make_shared(std::move(response)), {} }; +} + std::pair TestKeeperMultiRequest::process(TestKeeper::Container & container, int64_t zxid) const { MultiResponse response; @@ -505,6 +512,7 @@ ResponsePtr TestKeeperSetRequest::createResponse() const { return std::make_shar ResponsePtr TestKeeperListRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperCheckRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperSyncRequest::createResponse() const { return std::make_shared(); } +ResponsePtr TestKeeperReconfigRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperMultiRequest::createResponse() const { return std::make_shared(); } @@ -828,6 +836,28 @@ void TestKeeper::sync( pushRequest(std::move(request_info)); } +void TestKeeper::reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) +{ + TestKeeperReconfigRequest req; + req.joining = joining; + req.leaving = leaving; + req.new_members = new_members; + req.version = version; + + pushRequest({ + .request = std::make_shared(std::move(req)), + .callback = [callback](const Response & response) + { + callback(dynamic_cast(response)); + } + }); +} + void TestKeeper::multi( const Requests & requests, MultiCallback callback) diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 9bbd018cfb1..8615ed0fb77 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -87,6 +87,13 @@ public: const String & path, SyncCallback callback) override; + void reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) final; + void multi( const Requests & requests, MultiCallback callback) override; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 5dd7948276d..12b1d82133e 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -75,13 +75,14 @@ void ZooKeeper::init(ZooKeeperArgs args_) auto & host_string = host.host; try { - bool secure = startsWith(host_string, "secure://"); + const bool secure = startsWith(host_string, "secure://"); if (secure) host_string.erase(0, strlen("secure://")); - LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, Poco::Net::SocketAddress{host_string}.toString()); - nodes.emplace_back(Coordination::ZooKeeper::Node{Poco::Net::SocketAddress{host_string}, secure}); + const Poco::Net::SocketAddress host_socket_addr{host_string}; + LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, host_socket_addr.toString()); + nodes.emplace_back(Coordination::ZooKeeper::Node{host_socket_addr, secure}); } catch (const Poco::Net::HostNotFoundException & e) { @@ -191,12 +192,7 @@ std::vector ZooKeeper::shuffleHosts() const shuffle_hosts.emplace_back(shuffle_host); } - ::sort( - shuffle_hosts.begin(), shuffle_hosts.end(), - [](const ShuffleHost & lhs, const ShuffleHost & rhs) - { - return ShuffleHost::compare(lhs, rhs); - }); + ::sort(shuffle_hosts.begin(), shuffle_hosts.end(), ShuffleHost::compare); return shuffle_hosts; } @@ -231,7 +227,7 @@ Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::List), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::List, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -298,7 +294,7 @@ Coordination::Error ZooKeeper::createImpl(const std::string & path, const std::s if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Create), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Create, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -393,7 +389,7 @@ Coordination::Error ZooKeeper::removeImpl(const std::string & path, int32_t vers if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Remove), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Remove, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -425,7 +421,7 @@ Coordination::Error ZooKeeper::existsImpl(const std::string & path, Coordination if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Exists), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Exists, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -459,7 +455,7 @@ Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & r if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Get), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Get, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -531,7 +527,7 @@ Coordination::Error ZooKeeper::setImpl(const std::string & path, const std::stri if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Set), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Set, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -583,7 +579,7 @@ Coordination::Error ZooKeeper::multiImpl(const Coordination::Requests & requests if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Multi), requests[0]->getPath())); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Multi, requests[0]->getPath())); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -617,7 +613,7 @@ Coordination::Error ZooKeeper::syncImpl(const std::string & path, std::string & if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Sync), path)); + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Sync, path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -1229,7 +1225,7 @@ size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination:: if (!Coordination::isUserError(exception_code)) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "There are no failed OPs because '{}' is not valid response code for that", - std::string(Coordination::errorMessage(exception_code))); + exception_code); throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "There is no failed OpResult"); } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 5031af38812..c24eecbafd8 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -36,7 +36,7 @@ std::string ZooKeeperRequest::toString() const "OpNum = {}\n" "Additional info:\n{}", xid, - Coordination::toString(getOpNum()), + getOpNum(), toStringImpl()); } @@ -76,6 +76,41 @@ void ZooKeeperSyncResponse::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +void ZooKeeperReconfigRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(joining, out); + Coordination::write(leaving, out); + Coordination::write(new_members, out); + Coordination::write(version, out); +} + +void ZooKeeperReconfigRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(joining, in); + Coordination::read(leaving, in); + Coordination::read(new_members, in); + Coordination::read(version, in); +} + +std::string ZooKeeperReconfigRequest::toStringImpl() const +{ + return fmt::format( + "joining = {}\nleaving = {}\nnew_members = {}\nversion = {}", + joining, leaving, new_members, version); +} + +void ZooKeeperReconfigResponse::readImpl(ReadBuffer & in) +{ + Coordination::read(value, in); + Coordination::read(stat, in); +} + +void ZooKeeperReconfigResponse::writeImpl(WriteBuffer & out) const +{ + Coordination::write(value, out); + Coordination::write(stat, out); +} + void ZooKeeperWatchResponse::readImpl(ReadBuffer & in) { Coordination::read(type, in); @@ -664,6 +699,7 @@ ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return setTi ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperReconfigRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperSimpleListRequest::makeResponse() const { return setTime(std::make_shared()); } @@ -861,7 +897,8 @@ void ZooKeeperMultiResponse::fillLogElements(LogElements & elems, size_t idx) co void ZooKeeperRequestFactory::registerRequest(OpNum op_num, Creator creator) { if (!op_num_to_request.try_emplace(op_num, creator).second) - throw Coordination::Exception("Request type " + toString(op_num) + " already registered", Coordination::Error::ZRUNTIMEINCONSISTENCY); + throw Coordination::Exception(Coordination::Error::ZRUNTIMEINCONSISTENCY, + "Request type {} already registered", op_num); } std::shared_ptr ZooKeeperRequest::read(ReadBuffer & in) @@ -916,7 +953,7 @@ ZooKeeperRequestPtr ZooKeeperRequestFactory::get(OpNum op_num) const { auto it = op_num_to_request.find(op_num); if (it == op_num_to_request.end()) - throw Exception("Unknown operation type " + toString(op_num), Error::ZBADARGUMENTS); + throw Exception(Error::ZBADARGUMENTS, "Unknown operation type {}", op_num); return it->second(); } @@ -960,6 +997,7 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory() registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 5f00698423e..131d19f1ca4 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -117,6 +117,35 @@ struct ZooKeeperSyncResponse final : SyncResponse, ZooKeeperResponse OpNum getOpNum() const override { return OpNum::Sync; } }; +struct ZooKeeperReconfigRequest final : ZooKeeperRequest +{ + String joining; + String leaving; + String new_members; + int64_t version; // kazoo sends a 64bit integer in this request + + String getPath() const override { return keeper_config_path; } + OpNum getOpNum() const override { return OpNum::Reconfig; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + std::string toStringImpl() const override; + ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return false; } + + size_t bytesSize() const override + { + return ZooKeeperRequest::bytesSize() + joining.size() + leaving.size() + new_members.size() + + sizeof(version); + } +}; + +struct ZooKeeperReconfigResponse final : ReconfigResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer & in) override; + void writeImpl(WriteBuffer & out) const override; + OpNum getOpNum() const override { return OpNum::Reconfig; } +}; + struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse { void readImpl(ReadBuffer &) override {} diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.cpp b/src/Common/ZooKeeper/ZooKeeperConstants.cpp index 86f70ea547a..9bb9c7b0488 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.cpp +++ b/src/Common/ZooKeeper/ZooKeeperConstants.cpp @@ -19,6 +19,7 @@ static const std::unordered_set VALID_OPERATIONS = static_cast(OpNum::Heartbeat), static_cast(OpNum::List), static_cast(OpNum::Check), + static_cast(OpNum::Reconfig), static_cast(OpNum::Multi), static_cast(OpNum::MultiRead), static_cast(OpNum::Auth), @@ -29,55 +30,6 @@ static const std::unordered_set VALID_OPERATIONS = static_cast(OpNum::CheckNotExists), }; -std::string toString(OpNum op_num) -{ - switch (op_num) - { - case OpNum::Close: - return "Close"; - case OpNum::Error: - return "Error"; - case OpNum::Create: - return "Create"; - case OpNum::Remove: - return "Remove"; - case OpNum::Exists: - return "Exists"; - case OpNum::Get: - return "Get"; - case OpNum::Set: - return "Set"; - case OpNum::SimpleList: - return "SimpleList"; - case OpNum::List: - return "List"; - case OpNum::Check: - return "Check"; - case OpNum::Multi: - return "Multi"; - case OpNum::MultiRead: - return "MultiRead"; - case OpNum::Sync: - return "Sync"; - case OpNum::Heartbeat: - return "Heartbeat"; - case OpNum::Auth: - return "Auth"; - case OpNum::SessionID: - return "SessionID"; - case OpNum::SetACL: - return "SetACL"; - case OpNum::GetACL: - return "GetACL"; - case OpNum::FilteredList: - return "FilteredList"; - case OpNum::CheckNotExists: - return "CheckNotExists"; - } - int32_t raw_op = static_cast(op_num); - throw Exception("Operation " + std::to_string(raw_op) + " is unknown", Error::ZUNIMPLEMENTED); -} - OpNum getOpNum(int32_t raw_op_num) { if (!VALID_OPERATIONS.contains(raw_op_num)) diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index 6b50c5c5d09..a773fbbab74 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -31,6 +31,7 @@ enum class OpNum : int32_t List = 12, Check = 13, Multi = 14, + Reconfig = 16, MultiRead = 22, Auth = 100, @@ -41,7 +42,6 @@ enum class OpNum : int32_t SessionID = 997, /// Special internal request }; -std::string toString(OpNum op_num); OpNum getOpNum(int32_t raw_op_num); static constexpr int32_t ZOOKEEPER_PROTOCOL_VERSION = 0; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 0f27d078234..5e16a437be3 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -35,6 +35,7 @@ namespace ProfileEvents extern const Event ZooKeeperRemove; extern const Event ZooKeeperExists; extern const Event ZooKeeperMulti; + extern const Event ZooKeeperReconfig; extern const Event ZooKeeperGet; extern const Event ZooKeeperSet; extern const Event ZooKeeperList; @@ -571,7 +572,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) if (err != Error::ZOK) throw Exception(Error::ZMARSHALLINGERROR, "Error received in reply to auth request. Code: {}. Message: {}", - static_cast(err), errorMessage(err)); + static_cast(err), err); } void ZooKeeper::sendThread() @@ -697,7 +698,7 @@ void ZooKeeper::receiveThread() if (earliest_operation) { throw Exception(Error::ZOPERATIONTIMEOUT, "Operation timeout (no response in {} ms) for request {} for path: {}", - args.operation_timeout_ms, toString(earliest_operation->request->getOpNum()), earliest_operation->request->getPath()); + args.operation_timeout_ms, earliest_operation->request->getOpNum(), earliest_operation->request->getPath()); } waited_us += max_wait_us; if (waited_us >= args.session_timeout_ms * 1000) @@ -738,7 +739,7 @@ void ZooKeeper::receiveEvent() if (xid == PING_XID) { if (err != Error::ZOK) - throw Exception(Error::ZRUNTIMEINCONSISTENCY, "Received error in heartbeat response: {}", errorMessage(err)); + throw Exception(Error::ZRUNTIMEINCONSISTENCY, "Received error in heartbeat response: {}", err); response = std::make_shared(); } @@ -1195,7 +1196,6 @@ void ZooKeeper::create( ProfileEvents::increment(ProfileEvents::ZooKeeperCreate); } - void ZooKeeper::remove( const String & path, int32_t version, @@ -1335,6 +1335,26 @@ void ZooKeeper::sync( ProfileEvents::increment(ProfileEvents::ZooKeeperSync); } +void ZooKeeper::reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) +{ + ZooKeeperReconfigRequest request; + request.joining = joining; + request.leaving = leaving; + request.new_members = new_members; + request.version = version; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; + + pushRequest(std::move(request_info)); + ProfileEvents::increment(ProfileEvents::ZooKeeperReconfig); +} void ZooKeeper::multi( const Requests & requests, diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 44ea993947e..7e27608d0a1 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -178,6 +178,13 @@ public: const String & path, SyncCallback callback) override; + void reconfig( + std::string_view joining, + std::string_view leaving, + std::string_view new_members, + int32_t version, + ReconfigCallback callback) final; + void multi( const Requests & requests, MultiCallback callback) override; diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 84cbb0ab7c5..675001d51e0 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -1,5 +1,4 @@ #pragma once - #include namespace DB @@ -14,8 +13,8 @@ enum class KeeperApiVersion : uint8_t WITH_CHECK_NOT_EXISTS, }; -const std::string keeper_system_path = "/keeper"; -const std::string keeper_api_version_path = keeper_system_path + "/api_version"; -const std::string keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; - +const String keeper_system_path = "/keeper"; +const String keeper_api_version_path = keeper_system_path + "/api_version"; +const String keeper_api_feature_flags_path = keeper_system_path + "/feature_flags"; +const String keeper_config_path = keeper_system_path + "/config"; } diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 229dbd51ab2..4fb552f20a3 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -1,10 +1,8 @@ #pragma once - -#include - #include -#include #include +#include +#include #include #include @@ -12,6 +10,8 @@ namespace DB { +class KeeperDispatcher; + class KeeperContext { public: @@ -51,6 +51,7 @@ public: const KeeperFeatureFlags & getFeatureFlags() const; void dumpConfiguration(WriteBufferFromOwnString & buf) const; + private: /// local disk defined using path or disk name using Storage = std::variant; @@ -85,6 +86,7 @@ private: std::unordered_map system_nodes_with_data; KeeperFeatureFlags feature_flags; + KeeperDispatcher * dispatcher{nullptr}; }; using KeeperContextPtr = std::shared_ptr; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9d9df5c7f30..178453b2f5b 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -38,6 +38,8 @@ namespace ProfileEvents extern const Event MemoryAllocatorPurgeTimeMicroseconds; } +using namespace std::chrono_literals; + namespace DB { @@ -336,6 +338,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf keeper_context = std::make_shared(standalone_keeper); keeper_context->initialize(config); + keeper_context->dispatcher = this; server = std::make_unique( configuration_and_settings, @@ -392,7 +395,10 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf /// Start it after keeper server start session_cleaner_thread = ThreadFromGlobalPool([this] { sessionCleanerTask(); }); - update_configuration_thread = ThreadFromGlobalPool([this] { updateConfigurationThread(); }); + + update_configuration_thread = reconfigEnabled() + ? ThreadFromGlobalPool([this] { clusterUpdateThread(); }) + : ThreadFromGlobalPool([this] { clusterUpdateWithReconfigDisabledThread(); }); LOG_DEBUG(log, "Dispatcher initialized"); } @@ -429,7 +435,7 @@ void KeeperDispatcher::shutdown() if (snapshot_thread.joinable()) snapshot_thread.join(); - update_configuration_queue.finish(); + cluster_update_queue.finish(); if (update_configuration_thread.joinable()) update_configuration_thread.join(); } @@ -608,7 +614,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession "Could not push error response xid {} zxid {} error message {} to responses queue", response->xid, response->zxid, - errorMessage(error)); + error); } } @@ -653,7 +659,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) { if (response->getOpNum() != Coordination::OpNum::SessionID) promise->set_exception(std::make_exception_ptr(Exception(ErrorCodes::LOGICAL_ERROR, - "Incorrect response of type {} instead of SessionID response", Coordination::toString(response->getOpNum())))); + "Incorrect response of type {} instead of SessionID response", response->getOpNum()))); auto session_id_response = dynamic_cast(*response); if (session_id_response.internal_id != internal_id) @@ -685,17 +691,12 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) return future.get(); } - -void KeeperDispatcher::updateConfigurationThread() +void KeeperDispatcher::clusterUpdateWithReconfigDisabledThread() { - while (true) + while (!shutdown_called) { - if (shutdown_called) - return; - try { - using namespace std::chrono_literals; if (!server->checkInit()) { LOG_INFO(log, "Server still not initialized, will not apply configuration until initialization finished"); @@ -710,11 +711,10 @@ void KeeperDispatcher::updateConfigurationThread() continue; } - ConfigUpdateAction action; - if (!update_configuration_queue.pop(action)) + ClusterUpdateAction action; + if (!cluster_update_queue.pop(action)) break; - /// We must wait this update from leader or apply it ourself (if we are leader) bool done = false; while (!done) @@ -727,15 +727,13 @@ void KeeperDispatcher::updateConfigurationThread() if (isLeader()) { - server->applyConfigurationUpdate(action); + server->applyConfigUpdateWithReconfigDisabled(action); done = true; } - else - { - done = server->waitConfigurationUpdate(action); - if (!done) - LOG_INFO(log, "Cannot wait for configuration update, maybe we become leader, or maybe update is invalid, will try to wait one more time"); - } + else if (done = server->waitForConfigUpdateWithReconfigDisabled(action); !done) + LOG_INFO(log, + "Cannot wait for configuration update, maybe we became leader " + "or maybe update is invalid, will try to wait one more time"); } } catch (...) @@ -745,6 +743,46 @@ void KeeperDispatcher::updateConfigurationThread() } } +void KeeperDispatcher::clusterUpdateThread() +{ + while (!shutdown_called) + { + ClusterUpdateAction action; + if (!cluster_update_queue.pop(action)) + return; + + if (server->applyConfigUpdate(action)) + LOG_DEBUG(log, "Processing config update {}: accepted", action); + else // TODO (myrrc) sleep a random amount? sleep less? + { + (void)cluster_update_queue.pushFront(action); + LOG_DEBUG(log, "Processing config update {}: declined, backoff", action); + std::this_thread::sleep_for(50ms); + } + } +} + +void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions&& actions) +{ + if (shutdown_called) return; + for (auto && action : actions) + { + if (!cluster_update_queue.push(std::move(action))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot push configuration update"); + LOG_DEBUG(log, "Processing config update {}: pushed", action); + } +} + +bool KeeperDispatcher::clusterUpdateQueueEmpty() const +{ + return cluster_update_queue.empty(); +} + +bool KeeperDispatcher::reconfigEnabled() const +{ + return server->reconfigEnabled(); +} + bool KeeperDispatcher::isServerActive() const { return checkInit() && hasLeader() && !server->isRecovering(); @@ -752,20 +790,25 @@ bool KeeperDispatcher::isServerActive() const void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros) { - auto diff = server->getConfigurationDiff(config); + auto diff = server->getRaftConfigurationDiff(config); + if (diff.empty()) - LOG_TRACE(log, "Configuration update triggered, but nothing changed for RAFT"); + LOG_TRACE(log, "Configuration update triggered, but nothing changed for Raft"); + else if (reconfigEnabled()) + LOG_WARNING(log, + "Raft configuration changed, but keeper_server.enable_reconfiguration is on. " + "This update will be ignored. Use \"reconfig\" instead"); else if (diff.size() > 1) - LOG_WARNING(log, "Configuration changed for more than one server ({}) from cluster, it's strictly not recommended", diff.size()); + LOG_WARNING(log, + "Configuration changed for more than one server ({}) from cluster, " + "it's strictly not recommended", diff.size()); else LOG_DEBUG(log, "Configuration change size ({})", diff.size()); - for (auto & change : diff) - { - bool push_result = update_configuration_queue.push(change); - if (!push_result) - throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue"); - } + if (!reconfigEnabled()) + for (auto & change : diff) + if (!cluster_update_queue.push(change)) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue"); snapshot_s3.updateS3Configuration(config, macros); } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 1b44f0f6ced..a9b3d33eb51 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -31,7 +31,7 @@ private: using RequestsQueue = ConcurrentBoundedQueue; using SessionToResponseCallback = std::unordered_map; - using UpdateConfigurationQueue = ConcurrentBoundedQueue; + using ClusterUpdateQueue = ConcurrentBoundedQueue; /// Size depends on coordination settings std::unique_ptr requests_queue; @@ -39,7 +39,7 @@ private: SnapshotsQueue snapshots_queue{1}; /// More than 1k updates is definitely misconfiguration. - UpdateConfigurationQueue update_configuration_queue{1000}; + ClusterUpdateQueue cluster_update_queue{1000}; std::atomic shutdown_called{false}; @@ -91,8 +91,10 @@ private: void sessionCleanerTask(); /// Thread create snapshots in the background void snapshotThread(); - /// Thread apply or wait configuration changes from leader - void updateConfigurationThread(); + + // TODO (myrrc) this should be removed once "reconfig" is stabilized + void clusterUpdateWithReconfigDisabledThread(); + void clusterUpdateThread(); void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); @@ -132,10 +134,10 @@ public: /// and achieved quorum bool isServerActive() const; - /// Registered in ConfigReloader callback. Add new configuration changes to - /// update_configuration_queue. Keeper Dispatcher apply them asynchronously. - /// 'macros' are used to substitute macros in endpoint of disks void updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); + void pushClusterUpdates(ClusterUpdateActions&& actions); + bool clusterUpdateQueueEmpty() const; + bool reconfigEnabled() const; /// Shutdown internal keeper parts (server, state machine, log storage, etc) void shutdown(); diff --git a/src/Coordination/KeeperReconfiguration.cpp b/src/Coordination/KeeperReconfiguration.cpp new file mode 100644 index 00000000000..dec3e1f155f --- /dev/null +++ b/src/Coordination/KeeperReconfiguration.cpp @@ -0,0 +1,92 @@ +#include "KeeperReconfiguration.h" +#include +#include +#include +#include + +namespace DB +{ +ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining) +{ + ClusterUpdateActions out; + std::unordered_set endpoints; + + for (const auto & server : cfg->get_servers()) + endpoints.emplace(server->get_endpoint()); + + // We can either add new servers or change weight of existing ones. + // It makes no sense having a server in _joining_ which is identical to existing one including + // weight, so such requests are declined. + for (const RaftServerConfig & update : parseRaftServers(joining)) + if (auto server_ptr = cfg->get_server(update.id)) + { + if (update.endpoint != server_ptr->get_endpoint() || update.learner != server_ptr->is_learner() + || update.priority == server_ptr->get_priority()) + return {}; // can't change server endpoint/type due to NuRaft API limitations + out.emplace_back(UpdateRaftServerPriority{.id = update.id, .priority = update.priority}); + } + else if (endpoints.contains(update.endpoint)) + return {}; + else + out.emplace_back(AddRaftServer{update}); + + return out; +} + +ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view leaving) +{ + std::vector leaving_arr; + splitInto<','>(leaving_arr, leaving); + if (leaving_arr.size() >= cfg->get_servers().size()) + return {}; + + std::unordered_set remove_ids; + ClusterUpdateActions out; + + for (std::string_view leaving_server : leaving_arr) + { + int id; + if (std::from_chars(leaving_server.begin(), leaving_server.end(), id).ec != std::error_code{}) + return {}; + + if (remove_ids.contains(id)) + continue; + + if (auto ptr = cfg->get_server(id)) + out.emplace_back(RemoveRaftServer{.id = id}); + else + return {}; + + remove_ids.emplace(id); + } + + return out; +} + +String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateActions & updates) +{ + RaftServers new_config; + std::unordered_set remove_update_ids; + + for (const auto & update : updates) + { + if (const auto * add = std::get_if(&update)) + new_config.emplace_back(*add); + else if (const auto * remove = std::get_if(&update)) + remove_update_ids.insert(remove->id); + else if (const auto * priority = std::get_if(&update)) + { + remove_update_ids.insert(priority->id); + new_config.emplace_back(RaftServerConfig{*cfg->get_server(priority->id)}); + } + else + UNREACHABLE(); + } + + for (const auto & item : cfg->get_servers()) + if (!remove_update_ids.contains(item->get_id())) + new_config.emplace_back(RaftServerConfig{*item}); + + return fmt::format("{}", fmt::join(new_config.begin(), new_config.end(), "\n")); +} +} diff --git a/src/Coordination/KeeperReconfiguration.h b/src/Coordination/KeeperReconfiguration.h new file mode 100644 index 00000000000..71958f2035e --- /dev/null +++ b/src/Coordination/KeeperReconfiguration.h @@ -0,0 +1,10 @@ +#pragma once +#include "Coordination/KeeperSnapshotManager.h" +#include "Coordination/RaftServerConfig.h" + +namespace DB +{ +ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining); +ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view leaving); +String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateActions & updates = {}); +} diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 82c843287c1..1cde957ef3a 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -27,6 +27,7 @@ #include #include #include +#include namespace DB { @@ -40,6 +41,8 @@ namespace ErrorCodes extern const int INVALID_CONFIG_PARAMETER; } +using namespace std::chrono_literals; + namespace { @@ -118,6 +121,7 @@ KeeperServer::KeeperServer( , is_recovering(config.getBool("keeper_server.force_recovery", false)) , keeper_context{std::move(keeper_context_)} , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) + , enable_reconfiguration(config.getBool("keeper_server.enable_reconfiguration", false)) { if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); @@ -450,7 +454,7 @@ void KeeperServer::shutdownRaftServer() size_t count = 0; while (asio_service->get_active_workers() != 0 && count < timeout * 100) { - std::this_thread::sleep_for(std::chrono::milliseconds(10)); + std::this_thread::sleep_for(10ms); count++; } } @@ -715,10 +719,12 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ if (next_index < last_commited || next_index - last_commited <= 1) commited_store = true; - auto set_initialized = [this]() + auto set_initialized = [this] { - std::lock_guard lock(initialized_mutex); - initialized_flag = true; + { + std::lock_guard lock(initialized_mutex); + initialized_flag = true; + } initialized_cv.notify_all(); }; @@ -783,9 +789,42 @@ std::vector KeeperServer::getDeadSessions() return state_machine->getDeadSessions(); } -ConfigUpdateActions KeeperServer::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) +bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) { - auto diff = state_manager->getConfigurationDiff(config); + std::lock_guard _{server_write_mutex}; + + if (const auto* add = std::get_if(&action)) + return raft_instance->get_srv_config(add->id) != nullptr + || raft_instance->add_srv(static_cast(*add))->get_accepted(); + else if (const auto* remove = std::get_if(&action)) + { + if (isLeader() && remove->id == state_manager->server_id()) + { + raft_instance->yield_leadership(); + return false; + } + + return raft_instance->get_srv_config(remove->id) == nullptr + || raft_instance->remove_srv(remove->id)->get_accepted(); + } + else if (const auto* update = std::get_if(&action)) + { + if (auto ptr = raft_instance->get_srv_config(update->id); ptr == nullptr) + throw Exception(ErrorCodes::RAFT_ERROR, + "Attempt to apply {} but server is not present in Raft", + action); + else if (ptr->get_priority() == update->priority) + return true; + + raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); + return true; + } + UNREACHABLE(); +} + +ClusterUpdateActions KeeperServer::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) +{ + auto diff = state_manager->getRaftConfigurationDiff(config); if (!diff.empty()) { @@ -796,160 +835,103 @@ ConfigUpdateActions KeeperServer::getConfigurationDiff(const Poco::Util::Abstrac return diff; } -void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task) +void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action) { - std::lock_guard lock{server_write_mutex}; - if (is_recovering) - return; + std::lock_guard _{server_write_mutex}; + if (is_recovering) return; + constexpr auto sleep_time = 500ms; - size_t sleep_ms = 500; - if (task.action_type == ConfigUpdateActionType::AddServer) + LOG_INFO(log, "Will try to apply {}", action); + + auto applied = [&] { LOG_INFO(log, "Applied {}", action); }; + auto not_leader = [&] { LOG_INFO(log, "Not leader anymore, aborting"); }; + auto backoff_on_refusal = [&](size_t i) + { + LOG_INFO(log, "Update was not accepted (try {}), backing off for {}", i + 1, sleep_time * (i + 1)); + std::this_thread::sleep_for(sleep_time * (i + 1)); + }; + + if (const auto* add = std::get_if(&action)) { - LOG_INFO(log, "Will try to add server with id {}", task.server->get_id()); - bool added = false; for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) != nullptr) - { - LOG_INFO(log, "Server with id {} was successfully added", task.server->get_id()); - added = true; - break; - } - + if (raft_instance->get_srv_config(add->id) != nullptr) + return applied(); if (!isLeader()) - { - LOG_INFO(log, "We are not leader anymore, will not try to add server {}", task.server->get_id()); - break; - } - - auto result = raft_instance->add_srv(*task.server); - if (!result->get_accepted()) - LOG_INFO( - log, - "Command to add server {} was not accepted for the {} time, will sleep for {} ms and retry", - task.server->get_id(), - i + 1, - sleep_ms * (i + 1)); - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return not_leader(); + if (!raft_instance->add_srv(static_cast(*add))->get_accepted()) + backoff_on_refusal(i); } - if (!added) - throw Exception( - ErrorCodes::RAFT_ERROR, - "Configuration change to add server (id {}) was not accepted by RAFT after all {} retries", - task.server->get_id(), - coordination_settings->configuration_change_tries_count); } - else if (task.action_type == ConfigUpdateActionType::RemoveServer) + else if (const auto* remove = std::get_if(&action)) { - LOG_INFO(log, "Will try to remove server with id {}", task.server->get_id()); - - bool removed = false; - if (task.server->get_id() == state_manager->server_id()) + if (remove->id == state_manager->server_id()) { - LOG_INFO( - log, - "Trying to remove leader node (ourself), so will yield leadership and some other node (new leader) will try remove us. " + LOG_INFO(log, + "Trying to remove leader node (ourself), so will yield leadership and some other node " + "(new leader) will try to remove us. " "Probably you will have to run SYSTEM RELOAD CONFIG on the new leader node"); - - raft_instance->yield_leadership(); - return; + return raft_instance->yield_leadership(); } for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) == nullptr) - { - LOG_INFO(log, "Server with id {} was successfully removed", task.server->get_id()); - removed = true; - break; - } - + if (raft_instance->get_srv_config(remove->id) == nullptr) + return applied(); if (!isLeader()) - { - LOG_INFO(log, "We are not leader anymore, will not try to remove server {}", task.server->get_id()); - break; - } - - auto result = raft_instance->remove_srv(task.server->get_id()); - if (!result->get_accepted()) - LOG_INFO( - log, - "Command to remove server {} was not accepted for the {} time, will sleep for {} ms and retry", - task.server->get_id(), - i + 1, - sleep_ms * (i + 1)); - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return not_leader(); + if (!raft_instance->remove_srv(remove->id)->get_accepted()) + backoff_on_refusal(i); } - if (!removed) - throw Exception( - ErrorCodes::RAFT_ERROR, - "Configuration change to remove server (id {}) was not accepted by RAFT after all {} retries", - task.server->get_id(), - coordination_settings->configuration_change_tries_count); } - else if (task.action_type == ConfigUpdateActionType::UpdatePriority) - raft_instance->set_priority(task.server->get_id(), task.server->get_priority()); - else - LOG_WARNING(log, "Unknown configuration update type {}", static_cast(task.action_type)); + else if (const auto* update = std::get_if(&action)) + { + raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); + return; + } + + throw Exception(ErrorCodes::RAFT_ERROR, + "Configuration change {} was not accepted by Raft after {} retries", + action, coordination_settings->configuration_change_tries_count); } - -bool KeeperServer::waitConfigurationUpdate(const ConfigUpdateAction & task) +bool KeeperServer::waitForConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action) { - if (is_recovering) - return false; + if (is_recovering) return false; + constexpr auto sleep_time = 500ms; - size_t sleep_ms = 500; - if (task.action_type == ConfigUpdateActionType::AddServer) + LOG_INFO(log, "Will try to wait for {}", action); + + auto applied = [&] { LOG_INFO(log, "Applied {}", action); return true; }; + auto became_leader = [&] { LOG_INFO(log, "Became leader, aborting"); return false; }; + auto backoff = [&](size_t i) { std::this_thread::sleep_for(sleep_time * (i + 1)); }; + + if (const auto* add = std::get_if(&action)) { - LOG_INFO(log, "Will try to wait server with id {} to be added", task.server->get_id()); for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) != nullptr) - { - LOG_INFO(log, "Server with id {} was successfully added by leader", task.server->get_id()); - return true; - } - + if (raft_instance->get_srv_config(add->id) != nullptr) + return applied(); if (isLeader()) - { - LOG_INFO(log, "We are leader now, probably we will have to add server {}", task.server->get_id()); - return false; - } - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return became_leader(); + backoff(i); } - return false; } - else if (task.action_type == ConfigUpdateActionType::RemoveServer) + else if (const auto* remove = std::get_if(&action)) { - LOG_INFO(log, "Will try to wait remove of server with id {}", task.server->get_id()); - for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { - if (raft_instance->get_srv_config(task.server->get_id()) == nullptr) - { - LOG_INFO(log, "Server with id {} was successfully removed by leader", task.server->get_id()); - return true; - } - + if (raft_instance->get_srv_config(remove->id) == nullptr) + return applied(); if (isLeader()) - { - LOG_INFO(log, "We are leader now, probably we will have to remove server {}", task.server->get_id()); - return false; - } - - std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + return became_leader(); + backoff(i); } - return false; } - else if (task.action_type == ConfigUpdateActionType::UpdatePriority) + else if (std::get_if(&action) != nullptr) return true; - else - LOG_WARNING(log, "Unknown configuration update type {}", static_cast(task.action_type)); - return true; + + return false; } Keeper4LWInfo KeeperServer::getPartiallyFilled4LWInfo() const diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 8f416b1f48c..61e29b67bbd 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -10,12 +10,15 @@ #include #include #include +#include namespace DB { using RaftAppendResult = nuraft::ptr>>; +class KeeperDispatcher; + class KeeperServer { private: @@ -28,9 +31,10 @@ private: nuraft::ptr state_manager; struct KeeperRaftServer; - nuraft::ptr raft_instance; + nuraft::ptr raft_instance; // TSA_GUARDED_BY(server_write_mutex); nuraft::ptr asio_service; std::vector> asio_listeners; + // because some actions can be applied // when we are sure that there are no requests currently being // processed (e.g. recovery) we do all write actions @@ -65,6 +69,7 @@ private: std::shared_ptr keeper_context; const bool create_snapshot_on_exit; + const bool enable_reconfiguration; public: KeeperServer( @@ -84,6 +89,7 @@ public: void putLocalReadRequest(const KeeperStorage::RequestForSession & request); bool isRecovering() const { return is_recovering; } + bool reconfigEnabled() const { return enable_reconfiguration; } /// Put batch of requests into Raft and get result of put. Responses will be set separately into /// responses_queue. @@ -122,17 +128,12 @@ public: int getServerID() const { return server_id; } - /// Get configuration diff between current configuration in RAFT and in XML file - ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config); + bool applyConfigUpdate(const ClusterUpdateAction& action); - /// Apply action for configuration update. Actually call raft_instance->remove_srv or raft_instance->add_srv. - /// Synchronously check for update results with retries. - void applyConfigurationUpdate(const ConfigUpdateAction & task); - - - /// Wait configuration update for action. Used by followers. - /// Return true if update was successfully received. - bool waitConfigurationUpdate(const ConfigUpdateAction & task); + // TODO (myrrc) these functions should be removed once "reconfig" is stabilized + void applyConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action); + bool waitForConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action); + ClusterUpdateActions getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config); uint64_t createSnapshot(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5c84f23fc60..3e9850caa40 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -2,17 +2,20 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include #include #include #include "Coordination/KeeperStorage.h" +#include "Coordination/KeeperReconfiguration.h" #include @@ -146,7 +149,7 @@ void assertDigest( "Digest for nodes is not matching after {} request of type '{}'.\nExpected digest - {}, actual digest - {} (digest " "{}). Keeper will terminate to avoid inconsistencies.\nExtra information about the request:\n{}", committing ? "committing" : "preprocessing", - Coordination::toString(request.getOpNum()), + request.getOpNum(), first.value, second.value, first.version, @@ -261,7 +264,8 @@ std::shared_ptr KeeperStateMachine::parseReque bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session) { - if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) + const auto op_num = request_for_session.request->getOpNum(); + if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig) return true; std::lock_guard lock(storage_and_responses_lock); @@ -291,14 +295,89 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req return true; } +KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( + const KeeperStorage::RequestForSession& request_for_session) +{ + const auto& request = static_cast(*request_for_session.request); + const int64_t session_id = request_for_session.session_id; + const int64_t zxid = request_for_session.zxid; + + using enum Coordination::Error; + auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorage::ResponseForSession + { + auto res = std::make_shared(); + res->xid = request.xid; + res->zxid = zxid; + res->error = code; + return { session_id, std::move(res) }; + }; + + KeeperDispatcher& dispatcher = *keeper_context->dispatcher; + if (!dispatcher.reconfigEnabled()) + return bad_request(ZUNIMPLEMENTED); + if (!dispatcher.clusterUpdateQueueEmpty()) + return bad_request(ZRECONFIGINPROGRESS); + if (request.version != -1) + return bad_request(ZBADVERSION); + + const bool has_new_members = !request.new_members.empty(); + const bool has_joining = !request.joining.empty(); + const bool has_leaving = !request.leaving.empty(); + const bool incremental_reconfig = (has_joining || has_leaving) && !has_new_members; + if (!incremental_reconfig) + return bad_request(); + + const ClusterConfigPtr config = getClusterConfig(); + if (!config) // Server can be uninitialized yet + return bad_request(); + + ClusterUpdateActions updates; + + if (has_joining) + { + if (auto join_updates = joiningToClusterUpdates(config, request.joining); !join_updates.empty()) + moveExtend(updates, std::move(join_updates)); + else + return bad_request(); + } + + if (has_leaving) + { + if (auto leave_updates = leavingToClusterUpdates(config, request.leaving); !leave_updates.empty()) + moveExtend(updates, std::move(leave_updates)); + else + return bad_request(); + } + + auto response = std::make_shared(); + response->xid = request.xid; + response->zxid = zxid; + response->error = Coordination::Error::ZOK; + response->value = serializeClusterConfig(config, updates); + + dispatcher.pushClusterUpdates(std::move(updates)); + return { session_id, std::move(response) }; +} + nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) { auto request_for_session = parseRequest(data, true); if (!request_for_session->zxid) request_for_session->zxid = log_idx; - /// Special processing of session_id request - if (request_for_session->request->getOpNum() == Coordination::OpNum::SessionID) + auto try_push = [this](const KeeperStorage::ResponseForSession& response) + { + if (!responses_queue.push(response)) + { + ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); + LOG_WARNING(log, + "Failed to push response with session id {} to the queue, probably because of shutdown", + response.session_id); + } + }; + + const auto op_num = request_for_session->request->getOpNum(); + if (op_num == Coordination::OpNum::SessionID) { const Coordination::ZooKeeperSessionIDRequest & session_id_request = dynamic_cast(*request_for_session->request); @@ -309,21 +388,24 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n KeeperStorage::ResponseForSession response_for_session; response_for_session.session_id = -1; response_for_session.response = response; - { - std::lock_guard lock(storage_and_responses_lock); - session_id = storage->getSessionID(session_id_request.session_timeout_ms); - LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); - response->session_id = session_id; - if (!responses_queue.push(response_for_session)) - { - ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); - LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", session_id); - } - } + + std::lock_guard lock(storage_and_responses_lock); + session_id = storage->getSessionID(session_id_request.session_timeout_ms); + LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); + response->session_id = session_id; + try_push(response_for_session); + } + // Processing reconfig request as an ordinary one (in KeeperStorage) brings multiple inconsistencies + // regarding replays of old reconfigurations in new nodes. Thus the storage is not involved. + // See https://github.com/ClickHouse/ClickHouse/pull/49450 for details + else if (op_num == Coordination::OpNum::Reconfig) + { + std::lock_guard lock(storage_and_responses_lock); + try_push(processReconfiguration(*request_for_session)); } else { - if (request_for_session->request->getOpNum() == Coordination::OpNum::Close) + if (op_num == Coordination::OpNum::Close) { std::lock_guard lock(request_cache_mutex); parsed_request_cache.erase(request_for_session->session_id); @@ -333,14 +415,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); for (auto & response_for_session : responses_for_sessions) - if (!responses_queue.push(response_for_session)) - { - ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); - LOG_WARNING( - log, - "Failed to push response with session id {} to the queue, probably because of shutdown", - response_for_session.session_id); - } + try_push(response_for_session); if (keeper_context->digestEnabled() && request_for_session->digest) assertDigest(*request_for_session->digest, storage->getNodesDigest(true), *request_for_session->request, true); @@ -782,5 +857,4 @@ void KeeperStateMachine::recalculateStorageStats() storage->recalculateStats(); LOG_INFO(log, "Done recalculating storage stats"); } - } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index b47a9b5cc42..3b239adae45 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -12,6 +12,7 @@ namespace DB { +class KeeperDispatcher; using ResponsesQueue = ConcurrentBoundedQueue; using SnapshotsQueue = ConcurrentBoundedQueue; @@ -67,7 +68,9 @@ public: // (can happen in case of exception during preprocessing) void rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); - void rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); + void rollbackRequestNoLock( + const KeeperStorage::RequestForSession & request_for_session, + bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS; uint64_t last_commit_index() override { return last_committed_idx; } @@ -87,8 +90,10 @@ public: int read_logical_snp_obj( nuraft::snapshot & s, void *& user_snp_ctx, uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) override; - /// just for test - KeeperStorage & getStorage() { return *storage; } + KeeperStorage & getStorageForUnitTests() TSA_NO_THREAD_SAFETY_ANALYSIS + { + return *storage; + } void shutdownStorage(); @@ -122,6 +127,7 @@ public: uint64_t getLatestSnapshotBufSize() const; void recalculateStorageStats(); + private: CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored @@ -133,7 +139,7 @@ private: CoordinationSettingsPtr coordination_settings; /// Main state machine logic - KeeperStoragePtr storage; + KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock); /// Save/Load and Serialize/Deserialize logic for snapshots. KeeperSnapshotManager snapshot_manager; @@ -178,6 +184,8 @@ private: KeeperContextPtr keeper_context; KeeperSnapshotManagerS3 * snapshot_manager_s3; + + KeeperStorage::ResponseForSession processReconfiguration(const KeeperStorage::RequestForSession& request_for_session); }; } diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 450fd04b61d..cf1bad8c5fa 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -451,7 +451,7 @@ nuraft::ptr KeeperStateManager::read_state() return nullptr; } -ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const +ClusterUpdateActions KeeperStateManager::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const { auto new_configuration_wrapper = parseServersConfiguration(config, true); @@ -465,14 +465,14 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A old_ids[old_server->get_id()] = old_server; } - ConfigUpdateActions result; + ClusterUpdateActions result; /// First of all add new servers for (const auto & [new_id, server_config] : new_ids) { auto old_server_it = old_ids.find(new_id); if (old_server_it == old_ids.end()) - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::AddServer, server_config}); + result.emplace_back(AddRaftServer{RaftServerConfig{*server_config}}); else { const auto & old_endpoint = old_server_it->second->get_endpoint(); @@ -491,10 +491,8 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A /// After that remove old ones for (auto [old_id, server_config] : old_ids) - { if (!new_ids.contains(old_id)) - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::RemoveServer, server_config}); - } + result.emplace_back(RemoveRaftServer{old_id}); { std::lock_guard lock(configuration_wrapper_mutex); @@ -507,7 +505,10 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A { if (old_server->get_priority() != new_server->get_priority()) { - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::UpdatePriority, new_server}); + result.emplace_back(UpdateRaftServerPriority{ + .id = new_server->get_id(), + .priority = new_server->get_priority() + }); } break; } diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index f24f0c2b1e5..5abeea604b5 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -7,31 +7,13 @@ #include #include #include "Coordination/KeeperStateMachine.h" +#include "Coordination/RaftServerConfig.h" #include namespace DB { - using KeeperServerConfigPtr = nuraft::ptr; -/// When our configuration changes the following action types -/// can happen -enum class ConfigUpdateActionType -{ - RemoveServer, - AddServer, - UpdatePriority, -}; - -/// Action to update configuration -struct ConfigUpdateAction -{ - ConfigUpdateActionType action_type; - KeeperServerConfigPtr server; -}; - -using ConfigUpdateActions = std::vector; - /// Responsible for managing our and cluster configuration class KeeperStateManager : public nuraft::state_mgr { @@ -74,7 +56,11 @@ public: int32_t server_id() override { return my_server_id; } - nuraft::ptr get_srv_config() const { return configuration_wrapper.config; } /// NOLINT + nuraft::ptr get_srv_config() const + { + std::lock_guard lk(configuration_wrapper_mutex); + return configuration_wrapper.config; + } void system_exit(const int exit_code) override; /// NOLINT @@ -106,8 +92,8 @@ public: /// Read all log entries in log store from the begging and return latest config (with largest log_index) ClusterConfigPtr getLatestConfigFromLogStore() const; - /// Get configuration diff between proposed XML and current state in RAFT - ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; + // TODO (myrrc) This should be removed once "reconfig" is stabilized + ClusterUpdateActions getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; private: const String & getOldServerStatePath(); @@ -133,7 +119,7 @@ private: std::string config_prefix; mutable std::mutex configuration_wrapper_mutex; - KeeperConfigurationWrapper configuration_wrapper; + KeeperConfigurationWrapper configuration_wrapper TSA_GUARDED_BY(configuration_wrapper_mutex); nuraft::ptr log_store; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 884aacc4558..2b245a455b7 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -20,10 +20,10 @@ #include #include +#include #include +#include -#include -#include #include #include #include @@ -53,7 +53,6 @@ namespace ErrorCodes namespace { - String getSHA1(const String & userdata) { Poco::SHA1Engine engine; @@ -1060,7 +1059,8 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce ProfileEvents::increment(ProfileEvents::KeeperGetRequest); Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); - if (request.path == Coordination::keeper_api_feature_flags_path) + if (request.path == Coordination::keeper_api_feature_flags_path + || request.path == Coordination::keeper_config_path) return {}; if (!storage.uncommitted_state.getNode(request.path)) @@ -1085,6 +1085,14 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } } + if (request.path == Coordination::keeper_config_path) + { + response.data = serializeClusterConfig( + storage.keeper_context->dispatcher->getStateMachine().getClusterConfig()); + response.error = Coordination::Error::ZOK; + return response_ptr; + } + auto & container = storage.container; auto node_it = container.find(request.path); if (node_it == container.end()) @@ -1784,7 +1792,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro throw DB::Exception( ErrorCodes::BAD_ARGUMENTS, "Illegal command as part of multi ZooKeeper request {}", - Coordination::toString(sub_zk_request->getOpNum())); + sub_zk_request->getOpNum()); } } @@ -1975,7 +1983,7 @@ public: { auto request_it = op_num_to_request.find(zk_request->getOpNum()); if (request_it == op_num_to_request.end()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown operation type {}", toString(zk_request->getOpNum())); + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown operation type {}", zk_request->getOpNum()); return request_it->second(zk_request); } diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp new file mode 100644 index 00000000000..42923dd0b29 --- /dev/null +++ b/src/Coordination/RaftServerConfig.cpp @@ -0,0 +1,96 @@ +#include "RaftServerConfig.h" +#include +#include +#include +#include + +namespace DB +{ +RaftServerConfig::RaftServerConfig(const nuraft::srv_config & cfg) noexcept + : id(cfg.get_id()), endpoint(cfg.get_endpoint()), learner(cfg.is_learner()), priority(cfg.get_priority()) +{ +} + +RaftServerConfig::operator nuraft::srv_config() const noexcept +{ + return {id, 0, endpoint, "", learner, priority}; +} + +std::optional RaftServerConfig::parse(std::string_view server) noexcept +{ + std::vector parts; + splitInto<';', '='>(parts, server); + + const bool with_id_endpoint = parts.size() == 2; + const bool with_server_type = parts.size() == 3; + const bool with_priority = parts.size() == 4; + if (!with_id_endpoint && !with_server_type && !with_priority) + return std::nullopt; + + const std::string_view id_str = parts[0]; + if (!id_str.starts_with("server.")) + return std::nullopt; + + int id; + if (std::from_chars(std::next(id_str.begin(), 7), id_str.end(), id).ec != std::error_code{}) + return std::nullopt; + if (id <= 0) + return std::nullopt; + + const std::string_view endpoint = parts[1]; + const size_t port_delimiter = endpoint.find_last_of(':'); + if (port_delimiter == std::string::npos) + return {}; + const std::string_view port = endpoint.substr(port_delimiter + 1); + + uint16_t port_tmp; + if (std::from_chars(port.begin(), port.end(), port_tmp).ec != std::error_code{}) + return std::nullopt; + + RaftServerConfig out{id, endpoint}; + + if (with_id_endpoint) + return out; + + if (parts[2] != "learner" && parts[2] != "participant") + return std::nullopt; + out.learner = parts[2] == "learner"; + if (with_server_type) + return out; + + const std::string_view priority = parts[3]; + if (std::from_chars(priority.begin(), priority.end(), out.priority).ec != std::error_code{}) + return std::nullopt; + if (out.priority < 0) + return std::nullopt; + + return out; +} + +RaftServers parseRaftServers(std::string_view servers) +{ + std::vector server_arr; + std::unordered_set ids; + std::unordered_set endpoints; + RaftServers out; + + for (auto & server : splitInto<','>(server_arr, servers)) + if (auto maybe_server = RaftServerConfig::parse(server)) + { + String endpoint = maybe_server->endpoint; + if (endpoints.contains(endpoint)) + return {}; + const int id = maybe_server->id; + if (ids.contains(id)) + return {}; + + out.emplace_back(std::move(*maybe_server)); + endpoints.emplace(std::move(endpoint)); + ids.emplace(id); + } + else + return {}; + + return out; +} +} diff --git a/src/Coordination/RaftServerConfig.h b/src/Coordination/RaftServerConfig.h new file mode 100644 index 00000000000..451d61a436e --- /dev/null +++ b/src/Coordination/RaftServerConfig.h @@ -0,0 +1,78 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ +// default- and copy-constructible version of nuraft::srv_config +struct RaftServerConfig +{ + int id; + String endpoint; + bool learner; + int priority; + + constexpr RaftServerConfig() = default; + constexpr RaftServerConfig(int id_, std::string_view endpoint_, bool learner_ = false, int priority_ = 1) + : id(id_), endpoint(endpoint_), learner(learner_), priority(priority_) + { + } + + constexpr bool operator==(const RaftServerConfig &) const = default; + explicit RaftServerConfig(const nuraft::srv_config & cfg) noexcept; + explicit operator nuraft::srv_config() const noexcept; + + /// Parse server in format "server.id=host:port[;learner][;priority]" + static std::optional parse(std::string_view server) noexcept; +}; + +using RaftServers = std::vector; +/// Parse comma-delimited servers. Check for duplicate endpoints and ids. +/// @returns {} on parsing or validation error. +RaftServers parseRaftServers(std::string_view servers); + +struct AddRaftServer : RaftServerConfig +{ +}; + +struct RemoveRaftServer +{ + int id; +}; + +struct UpdateRaftServerPriority +{ + int id; + int priority; +}; + +using ClusterUpdateAction = std::variant; +using ClusterUpdateActions = std::vector; +} + +template <> +struct fmt::formatter : fmt::formatter +{ + constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx) + { + return fmt::format_to( + ctx.out(), "server.{}={};{};{}", server.id, server.endpoint, server.learner ? "learner" : "participant", server.priority); + } +}; + +template <> +struct fmt::formatter : fmt::formatter +{ + constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx) + { + if (const auto * add = std::get_if(&action)) + return fmt::format_to(ctx.out(), "(Add server {})", add->id); + if (const auto * remove = std::get_if(&action)) + return fmt::format_to(ctx.out(), "(Remove server {})", remove->id); + if (const auto * update = std::get_if(&action)) + return fmt::format_to(ctx.out(), "(Change server {} priority to {})", update->id, update->priority); + UNREACHABLE(); + } +}; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 0f60c960b8b..b302f9b13ca 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -84,6 +84,47 @@ protected: } }; +TEST_P(CoordinationTest, RaftServerConfigParse) +{ + auto parse = Coordination::RaftServerConfig::parse; + using Cfg = std::optional; + + EXPECT_EQ(parse(""), std::nullopt); + EXPECT_EQ(parse("="), std::nullopt); + EXPECT_EQ(parse("=;"), std::nullopt); + EXPECT_EQ(parse("=;;"), std::nullopt); + EXPECT_EQ(parse("=:80"), std::nullopt); + EXPECT_EQ(parse("server."), std::nullopt); + EXPECT_EQ(parse("server.=:80"), std::nullopt); + EXPECT_EQ(parse("server.-5=1:2"), std::nullopt); + EXPECT_EQ(parse("server.1=host;-123"), std::nullopt); + EXPECT_EQ(parse("server.1=host:999"), (Cfg{{1, "host:999"}})); + EXPECT_EQ(parse("server.1=host:999;learner"), (Cfg{{1, "host:999", true}})); + EXPECT_EQ(parse("server.1=host:999;participant"), (Cfg{{1, "host:999", false}})); + EXPECT_EQ(parse("server.1=host:999;learner;25"), (Cfg{{1, "host:999", true, 25}})); + + EXPECT_EQ(parse("server.1=127.0.0.1:80"), (Cfg{{1, "127.0.0.1:80"}})); + EXPECT_EQ( + parse("server.1=2001:0db8:85a3:0000:0000:8a2e:0370:7334:80"), + (Cfg{{1, "2001:0db8:85a3:0000:0000:8a2e:0370:7334:80"}})); +} + +TEST_P(CoordinationTest, RaftServerClusterConfigParse) +{ + auto parse = Coordination::parseRaftServers; + using Cfg = DB::RaftServerConfig; + using Servers = DB::RaftServers; + + EXPECT_EQ(parse(""), Servers{}); + EXPECT_EQ(parse(","), Servers{}); + EXPECT_EQ(parse("1,2"), Servers{}); + EXPECT_EQ(parse("server.1=host:80,server.1=host2:80"), Servers{}); + EXPECT_EQ(parse("server.1=host:80,server.2=host:80"), Servers{}); + EXPECT_EQ( + parse("server.1=host:80,server.2=host:81"), + (Servers{Cfg{1, "host:80"}, Cfg{2, "host:81"}})); +} + TEST_P(CoordinationTest, BuildTest) { DB::InMemoryLogStore store; @@ -1575,8 +1616,8 @@ void testLogAndStateMachine( restore_machine->commit(i, changelog.entry_at(i)->get_buf()); } - auto & source_storage = state_machine->getStorage(); - auto & restored_storage = restore_machine->getStorage(); + auto & source_storage = state_machine->getStorageForUnitTests(); + auto & restored_storage = restore_machine->getStorageForUnitTests(); EXPECT_EQ(source_storage.container.size(), restored_storage.container.size()); for (size_t i = 1; i < total_logs + 1; ++i) @@ -1678,7 +1719,7 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) auto entry_c = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), request_c); state_machine->pre_commit(1, entry_c->get_buf()); state_machine->commit(1, entry_c->get_buf()); - const auto & storage = state_machine->getStorage(); + const auto & storage = state_machine->getStorageForUnitTests(); EXPECT_EQ(storage.ephemerals.size(), 1); std::shared_ptr request_d = std::make_shared(); @@ -1727,7 +1768,7 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); state_machine->pre_commit(2, create_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; ASSERT_TRUE(uncommitted_state.nodes.contains(node_path)); // commit log entries @@ -1790,7 +1831,7 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) state_machine->commit(2, create_entry->get_buf()); state_machine->commit(3, set_acl_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorage().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; auto node = uncommitted_state.getNode(node_path); ASSERT_NE(node, nullptr); diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index 48f4d510af7..2231a58c6a9 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -73,6 +73,7 @@ NamesAndTypesList ZooKeeperLogElement::getNamesAndTypes() {"Create", static_cast(Coordination::OpNum::Create)}, {"Remove", static_cast(Coordination::OpNum::Remove)}, {"Exists", static_cast(Coordination::OpNum::Exists)}, + {"Reconfig", static_cast(Coordination::OpNum::Reconfig)}, {"Get", static_cast(Coordination::OpNum::Get)}, {"Set", static_cast(Coordination::OpNum::Set)}, {"GetACL", static_cast(Coordination::OpNum::GetACL)}, diff --git a/src/Storages/DataLakes/HudiMetadataParser.cpp b/src/Storages/DataLakes/HudiMetadataParser.cpp index a1f35a5ae42..78d69c83989 100644 --- a/src/Storages/DataLakes/HudiMetadataParser.cpp +++ b/src/Storages/DataLakes/HudiMetadataParser.cpp @@ -67,7 +67,8 @@ struct HudiMetadataParser::Impl { auto key_file = std::filesystem::path(key); Strings file_parts; - splitInto<'_'>(file_parts, key_file.stem()); + const String stem = key_file.stem(); + splitInto<'_'>(file_parts, stem); if (file_parts.size() != 3) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format for file: {}", key); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index bcc4dc749fb..07cfced8362 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -492,7 +492,7 @@ size_t ReplicatedMergeTreeCleanupThread::clearOldBlocks(const String & blocks_di } else { - LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", path, Coordination::errorMessage(rc)); + LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", path, rc); } first_outdated_block++; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 792843cbe18..b08b9de12a3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -494,7 +494,7 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper( if (code != Coordination::Error::ZOK) LOG_ERROR(log, "Couldn't set value of nodes for insert times " "({}/min_unprocessed_insert_time, max_processed_insert_time): {}. " - "This shouldn't happen often.", replica_path, Coordination::errorMessage(code)); + "This shouldn't happen often.", replica_path, code); } } @@ -551,7 +551,7 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / entry->znode_name); if (code != Coordination::Error::ZOK) - LOG_ERROR(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, Coordination::errorMessage(code)); + LOG_ERROR(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, code); updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed); } @@ -1144,7 +1144,7 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / znode_name); if (code != Coordination::Error::ZOK) - LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / znode_name).string(), Coordination::errorMessage(code)); + LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / znode_name).string(), code); updateStateOnQueueEntryRemoval( *it, /* is_successful = */ false, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 4128654a632..1e033566fed 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -723,7 +723,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: retries_ctl.setUserError( ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, "Insert failed due to zookeeper error. Please retry. Reason: {}", - Coordination::errorMessage(write_part_info_keeper_error)); + write_part_info_keeper_error); } retries_ctl.stopRetries(); @@ -1033,7 +1033,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: retries_ctl.setUserError( ErrorCodes::UNKNOWN_STATUS_OF_INSERT, "Unknown status, client must retry. Reason: {}", - Coordination::errorMessage(multi_code)); + multi_code); return; } else if (Coordination::isUserError(multi_code)) @@ -1109,7 +1109,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: "Unexpected logical error while adding block {} with ID '{}': {}, path {}", block_number, toString(block_id), - Coordination::errorMessage(multi_code), + multi_code, failed_op_path); } } @@ -1122,7 +1122,7 @@ std::pair, bool> ReplicatedMergeTreeSinkImpl:: "Unexpected ZooKeeper error while adding block {} with ID '{}': {}", block_number, toString(block_id), - Coordination::errorMessage(multi_code)); + multi_code); } }, [&zookeeper]() { zookeeper->cleanupEphemeralNodes(); }); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index dac9e6923a5..06e9d88a954 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1037,7 +1037,7 @@ void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, con code = zookeeper->tryMulti(ops, res); if (code != Coordination::Error::ZOK) LOG_WARNING(logger, "Cannot quickly remove nodes without children: {} (replica: {}). Will remove recursively.", - Coordination::errorMessage(code), remote_replica_path); + code, remote_replica_path); /// And finally remove everything else recursively /// It may left some garbage if replica_path subtree is concurrently modified @@ -1145,7 +1145,7 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper auto code = zookeeper->tryMulti(ops, res); if (code != Coordination::Error::ZOK) LOG_WARNING(logger, "Cannot quickly remove nodes without children: {} (table: {}). Will remove recursively.", - Coordination::errorMessage(code), zookeeper_path); + code, zookeeper_path); Strings children; code = zookeeper->tryGetChildren(zookeeper_path, children); @@ -1893,7 +1893,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che else if (code == Coordination::Error::ZBADVERSION || code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part {} as failed. Code: {}", - entry.new_part_name, Coordination::errorMessage(code)); + entry.new_part_name, code); } else throw Coordination::Exception(code); @@ -3098,7 +3098,7 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke if (get_is_lost.error != Coordination::Error::ZOK) { - LOG_INFO(log, "Not cloning {}, cannot get '/is_lost': {}", source_replica_name, Coordination::errorMessage(get_is_lost.error)); + LOG_INFO(log, "Not cloning {}, cannot get '/is_lost': {}", source_replica_name, get_is_lost.error); continue; } else if (get_is_lost.data != "0") @@ -3109,12 +3109,12 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke if (get_log_pointer.error != Coordination::Error::ZOK) { - LOG_INFO(log, "Not cloning {}, cannot get '/log_pointer': {}", source_replica_name, Coordination::errorMessage(get_log_pointer.error)); + LOG_INFO(log, "Not cloning {}, cannot get '/log_pointer': {}", source_replica_name, get_log_pointer.error); continue; } if (get_queue.error != Coordination::Error::ZOK) { - LOG_INFO(log, "Not cloning {}, cannot get '/queue': {}", source_replica_name, Coordination::errorMessage(get_queue.error)); + LOG_INFO(log, "Not cloning {}, cannot get '/queue': {}", source_replica_name, get_queue.error); continue; } @@ -7203,7 +7203,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( { for (size_t i = 0; i < delete_requests.size(); ++i) if (delete_responses[i]->error != Coordination::Error::ZOK) - LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", delete_requests[i]->getPath(), Coordination::errorMessage(delete_responses[i]->error)); + LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", delete_requests[i]->getPath(), delete_responses[i]->error); } LOG_TRACE(log, "Deleted {} deduplication block IDs in partition ID {}", delete_requests.size(), partition_id); @@ -8717,7 +8717,7 @@ std::pair> getParentLockedBlobs(const ZooKeeperWith zookeeper_ptr->tryGet(fs::path(zero_copy_part_path_prefix) / part_candidate_info_str, files_not_to_remove_str, nullptr, nullptr, &code); if (code != Coordination::Error::ZOK) { - LOG_TRACE(log, "Cannot get parent files from ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), errorMessage(code)); + LOG_TRACE(log, "Cannot get parent files from ZooKeeper on path ({}), error {}", (fs::path(zero_copy_part_path_prefix) / part_candidate_info_str).string(), code); return {true, std::nullopt}; } diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3b909194b63..3da1d5bd7b0 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -1,5 +1,6 @@ import socket import time +from kazoo.client import KazooClient def get_keeper_socket(cluster, node, port=9181): @@ -26,9 +27,17 @@ def send_4lw_cmd(cluster, node, cmd="ruok", port=9181): NOT_SERVING_REQUESTS_ERROR_MSG = "This instance is not currently serving requests" -def wait_until_connected(cluster, node, port=9181): +def wait_until_connected(cluster, node, port=9181, timeout=30.0): + elapsed = 0.0 + while send_4lw_cmd(cluster, node, "mntr", port) == NOT_SERVING_REQUESTS_ERROR_MSG: time.sleep(0.1) + elapsed += 0.1 + + if elapsed >= timeout: + raise Exception( + f"{timeout}s timeout while waiting for {node.name} to start serving requests" + ) def wait_until_quorum_lost(cluster, node, port=9181): @@ -51,3 +60,25 @@ def get_leader(cluster, nodes): if is_leader(cluster, node): return node raise Exception("No leader in Keeper cluster.") + + +def get_fake_zk(cluster, node, timeout: float = 30.0) -> KazooClient: + _fake = KazooClient( + hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout + ) + _fake.start() + return _fake + + +def get_config_str(zk: KazooClient) -> str: + """ + Return decoded contents of /keeper/config node + """ + return zk.get("/keeper/config")[0].decode("utf-8") + + +def configs_equal(left: str, right: str) -> bool: + """ + Check whether /keeper/config nodes are equal + """ + return sorted(left.split("\n")) == sorted(right.split("\n")) diff --git a/tests/integration/test_keeper_nodes_move/test.py b/tests/integration/test_keeper_nodes_move/test.py index 6884ff29607..8ac7bc9b5e2 100644 --- a/tests/integration/test_keeper_nodes_move/test.py +++ b/tests/integration/test_keeper_nodes_move/test.py @@ -1,12 +1,7 @@ #!/usr/bin/env python3 - -#!/usr/bin/env python3 - import pytest from helpers.cluster import ClickHouseCluster -import random -import string import os import time from multiprocessing.dummy import Pool diff --git a/tests/integration/test_keeper_reconfig_add/__init__.py b/tests/integration/test_keeper_reconfig_add/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_add/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_add/configs/keeper1.xml new file mode 100644 index 00000000000..44e2090e9d8 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/configs/keeper1.xml @@ -0,0 +1,20 @@ + + + true + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + 1 node1 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_add/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_add/configs/keeper2.xml new file mode 100644 index 00000000000..e9249f7091c --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/configs/keeper2.xml @@ -0,0 +1,21 @@ + + + true + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + 1 node1 9234 + 2 node2 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_add/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_add/configs/keeper3.xml new file mode 100644 index 00000000000..a7ff1f6de28 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/configs/keeper3.xml @@ -0,0 +1,22 @@ + + + true + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + 1 node1 9234 + 2 node2 9234 + 3 node3 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_add/test.py b/tests/integration/test_keeper_reconfig_add/test.py new file mode 100644 index 00000000000..c80279a0727 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_add/test.py @@ -0,0 +1,155 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as ku +import os +from kazoo.client import KazooClient +from kazoo.exceptions import BadArgumentsException + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") + +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", stay_alive=True) +node3 = cluster.add_instance("node3", stay_alive=True) + +server_join_msg = "confirms it will join" +part_of_cluster = "now this node is the part of cluster" +zk1, zk2, zk3 = None, None, None + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node2.stop_clickhouse() + node2.copy_file_to_container( + os.path.join(CONFIG_DIR, "keeper2.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + node3.stop_clickhouse() + node3.copy_file_to_container( + os.path.join(CONFIG_DIR, "keeper3.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + yield cluster + + finally: + for conn in [zk1, zk2, zk3]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def test_reconfig_add(started_cluster): + """ + Add a node to another node. Then add another node to two. + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + print("Initial config", config) + + assert len(config.split("\n")) == 1 + assert "node1" in config + assert "node2" not in config + assert "node3" not in config + + with pytest.raises(BadArgumentsException): + # duplicate id with different endpoint + zk1.reconfig(joining="server.1=localhost:1337", leaving=None, new_members=None) + + with pytest.raises(BadArgumentsException): + # duplicate endpoint + zk1.reconfig(joining="server.8=node1:9234", leaving=None, new_members=None) + + for i in range(100): + zk1.create(f"/test_three_{i}", b"somedata") + + node2.start_clickhouse() + config, _ = zk1.reconfig( + joining="server.2=node2:9234", leaving=None, new_members=None + ) + ku.wait_until_connected(cluster, node2) + + config = config.decode("utf-8") + print("After adding 2", config) + + assert len(config.split("\n")) == 2 + assert "node1" in config + assert "node2" in config + assert "node3" not in config + + zk2 = get_fake_zk(node2) + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + for i in range(100): + assert zk2.exists(f"/test_three_{i}") is not None + zk2.create(f"/test_three_{100 + i}", b"somedata") + + # Why not both? + # One node will process add_srv request, other will pull out updated config, apply + # and return true in config update thread (without calling add_srv again) + assert node1.contains_in_log(server_join_msg) or node2.contains_in_log( + server_join_msg + ) + + assert node2.contains_in_log(part_of_cluster) + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_three_0") + + for i in range(200): + assert zk1.exists(f"/test_three_{i}") is not None + + for i in range(100): + zk2.create(f"/test_four_{i}", b"somedata") + + node3.start_clickhouse() + config, _ = zk2.reconfig( + joining="server.3=node3:9234", leaving=None, new_members=None + ) + ku.wait_until_connected(cluster, node3) + + config = config.decode("utf-8") + print("After adding 3", config) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + + zk3 = get_fake_zk(node3) + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk3.exists(f"/test_four_{i}") is not None + zk3.create(f"/test_four_{100 + i}", b"somedata") + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_four_0") + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + + for i in range(200): + assert zk1.exists(f"/test_four_{i}") is not None + assert zk2.exists(f"/test_four_{i}") is not None + + assert node3.contains_in_log(part_of_cluster) diff --git a/tests/integration/test_keeper_reconfig_remove/__init__.py b/tests/integration/test_keeper_reconfig_remove/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml new file mode 100644 index 00000000000..bbadc2741af --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/configs/keeper1.xml @@ -0,0 +1,37 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + 0 + + + 3 + node3 + 9234 + true + 0 + + + + diff --git a/tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml new file mode 100644 index 00000000000..0191a522a50 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/configs/keeper2.xml @@ -0,0 +1,37 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + 0 + + + 3 + node3 + 9234 + true + 0 + + + + diff --git a/tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml new file mode 100644 index 00000000000..345bf402336 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/configs/keeper3.xml @@ -0,0 +1,37 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + 0 + + + 3 + node3 + 9234 + true + 0 + + + + diff --git a/tests/integration/test_keeper_reconfig_remove/test.py b/tests/integration/test_keeper_reconfig_remove/test.py new file mode 100644 index 00000000000..7f0b1ee92c6 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove/test.py @@ -0,0 +1,145 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as ku +import os +from kazoo.client import KazooClient +from kazoo.exceptions import BadVersionException, BadArgumentsException + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", main_configs=["configs/keeper2.xml"]) +node3 = cluster.add_instance("node3", main_configs=["configs/keeper3.xml"]) + +log_msg_removed = "has been removed from the cluster" +zk1, zk2, zk3 = None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + for conn in [zk1, zk2, zk3]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_remove_followers_from_3(started_cluster): + """ + Remove 1 follower node from cluster of 3. + Then remove another follower from two left nodes. + Check that remaining node is in standalone mode. + """ + + zk1 = get_fake_zk(node1) + config, _ = zk1.get("/keeper/config") + config = config.decode("utf-8") + print("Initial config", config) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + + with pytest.raises(BadVersionException): + zk1.reconfig(joining=None, leaving="1", new_members=None, from_config=20) + with pytest.raises(BadArgumentsException): + zk1.reconfig(joining=None, leaving=None, new_members=None) + with pytest.raises(BadArgumentsException): + # bulk reconfiguration is not supported + zk1.reconfig(joining=None, leaving=None, new_members="3") + with pytest.raises(BadArgumentsException): + zk1.reconfig(joining="1", leaving="1", new_members="3") + with pytest.raises(BadArgumentsException): + # at least one node must be left + zk1.reconfig(joining=None, leaving="1,2,3", new_members=None) + + for i in range(100): + zk1.create(f"/test_two_{i}", b"somedata") + + zk2 = get_fake_zk(node2) + zk2.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"test_two_{i}") is not None + assert zk3.exists(f"test_two_{i}") is not None + + config, _ = zk1.reconfig(joining=None, leaving="3", new_members=None) + config = config.decode("utf-8") + print("After removing 3", config) + + assert len(config.split("\n")) == 2 + assert "node1" in config + assert "node2" in config + assert "node3" not in config + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + for i in range(100): + assert zk2.exists(f"test_two_{i}") is not None + zk2.create(f"/test_two_{100 + i}", b"otherdata") + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_two_0") + + for i in range(200): + assert zk1.exists(f"test_two_{i}") is not None + + with pytest.raises(Exception): + zk3.stop() + zk3.close() + zk3 = get_fake_zk(node3) + zk3.sync("/test_two_0") + + assert node3.contains_in_log(log_msg_removed) + + for i in range(100): + zk2.create(f"/test_two_{200 + i}", b"otherdata") + + config, _ = zk1.reconfig(joining=None, leaving="2", new_members=None) + config = config.decode("utf-8") + + print("After removing 2", config) + assert len(config.split("\n")) == 1 + assert "node1" in config + assert "node2" not in config + assert "node3" not in config + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_two_0") + + for i in range(300): + assert zk1.exists(f"test_two_{i}") is not None + + with pytest.raises(Exception): + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_two_0") + + assert not node1.contains_in_log(log_msg_removed) + assert node2.contains_in_log(log_msg_removed) + assert "Mode: standalone" in zk1.command(b"stat") diff --git a/tests/integration/test_keeper_reconfig_remove_many/__init__.py b/tests/integration/test_keeper_reconfig_remove_many/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml new file mode 100644 index 00000000000..9976169624b --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper1.xml @@ -0,0 +1,47 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml new file mode 100644 index 00000000000..edc43142464 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper2.xml @@ -0,0 +1,47 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml new file mode 100644 index 00000000000..8cebcbc0808 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper3.xml @@ -0,0 +1,47 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml new file mode 100644 index 00000000000..99ac7e53f30 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper4.xml @@ -0,0 +1,47 @@ + + + 9181 + 4 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml new file mode 100644 index 00000000000..92102ad486b --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/configs/keeper5.xml @@ -0,0 +1,47 @@ + + + 9181 + 5 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + 4 + node4 + 9234 + true + + + 5 + node5 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_remove_many/test.py b/tests/integration/test_keeper_reconfig_remove_many/test.py new file mode 100644 index 00000000000..6bf477ff9c9 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_remove_many/test.py @@ -0,0 +1,149 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import helpers.keeper_utils as ku +import os +from kazoo.client import KazooClient, KazooState +from kazoo.exceptions import BadVersionException, BadArgumentsException + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") + +nodes = [ + cluster.add_instance(f"node{i}", main_configs=[f"configs/keeper{i}.xml"]) + for i in range(1, 6) +] +node1, node2, node3, node4, node5 = nodes + +log_msg_removed = "has been removed from the cluster" +zk1, zk2, zk3, zk4, zk5 = None, None, None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + for conn in [zk1, zk2, zk3, zk4, zk5]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_remove_2_and_leader(started_cluster): + """ + Remove 2 followers from a cluster of 5. Remove leader from 3 nodes. + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + print("Initial config", config) + + assert len(config.split("\n")) == 5 + + for i in range(100): + zk1.create(f"/test_two_{i}", b"somedata") + + zk4 = get_fake_zk(node4) + zk4.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk4)) + + zk5 = get_fake_zk(node5) + zk5.sync("/test_two_0") + assert ku.configs_equal(config, ku.get_config_str(zk5)) + + for i in range(100): + assert zk4.exists(f"test_two_{i}") is not None + assert zk5.exists(f"test_two_{i}") is not None + + zk4.create(f"/test_two_{100 + i}", b"otherdata") + + zk2 = get_fake_zk(node2) + config, _ = zk2.reconfig(joining=None, leaving="4,5", new_members=None) + config = config.decode("utf-8") + + print("After removing 4,5", config) + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + assert "node5" not in config + + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_two_0") + + assert ku.configs_equal(config, ku.get_config_str(zk1)) + + for i in range(200): + assert zk1.exists(f"test_two_{i}") is not None + assert zk2.exists(f"test_two_{i}") is not None + + with pytest.raises(Exception): + zk4.stop() + zk4.close() + zk4 = get_fake_zk(node4) + zk4.sync("/test_two_0") + + with pytest.raises(Exception): + zk5.stop() + zk5.close() + zk5 = get_fake_zk(node5) + zk5.sync("/test_two_0") + + assert not node1.contains_in_log(log_msg_removed) + assert not node2.contains_in_log(log_msg_removed) + assert not node3.contains_in_log(log_msg_removed) + assert node4.contains_in_log(log_msg_removed) + assert node5.contains_in_log(log_msg_removed) + + assert ku.is_leader(cluster, node1) + + for i in range(100): + zk1.create(f"/test_leader_{i}", b"somedata") + + # when a leader gets a remove request, it must yield leadership + config, _ = zk1.reconfig(joining=None, leaving="1", new_members=None) + config = config.decode("utf-8") + print("After removing 1 (leader)", config) + + assert len(config.split("\n")) == 2 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + assert "node5" not in config + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_leader_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_leader_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"test_leader_{i}") is not None + assert zk3.exists(f"test_leader_{i}") is not None + + with pytest.raises(Exception): + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_leader_0") + + assert node1.contains_in_log(log_msg_removed) + assert not node2.contains_in_log(log_msg_removed) + assert not node3.contains_in_log(log_msg_removed) diff --git a/tests/integration/test_keeper_reconfig_replace_leader/__init__.py b/tests/integration/test_keeper_reconfig_replace_leader/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml new file mode 100644 index 00000000000..71f3403aca3 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper1.xml @@ -0,0 +1,35 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml new file mode 100644 index 00000000000..faefb4d1102 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper2.xml @@ -0,0 +1,35 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml new file mode 100644 index 00000000000..80a9caa92c2 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper3.xml @@ -0,0 +1,35 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml new file mode 100644 index 00000000000..9fd88fe5d63 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/configs/keeper4.xml @@ -0,0 +1,21 @@ + + + 9181 + 4 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + 2 node2 9234 + 3 node3 9234 + 4 node4 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader/test.py b/tests/integration/test_keeper_reconfig_replace_leader/test.py new file mode 100644 index 00000000000..1b23aa056c6 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader/test.py @@ -0,0 +1,127 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +from os.path import join, dirname, realpath +import time +import helpers.keeper_utils as ku +from kazoo.client import KazooClient, KazooState + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = join(dirname(realpath(__file__)), "configs") + +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", main_configs=["configs/keeper2.xml"]) +node3 = cluster.add_instance("node3", main_configs=["configs/keeper3.xml"]) +node4 = cluster.add_instance("node4", stay_alive=True) +zk1, zk2, zk3, zk4 = None, None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node4.stop_clickhouse() + node4.copy_file_to_container( + join(CONFIG_DIR, "keeper4.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + yield cluster + + finally: + for conn in [zk1, zk2, zk3, zk4]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_replace_leader(started_cluster): + """ + Remove leader from a cluster of 3 and add a new node via two commands. + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + + for i in range(100): + zk1.create(f"/test_four_{i}", b"somedata") + + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"/test_four_{i}") is not None + assert zk3.exists(f"/test_four_{i}") is not None + + assert ku.is_leader(cluster, node1) + config, _ = zk2.reconfig(joining=None, leaving="1", new_members=None) + config = config.decode("utf-8") + + print("After removing 1 (leader)", config) + assert len(config.split("\n")) == 2 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + + with pytest.raises(Exception): + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_four_0") + + node4.start_clickhouse() + config, _ = zk2.reconfig( + joining="server.4=node4:9234", leaving=None, new_members=None + ) + config = config.decode("utf-8") + ku.wait_until_connected(cluster, node4) + + print("After adding 4", config) + assert len(config.split("\n")) == 3 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" in config + + zk4 = get_fake_zk(node4) + assert ku.configs_equal(config, ku.get_config_str(zk4)) + + for i in range(100): + assert zk4.exists(f"test_four_{i}") is not None + zk4.create(f"/test_four_{100 + i}", b"somedata") + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3.stop() + zk3.close() + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(200): + assert zk2.exists(f"test_four_{i}") is not None + assert zk3.exists(f"test_four_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml new file mode 100644 index 00000000000..71f3403aca3 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper1.xml @@ -0,0 +1,35 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml new file mode 100644 index 00000000000..faefb4d1102 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper2.xml @@ -0,0 +1,35 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml new file mode 100644 index 00000000000..80a9caa92c2 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper3.xml @@ -0,0 +1,35 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml new file mode 100644 index 00000000000..9fd88fe5d63 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/configs/keeper4.xml @@ -0,0 +1,21 @@ + + + 9181 + 4 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + 2 node2 9234 + 3 node3 9234 + 4 node4 9234 + + + diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py new file mode 100644 index 00000000000..c7aed945097 --- /dev/null +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -0,0 +1,120 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +from os.path import join, dirname, realpath +import time +import helpers.keeper_utils as ku +from kazoo.client import KazooClient, KazooState + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = join(dirname(realpath(__file__)), "configs") + +node1 = cluster.add_instance("node1", main_configs=["configs/keeper1.xml"]) +node2 = cluster.add_instance("node2", main_configs=["configs/keeper2.xml"]) +node3 = cluster.add_instance("node3", main_configs=["configs/keeper3.xml"]) +node4 = cluster.add_instance("node4", stay_alive=True) +zk1, zk2, zk3, zk4 = None, None, None, None + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node4.stop_clickhouse() + node4.copy_file_to_container( + join(CONFIG_DIR, "keeper4.xml"), + "/etc/clickhouse-server/config.d/keeper.xml", + ) + + yield cluster + + finally: + for conn in [zk1, zk2, zk3, zk4]: + if conn: + conn.stop() + conn.close() + + cluster.shutdown() + + +def get_fake_zk(node): + return ku.get_fake_zk(cluster, node) + + +def test_reconfig_replace_leader_in_one_command(started_cluster): + """ + Remove leader from a cluster of 3 and add a new node to this cluster in a single command + """ + + zk1 = get_fake_zk(node1) + config = ku.get_config_str(zk1) + + assert len(config.split("\n")) == 3 + assert "node1" in config + assert "node2" in config + assert "node3" in config + assert "node4" not in config + + for i in range(100): + zk1.create(f"/test_four_{i}", b"somedata") + + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(100): + assert zk2.exists(f"/test_four_{i}") is not None + assert zk3.exists(f"/test_four_{i}") is not None + + assert ku.is_leader(cluster, node1) + node4.start_clickhouse() + config, _ = zk2.reconfig( + joining="server.4=node4:9234", leaving="1", new_members=None + ) + config = config.decode("utf-8") + + print("After removing 1 and adding 4", config) + assert len(config.split("\n")) == 3 + assert "node1" not in config + assert "node2" in config + assert "node3" in config + assert "node4" in config + + ku.wait_until_connected(cluster, node4) + time.sleep(1) + + zk4 = get_fake_zk(node4) + zk4.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk4)) + + for i in range(100): + assert zk4.exists(f"test_four_{i}") is not None + zk4.create(f"/test_four_{100 + i}", b"somedata") + + with pytest.raises(Exception): + zk1.stop() + zk1.close() + zk1 = get_fake_zk(node1) + zk1.sync("/test_four_0") + + zk2.stop() + zk2.close() + zk2 = get_fake_zk(node2) + zk2.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk2)) + + zk3.stop() + zk3.close() + zk3 = get_fake_zk(node3) + zk3.sync("/test_four_0") + assert ku.configs_equal(config, ku.get_config_str(zk3)) + + for i in range(200): + assert zk2.exists(f"test_four_{i}") is not None + assert zk3.exists(f"test_four_{i}") is not None diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 5a6fd15d72c..8d685d65d1d 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -15,7 +15,7 @@ using namespace DB; void dumpMachine(std::shared_ptr machine) { - auto & storage = machine->getStorage(); + auto & storage = machine->getStorageForUnitTests(); std::queue keys; keys.push("/"); From b4f750ed66a7579580b88deaaedf15ac153785d0 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 10:11:44 +0000 Subject: [PATCH 0425/2047] review fixes --- base/base/find_symbols.h | 2 +- src/Common/ZooKeeper/IKeeper.h | 2 +- src/Coordination/KeeperContext.cpp | 3 ++- src/Coordination/KeeperContext.h | 5 +++-- src/Coordination/KeeperDispatcher.cpp | 5 ++--- src/Coordination/KeeperReconfiguration.cpp | 5 ++--- src/Coordination/KeeperReconfiguration.h | 4 ++-- src/Coordination/KeeperServer.cpp | 14 +++++++------- src/Coordination/KeeperServer.h | 2 -- src/Coordination/KeeperStateMachine.cpp | 10 ++++++---- src/Coordination/KeeperStateMachine.h | 2 -- src/Coordination/KeeperStorage.cpp | 2 +- src/Coordination/RaftServerConfig.cpp | 2 +- 13 files changed, 28 insertions(+), 30 deletions(-) diff --git a/base/base/find_symbols.h b/base/base/find_symbols.h index f7d24ccfc11..fda94edaa88 100644 --- a/base/base/find_symbols.h +++ b/base/base/find_symbols.h @@ -448,7 +448,7 @@ inline char * find_last_not_symbols_or_null(char * begin, char * end) /// See https://github.com/boostorg/algorithm/issues/63 /// And https://bugs.llvm.org/show_bug.cgi?id=41141 template -inline To& splitInto(To & to, std::string_view what, bool token_compress = false) +inline To & splitInto(To & to, std::string_view what, bool token_compress = false) { const char * pos = what.data(); const char * end = pos + what.size(); diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 20ce2a748e6..8567a53699e 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -573,7 +573,7 @@ public: template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(Coordination::Error code, auto& ctx) + constexpr auto format(Coordination::Error code, auto & ctx) { return formatter::format(Coordination::errorMessage(code), ctx); } diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 3c3c0500540..0c083971f74 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -32,8 +32,9 @@ KeeperContext::KeeperContext(bool standalone_keeper_) system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } -void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) +void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_) { + dispatcher = dispatcher_; digest_enabled = config.getBool("keeper_server.digest_enabled", false); ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false); diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 4fb552f20a3..ba1a81b4423 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -24,7 +24,7 @@ public: SHUTDOWN }; - void initialize(const Poco::Util::AbstractConfiguration & config); + void initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_); Phase getServerState() const; void setServerState(Phase server_state_); @@ -52,6 +52,8 @@ public: void dumpConfiguration(WriteBufferFromOwnString & buf) const; + constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; } + private: /// local disk defined using path or disk name using Storage = std::variant; @@ -90,5 +92,4 @@ private: }; using KeeperContextPtr = std::shared_ptr; - } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 178453b2f5b..26be2881780 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -337,8 +337,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshot_s3.startup(config, macros); keeper_context = std::make_shared(standalone_keeper); - keeper_context->initialize(config); - keeper_context->dispatcher = this; + keeper_context->initialize(config, this); server = std::make_unique( configuration_and_settings, @@ -762,7 +761,7 @@ void KeeperDispatcher::clusterUpdateThread() } } -void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions&& actions) +void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions && actions) { if (shutdown_called) return; for (auto && action : actions) diff --git a/src/Coordination/KeeperReconfiguration.cpp b/src/Coordination/KeeperReconfiguration.cpp index dec3e1f155f..f262a07209d 100644 --- a/src/Coordination/KeeperReconfiguration.cpp +++ b/src/Coordination/KeeperReconfiguration.cpp @@ -1,5 +1,4 @@ #include "KeeperReconfiguration.h" -#include #include #include #include @@ -45,8 +44,8 @@ ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std:: for (std::string_view leaving_server : leaving_arr) { - int id; - if (std::from_chars(leaving_server.begin(), leaving_server.end(), id).ec != std::error_code{}) + int32_t id; + if (!tryParse(id, leaving_server)) return {}; if (remove_ids.contains(id)) diff --git a/src/Coordination/KeeperReconfiguration.h b/src/Coordination/KeeperReconfiguration.h index 71958f2035e..fdd81708da2 100644 --- a/src/Coordination/KeeperReconfiguration.h +++ b/src/Coordination/KeeperReconfiguration.h @@ -1,6 +1,6 @@ #pragma once -#include "Coordination/KeeperSnapshotManager.h" -#include "Coordination/RaftServerConfig.h" +#include +#include namespace DB { diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 1cde957ef3a..f6715b0da3f 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -793,10 +793,10 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) { std::lock_guard _{server_write_mutex}; - if (const auto* add = std::get_if(&action)) + if (const auto * add = std::get_if(&action)) return raft_instance->get_srv_config(add->id) != nullptr || raft_instance->add_srv(static_cast(*add))->get_accepted(); - else if (const auto* remove = std::get_if(&action)) + else if (const auto * remove = std::get_if(&action)) { if (isLeader() && remove->id == state_manager->server_id()) { @@ -807,7 +807,7 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) return raft_instance->get_srv_config(remove->id) == nullptr || raft_instance->remove_srv(remove->id)->get_accepted(); } - else if (const auto* update = std::get_if(&action)) + else if (const auto * update = std::get_if(&action)) { if (auto ptr = raft_instance->get_srv_config(update->id); ptr == nullptr) throw Exception(ErrorCodes::RAFT_ERROR, @@ -851,7 +851,7 @@ void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateActi std::this_thread::sleep_for(sleep_time * (i + 1)); }; - if (const auto* add = std::get_if(&action)) + if (const auto * add = std::get_if(&action)) { for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i) { @@ -863,7 +863,7 @@ void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateActi backoff_on_refusal(i); } } - else if (const auto* remove = std::get_if(&action)) + else if (const auto * remove = std::get_if(&action)) { if (remove->id == state_manager->server_id()) { @@ -884,7 +884,7 @@ void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateActi backoff_on_refusal(i); } } - else if (const auto* update = std::get_if(&action)) + else if (const auto * update = std::get_if(&action)) { raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); return; @@ -928,7 +928,7 @@ bool KeeperServer::waitForConfigUpdateWithReconfigDisabled(const ClusterUpdateAc backoff(i); } } - else if (std::get_if(&action) != nullptr) + else if (std::holds_alternative(action)) return true; return false; diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 61e29b67bbd..50d229c9e63 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -17,8 +17,6 @@ namespace DB using RaftAppendResult = nuraft::ptr>>; -class KeeperDispatcher; - class KeeperServer { private: diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 3e9850caa40..6ec03235a2d 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -14,15 +16,13 @@ #include #include #include -#include "Coordination/KeeperStorage.h" -#include "Coordination/KeeperReconfiguration.h" - #include namespace ProfileEvents { extern const Event KeeperCommits; + extern const Event KeeperReconfigRequest; extern const Event KeeperCommitsFailed; extern const Event KeeperSnapshotCreations; extern const Event KeeperSnapshotCreationsFailed; @@ -298,6 +298,8 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( const KeeperStorage::RequestForSession& request_for_session) { + ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest); + const auto& request = static_cast(*request_for_session.request); const int64_t session_id = request_for_session.session_id; const int64_t zxid = request_for_session.zxid; @@ -312,7 +314,7 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( return { session_id, std::move(res) }; }; - KeeperDispatcher& dispatcher = *keeper_context->dispatcher; + KeeperDispatcher& dispatcher = *keeper_context->getDispatcher(); if (!dispatcher.reconfigEnabled()) return bad_request(ZUNIMPLEMENTED); if (!dispatcher.clusterUpdateQueueEmpty()) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 3b239adae45..4ff46394fcc 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -12,8 +12,6 @@ namespace DB { -class KeeperDispatcher; - using ResponsesQueue = ConcurrentBoundedQueue; using SnapshotsQueue = ConcurrentBoundedQueue; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 2b245a455b7..7fe85857ccb 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1088,7 +1088,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce if (request.path == Coordination::keeper_config_path) { response.data = serializeClusterConfig( - storage.keeper_context->dispatcher->getStateMachine().getClusterConfig()); + storage.keeper_context->getDispatcher()->getStateMachine().getClusterConfig()); response.error = Coordination::Error::ZOK; return response_ptr; } diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp index 42923dd0b29..45b6d5d1dad 100644 --- a/src/Coordination/RaftServerConfig.cpp +++ b/src/Coordination/RaftServerConfig.cpp @@ -31,7 +31,7 @@ std::optional RaftServerConfig::parse(std::string_view server) if (!id_str.starts_with("server.")) return std::nullopt; - int id; + Int32 id; if (std::from_chars(std::next(id_str.begin(), 7), id_str.end(), id).ec != std::error_code{}) return std::nullopt; if (id <= 0) From 297d566600c3b36a552b456f8371440c5939b1d7 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 14:13:26 +0000 Subject: [PATCH 0426/2047] acl check --- src/Coordination/KeeperStateMachine.cpp | 3 +++ src/Coordination/KeeperStateMachine.h | 5 +++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 6ec03235a2d..e053e481b6b 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -314,6 +314,9 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( return { session_id, std::move(res) }; }; + if (!storage->checkACL(keeper_config_path, Coordination::ACL::Write, session_id, true)) + return bad_request(ZNOAUTH); + KeeperDispatcher& dispatcher = *keeper_context->getDispatcher(); if (!dispatcher.reconfigEnabled()) return bad_request(ZUNIMPLEMENTED); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 4ff46394fcc..997a03a04d5 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -183,7 +183,8 @@ private: KeeperSnapshotManagerS3 * snapshot_manager_s3; - KeeperStorage::ResponseForSession processReconfiguration(const KeeperStorage::RequestForSession& request_for_session); + KeeperStorage::ResponseForSession processReconfiguration( + const KeeperStorage::RequestForSession& request_for_session) + TSA_REQUIRES(storage_and_responses_lock); }; - } From 1bef6fc76cf70b6faeb82b25e53e708bbf309bc6 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 15:04:31 +0000 Subject: [PATCH 0427/2047] process reconfig in keeper dispatcher --- src/Coordination/KeeperDispatcher.cpp | 6 ++++++ src/Coordination/KeeperStateMachine.cpp | 21 +++++++++++++-------- src/Coordination/KeeperStateMachine.h | 2 ++ 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 26be2881780..9039b3a6d11 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -90,6 +90,12 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; + if (request.request->getOpNum() == Coordination::OpNum::Reconfig) + { + server->getKeeperStateMachine()->reconfigure(request); + continue; + } + KeeperStorage::RequestsForSessions current_batch; size_t current_batch_bytes_size = 0; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e053e481b6b..c837b93ffdd 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -295,6 +295,19 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req return true; } +void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session) +{ + std::lock_guard _(storage_and_responses_lock); + KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session); + if (!responses_queue.push(response)) + { + ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); + LOG_WARNING(log, + "Failed to push response with session id {} to the queue, probably because of shutdown", + response.session_id); + } +} + KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( const KeeperStorage::RequestForSession& request_for_session) { @@ -400,14 +413,6 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n response->session_id = session_id; try_push(response_for_session); } - // Processing reconfig request as an ordinary one (in KeeperStorage) brings multiple inconsistencies - // regarding replays of old reconfigurations in new nodes. Thus the storage is not involved. - // See https://github.com/ClickHouse/ClickHouse/pull/49450 for details - else if (op_num == Coordination::OpNum::Reconfig) - { - std::lock_guard lock(storage_and_responses_lock); - try_push(processReconfiguration(*request_for_session)); - } else { if (op_num == Coordination::OpNum::Close) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 997a03a04d5..5762476886c 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -126,6 +126,8 @@ public: void recalculateStorageStats(); + void reconfigure(const KeeperStorage::RequestForSession& request_for_session); + private: CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored From 4550b15876c7e57533e7aa700b1376682c95de69 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 16:38:26 +0000 Subject: [PATCH 0428/2047] try updating tests to wait for cluster configs to come in sync --- tests/integration/helpers/keeper_utils.py | 14 +++++++++++--- tests/integration/test_keeper_reconfig_add/test.py | 4 ++-- .../test_keeper_reconfig_remove/test.py | 6 +++--- .../test_keeper_reconfig_remove_many/test.py | 10 +++++----- .../test_keeper_reconfig_replace_leader/test.py | 10 +++++----- .../test.py | 10 +++++----- 6 files changed, 31 insertions(+), 23 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3da1d5bd7b0..3970aa325ad 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -77,8 +77,16 @@ def get_config_str(zk: KazooClient) -> str: return zk.get("/keeper/config")[0].decode("utf-8") -def configs_equal(left: str, right: str) -> bool: +def wait_configs_equal(left_config: str, right_zk: KazooClient, timeout: float = 30.0): """ - Check whether /keeper/config nodes are equal + Check whether get /keeper/config result in left_config is equal + to get /keeper/config on right_zk ZK connection. """ - return sorted(left.split("\n")) == sorted(right.split("\n")) + elapsed: float = 0. + while sorted(left_config.split("\n")) != sorted(get_config_str(right_zk).split("\n")): + time.sleep(1) + elapsed += 1 + if elapsed >= timeout: + raise Exception( + f"timeout while checking nodes configs to get equal. " + f"Left: {left_config}, right: {get_config_str(right_zk)}") diff --git a/tests/integration/test_keeper_reconfig_add/test.py b/tests/integration/test_keeper_reconfig_add/test.py index c80279a0727..2c2da7403a1 100644 --- a/tests/integration/test_keeper_reconfig_add/test.py +++ b/tests/integration/test_keeper_reconfig_add/test.py @@ -91,7 +91,7 @@ def test_reconfig_add(started_cluster): assert "node3" not in config zk2 = get_fake_zk(node2) - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) for i in range(100): assert zk2.exists(f"/test_three_{i}") is not None @@ -132,7 +132,7 @@ def test_reconfig_add(started_cluster): assert "node3" in config zk3 = get_fake_zk(node3) - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk3.exists(f"/test_four_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_remove/test.py b/tests/integration/test_keeper_reconfig_remove/test.py index 7f0b1ee92c6..fb0a9472df3 100644 --- a/tests/integration/test_keeper_reconfig_remove/test.py +++ b/tests/integration/test_keeper_reconfig_remove/test.py @@ -70,11 +70,11 @@ def test_reconfig_remove_followers_from_3(started_cluster): zk2 = get_fake_zk(node2) zk2.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"test_two_{i}") is not None @@ -92,7 +92,7 @@ def test_reconfig_remove_followers_from_3(started_cluster): zk2.stop() zk2.close() zk2 = get_fake_zk(node2) - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) for i in range(100): assert zk2.exists(f"test_two_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_remove_many/test.py b/tests/integration/test_keeper_reconfig_remove_many/test.py index 6bf477ff9c9..ec0d8b95eff 100644 --- a/tests/integration/test_keeper_reconfig_remove_many/test.py +++ b/tests/integration/test_keeper_reconfig_remove_many/test.py @@ -54,11 +54,11 @@ def test_reconfig_remove_2_and_leader(started_cluster): zk4 = get_fake_zk(node4) zk4.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk4)) + ku.wait_configs_equal(config, zk4) zk5 = get_fake_zk(node5) zk5.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk5)) + ku.wait_configs_equal(config, zk5) for i in range(100): assert zk4.exists(f"test_two_{i}") is not None @@ -83,7 +83,7 @@ def test_reconfig_remove_2_and_leader(started_cluster): zk1 = get_fake_zk(node1) zk1.sync("/test_two_0") - assert ku.configs_equal(config, ku.get_config_str(zk1)) + ku.wait_configs_equal(config, zk1) for i in range(200): assert zk1.exists(f"test_two_{i}") is not None @@ -128,11 +128,11 @@ def test_reconfig_remove_2_and_leader(started_cluster): zk2.close() zk2 = get_fake_zk(node2) zk2.sync("/test_leader_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_leader_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"test_leader_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_replace_leader/test.py b/tests/integration/test_keeper_reconfig_replace_leader/test.py index 1b23aa056c6..ca1ec3a0c92 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader/test.py @@ -62,11 +62,11 @@ def test_reconfig_replace_leader(started_cluster): zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"/test_four_{i}") is not None @@ -104,7 +104,7 @@ def test_reconfig_replace_leader(started_cluster): assert "node4" in config zk4 = get_fake_zk(node4) - assert ku.configs_equal(config, ku.get_config_str(zk4)) + ku.wait_configs_equal(config, zk4) for i in range(100): assert zk4.exists(f"test_four_{i}") is not None @@ -114,13 +114,13 @@ def test_reconfig_replace_leader(started_cluster): zk2.close() zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3.stop() zk3.close() zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(200): assert zk2.exists(f"test_four_{i}") is not None diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py index c7aed945097..76aed1c7f3a 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -62,11 +62,11 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.wait_configs_equal(config, zk3) for i in range(100): assert zk2.exists(f"/test_four_{i}") is not None @@ -91,7 +91,7 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk4 = get_fake_zk(node4) zk4.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk4)) + ku.wait_configs_equal(config, zk4) for i in range(100): assert zk4.exists(f"test_four_{i}") is not None @@ -107,13 +107,13 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk2.close() zk2 = get_fake_zk(node2) zk2.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk2)) + ku.wait_configs_equal(config, zk2) zk3.stop() zk3.close() zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - assert ku.configs_equal(config, ku.get_config_str(zk3)) + ku.configs_equal(config, zk3) for i in range(200): assert zk2.exists(f"test_four_{i}") is not None From 2f0cd054970015799b394588b7ecf79ca34a6e9a Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 17:35:25 +0000 Subject: [PATCH 0429/2047] handle leader removal corner cases --- src/Coordination/KeeperServer.cpp | 7 +++++-- .../test.py | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index f6715b0da3f..fd82f220f9b 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -798,9 +798,12 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) || raft_instance->add_srv(static_cast(*add))->get_accepted(); else if (const auto * remove = std::get_if(&action)) { - if (isLeader() && remove->id == state_manager->server_id()) + if (remove->id == raft_instance->get_leader()) { - raft_instance->yield_leadership(); + if (isLeader()) + raft_instance->yield_leadership(); + else + raft_instance->request_leadership(); return false; } diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py index 76aed1c7f3a..e23d0674c12 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -113,7 +113,7 @@ def test_reconfig_replace_leader_in_one_command(started_cluster): zk3.close() zk3 = get_fake_zk(node3) zk3.sync("/test_four_0") - ku.configs_equal(config, zk3) + ku.wait_configs_equal(config, zk3) for i in range(200): assert zk2.exists(f"test_four_{i}") is not None From c2a0607cf890f95e94db2751a3e68b7acc59a5bf Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Mon, 3 Jul 2023 17:52:57 +0000 Subject: [PATCH 0430/2047] fix --- tests/integration/helpers/keeper_utils.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 3970aa325ad..93ea3fa74b7 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -82,11 +82,14 @@ def wait_configs_equal(left_config: str, right_zk: KazooClient, timeout: float = Check whether get /keeper/config result in left_config is equal to get /keeper/config on right_zk ZK connection. """ - elapsed: float = 0. - while sorted(left_config.split("\n")) != sorted(get_config_str(right_zk).split("\n")): + elapsed: float = 0.0 + while sorted(left_config.split("\n")) != sorted( + get_config_str(right_zk).split("\n") + ): time.sleep(1) elapsed += 1 if elapsed >= timeout: raise Exception( f"timeout while checking nodes configs to get equal. " - f"Left: {left_config}, right: {get_config_str(right_zk)}") + f"Left: {left_config}, right: {get_config_str(right_zk)}" + ) From c46b125d0a8501241a4a726a32141e1215a2cbf5 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Wed, 5 Jul 2023 18:23:34 +0000 Subject: [PATCH 0431/2047] review fixes --- src/Coordination/KeeperDispatcher.cpp | 2 ++ src/Coordination/KeeperStateMachine.cpp | 2 +- src/Coordination/KeeperStateMachine.h | 5 ++++- src/Coordination/tests/gtest_coordination.cpp | 10 +++++----- utils/keeper-data-dumper/main.cpp | 2 +- 5 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9039b3a6d11..90996dfaff7 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -121,6 +121,8 @@ void KeeperDispatcher::requestThread() std::lock_guard lock(read_request_queue_mutex); read_request_queue[last_request.session_id][last_request.request->xid].push_back(request); } + else if (request.request->getOpNum() == Coordination::OpNum::Reconfig) + server->getKeeperStateMachine()->reconfigure(request); else { current_batch_bytes_size += request.request->bytesSize(); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index c837b93ffdd..45c776e105b 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -313,7 +313,7 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( { ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest); - const auto& request = static_cast(*request_for_session.request); + const auto & request = static_cast(*request_for_session.request); const int64_t session_id = request_for_session.session_id; const int64_t zxid = request_for_session.zxid; diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 5762476886c..116fa9257a0 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -88,7 +88,10 @@ public: int read_logical_snp_obj( nuraft::snapshot & s, void *& user_snp_ctx, uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) override; - KeeperStorage & getStorageForUnitTests() TSA_NO_THREAD_SAFETY_ANALYSIS + // This should be used only for tests or keeper-data-dumper because it violates + // TSA -- we can't acquire the lock outside of this class or return a storage under lock + // in a reasonable way. + KeeperStorage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS { return *storage; } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index b302f9b13ca..03ce23e9233 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1616,8 +1616,8 @@ void testLogAndStateMachine( restore_machine->commit(i, changelog.entry_at(i)->get_buf()); } - auto & source_storage = state_machine->getStorageForUnitTests(); - auto & restored_storage = restore_machine->getStorageForUnitTests(); + auto & source_storage = state_machine->getStorageUnsafe(); + auto & restored_storage = restore_machine->getStorageUnsafe(); EXPECT_EQ(source_storage.container.size(), restored_storage.container.size()); for (size_t i = 1; i < total_logs + 1; ++i) @@ -1719,7 +1719,7 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) auto entry_c = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), request_c); state_machine->pre_commit(1, entry_c->get_buf()); state_machine->commit(1, entry_c->get_buf()); - const auto & storage = state_machine->getStorageForUnitTests(); + const auto & storage = state_machine->getStorageUnsafe(); EXPECT_EQ(storage.ephemerals.size(), 1); std::shared_ptr request_d = std::make_shared(); @@ -1768,7 +1768,7 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); state_machine->pre_commit(2, create_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state; ASSERT_TRUE(uncommitted_state.nodes.contains(node_path)); // commit log entries @@ -1831,7 +1831,7 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) state_machine->commit(2, create_entry->get_buf()); state_machine->commit(3, set_acl_entry->get_buf()); - const auto & uncommitted_state = state_machine->getStorageForUnitTests().uncommitted_state; + const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state; auto node = uncommitted_state.getNode(node_path); ASSERT_NE(node, nullptr); diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 8d685d65d1d..22e5f47687a 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -15,7 +15,7 @@ using namespace DB; void dumpMachine(std::shared_ptr machine) { - auto & storage = machine->getStorageForUnitTests(); + auto & storage = machine->getStorageUnsafe(); std::queue keys; keys.push("/"); From bafcc3afdc79463915b53b5e441758cbb958b958 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 6 Jul 2023 15:18:49 +0000 Subject: [PATCH 0432/2047] remove reconfig in process flag as it's useless --- src/Common/ZooKeeper/IKeeper.h | 1 - src/Coordination/KeeperDispatcher.cpp | 5 ----- src/Coordination/KeeperDispatcher.h | 1 - src/Coordination/KeeperStateMachine.cpp | 4 +--- src/Coordination/RaftServerConfig.cpp | 9 ++++----- 5 files changed, 5 insertions(+), 15 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 8567a53699e..5240acc2616 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -82,7 +82,6 @@ enum class Error : int32_t ZOPERATIONTIMEOUT = -7, /// Operation timeout ZBADARGUMENTS = -8, /// Invalid arguments ZINVALIDSTATE = -9, /// Invalid zhandle state - ZRECONFIGINPROGRESS = -14, /// Another reconfig is running /** API errors. * This is never thrown by the server, it shouldn't be used other than diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 90996dfaff7..b956bba4031 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -780,11 +780,6 @@ void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions && actions) } } -bool KeeperDispatcher::clusterUpdateQueueEmpty() const -{ - return cluster_update_queue.empty(); -} - bool KeeperDispatcher::reconfigEnabled() const { return server->reconfigEnabled(); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index a9b3d33eb51..40f1dac1570 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -136,7 +136,6 @@ public: void updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); void pushClusterUpdates(ClusterUpdateActions&& actions); - bool clusterUpdateQueueEmpty() const; bool reconfigEnabled() const; /// Shutdown internal keeper parts (server, state machine, log storage, etc) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 45c776e105b..b821050cccf 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -309,7 +309,7 @@ void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& req } KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( - const KeeperStorage::RequestForSession& request_for_session) + const KeeperStorage::RequestForSession & request_for_session) { ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest); @@ -333,8 +333,6 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( KeeperDispatcher& dispatcher = *keeper_context->getDispatcher(); if (!dispatcher.reconfigEnabled()) return bad_request(ZUNIMPLEMENTED); - if (!dispatcher.clusterUpdateQueueEmpty()) - return bad_request(ZRECONFIGINPROGRESS); if (request.version != -1) return bad_request(ZBADVERSION); diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp index 45b6d5d1dad..9090ed68fb6 100644 --- a/src/Coordination/RaftServerConfig.cpp +++ b/src/Coordination/RaftServerConfig.cpp @@ -1,7 +1,6 @@ #include "RaftServerConfig.h" -#include -#include #include +#include #include namespace DB @@ -32,7 +31,7 @@ std::optional RaftServerConfig::parse(std::string_view server) return std::nullopt; Int32 id; - if (std::from_chars(std::next(id_str.begin(), 7), id_str.end(), id).ec != std::error_code{}) + if (!tryParse(id, std::next(id_str.begin(), 7))) return std::nullopt; if (id <= 0) return std::nullopt; @@ -44,7 +43,7 @@ std::optional RaftServerConfig::parse(std::string_view server) const std::string_view port = endpoint.substr(port_delimiter + 1); uint16_t port_tmp; - if (std::from_chars(port.begin(), port.end(), port_tmp).ec != std::error_code{}) + if (!tryParse(port_tmp, port)) return std::nullopt; RaftServerConfig out{id, endpoint}; @@ -59,7 +58,7 @@ std::optional RaftServerConfig::parse(std::string_view server) return out; const std::string_view priority = parts[3]; - if (std::from_chars(priority.begin(), priority.end(), out.priority).ec != std::error_code{}) + if (!tryParse(out.priority, priority)) return std::nullopt; if (out.priority < 0) return std::nullopt; From 5302b478a4b512d080068563d4b5b983e4b13d77 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Thu, 6 Jul 2023 17:12:24 +0000 Subject: [PATCH 0433/2047] proper reconfig batch handling --- src/Common/ZooKeeper/IKeeper.cpp | 1 - src/Coordination/KeeperDispatcher.cpp | 32 +++++++++++++++++---------- 2 files changed, 20 insertions(+), 13 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 50160279506..f0a07241735 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -110,7 +110,6 @@ const char * errorMessage(Error code) case Error::ZCLOSING: return "ZooKeeper is closing"; case Error::ZNOTHING: return "(not error) no server responses to process"; case Error::ZSESSIONMOVED: return "Session moved to another server, so operation is ignored"; - case Error::ZRECONFIGINPROGRESS: return "Another reconfiguration is progress"; } UNREACHABLE(); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index b956bba4031..daa65de0d89 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -82,6 +82,7 @@ void KeeperDispatcher::requestThread() /// requests into a batch we must check that the new request is not read request. Otherwise we have to /// process all already accumulated write requests, wait them synchronously and only after that process /// read request. So reads are some kind of "separator" for writes. + /// Also there is a special reconfig request also being a separator. try { if (requests_queue->tryPop(request, max_wait)) @@ -90,20 +91,17 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; - if (request.request->getOpNum() == Coordination::OpNum::Reconfig) - { - server->getKeeperStateMachine()->reconfigure(request); - continue; - } - KeeperStorage::RequestsForSessions current_batch; size_t current_batch_bytes_size = 0; bool has_read_request = false; + bool has_reconfig_request = false; - /// If new request is not read request or we must to process it through quorum. + /// If new request is not read request or reconfig request we must process it through quorum. /// Otherwise we will process it locally. - if (coordination_settings->quorum_reads || !request.request->isReadRequest()) + if (request.request->getOpNum() == Coordination::OpNum::Reconfig) + has_reconfig_request = true; + else if (coordination_settings->quorum_reads || !request.request->isReadRequest()) { current_batch_bytes_size += request.request->bytesSize(); current_batch.emplace_back(request); @@ -122,7 +120,10 @@ void KeeperDispatcher::requestThread() read_request_queue[last_request.session_id][last_request.request->xid].push_back(request); } else if (request.request->getOpNum() == Coordination::OpNum::Reconfig) - server->getKeeperStateMachine()->reconfigure(request); + { + has_reconfig_request = true; + return false; + } else { current_batch_bytes_size += request.request->bytesSize(); @@ -138,6 +139,7 @@ void KeeperDispatcher::requestThread() /// TODO: Deprecate max_requests_quick_batch_size and use only max_requests_batch_size and max_requests_batch_bytes_size size_t max_quick_batch_size = coordination_settings->max_requests_quick_batch_size; while (!shutdown_called && !has_read_request && + !has_reconfig_request && current_batch.size() < max_quick_batch_size && current_batch_bytes_size < max_batch_bytes_size && try_get_request()) ; @@ -150,8 +152,10 @@ void KeeperDispatcher::requestThread() }; /// Waiting until previous append will be successful, or batch is big enough - while (!shutdown_called && !has_read_request && !prev_result_done() && - current_batch.size() <= max_batch_size && current_batch_bytes_size < max_batch_bytes_size) + while (!shutdown_called && !has_read_request && + !has_reconfig_request && !prev_result_done() && + current_batch.size() <= max_batch_size + && current_batch_bytes_size < max_batch_bytes_size) { try_get_request(); } @@ -175,7 +179,8 @@ void KeeperDispatcher::requestThread() if (result) { - if (has_read_request) /// If we will execute read request next, than we have to process result now + /// If we will execute read or reconfig next, we have to process result now + if (has_read_request || has_reconfig_request) forceWaitAndProcessResult(result, current_batch); } else @@ -189,6 +194,9 @@ void KeeperDispatcher::requestThread() prev_result = result; } + if (has_reconfig_request) + server->getKeeperStateMachine()->reconfigure(request); + /// Read request always goes after write batch (last request) if (has_read_request) { From ef3551fea00b6eeaa76884880a977e9a0768bb82 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 7 Jul 2023 03:54:10 +0200 Subject: [PATCH 0434/2047] Maybe better tests --- tests/queries/0_stateless/00995_exception_while_insert.sh | 3 +-- tests/queries/0_stateless/01030_limit_by_with_ties_error.sh | 3 +-- tests/queries/0_stateless/01187_set_profile_as_setting.sh | 2 +- tests/queries/0_stateless/01442_merge_detach_attach_long.sh | 3 +-- tests/queries/0_stateless/01515_logtrace_function.sh | 3 +-- .../01583_parallel_parsing_exception_with_offset.sh | 3 +-- tests/queries/0_stateless/02359_send_logs_source_regexp.sh | 2 +- .../0_stateless/02360_rename_table_along_with_log_name.sh | 2 +- 8 files changed, 8 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/00995_exception_while_insert.sh b/tests/queries/0_stateless/00995_exception_while_insert.sh index e0cd264a2b7..927ac6a54e5 100755 --- a/tests/queries/0_stateless/00995_exception_while_insert.sh +++ b/tests/queries/0_stateless/00995_exception_while_insert.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS check;" $CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64, y UInt64 DEFAULT throwIf(x > 1500000)) ENGINE = Memory;" diff --git a/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh b/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh index 711a015f044..c3414838789 100755 --- a/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh +++ b/tests/queries/0_stateless/01030_limit_by_with_ties_error.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - $CLICKHOUSE_CLIENT --query=""" SELECT * FROM (SELECT number % 5 AS a, count() AS b, c FROM numbers(10) ARRAY JOIN [1,2] AS c GROUP BY a,c) AS table diff --git a/tests/queries/0_stateless/01187_set_profile_as_setting.sh b/tests/queries/0_stateless/01187_set_profile_as_setting.sh index dacb609d790..fccac57aea8 100755 --- a/tests/queries/0_stateless/01187_set_profile_as_setting.sh +++ b/tests/queries/0_stateless/01187_set_profile_as_setting.sh @@ -4,13 +4,13 @@ unset CLICKHOUSE_LOG_COMMENT CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -n -m -q "select value, changed from system.settings where name='readonly';" $CLICKHOUSE_CLIENT -n -m -q "set profile='default'; select value, changed from system.settings where name='readonly';" $CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" 2>&1| grep -Fa "Cannot modify 'send_logs_level' setting in readonly mode" > /dev/null && echo "OK" -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=fatal/g') $CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=select+value,changed+from+system.settings+where+name='readonly'" diff --git a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh index c080dded1c8..acb2550d48c 100755 --- a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh +++ b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh @@ -4,11 +4,10 @@ set -e CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY tuple()" diff --git a/tests/queries/0_stateless/01515_logtrace_function.sh b/tests/queries/0_stateless/01515_logtrace_function.sh index 131ec0edb9e..4ebecd0cc18 100755 --- a/tests/queries/0_stateless/01515_logtrace_function.sh +++ b/tests/queries/0_stateless/01515_logtrace_function.sh @@ -2,9 +2,8 @@ # Tags: race CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=debug # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') - ${CLICKHOUSE_CLIENT} --query="SELECT logTrace('logTrace Function Test');" 2>&1 | grep -q "logTrace Function Test" && echo "OK" || echo "FAIL" diff --git a/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh b/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh index aa3a25096c0..00d22cb8e83 100755 --- a/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh +++ b/tests/queries/0_stateless/01583_parallel_parsing_exception_with_offset.sh @@ -1,11 +1,10 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS check;" $CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64) ENGINE = Memory;" diff --git a/tests/queries/0_stateless/02359_send_logs_source_regexp.sh b/tests/queries/0_stateless/02359_send_logs_source_regexp.sh index d3b60bc59f4..f287e323ca7 100755 --- a/tests/queries/0_stateless/02359_send_logs_source_regexp.sh +++ b/tests/queries/0_stateless/02359_send_logs_source_regexp.sh @@ -1,11 +1,11 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh [ ! -z "$CLICKHOUSE_CLIENT_REDEFINED" ] && CLICKHOUSE_CLIENT=$CLICKHOUSE_CLIENT_REDEFINED -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') regexp="executeQuery|InterpreterSelectQuery" $CLICKHOUSE_CLIENT --send_logs_source_regexp "$regexp" -q "SELECT 1;" 2> >(grep -v -E "$regexp" 1>&2) diff --git a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh index e8c7f844b5c..c07dcdd549b 100755 --- a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh +++ b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh @@ -11,7 +12,6 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS y;" $CLICKHOUSE_CLIENT -q "CREATE TABLE x(i int) ENGINE MergeTree ORDER BY i;" $CLICKHOUSE_CLIENT -q "RENAME TABLE x TO y;" -CLICKHOUSE_CLIENT_WITH_LOG=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') regexp="${CLICKHOUSE_DATABASE}\\.x" # Check if there are still log entries with old table name $CLICKHOUSE_CLIENT_WITH_LOG --send_logs_source_regexp "$regexp" -q "INSERT INTO y VALUES(1);" From acd17c7974637714138a76fb83f73ec31946aa79 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 10:40:04 +0800 Subject: [PATCH 0435/2047] Make a deal with the "Style check" --- src/Loggers/Loggers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 1e169190ca4..4c85ea79a63 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -39,7 +39,7 @@ static std::string renderFileNameTemplate(time_t now, const std::string & file_p fs::path path{file_path}; std::tm buf; localtime_r(&now, &buf); - std::stringstream ss; + std::ostringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM ss << std::put_time(&buf, file_path.c_str()); return path.replace_filename(ss.str()); } From 7080d85d2de6c743cc5759fa2a50d1ada1d51068 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 10:54:36 +0800 Subject: [PATCH 0436/2047] Amend the documentation --- .../settings.md | 49 +++++++++++++++++- .../settings.md | 50 ++++++++++++++++++- 2 files changed, 95 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index bad7e388377..48361b0f157 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1201,13 +1201,58 @@ Keys: - `console` – Send `log` and `errorlog` to the console instead of file. To enable, set to `1` or `true`. - `stream_compress` – Compress `log` and `errorlog` with `lz4` stream compression. To enable, set to `1` or `true`. +Both log and error log file names (only file names, not directories) support date and time format placeholders. + +**Placeholders** +Using the following placeholders, you can define a pattern for the resulting file name. “Example” column shows formatting result for `2023-07-06 18:32:07`. + +| Placeholder | Description | Example | +|-------------|---------------------------------------------------------------------------------------------------------------------|--------------------------| +| %% | Literal % | % | +| %n | New-line character | | +| %t | Horizontal tab character | | +| %Y | Year as a decimal number, e.g. 2017 | 2023 | +| %y | Last 2 digits of year as a decimal number (range [00,99]) | 23 | +| %C | First 2 digits of year as a decimal number (range [00,99]) | 20 | +| %G | Four-digit [ISO 8601 week-based year](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), i.e. the year that contains the specified week. Normally useful only with %V | 2023 | +| %g | Last 2 digits of [ISO 8601 week-based year](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), i.e. the year that contains the specified week. | 23 | +| %b | Abbreviated month name, e.g. Oct (locale dependent) | Jul | +| %h | Synonym of %b | Jul | +| %B | Full month name, e.g. October (locale dependent) | July | +| %m | Month as a decimal number (range [01,12]) | 07 | +| %U | Week of the year as a decimal number (Sunday is the first day of the week) (range [00,53]) | 27 | +| %W | Week of the year as a decimal number (Monday is the first day of the week) (range [00,53]) | 27 | +| %V | ISO 8601 week number (range [01,53]) | 27 | +| %j | Day of the year as a decimal number (range [001,366]) | 187 | +| %d | Day of the month as a zero-padded decimal number (range [01,31]). Single digit is preceded by zero. | 06 | +| %e | Day of the month as a space-padded decimal number (range [1,31]). Single digit is preceded by a space. |   6 | +| %a | Abbreviated weekday name, e.g. Fri (locale dependent) | Thu | +| %A | Full weekday name, e.g. Friday (locale dependent) | Thursday | +| %w | Weekday as a integer number with Sunday as 0 (range [0-6]) | 4 | +| %u | Weekday as a decimal number, where Monday is 1 (ISO 8601 format) (range [1-7]) | 4 | +| %H | Hour as a decimal number, 24 hour clock (range [00-23]) | 18 | +| %I | Hour as a decimal number, 12 hour clock (range [01,12]) | 06 | +| %M | Minute as a decimal number (range [00,59]) | 32 | +| %S | Second as a decimal number (range [00,60]) | 07 | +| %c | Standard date and time string, e.g. Sun Oct 17 04:41:13 2010 (locale dependent) | Thu Jul 6 18:32:07 2023 | +| %x | Localized date representation (locale dependent) | 07/06/23 | +| %X | Localized time representation, e.g. 18:40:20 or 6:40:20 PM (locale dependent) | 18:32:07 | +| %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 07/06/23 | +| %F | Short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2023-07-06 | +| %r | Localized 12-hour clock time (locale dependent) | 06:32:07 PM | +| %R | Equivalent to "%H:%M" | 18:32 | +| %T | Equivalent to "%H:%M:%S" (the ISO 8601 time format) | 18:32:07 | +| %p | Localized a.m. or p.m. designation (locale dependent) | PM | +| %z | Offset from UTC in the ISO 8601 format (e.g. -0430), or no characters if the time zone information is not available | +0800 | +| %Z | Locale-dependent time zone name or abbreviation, or no characters if the time zone information is not available | Z AWST | + **Example** ``` xml trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.err.log 1000M 10 true diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 5430469ea18..421df3fe3eb 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -575,14 +575,60 @@ ClickHouse поддерживает динамическое изменение - `errorlog` - Файл лога ошибок. - `size` - Размер файла. Действует для `log` и `errorlog`. Как только файл достиг размера `size`, ClickHouse архивирует и переименовывает его, а на его месте создает новый файл лога. - `count` - Количество заархивированных файлов логов, которые сохраняет ClickHouse. +- `stream_compress` – Сжимать `log` и `errorlog` с помощью алгоритма `lz4`. Чтобы активировать, узтановите значение `1` или `true`. + +Имена файлов `log` и `errorlog` (только имя файла, а не директорий) поддерживают спецификаторы шаблонов даты и времени. + +**Спецификаторы форматирования** +С помощью следующих спецификаторов, можно определить шаблон для формирования имени файла. Столбец “Пример” показывает возможные значения на момент времени `2023-07-06 18:32:07`. + +| Спецификатор | Описание | Пример | +|--------------|---------------------------------------------------------------------------------------------------------------------|--------------------------| +| %% | Литерал % | % | +| %n | Символ новой строки | | +| %t | Символ горизонтальной табуляции | | +| %Y | Год как десятичное число, например, 2017 | 2023 | +| %y | Последние 2 цифры года в виде десятичного числа (диапазон [00,99]) | 23 | +| %C | Первые 2 цифры года в виде десятичного числа (диапазон [00,99]) | 20 | +| %G | Год по неделям согласно [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), то есть год, который содержит указанную неделю. Обычно используется вместе с %V. | 2023 | +| %g | Последние 2 цифры [года по неделям ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), т.е. года, содержащего указанную неделю (диапазон [00,99]). | 23 | +| %b | Сокращённое название месяца, например Oct (зависит от локали) | Jul | +| %h | Синоним %b | Jul | +| %B | Полное название месяца, например, October (зависит от локали) | July | +| %m | Месяц в виде десятичного числа (диапазон [01,12]) | 07 | +| %U | Неделя года в виде десятичного числа (воскресенье - первый день недели) (диапазон [00,53]) | 27 | +| %W | Неделя года в виде десятичного числа (понедельник - первый день недели) (диапазон [00,53]) | 27 | +| %V | Неделя года ISO 8601 (диапазон [01,53]) | 27 | +| %j | День года в виде десятичного числа (диапазон [001,366]) | 187 | +| %d | День месяца в виде десятичного числа (диапазон [01,31]) Перед одиночной цифрой ставится ноль. | 06 | +| %e | День месяца в виде десятичного числа (диапазон [1,31]). Перед одиночной цифрой ставится пробел. |   6 | +| %a | Сокращённое название дня недели, например, Fri (зависит от локали) | Thu | +| %A | Полный день недели, например, Friday (зависит от локали) | Thursday | +| %w | День недели в виде десятичного числа, где воскресенье равно 0 (диапазон [0-6]) | 4 | +| %u | День недели в виде десятичного числа, где понедельник равен 1 (формат ISO 8601) (диапазон [1-7]) | 4 | +| %H | Час в виде десятичного числа, 24-часовой формат (диапазон [00-23]) | 18 | +| %I | Час в виде десятичного числа, 12-часовой формат (диапазон [01,12]) | 06 | +| %M | Минуты в виде десятичного числа (диапазон [00,59]) | 32 | +| %S | Секунды как десятичное число (диапазон [00,60]) | 07 | +| %c | Стандартная строка даты и времени, например, Sun Oct 17 04:41:13 2010 (зависит от локали) | Thu Jul 6 18:32:07 2023 | +| %x | Локализованное представление даты (зависит от локали) | 07/06/23 | +| %X | Локализованное представление времени, например, 18:40:20 или 6:40:20 PM (зависит от локали) | 18:32:07 | +| %D | Эквивалентно "%m/%d/%y" | 07/06/23 | +| %F | Эквивалентно "%Y-%m-%d" (формат даты ISO 8601) | 2023-07-06 | +| %r | Локализованное 12-часовое время (зависит от локали) | 06:32:07 PM | +| %R | Эквивалентно "%H:%M" | 18:32 | +| %T | Эквивалентно "%H:%M:%S" (формат времени ISO 8601) | 18:32:07 | +| %p | Локализованное обозначение a.m. или p.m. (зависит от локали) | PM | +| %z | Смещение от UTC в формате ISO 8601 (например, -0430), или без символов, если информация о часовом поясе недоступна | +0800 | +| %Z | Зависящее от локали название или аббревиатура часового пояса, если информация о часовом поясе доступна | Z AWST | **Пример** ``` xml trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.err.log 1000M 10 From 63fbde41fee5fb8c0133dc5a576ed4e3caa5c3f2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 11:01:39 +0800 Subject: [PATCH 0437/2047] Reformat cluster.py (add empty line) --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5b583b865de..d4b1ee76712 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -72,6 +72,7 @@ CLICKHOUSE_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.log" CLICKHOUSE_ERROR_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.err.log" + # to create docker-compose env file def _create_env_file(path, variables): logging.debug(f"Env {variables} stored in {path}") From 3edee4174c040b079015ce6524c0d4c56926e348 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 11:34:03 +0800 Subject: [PATCH 0438/2047] Add AWST time zone abbreviation to the ignore list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2802e52c288..6ddca6db538 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -12,6 +12,7 @@ ARMv ASLR ASOF ASan +AWST Actian ActionsMenu ActiveRecord From 87ea1b6667ed9a79272e3b77c529369f2acc4e4e Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 13:01:30 +0800 Subject: [PATCH 0439/2047] Recover the integration test --- .../test.py | 58 +++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/tests/integration/test_render_log_file_name_templates/test.py b/tests/integration/test_render_log_file_name_templates/test.py index e69de29bb2d..9fa87056d2c 100644 --- a/tests/integration/test_render_log_file_name_templates/test.py +++ b/tests/integration/test_render_log_file_name_templates/test.py @@ -0,0 +1,58 @@ +import pytest +import logging +from helpers.cluster import ClickHouseCluster +from datetime import datetime + + +log_dir = "/var/log/clickhouse-server/" +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def started_cluster(): + cluster.add_instance( + "file-names-from-config", + main_configs=["configs/config-file-template.xml"], + clickhouse_log_file=None, + clickhouse_error_log_file=None, + ) + cluster.add_instance( + "file-names-from-params", + clickhouse_log_file=log_dir + "clickhouse-server-%Y-%m.log", + clickhouse_error_log_file=log_dir + "clickhouse-server-%Y-%m.err.log", + ) + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_check_file_names(started_cluster): + now = datetime.now() + log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" + ) + err_log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" + ) + logging.debug(f"log_file {log_file} err_log_file {err_log_file}") + + for name, instance in started_cluster.instances.items(): + files = instance.exec_in_container( + ["bash", "-c", f"ls -lh {log_dir}"], nothrow=True + ) + + logging.debug(f"check instance '{name}': {log_dir} contains: {files}") + + assert ( + instance.exec_in_container(["bash", "-c", f"ls {log_file}"], nothrow=True) + == log_file + "\n" + ) + + assert ( + instance.exec_in_container( + ["bash", "-c", f"ls {err_log_file}"], nothrow=True + ) + == err_log_file + "\n" + ) From 95fedaedff3ad3e3cdb15d3cc2b06ab6d9ea1e9b Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 15:16:10 +0800 Subject: [PATCH 0440/2047] Refine the integration test code --- .../__init__.py | 58 ------------------- .../test.py | 8 +-- 2 files changed, 2 insertions(+), 64 deletions(-) diff --git a/tests/integration/test_render_log_file_name_templates/__init__.py b/tests/integration/test_render_log_file_name_templates/__init__.py index 9fa87056d2c..e69de29bb2d 100644 --- a/tests/integration/test_render_log_file_name_templates/__init__.py +++ b/tests/integration/test_render_log_file_name_templates/__init__.py @@ -1,58 +0,0 @@ -import pytest -import logging -from helpers.cluster import ClickHouseCluster -from datetime import datetime - - -log_dir = "/var/log/clickhouse-server/" -cluster = ClickHouseCluster(__file__) - - -@pytest.fixture(scope="module") -def started_cluster(): - cluster.add_instance( - "file-names-from-config", - main_configs=["configs/config-file-template.xml"], - clickhouse_log_file=None, - clickhouse_error_log_file=None, - ) - cluster.add_instance( - "file-names-from-params", - clickhouse_log_file=log_dir + "clickhouse-server-%Y-%m.log", - clickhouse_error_log_file=log_dir + "clickhouse-server-%Y-%m.err.log", - ) - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_check_file_names(started_cluster): - now = datetime.now() - log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" - ) - err_log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" - ) - logging.debug(f"log_file {log_file} err_log_file {err_log_file}") - - for name, instance in started_cluster.instances.items(): - files = instance.exec_in_container( - ["bash", "-c", f"ls -lh {log_dir}"], nothrow=True - ) - - logging.debug(f"check instance '{name}': {log_dir} contains: {files}") - - assert ( - instance.exec_in_container(["bash", "-c", f"ls {log_file}"], nothrow=True) - == log_file + "\n" - ) - - assert ( - instance.exec_in_container( - ["bash", "-c", f"ls {err_log_file}"], nothrow=True - ) - == err_log_file + "\n" - ) diff --git a/tests/integration/test_render_log_file_name_templates/test.py b/tests/integration/test_render_log_file_name_templates/test.py index 9fa87056d2c..58df32b823e 100644 --- a/tests/integration/test_render_log_file_name_templates/test.py +++ b/tests/integration/test_render_log_file_name_templates/test.py @@ -30,12 +30,8 @@ def started_cluster(): def test_check_file_names(started_cluster): now = datetime.now() - log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" - ) - err_log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" - ) + log_file = log_dir + f"clickhouse-server-{now.strftime('%Y-%m')}.log" + err_log_file = log_dir + f"clickhouse-server-{now.strftime('%Y-%m')}.err.log" logging.debug(f"log_file {log_file} err_log_file {err_log_file}") for name, instance in started_cluster.instances.items(): From 5b102ce7d44d678a674d29e4140a60950c69f537 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 17:21:47 +0800 Subject: [PATCH 0441/2047] Amend English version of settings.md --- .../server-configuration-parameters/settings.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 48361b0f157..82dac74e647 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1201,12 +1201,12 @@ Keys: - `console` – Send `log` and `errorlog` to the console instead of file. To enable, set to `1` or `true`. - `stream_compress` – Compress `log` and `errorlog` with `lz4` stream compression. To enable, set to `1` or `true`. -Both log and error log file names (only file names, not directories) support date and time format placeholders. +Both log and error log file names (only file names, not directories) support date and time format specifiers. -**Placeholders** -Using the following placeholders, you can define a pattern for the resulting file name. “Example” column shows formatting result for `2023-07-06 18:32:07`. +**Format specifiers** +Using the following format specifiers, you can define a pattern for the resulting file name. “Example” column shows possible results for `2023-07-06 18:32:07`. -| Placeholder | Description | Example | +| Specifier | Description | Example | |-------------|---------------------------------------------------------------------------------------------------------------------|--------------------------| | %% | Literal % | % | | %n | New-line character | | From cd3080428ea3da6a71169c929e959a0c3f9c5d5b Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 7 Jul 2023 10:58:01 +0000 Subject: [PATCH 0442/2047] Fix async connect to hosts with multiple ips --- src/Client/Connection.cpp | 2 + src/Client/Connection.h | 4 ++ src/Client/ConnectionEstablisher.cpp | 7 +- src/Client/ConnectionEstablisher.h | 2 + .../configs/enable_hedged.xml | 8 +++ .../configs/listen_host.xml | 4 ++ .../test.py | 65 +++++++++++++++++++ 7 files changed, 91 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml create mode 100644 tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml create mode 100644 tests/integration/test_async_connect_to_multiple_ips/test.py diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index cd102f46ffe..cac5600fbcb 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -105,6 +105,8 @@ void Connection::connect(const ConnectionTimeouts & timeouts) for (auto it = addresses.begin(); it != addresses.end();) { + have_more_addresses_to_connect = it != std::prev(addresses.end()); + if (connected) disconnect(); diff --git a/src/Client/Connection.h b/src/Client/Connection.h index cb3f2507cb9..f4daf8e3aeb 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -159,6 +159,8 @@ public: out->setAsyncCallback(async_callback); } + bool haveMoreAddressesToConnect() const { return have_more_addresses_to_connect; } + private: String host; UInt16 port; @@ -227,6 +229,8 @@ private: std::shared_ptr maybe_compressed_out; std::unique_ptr block_out; + bool have_more_addresses_to_connect = false; + /// Logger is created lazily, for avoid to run DNS request in constructor. class LoggerWrapper { diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 897fb5fde73..439025447ca 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -179,7 +179,7 @@ bool ConnectionEstablisherAsync::checkTimeout() is_timeout_alarmed = true; } - if (is_timeout_alarmed && !is_socket_ready) + if (is_timeout_alarmed && !is_socket_ready && !haveMoreAddressesToConnect()) { /// In not async case timeout exception would be thrown and caught in ConnectionEstablisher::run, /// but in async case we process timeout outside and cannot throw exception. So, we just save fail message. @@ -225,6 +225,11 @@ void ConnectionEstablisherAsync::resetResult() } } +bool ConnectionEstablisherAsync::haveMoreAddressesToConnect() +{ + return !result.entry.isNull() && result.entry->haveMoreAddressesToConnect(); +} + #endif } diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index 5b58563dc01..a8126900d3b 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -104,6 +104,8 @@ private: void resetResult(); + bool haveMoreAddressesToConnect(); + ConnectionEstablisher connection_establisher; TryResult result; std::string fail_message; diff --git a/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml new file mode 100644 index 00000000000..238370176af --- /dev/null +++ b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml @@ -0,0 +1,8 @@ + + + + 1 + 0 + + + diff --git a/tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml b/tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml new file mode 100644 index 00000000000..df0247fd651 --- /dev/null +++ b/tests/integration/test_async_connect_to_multiple_ips/configs/listen_host.xml @@ -0,0 +1,4 @@ + + :: + + diff --git a/tests/integration/test_async_connect_to_multiple_ips/test.py b/tests/integration/test_async_connect_to_multiple_ips/test.py new file mode 100644 index 00000000000..0c18a316d4b --- /dev/null +++ b/tests/integration/test_async_connect_to_multiple_ips/test.py @@ -0,0 +1,65 @@ +import pytest +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def cluster_without_dns_cache_update(): + try: + cluster.start() + + yield cluster + + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + pass + + +node1 = cluster.add_instance( + "node1", + main_configs=["configs/listen_host.xml"], + user_configs=["configs/enable_hedged.xml"], + with_zookeeper=True, + ipv4_address="10.5.95.11", +) + +node2 = cluster.add_instance( + "node2", + main_configs=["configs/listen_host.xml"], + user_configs=["configs/enable_hedged.xml"], + with_zookeeper=True, + ipv4_address="10.5.95.12", +) + +# node1 - source with table, have invalid ipv6 +# node2 - destination, doing remote query +def test(cluster_without_dns_cache_update): + node1.query( + "CREATE TABLE test(t Date, label UInt8) ENGINE = MergeTree PARTITION BY t ORDER BY label;" + ) + node1.query( + "INSERT INTO test SELECT toDate('2022-12-28'), 1;" + ) + assert node1.query( + 'SELECT count(*) FROM test' + ) == '1\n' + + wrong_ip = '2001:3984:3989::1:1118' + + node2.exec_in_container( + (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(wrong_ip, node1.name)]) + ) + node2.exec_in_container( + (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(node1.ipv4_address, node1.name)]) + ) + + assert node1.query("SELECT count(*) from test") == "1\n" + node2.query("SYSTEM DROP DNS CACHE") + node1.query("SYSTEM DROP DNS CACHE") + assert node2.query(f"SELECT count(*) FROM remote('{node1.name}', default.test) limit 1;") == "1\n" + From fc94cc8b87fceb8b6631b72e34a6c10fdc197f83 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 7 Jul 2023 10:59:27 +0000 Subject: [PATCH 0443/2047] Update config for test --- .../test_async_connect_to_multiple_ips/configs/enable_hedged.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml index 238370176af..399d886ee6a 100644 --- a/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml +++ b/tests/integration/test_async_connect_to_multiple_ips/configs/enable_hedged.xml @@ -2,7 +2,6 @@ 1 - 0 From e618dd05cc73b7ad38296e7c28f66b6f077343f8 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 7 Jul 2023 13:03:44 +0200 Subject: [PATCH 0444/2047] Fix clang tidy and race --- programs/server/Server.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 16 ++++++++++++---- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 58cf3e5d210..4b47da9affb 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2285,6 +2285,7 @@ void Server::updateServers( Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : this->config(); std::vector all_servers; + all_servers.reserve(servers.size() + servers_to_start_before_tables.size()); for (auto & server : servers) all_servers.push_back(&server); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0f5a52b275c..504cf0326f0 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4890,14 +4890,23 @@ void StorageReplicatedMergeTree::shutdown() if (shutdown_called.exchange(true)) return; - if (!shutdown_prepared_called.load()) - flushAndPrepareForShutdown(); + flushAndPrepareForShutdown(); auto settings_ptr = getSettings(); if (!shutdown_deadline.has_value()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Shutdown deadline is not set in shutdown"); - waitForUniquePartsToBeFetchedByOtherReplicas(*shutdown_deadline); + try + { + waitForUniquePartsToBeFetchedByOtherReplicas(*shutdown_deadline); + } + catch (const Exception & ex) + { + if (ex.code() == ErrorCodes::LOGICAL_ERROR) + throw; + + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } session_expired_callback_handler.reset(); stopOutdatedDataPartsLoadingTask(); @@ -4905,7 +4914,6 @@ void StorageReplicatedMergeTree::shutdown() partialShutdown(); part_moves_between_shards_orchestrator.shutdown(); - background_operations_assignee.finish(); { auto lock = queue.lockQueue(); From 3dd9c09579887d5627a2486b3e0cddcc15b2487d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Jul 2023 11:25:48 +0000 Subject: [PATCH 0445/2047] Just save --- .../Optimizations/optimizePrewhere.cpp | 79 +++++++++++++++++++ .../MergeTreeBaseSelectProcessor.cpp | 2 + 2 files changed, 81 insertions(+) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index ca8a412bf2e..bcd3244b5a9 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -60,6 +61,74 @@ void matchDAGOutputNodesOrderWithHeader(ActionsDAGPtr & actions_dag, const Block namespace QueryPlanOptimizations { +#ifdef WHATEVERSOMETHING +static void removeAliases(ActionsDAG * dag) +{ + using Node = ActionsDAG::Node; + struct Frame + { + const ActionsDAG::Node * node; + const ActionsDAG::Node * parent; + size_t next_child = 0; + }; + std::vector stack; + std::vector> aliases; + + /// collect aliases + auto output_nodes = dag->getOutputs(); + for (const auto * output_node : output_nodes) + { + stack.push_back({output_node, nullptr}); + while (!stack.empty()) + { + auto & frame = stack.back(); + const auto * parent = frame.parent; + const auto * node = frame.node; + + if (frame.next_child < node->children.size()) + { + auto next_frame = Frame{.node = node->children[frame.next_child], .parent = node}; + ++frame.next_child; + stack.push_back(next_frame); + continue; + } + + if (parent && node->type == ActionsDAG::ActionType::ALIAS) + aliases.emplace_back(const_cast(node), const_cast(parent)); + + stack.pop_back(); + } + } + + /// remove aliases from output nodes if any + for(auto it = output_nodes.begin(); it != output_nodes.end();) + { + if ((*it)->type == ActionsDAG::ActionType::ALIAS) + it = output_nodes.erase(it); + else + ++it; + } + + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "aliases found: {}", aliases.size()); + + /// disconnect aliases + for(auto [alias, parent]: aliases) + { + /// find alias in parent's children and replace it with alias child + for (auto & child : parent->children) + { + if (child == alias) + { + child = alias->children.front(); + break; + } + } + } + + /// remove aliases + dag->removeUnusedActions(); +} +#endif void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) { @@ -162,6 +231,8 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) storage.supportedPrewhereColumns(), &Poco::Logger::get("QueryPlanOptimizePrewhere")}; + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "filter expression\n{}", filter_step->getExpression()->dumpDAG()); + auto optimize_result = where_optimizer.optimize(filter_step->getExpression(), filter_step->getFilterColumnName(), read_from_merge_tree->getContext(), @@ -178,6 +249,10 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) prewhere_info->need_filter = true; auto & prewhere_filter_actions = optimize_result->prewhere_filter_actions; + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "prewhere_filter_actions\n{}", prewhere_filter_actions->dumpDAG()); + + // removeAliases(prewhere_filter_actions.get()); + // LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "removeAliases\n{}", prewhere_filter_actions->dumpDAG()); ActionsChain actions_chain; @@ -260,7 +335,9 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) prewhere_info->prewhere_column_name = prewere_filter_node_name; prewhere_info->remove_prewhere_column = !prewhere_actions_chain_node->getChildRequiredOutputColumnsNames().contains(prewere_filter_node_name); + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "header BEFORE prewhere update\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); read_from_merge_tree->updatePrewhereInfo(prewhere_info); + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "header AFTER prewhere update\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); QueryPlan::Node * replace_old_filter_node = nullptr; bool remove_filter_node = false; @@ -321,10 +398,12 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) bool apply_match_step = false; + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "read header\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); /// If column order does not match old filter step column order, match dag output nodes with header if (!blocksHaveEqualStructure(read_from_merge_tree->getOutputStream().header, filter_step->getOutputStream().header)) { apply_match_step = true; + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "rename_actions_dag\n{}", rename_actions_dag->dumpDAG()); matchDAGOutputNodesOrderWithHeader(rename_actions_dag, filter_step->getOutputStream().header); } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 48adf36e678..d3d8c0f2bc8 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -644,6 +644,7 @@ Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const Prewher if (prewhere_info->prewhere_actions) { block = prewhere_info->prewhere_actions->updateHeader(std::move(block)); + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "updateHeader()\n{}", block.dumpStructure()); auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) @@ -654,6 +655,7 @@ Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const Prewher if (prewhere_info->remove_prewhere_column) { + LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "remove_column_name\n{}", prewhere_info->prewhere_column_name); block.erase(prewhere_info->prewhere_column_name); } else if (prewhere_info->need_filter) From 39d0b309bd730748b52acfb32de729e8f8496f83 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 7 Jul 2023 13:15:26 +0000 Subject: [PATCH 0446/2047] Make own function with slices --- src/Functions/HasSubsequenceImpl.h | 187 ++++++++---------- src/Functions/hasSubsequence.cpp | 2 +- .../hasSubsequenceCaseInsensitive.cpp | 2 +- .../hasSubsequenceCaseInsensitiveUTF8.cpp | 2 +- src/Functions/hasSubsequenceUTF8.cpp | 2 +- 5 files changed, 84 insertions(+), 111 deletions(-) diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h index bcb8e8e99e6..1396e64ade5 100644 --- a/src/Functions/HasSubsequenceImpl.h +++ b/src/Functions/HasSubsequenceImpl.h @@ -1,124 +1,109 @@ #pragma once + +#include +#include +#include +#include +#include namespace DB { namespace { -template -struct HasSubsequenceImpl -{ - using ResultType = UInt8; - static constexpr bool use_default_implementation_for_constants = false; - static constexpr bool supports_start_pos = false; +using namespace GatherUtils; + +template +class FunctionsHasSubsequenceImpl : public IFunction +{ +public: static constexpr auto name = Name::name; - static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {};} + static FunctionPtr create(ContextPtr) { return std::make_shared(); } - static void vectorConstant( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const String & needle, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - [[maybe_unused]] ColumnUInt8 * /*res_null*/) + String getName() const override { return name; } + + bool isVariadic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + size_t getNumberOfArguments() const override { return 2; } + + bool useDefaultImplementationForConstants() const override { return false; } + + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {};} + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (needle.empty()) - { - for (auto & r : res) - r = 1; - return; - } + if (!isString(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[0]->getName(), getName()); - ColumnString::Offset prev_haystack_offset = 0; - for (size_t i = 0; i < haystack_offsets.size(); ++i) - { - size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; - const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); - res[i] = hasSubsequence(haystack, haystack_size, needle.c_str(), needle.size()); - prev_haystack_offset = haystack_offsets[i]; - } + if (!isString(arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[1]->getName(), getName()); + + return std::make_shared>(); } - static void vectorVector( - const ColumnString::Chars & haystack_data, - const ColumnString::Offsets & haystack_offsets, - const ColumnString::Chars & needle_data, - const ColumnString::Offsets & needle_offsets, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - ColumnUInt8 * /*res_null*/) + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override { - ColumnString::Offset prev_haystack_offset = 0; - ColumnString::Offset prev_needle_offset = 0; + const ColumnPtr & column_haystack = arguments[0].column; + const ColumnPtr & column_needle = arguments[1].column; - size_t size = haystack_offsets.size(); + const ColumnConst * haystack_const_string = checkAndGetColumnConst(column_haystack.get()); + const ColumnConst * needle_const_string = checkAndGetColumnConst(column_needle.get()); + const ColumnString * haystack_string = checkAndGetColumn(&*column_haystack); + const ColumnString * needle_string = checkAndGetColumn(&*column_needle); - for (size_t i = 0; i < size; ++i) - { - size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1; + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(input_rows_count); - if (0 == needle_size) - { - res[i] = 1; - } - else - { - const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); - const char * haystack = reinterpret_cast(&haystack_data[prev_haystack_offset]); - res[i] = hasSubsequence(haystack, haystack_size, needle, needle_size); - } + if (haystack_string && needle_string) + execute(StringSource{*haystack_string}, StringSource{*needle_string}, vec_res); + else if (haystack_string && needle_const_string) + execute(StringSource{*haystack_string}, ConstSource{*needle_const_string}, vec_res); + else if (haystack_const_string && needle_string) + execute(ConstSource{*haystack_const_string}, StringSource{*needle_string}, vec_res); + else if (haystack_const_string && needle_const_string) + execute(ConstSource{*haystack_const_string}, ConstSource{*needle_const_string}, vec_res); + else + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {}, first argument of function {} must be a string", + arguments[0].column->getName(), + getName()); - prev_haystack_offset = haystack_offsets[i]; - prev_needle_offset = needle_offsets[i]; - } + return col_res; } - static void constantVector( - const String & haystack, - const ColumnString::Chars & needle_data, - const ColumnString::Offsets & needle_offsets, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - ColumnUInt8 * /*res_null*/) +private: + + template + void execute( + SourceHaystack && haystacks, + SourceNeedle && needles, + PaddedPODArray & res_data) const { - ColumnString::Offset prev_needle_offset = 0; + size_t row_num = 0; - size_t size = needle_offsets.size(); - - for (size_t i = 0; i < size; ++i) + while (!haystacks.isEnd()) { - size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; + [[maybe_unused]] auto haystack_slice = haystacks.getWhole(); + [[maybe_unused]] auto needle_slice = needles.getWhole(); - if (0 == needle_size) - { - res[i] = 1; - } - else - { - const char * needle = reinterpret_cast(&needle_data[prev_needle_offset]); - res[i] = hasSubsequence(haystack.c_str(), haystack.size(), needle, needle_size); - } - prev_needle_offset = needle_offsets[i]; - } - } + auto haystack = std::string(reinterpret_cast(haystack_slice.data), haystack_slice.size); + auto needle = std::string(reinterpret_cast(needle_slice.data), needle_slice.size); - static void constantConstant( - String haystack, - String needle, - const ColumnPtr & /*start_pos*/, - PaddedPODArray & res, - ColumnUInt8 * /*res_null*/) - { - size_t size = res.size(); - Impl::toLowerIfNeed(haystack); - Impl::toLowerIfNeed(needle); + Impl::toLowerIfNeed(haystack); + Impl::toLowerIfNeed(needle); - UInt8 result = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); - - for (size_t i = 0; i < size; ++i) - { - res[i] = result; + res_data[row_num] = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + haystacks.next(); + needles.next(); + ++row_num; } } @@ -130,18 +115,6 @@ struct HasSubsequenceImpl ++j; return j == needle_size; } - - template - static void vectorFixedConstant(Args &&...) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); - } - - template - static void vectorFixedVector(Args &&...) - { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); - } }; } diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp index bb1f295cee4..900e80f5524 100644 --- a/src/Functions/hasSubsequence.cpp +++ b/src/Functions/hasSubsequence.cpp @@ -18,7 +18,7 @@ struct NameHasSubsequence static constexpr auto name = "hasSubsequence"; }; -using FunctionHasSubsequence = FunctionsStringSearch>; +using FunctionHasSubsequence = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequence) diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp index fe50ada9be9..dbac62d7f09 100644 --- a/src/Functions/hasSubsequenceCaseInsensitive.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -17,7 +17,7 @@ struct NameHasSubsequenceCaseInsensitive static constexpr auto name = "hasSubsequenceCaseInsensitive"; }; -using FunctionHasSubsequenceCaseInsensitive = FunctionsStringSearch>; +using FunctionHasSubsequenceCaseInsensitive = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceCaseInsensitive) diff --git a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp index 2908c284a25..c104ff52857 100644 --- a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp @@ -17,7 +17,7 @@ struct NameHasSubsequenceCaseInsensitiveUTF8 static constexpr auto name = "hasSubsequenceCaseInsensitiveUTF8"; }; -using FunctionHasSubsequenceCaseInsensitiveUTF8 = FunctionsStringSearch>; +using FunctionHasSubsequenceCaseInsensitiveUTF8 = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceCaseInsensitiveUTF8) diff --git a/src/Functions/hasSubsequenceUTF8.cpp b/src/Functions/hasSubsequenceUTF8.cpp index c0811de6575..c67ce7d9c74 100644 --- a/src/Functions/hasSubsequenceUTF8.cpp +++ b/src/Functions/hasSubsequenceUTF8.cpp @@ -18,7 +18,7 @@ struct NameHasSubsequenceUTF8 static constexpr auto name = "hasSubsequenceUTF8"; }; -using FunctionHasSubsequenceUTF8 = FunctionsStringSearch>; +using FunctionHasSubsequenceUTF8 = FunctionsHasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceUTF8) From ed37b01b515ea204223dd03cee5482ee6faad351 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 7 Jul 2023 14:39:11 +0000 Subject: [PATCH 0447/2047] Fix style --- .../__init__.py | 0 .../test.py | 29 ++++++++++++------- 2 files changed, 18 insertions(+), 11 deletions(-) create mode 100644 tests/integration/test_async_connect_to_multiple_ips/__init__.py diff --git a/tests/integration/test_async_connect_to_multiple_ips/__init__.py b/tests/integration/test_async_connect_to_multiple_ips/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_async_connect_to_multiple_ips/test.py b/tests/integration/test_async_connect_to_multiple_ips/test.py index 0c18a316d4b..acc4d24d0fa 100644 --- a/tests/integration/test_async_connect_to_multiple_ips/test.py +++ b/tests/integration/test_async_connect_to_multiple_ips/test.py @@ -36,30 +36,37 @@ node2 = cluster.add_instance( ipv4_address="10.5.95.12", ) + # node1 - source with table, have invalid ipv6 # node2 - destination, doing remote query def test(cluster_without_dns_cache_update): node1.query( "CREATE TABLE test(t Date, label UInt8) ENGINE = MergeTree PARTITION BY t ORDER BY label;" ) - node1.query( - "INSERT INTO test SELECT toDate('2022-12-28'), 1;" - ) - assert node1.query( - 'SELECT count(*) FROM test' - ) == '1\n' - - wrong_ip = '2001:3984:3989::1:1118' + node1.query("INSERT INTO test SELECT toDate('2022-12-28'), 1;") + assert node1.query("SELECT count(*) FROM test") == "1\n" + + wrong_ip = "2001:3984:3989::1:1118" node2.exec_in_container( (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(wrong_ip, node1.name)]) ) node2.exec_in_container( - (["bash", "-c", "echo '{} {}' >> /etc/hosts".format(node1.ipv4_address, node1.name)]) + ( + [ + "bash", + "-c", + "echo '{} {}' >> /etc/hosts".format(node1.ipv4_address, node1.name), + ] + ) ) assert node1.query("SELECT count(*) from test") == "1\n" node2.query("SYSTEM DROP DNS CACHE") node1.query("SYSTEM DROP DNS CACHE") - assert node2.query(f"SELECT count(*) FROM remote('{node1.name}', default.test) limit 1;") == "1\n" - + assert ( + node2.query( + f"SELECT count(*) FROM remote('{node1.name}', default.test) limit 1;" + ) + == "1\n" + ) From eed1ecb6ba7ba4fdebd1c572881d064c66a0a102 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 7 Jul 2023 15:01:23 +0000 Subject: [PATCH 0448/2047] Revert "Remove parts in order for object storage always" This reverts commit c35294317dbff31b8ff8b48f6256162d6d5dc02e. --- src/Storages/MergeTree/MergeTreeData.cpp | 34 +++++++++--------------- 1 file changed, 12 insertions(+), 22 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0ef71895999..fa9bfd38a23 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2137,20 +2137,20 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force) /// Please don't use "zero-copy replication" (a non-production feature) in production. /// It is not ready for production usage. Don't use it. - /// It also is disabled for any object storage, because it can lead to race conditions on blob removal. - /// (see comment at `clearPartsFromFilesystemImpl`). - bool need_remove_parts_in_order = false; + bool need_remove_parts_in_order = supportsReplication() && getSettings()->allow_remote_fs_zero_copy_replication; - if (supportsReplication()) + if (need_remove_parts_in_order) { + bool has_zero_copy_disk = false; for (const auto & disk : getDisks()) { - if (disk->isRemote()) + if (disk->supportZeroCopyReplication()) { - need_remove_parts_in_order = true; + has_zero_copy_disk = true; break; } } + need_remove_parts_in_order = has_zero_copy_disk; } std::vector parts_to_delete; @@ -2394,28 +2394,18 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t std::mutex part_names_mutex; auto runner = threadPoolCallbackRunner(getPartsCleaningThreadPool().get(), "PartsCleaning"); - /** Straightforward concurrent parts removal can be applied for the case - * when we have parts on object storage disk + at least some of them were mutated - * (thus, can contains hardlinks to files in the previous parts). - * If we are deleting parts that contains hardlinks to the same file we may face into race condition - * and delete only local metadata files, but not the blobs on object storage. - * Given that, we remove in parallel only "independent" parts that don't have such hardlinks. - * Note that it also may be applicable for the regular MergeTree, fixed only for Replicated. - * - * To avoid this we need to fix race conditions on parts and blob removal. - */ + /// This flag disallow straightforward concurrent parts removal. It's required only in case + /// when we have parts on zero-copy disk + at least some of them were mutated. bool remove_parts_in_order = false; - if (dynamic_cast(this) != nullptr) + if (settings->allow_remote_fs_zero_copy_replication && dynamic_cast(this) != nullptr) { remove_parts_in_order = std::any_of( parts_to_remove.begin(), parts_to_remove.end(), - [] (const auto & data_part) - { - return data_part->isStoredOnRemoteDisk() && data_part->info.getMutationVersion() > 0; - } + [] (const auto & data_part) { return data_part->isStoredOnRemoteDiskWithZeroCopySupport() && data_part->info.getMutationVersion() > 0; } ); } + if (!remove_parts_in_order) { /// NOTE: Under heavy system load you may get "Cannot schedule a task" from ThreadPool. @@ -2451,7 +2441,7 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t /// NOTE: Under heavy system load you may get "Cannot schedule a task" from ThreadPool. LOG_DEBUG( - log, "Removing {} parts from filesystem (concurrently in order): Parts: [{}]", parts_to_remove.size(), fmt::join(parts_to_remove, ", ")); + log, "Removing {} parts from filesystem (concurrently): Parts: [{}]", parts_to_remove.size(), fmt::join(parts_to_remove, ", ")); /// We have "zero copy replication" parts and we are going to remove them in parallel. /// The problem is that all parts in a mutation chain must be removed sequentially to avoid "key does not exits" issues. From 227e415d6d71ca49b486052513786c5f050a6279 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 7 Jul 2023 15:08:21 +0000 Subject: [PATCH 0449/2047] Check refcount in `RemoveManyObjectStorageOperation::finalize` instead of `execute` --- .../DiskObjectStorageTransaction.cpp | 39 ++++++++++++------- src/Disks/ObjectStorages/IMetadataStorage.h | 5 ++- .../MetadataStorageFromDisk.cpp | 7 +++- .../ObjectStorages/MetadataStorageFromDisk.h | 5 ++- ...taStorageFromDiskTransactionOperations.cpp | 5 +++ ...dataStorageFromDiskTransactionOperations.h | 12 ++++++ .../MetadataStorageFromPlainObjectStorage.cpp | 5 ++- .../MetadataStorageFromPlainObjectStorage.h | 5 ++- 8 files changed, 63 insertions(+), 20 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index bd66ada492f..f3dbac445a5 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -6,6 +6,8 @@ #include #include +#include + namespace DB { @@ -150,7 +152,15 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati RemoveBatchRequest remove_paths; bool keep_all_batch_data; NameSet file_names_remove_metadata_only; - StoredObjects objects_to_remove; + + struct ObjectsToRemove + { + StoredObjects objects; + UnlinkMetadataFileOperationOutcomePtr unlink_outcome; + }; + + std::vector objects_to_remove; + bool remove_from_cache = false; RemoveManyObjectStorageOperation( @@ -174,7 +184,6 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati { for (const auto & [path, if_exists] : remove_paths) { - if (!metadata_storage.exists(path)) { if (if_exists) @@ -188,14 +197,12 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati try { - uint32_t hardlink_count = metadata_storage.getHardlinkCount(path); - auto objects = metadata_storage.getStorageObjects(path); - - tx->unlinkMetadata(path); - - /// File is really redundant - if (hardlink_count == 0 && !keep_all_batch_data && !file_names_remove_metadata_only.contains(fs::path(path).filename())) - std::move(objects.begin(), objects.end(), std::back_inserter(objects_to_remove)); + auto unlink_outcome = tx->unlinkMetadata(path); + if (unlink_outcome && !keep_all_batch_data && !file_names_remove_metadata_only.contains(fs::path(path).filename())) + { + auto objects = metadata_storage.getStorageObjects(path); + objects_to_remove.emplace_back(ObjectsToRemove{std::move(objects), std::move(unlink_outcome)}); + } } catch (const Exception & e) { @@ -215,15 +222,21 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati void undo() override { - } void finalize() override { + StoredObjects remove_from_remote; + for (auto && [objects, unlink_outcome] : objects_to_remove) + { + if (unlink_outcome->num_hardlinks == 0) + std::move(objects.begin(), objects.end(), std::back_inserter(remove_from_remote)); + } + /// Read comment inside RemoveObjectStorageOperation class /// TL;DR Don't pay any attention to 404 status code - if (!objects_to_remove.empty()) - object_storage.removeObjectsIfExist(objects_to_remove); + if (!remove_from_remote.empty()) + object_storage.removeObjectsIfExist(remove_from_remote); } }; diff --git a/src/Disks/ObjectStorages/IMetadataStorage.h b/src/Disks/ObjectStorages/IMetadataStorage.h index 00150df9fa3..264c481ee08 100644 --- a/src/Disks/ObjectStorages/IMetadataStorage.h +++ b/src/Disks/ObjectStorages/IMetadataStorage.h @@ -22,6 +22,8 @@ namespace ErrorCodes } class IMetadataStorage; +struct UnlinkMetadataFileOperationOutcome; +using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; /// Tries to provide some "transactions" interface, which allow /// to execute (commit) operations simultaneously. We don't provide @@ -127,9 +129,10 @@ public: /// Unlink metadata file and do something special if required /// By default just remove file (unlink file). - virtual void unlinkMetadata(const std::string & path) + virtual UnlinkMetadataFileOperationOutcomePtr unlinkMetadata(const std::string & path) { unlinkFile(path); + return nullptr; } virtual ~IMetadataTransaction() = default; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp index 9461a82845f..53428c2f6e1 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp @@ -340,9 +340,12 @@ void MetadataStorageFromDiskTransaction::addBlobToMetadata(const std::string & p addOperation(std::make_unique(path, blob_name, metadata_storage.object_storage_root_path, size_in_bytes, *metadata_storage.disk, metadata_storage)); } -void MetadataStorageFromDiskTransaction::unlinkMetadata(const std::string & path) +UnlinkMetadataFileOperationOutcomePtr MetadataStorageFromDiskTransaction::unlinkMetadata(const std::string & path) { - addOperation(std::make_unique(path, *metadata_storage.disk, metadata_storage)); + auto operation = std::make_unique(path, *metadata_storage.getDisk(), metadata_storage); + auto result = operation->outcome; + addOperation(std::move(operation)); + return result; } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h index 5273f0b041e..b518f5e3622 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h @@ -11,6 +11,9 @@ namespace DB { +struct UnlinkMetadataFileOperationOutcome; +using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; + /// Store metadata on a separate disk /// (used for object storages, like S3 and related). class MetadataStorageFromDisk final : public IMetadataStorage @@ -131,7 +134,7 @@ public: void replaceFile(const std::string & path_from, const std::string & path_to) override; - void unlinkMetadata(const std::string & path) override; + UnlinkMetadataFileOperationOutcomePtr unlinkMetadata(const std::string & path) override; }; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp index 7463622cb06..78e8764f8fc 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp @@ -319,6 +319,8 @@ void UnlinkMetadataFileOperation::execute(std::unique_lock & metada write_operation = std::make_unique(path, disk, metadata->serializeToString()); write_operation->execute(metadata_lock); } + outcome->num_hardlinks = ref_count; + unlink_operation = std::make_unique(path, disk); unlink_operation->execute(metadata_lock); } @@ -334,6 +336,9 @@ void UnlinkMetadataFileOperation::undo() if (write_operation) write_operation->undo(); + + /// Update outcome to reflect the fact that we have restored the file. + outcome->num_hardlinks++; } void SetReadonlyFileOperation::execute(std::unique_lock & metadata_lock) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h index d8e4892a0a5..4662ebc3423 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h @@ -3,6 +3,8 @@ #include #include +#include + namespace DB { class MetadataStorageFromDisk; @@ -242,9 +244,19 @@ private: std::unique_ptr write_operation; }; +/// Return the result of operation to the caller. +/// It is used in `IDiskObjectStorageOperation::finalize` after metadata transaction executed to make decision on blob removal. +struct UnlinkMetadataFileOperationOutcome +{ + UInt32 num_hardlinks = std::numeric_limits::max(); +}; + +using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; struct UnlinkMetadataFileOperation final : public IMetadataOperation { + const UnlinkMetadataFileOperationOutcomePtr outcome = std::make_shared(); + UnlinkMetadataFileOperation( const std::string & path_, IDisk & disk_, diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index c119e9f3adc..3650c7eaac8 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -135,9 +135,10 @@ void MetadataStorageFromPlainObjectStorageTransaction::addBlobToMetadata( { /// Noop, local metadata files is only one file, it is the metadata file itself. } -void MetadataStorageFromPlainObjectStorageTransaction::unlinkMetadata(const std::string &) + +UnlinkMetadataFileOperationOutcomePtr MetadataStorageFromPlainObjectStorageTransaction::unlinkMetadata(const std::string &) { - /// Noop, no separate metadata. + return nullptr; } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index fb5b6d0757c..bd068c1362f 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -9,6 +9,9 @@ namespace DB { +struct UnlinkMetadataFileOperationOutcome; +using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; + /// Object storage is used as a filesystem, in a limited form: /// - no directory concept, files only /// - no stat/chmod/... @@ -104,7 +107,7 @@ public: void unlinkFile(const std::string & path) override; - void unlinkMetadata(const std::string & path) override; + UnlinkMetadataFileOperationOutcomePtr unlinkMetadata(const std::string & path) override; void commit() override { From 88911e1378900d6687e05f08c6cbe592b5d32001 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 7 Jul 2023 16:42:03 +0000 Subject: [PATCH 0450/2047] Check refcount in finalize for other RemoveObjectStorageOperations --- .../DiskObjectStorageTransaction.cpp | 50 +++++++++---------- 1 file changed, 24 insertions(+), 26 deletions(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index f3dbac445a5..0ae577602b1 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -65,11 +65,18 @@ struct PureMetadataObjectStorageOperation final : public IDiskObjectStorageOpera std::string getInfoForLog() const override { return fmt::format("PureMetadataObjectStorageOperation"); } }; + +struct ObjectsToRemove +{ + StoredObjects objects; + UnlinkMetadataFileOperationOutcomePtr unlink_outcome; +}; + struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation { std::string path; bool delete_metadata_only; - StoredObjects objects_to_remove; + ObjectsToRemove objects_to_remove; bool if_exists; bool remove_from_cache = false; @@ -105,15 +112,12 @@ struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation try { - uint32_t hardlink_count = metadata_storage.getHardlinkCount(path); auto objects = metadata_storage.getStorageObjects(path); - tx->unlinkMetadata(path); + auto unlink_outcome = tx->unlinkMetadata(path); - if (hardlink_count == 0) - { - objects_to_remove = std::move(objects); - } + if (unlink_outcome) + objects_to_remove = ObjectsToRemove{std::move(objects), std::move(unlink_outcome)}; } catch (const Exception & e) { @@ -142,8 +146,11 @@ struct RemoveObjectStorageOperation final : public IDiskObjectStorageOperation /// due to network error or similar. And when it will retry an operation it may receive /// a 404 HTTP code. We don't want to threat this code as a real error for deletion process /// (e.g. throwing some exceptions) and thus we just use method `removeObjectsIfExists` - if (!delete_metadata_only && !objects_to_remove.empty()) - object_storage.removeObjectsIfExist(objects_to_remove); + if (!delete_metadata_only && !objects_to_remove.objects.empty() + && objects_to_remove.unlink_outcome->num_hardlinks == 0) + { + object_storage.removeObjectsIfExist(objects_to_remove.objects); + } } }; @@ -153,12 +160,6 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati bool keep_all_batch_data; NameSet file_names_remove_metadata_only; - struct ObjectsToRemove - { - StoredObjects objects; - UnlinkMetadataFileOperationOutcomePtr unlink_outcome; - }; - std::vector objects_to_remove; bool remove_from_cache = false; @@ -197,10 +198,10 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati try { + auto objects = metadata_storage.getStorageObjects(path); auto unlink_outcome = tx->unlinkMetadata(path); if (unlink_outcome && !keep_all_batch_data && !file_names_remove_metadata_only.contains(fs::path(path).filename())) { - auto objects = metadata_storage.getStorageObjects(path); objects_to_remove.emplace_back(ObjectsToRemove{std::move(objects), std::move(unlink_outcome)}); } } @@ -244,10 +245,9 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOperation { std::string path; - std::unordered_map objects_to_remove; + std::unordered_map objects_to_remove_by_path; bool keep_all_batch_data; NameSet file_names_remove_metadata_only; - StoredObjects objects_to_remove_from_cache; RemoveRecursiveObjectStorageOperation( IObjectStorage & object_storage_, @@ -274,14 +274,11 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp { try { - uint32_t hardlink_count = metadata_storage.getHardlinkCount(path_to_remove); auto objects_paths = metadata_storage.getStorageObjects(path_to_remove); - - tx->unlinkMetadata(path_to_remove); - - if (hardlink_count == 0) + auto unlink_outcome = tx->unlinkMetadata(path_to_remove); + if (unlink_outcome) { - objects_to_remove[path_to_remove] = std::move(objects_paths); + objects_to_remove_by_path[path_to_remove] = ObjectsToRemove{std::move(objects_paths), std::move(unlink_outcome)}; } } catch (const Exception & e) @@ -331,11 +328,12 @@ struct RemoveRecursiveObjectStorageOperation final : public IDiskObjectStorageOp if (!keep_all_batch_data) { StoredObjects remove_from_remote; - for (auto && [local_path, remote_paths] : objects_to_remove) + for (auto && [local_path, objects_to_remove] : objects_to_remove_by_path) { if (!file_names_remove_metadata_only.contains(fs::path(local_path).filename())) { - std::move(remote_paths.begin(), remote_paths.end(), std::back_inserter(remove_from_remote)); + if (objects_to_remove.unlink_outcome->num_hardlinks == 0) + std::move(objects_to_remove.objects.begin(), objects_to_remove.objects.end(), std::back_inserter(remove_from_remote)); } } /// Read comment inside RemoveObjectStorageOperation class From 39a440fa0e99849d710e09bd031de5a52708fd6f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Jul 2023 22:08:31 +0000 Subject: [PATCH 0451/2047] Build sort description based on original header --- .../QueryPlan/ReadFromMergeTree.cpp | 41 +++++++++++++++---- 1 file changed, 34 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 32bfa6935df..3d1e2650188 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -139,17 +139,42 @@ static bool checkAllPartsOnRemoteFS(const RangesInDataParts & parts) /// build sort description for output stream static void updateSortDescriptionForOutputStream( - DataStream & output_stream, const Names & sorting_key_columns, const int sort_direction, InputOrderInfoPtr input_order_info) + DataStream & output_stream, const Names & sorting_key_columns, const int sort_direction, InputOrderInfoPtr input_order_info, PrewhereInfoPtr prewhere_info) { + Block original_header = output_stream.header.cloneEmpty(); + /// build original header + if (prewhere_info && prewhere_info->prewhere_actions) + { + FindOriginalNodeForOutputName original_column_finder(prewhere_info->prewhere_actions); + + for (auto & column : original_header) + { + const auto * original_node = original_column_finder.find(column.name); + if (original_node) + { + LOG_DEBUG( + &Poco::Logger::get(__PRETTY_FUNCTION__), + "Found original column '{}' for '{}'", + original_node->result_name, + column.name); + column.name = original_node->result_name; + } + } + } + SortDescription sort_description; const Block & header = output_stream.header; - for (const auto & column_name : sorting_key_columns) + for (const auto & sorting_key : sorting_key_columns) { - if (std::find_if(header.begin(), header.end(), [&](ColumnWithTypeAndName const & col) { return col.name == column_name; }) - == header.end()) + const auto it = std::find_if( + original_header.begin(), original_header.end(), [&sorting_key](const auto & column) { return column.name == sorting_key; }); + if (it == original_header.end()) break; - sort_description.emplace_back(column_name, sort_direction); + + const size_t column_pos = std::distance(original_header.begin(), it); + sort_description.emplace_back((header.begin() + column_pos)->name, sort_direction); } + if (!sort_description.empty()) { if (input_order_info) @@ -283,7 +308,8 @@ ReadFromMergeTree::ReadFromMergeTree( *output_stream, storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), getSortDirection(), - query_info.getInputOrderInfo()); + query_info.getInputOrderInfo(), + prewhere_info); } @@ -1575,7 +1601,8 @@ void ReadFromMergeTree::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info *output_stream, storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(), getSortDirection(), - query_info.getInputOrderInfo()); + query_info.getInputOrderInfo(), + prewhere_info); } bool ReadFromMergeTree::requestOutputEachPartitionThroughSeparatePort() From 8e4c8f118cf64fcd77524439508b838c05a58fcf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 8 Jul 2023 09:07:05 +0200 Subject: [PATCH 0452/2047] Fix disaster in integration tests, part 2 --- tests/integration/ci-runner.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index d6d17abe725..43184574e6e 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -406,9 +406,9 @@ class ClickhouseIntegrationTestsRunner: out_file_full = os.path.join(self.result_path, "runner_get_all_tests.log") cmd = ( "cd {repo_path}/tests/integration && " - "timeout -s 9 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' " - "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " - "| grep -v 'SKIPPED' | sort -u > {out_file}".format( + "timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' " + "| tee '{out_file_full}' | grep -F '::' | sed -r 's/ \(fixtures used:.*//g; s/^ *//g; s/ *$//g' " + "| grep -v -F 'SKIPPED' | sort --unique > {out_file}".format( repo_path=repo_path, runner_opts=self._get_runner_opts(), image_cmd=image_cmd, @@ -626,7 +626,7 @@ class ClickhouseIntegrationTestsRunner: info_basename = test_group_str + "_" + str(i) + ".nfo" info_path = os.path.join(repo_path, "tests/integration", info_basename) - test_cmd = " ".join([test for test in sorted(test_names)]) + test_cmd = " ".join([f"'{test}'" for test in sorted(test_names)]) parallel_cmd = ( " --parallel {} ".format(num_workers) if num_workers > 0 else "" ) From 62bfa4ed93fb3796eccb0df041a9dfa057583c9b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Jul 2023 02:21:48 +0200 Subject: [PATCH 0453/2047] Fix performance test for regexp cache --- src/Functions/Regexps.h | 4 +++- tests/performance/re2_regex_caching.xml | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Functions/Regexps.h b/src/Functions/Regexps.h index 4bfd10bdbf5..aa8ae5b4054 100644 --- a/src/Functions/Regexps.h +++ b/src/Functions/Regexps.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -21,6 +22,7 @@ # include #endif + namespace ProfileEvents { extern const Event RegexpCreated; @@ -86,7 +88,7 @@ public: private: constexpr static size_t CACHE_SIZE = 100; /// collision probability - std::hash hasher; + DefaultHash hasher; struct Bucket { String pattern; /// key diff --git a/tests/performance/re2_regex_caching.xml b/tests/performance/re2_regex_caching.xml index 6edc83097ba..9778a8d4c0c 100644 --- a/tests/performance/re2_regex_caching.xml +++ b/tests/performance/re2_regex_caching.xml @@ -24,8 +24,8 @@ '.*' || toString(number) || '.' '.*' || toString(number % 10) || '.' - - '([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number) + + '([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number % 10) + + s3 + s3_common_disk/ + http://localhost:11111/test/common/ + clickhouse + clickhouse + 20000 + s3 s3_disk/ diff --git a/tests/queries/0_stateless/02801_backup_native_copy.reference b/tests/queries/0_stateless/02801_backup_native_copy.reference new file mode 100644 index 00000000000..659df5e9b25 --- /dev/null +++ b/tests/queries/0_stateless/02801_backup_native_copy.reference @@ -0,0 +1,4 @@ +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 +RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 +RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 diff --git a/tests/queries/0_stateless/02801_backup_native_copy.sh b/tests/queries/0_stateless/02801_backup_native_copy.sh new file mode 100755 index 00000000000..966d7ae9ce8 --- /dev/null +++ b/tests/queries/0_stateless/02801_backup_native_copy.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag: no-fasttest - requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists data; + create table data (key Int) engine=MergeTree() order by tuple() settings disk='s3_common_disk'; + insert into data select * from numbers(10); +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" From 5835e72fd6d5dd0225a0dda2f81887d6f61015fb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 27 Jun 2023 16:20:27 +0200 Subject: [PATCH 0460/2047] More generic approach to disable native copy Previous patch implements this only for BACKUP/RESTORE, but it can be useful for regular disks as well, so add allow_native_copy for disks. Note, that there is s3_allow_native_copy query setting, since it looks redundant, since it make sense only for S3 disks, and not on a per query basis. Signed-off-by: Azat Khuzhin --- src/Backups/BackupIO_S3.cpp | 34 +++++++++++++------ src/Backups/BackupIO_S3.h | 6 ++-- .../ObjectStorages/S3/S3ObjectStorage.cpp | 14 ++++++-- src/IO/S3/copyS3File.cpp | 22 +++++++++++- src/IO/S3/copyS3File.h | 20 +++++++++-- src/Storages/StorageS3Settings.cpp | 2 ++ src/Storages/StorageS3Settings.h | 1 + 7 files changed, 80 insertions(+), 19 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 01e6bc78949..6531948c872 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -101,15 +101,16 @@ namespace BackupReaderS3::BackupReaderS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) : BackupReaderDefault(&Poco::Logger::get("BackupReaderS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} - , native_copy(native_copy_) { + request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint + request_settings.allow_native_copy = native_copy; } BackupReaderS3::~BackupReaderS3() = default; @@ -139,11 +140,10 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto destination_data_source_description = destination_disk->getDataSourceDescription(); - if (native_copy && destination_data_source_description.sameKind(data_source_description) + if (destination_data_source_description.sameKind(data_source_description) && (destination_data_source_description.is_encrypted == encrypted_in_backup)) { - /// Use native copy, the more optimal way. - LOG_TRACE(log, "Copying {} from S3 to disk {} using native copy", path_in_backup, destination_disk->getName()); + LOG_TRACE(log, "Copying {} from S3 to disk {}", path_in_backup, destination_disk->getName()); auto write_blob_function = [&](const Strings & blob_path, WriteMode mode, const std::optional & object_attributes) -> size_t { /// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files. @@ -152,7 +152,13 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s "Blob writing function called with unexpected blob_path.size={} or mode={}", blob_path.size(), mode); + auto create_read_buffer = [this, path_in_backup] + { + return readFile(path_in_backup); + }; + copyS3File( + create_read_buffer, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, @@ -178,16 +184,16 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s BackupWriterS3::BackupWriterS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} - , native_copy(native_copy_) { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint + request_settings.allow_native_copy = native_copy; } void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, @@ -196,15 +202,23 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto source_data_source_description = src_disk->getDataSourceDescription(); - if (native_copy && source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) + if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) { /// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in S3 bucket. /// In this case we can't use the native copy. if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) { - /// Use native copy, the more optimal way. - LOG_TRACE(log, "Copying file {} from disk {} to S3 using native copy", src_path, src_disk->getName()); + auto create_read_buffer = [src_disk, src_path, copy_encrypted, settings = read_settings.adjustBufferSize(start_pos + length)] + { + if (copy_encrypted) + return src_disk->readEncryptedFile(src_path, settings); + else + return src_disk->readFile(src_path, settings); + }; + + LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName()); copyS3File( + create_read_buffer, client, /* src_bucket */ blob_path[1], /* src_key= */ blob_path[0], diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index d02e45370f9..16b2abfea3d 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -17,7 +17,7 @@ namespace DB class BackupReaderS3 : public BackupReaderDefault { public: - BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_); + BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); ~BackupReaderS3() override; bool fileExists(const String & file_name) override; @@ -32,14 +32,13 @@ private: const std::shared_ptr client; S3Settings::RequestSettings request_settings; const DataSourceDescription data_source_description; - const bool native_copy; }; class BackupWriterS3 : public BackupWriterDefault { public: - BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_); + BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); ~BackupWriterS3() override; bool fileExists(const String & file_name) override; @@ -62,7 +61,6 @@ private: S3Settings::RequestSettings request_settings; std::optional supports_batch_delete; const DataSourceDescription data_source_description; - const bool native_copy; }; } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index e46ca3d0828..3c19af188dc 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -435,7 +435,12 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, + auto create_read_buffer = [this, object_from] + { + return readObject(object_from); + }; + + copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } else @@ -451,7 +456,12 @@ void S3ObjectStorage::copyObject( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, + auto create_read_buffer = [this, object_from] + { + return readObject(object_from); + }; + + copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 7886b84cd00..3f18d3b2145 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -809,7 +809,7 @@ void copyDataToS3File( } -void copyS3File( +void copyS3FileNative( const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -826,6 +826,26 @@ void copyS3File( helper.performCopy(); } +void copyS3File( + const CreateReadBuffer & create_read_buffer, + const std::shared_ptr & s3_client, + const String & src_bucket, + const String & src_key, + size_t src_offset, + size_t src_size, + const String & dest_bucket, + const String & dest_key, + const S3Settings::RequestSettings & settings, + const std::optional> & object_metadata, + ThreadPoolCallbackRunner schedule, + bool for_disk_s3) +{ + if (settings.allow_native_copy) + copyS3FileNative(s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + else + copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); +} + } #endif diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 618ef419a9b..d41f34c103c 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -19,9 +19,9 @@ using CreateReadBuffer = std::function()>; /// Copies a file from S3 to S3. /// The same functionality can be done by using the function copyData() and the classes ReadBufferFromS3 and WriteBufferFromS3 -/// however copyS3File() is faster and spends less network traffic and memory. +/// however copyS3FileNative() is faster and spends less network traffic and memory. /// The parameters `src_offset` and `src_size` specify a part in the source to copy. -void copyS3File( +void copyS3FileNative( const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -51,6 +51,22 @@ void copyDataToS3File( ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_s3 = false); +/// Tries to copy file using native copy (copyS3FileNative()), if this is not +/// possible it will fallback to read-write copy (copyDataToS3File()) +void copyS3File( + const CreateReadBuffer & create_read_buffer, + const std::shared_ptr & s3_client, + const String & src_bucket, + const String & src_key, + size_t src_offset, + size_t src_size, + const String & dest_bucket, + const String & dest_key, + const S3Settings::RequestSettings & settings, + const std::optional> & object_metadata = std::nullopt, + ThreadPoolCallbackRunner schedule_ = {}, + bool for_disk_s3 = false); + } #endif diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 89e6ee46b4d..0dc8d8d897b 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -182,6 +182,7 @@ S3Settings::RequestSettings::RequestSettings(const NamedCollection & collection) max_single_read_retries = collection.getOrDefault("max_single_read_retries", max_single_read_retries); max_connections = collection.getOrDefault("max_connections", max_connections); list_object_keys_size = collection.getOrDefault("list_object_keys_size", list_object_keys_size); + allow_native_copy = collection.getOrDefault("allow_native_copy", allow_native_copy); throw_on_zero_files_match = collection.getOrDefault("throw_on_zero_files_match", throw_on_zero_files_match); } @@ -197,6 +198,7 @@ S3Settings::RequestSettings::RequestSettings( max_connections = config.getUInt64(key + "max_connections", settings.s3_max_connections); check_objects_after_upload = config.getBool(key + "check_objects_after_upload", settings.s3_check_objects_after_upload); list_object_keys_size = config.getUInt64(key + "list_object_keys_size", settings.s3_list_object_keys_size); + allow_native_copy = config.getBool(key + "allow_native_copy", allow_native_copy); throw_on_zero_files_match = config.getBool(key + "throw_on_zero_files_match", settings.s3_throw_on_zero_files_match); retry_attempts = config.getUInt64(key + "retry_attempts", settings.s3_retry_attempts); request_timeout_ms = config.getUInt64(key + "request_timeout_ms", settings.s3_request_timeout_ms); diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 991e323acb6..581665a7dc5 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -71,6 +71,7 @@ struct S3Settings size_t retry_attempts = 10; size_t request_timeout_ms = 3000; size_t long_request_timeout_ms = 30000; // TODO: Take this from config like request_timeout_ms + bool allow_native_copy = true; bool throw_on_zero_files_match = false; From 5d63b8be0d317af7b2ee1fdfd7dc76daeeec3afd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 27 Jun 2023 16:48:30 +0200 Subject: [PATCH 0461/2047] Add a test for allow_native_copy using clickhouse-disks (first ever) Signed-off-by: Azat Khuzhin --- .../02802_clickhouse_disks_s3_copy.reference | 4 +++ .../02802_clickhouse_disks_s3_copy.sh | 26 +++++++++++++++++++ .../02802_clickhouse_disks_s3_copy.xml | 21 +++++++++++++++ 3 files changed, 51 insertions(+) create mode 100644 tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference create mode 100755 tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh create mode 100644 tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference new file mode 100644 index 00000000000..96860a2f90a --- /dev/null +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference @@ -0,0 +1,4 @@ +s3_plain_native_copy +Single operation copy has completed. +s3_plain_no_native_copy +Single part upload has completed. diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh new file mode 100755 index 00000000000..f879b7a5621 --- /dev/null +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +config="${BASH_SOURCE[0]/.sh/.xml}" + +function run_test_for_disk() +{ + local disk=$1 && shift + + echo "$disk" + + clickhouse-disks -C "$config" --disk "$disk" write --input "$config" $CLICKHOUSE_DATABASE/test + clickhouse-disks -C "$config" --log-level test --disk "$disk" copy $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy |& { + grep -o -e "Single part upload has completed." -e "Single operation copy has completed." + } + clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test + clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test.copy +} + +run_test_for_disk s3_plain_native_copy +run_test_for_disk s3_plain_no_native_copy diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml new file mode 100644 index 00000000000..d4235a70903 --- /dev/null +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml @@ -0,0 +1,21 @@ + + + + + s3_plain + http://localhost:11111/test/clickhouse-disks/ + clickhouse + clickhouse + true + + + + s3_plain + http://localhost:11111/test/clickhouse-disks/ + clickhouse + clickhouse + false + + + + From 84c720b33e9ffe44c79658af57f5985b38b8a728 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 10:52:00 +0200 Subject: [PATCH 0462/2047] Beatify test_backup_restore_s3 (using per-query profile events) Signed-off-by: Azat Khuzhin --- .../test_backup_restore_s3/test.py | 148 +++++++++--------- 1 file changed, 75 insertions(+), 73 deletions(-) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 0285500d044..bb14fa4824b 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -2,6 +2,7 @@ from typing import Dict, Iterable import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV +import uuid cluster = ClickHouseCluster(__file__) @@ -37,32 +38,31 @@ def new_backup_name(): return f"backup{backup_id_counter}" -def get_events(events_names: Iterable[str]) -> Dict[str, int]: - _events = TSV( +def get_events_for_query(query_id: str) -> Dict[str, int]: + events = TSV( node.query( - f"SELECT event, value FROM system.events WHERE event in {events_names} SETTINGS system_events_show_zero_values = 1;" + f""" + SYSTEM FLUSH LOGS; + + WITH arrayJoin(ProfileEvents) as pe + SELECT pe.1, pe.2 + FROM system.query_log + WHERE query_id = '{query_id}' + """ ) ) return { event: int(value) - for event, value in [line.split("\t") for line in _events.lines] + for event, value in [line.split("\t") for line in events.lines] } def check_backup_and_restore( - storage_policy, backup_destination, size=1000, backup_name=None, check_events=False + storage_policy, + backup_destination, + size=1000, + backup_name=None, ): - s3_backup_events = ( - "WriteBufferFromS3Microseconds", - "WriteBufferFromS3Bytes", - "WriteBufferFromS3RequestsErrors", - ) - s3_restore_events = ( - "ReadBufferFromS3Microseconds", - "ReadBufferFromS3Bytes", - "ReadBufferFromS3RequestsErrors", - ) - node.query( f""" DROP TABLE IF EXISTS data SYNC; @@ -72,16 +72,17 @@ def check_backup_and_restore( """ ) try: - events_before_backups = get_events(s3_backup_events) - node.query(f"BACKUP TABLE data TO {backup_destination}") - events_after_backups = get_events(s3_backup_events) - events_before_restore = get_events(s3_restore_events) + backup_query_id = uuid.uuid4().hex + node.query( + f"BACKUP TABLE data TO {backup_destination}", query_id=backup_query_id + ) + restore_query_id = uuid.uuid4().hex node.query( f""" RESTORE TABLE data AS data_restored FROM {backup_destination}; - """ + """, + query_id=restore_query_id, ) - events_after_restore = get_events(s3_restore_events) node.query( """ SELECT throwIf( @@ -91,55 +92,10 @@ def check_backup_and_restore( ); """ ) - if check_events and backup_name: - objects = node.cluster.minio_client.list_objects( - "root", f"data/backups/multipart/{backup_name}/" - ) - backup_meta_size = 0 - for obj in objects: - if ".backup" in obj.object_name: - backup_meta_size = obj.size - break - backup_total_size = int( - node.query( - f"SELECT sum(total_size) FROM system.backups WHERE status = 'BACKUP_CREATED' AND name like '%{backup_name}%'" - ).strip() - ) - restore_total_size = int( - node.query( - f"SELECT sum(total_size) FROM system.backups WHERE status = 'RESTORED' AND name like '%{backup_name}%'" - ).strip() - ) - # backup - # NOTE: ~35 bytes is used by .lock file, so set up 100 bytes to avoid flaky test - assert ( - abs( - backup_total_size - - ( - events_after_backups["WriteBufferFromS3Bytes"] - - events_before_backups["WriteBufferFromS3Bytes"] - - backup_meta_size - ) - ) - < 100 - ) - assert ( - events_after_backups["WriteBufferFromS3Microseconds"] - > events_before_backups["WriteBufferFromS3Microseconds"] - ) - assert events_after_backups["WriteBufferFromS3RequestsErrors"] == 0 - # restore - assert ( - events_after_restore["ReadBufferFromS3Bytes"] - - events_before_restore["ReadBufferFromS3Bytes"] - - backup_meta_size - == restore_total_size - ) - assert ( - events_after_restore["ReadBufferFromS3Microseconds"] - > events_before_restore["ReadBufferFromS3Microseconds"] - ) - assert events_after_restore["ReadBufferFromS3RequestsErrors"] == 0 + return [ + get_events_for_query(backup_query_id), + get_events_for_query(restore_query_id), + ] finally: node.query( """ @@ -224,17 +180,63 @@ def test_backup_to_s3_multipart(): storage_policy = "default" backup_name = new_backup_name() backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')" - check_backup_and_restore( + (backup_events, restore_events) = check_backup_and_restore( storage_policy, backup_destination, size=1000000, backup_name=backup_name, - check_events=True, ) assert node.contains_in_log( f"copyDataToS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}" ) + s3_backup_events = ( + "WriteBufferFromS3Microseconds", + "WriteBufferFromS3Bytes", + "WriteBufferFromS3RequestsErrors", + ) + s3_restore_events = ( + "ReadBufferFromS3Microseconds", + "ReadBufferFromS3Bytes", + "ReadBufferFromS3RequestsErrors", + ) + + objects = node.cluster.minio_client.list_objects( + "root", f"data/backups/multipart/{backup_name}/" + ) + backup_meta_size = 0 + for obj in objects: + if ".backup" in obj.object_name: + backup_meta_size = obj.size + break + backup_total_size = int( + node.query( + f"SELECT sum(total_size) FROM system.backups WHERE status = 'BACKUP_CREATED' AND name like '%{backup_name}%'" + ).strip() + ) + restore_total_size = int( + node.query( + f"SELECT sum(total_size) FROM system.backups WHERE status = 'RESTORED' AND name like '%{backup_name}%'" + ).strip() + ) + # backup + # NOTE: ~35 bytes is used by .lock file, so set up 100 bytes to avoid flaky test + assert ( + abs( + backup_total_size + - (backup_events["WriteBufferFromS3Bytes"] - backup_meta_size) + ) + < 100 + ) + assert backup_events["WriteBufferFromS3Microseconds"] > 0 + assert "WriteBufferFromS3RequestsErrors" not in backup_events + # restore + assert ( + restore_events["ReadBufferFromS3Bytes"] - backup_meta_size == restore_total_size + ) + assert restore_events["ReadBufferFromS3Microseconds"] > 0 + assert "ReadBufferFromS3RequestsErrors" not in restore_events + def test_backup_to_s3_native_copy(): storage_policy = "policy_s3" From 29dc9abfcab495f66689826fdbb8ee7a81ab4c7d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 10:58:23 +0200 Subject: [PATCH 0463/2047] Fix test_backup_restore_s3 after logging for native copying changed Check profile events instead of some odd logs. Signed-off-by: Azat Khuzhin --- .../test_backup_restore_s3/test.py | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index bb14fa4824b..8701bf0d832 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -244,9 +244,12 @@ def test_backup_to_s3_native_copy(): backup_destination = ( f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) - check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("BackupWriterS3.*using native copy") - assert node.contains_in_log("BackupReaderS3.*using native copy") + (backup_events, restore_events) = check_backup_and_restore( + storage_policy, backup_destination + ) + # single part upload + assert backup_events["S3CopyObject"] > 0 + assert restore_events["S3CopyObject"] > 0 assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" ) @@ -258,9 +261,12 @@ def test_backup_to_s3_native_copy_other_bucket(): backup_destination = ( f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) - check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("BackupWriterS3.*using native copy") - assert node.contains_in_log("BackupReaderS3.*using native copy") + (backup_events, restore_events) = check_backup_and_restore( + storage_policy, backup_destination + ) + # single part upload + assert backup_events["S3CopyObject"] > 0 + assert restore_events["S3CopyObject"] > 0 assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" ) @@ -270,9 +276,12 @@ def test_backup_to_s3_native_copy_multipart(): storage_policy = "policy_s3" backup_name = new_backup_name() backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')" - check_backup_and_restore(storage_policy, backup_destination, size=1000000) - assert node.contains_in_log("BackupWriterS3.*using native copy") - assert node.contains_in_log("BackupReaderS3.*using native copy") + (backup_events, restore_events) = check_backup_and_restore( + storage_policy, backup_destination, size=1000000 + ) + # multi part upload + assert backup_events["S3CreateMultipartUpload"] > 0 + assert restore_events["S3CreateMultipartUpload"] > 0 assert node.contains_in_log( f"copyS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}/" ) From 1590ffa3b1eee26d66ae3aec3ac32c63acdea153 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 17:22:57 +0200 Subject: [PATCH 0464/2047] Remove copyS3FileNative() Signed-off-by: Azat Khuzhin --- src/IO/S3/copyS3File.cpp | 22 ++++------------------ src/IO/S3/copyS3File.h | 25 +++++++------------------ 2 files changed, 11 insertions(+), 36 deletions(-) diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 3f18d3b2145..2c6557d97e7 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -809,23 +809,6 @@ void copyDataToS3File( } -void copyS3FileNative( - const std::shared_ptr & s3_client, - const String & src_bucket, - const String & src_key, - size_t src_offset, - size_t src_size, - const String & dest_bucket, - const String & dest_key, - const S3Settings::RequestSettings & settings, - const std::optional> & object_metadata, - ThreadPoolCallbackRunner schedule, - bool for_disk_s3) -{ - CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; - helper.performCopy(); -} - void copyS3File( const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, @@ -841,7 +824,10 @@ void copyS3File( bool for_disk_s3) { if (settings.allow_native_copy) - copyS3FileNative(s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + { + CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; + helper.performCopy(); + } else copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index d41f34c103c..2c848076e9b 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -19,9 +19,14 @@ using CreateReadBuffer = std::function()>; /// Copies a file from S3 to S3. /// The same functionality can be done by using the function copyData() and the classes ReadBufferFromS3 and WriteBufferFromS3 -/// however copyS3FileNative() is faster and spends less network traffic and memory. +/// however copyS3File() is faster and spends less network traffic and memory. /// The parameters `src_offset` and `src_size` specify a part in the source to copy. -void copyS3FileNative( +/// +/// Note, that it tries to copy file using native copy (CopyObject), but if it +/// has been disabled (with settings.allow_native_copy) it is fallbacks to +/// read-write copy (copyDataToS3File()). +void copyS3File( + const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -51,22 +56,6 @@ void copyDataToS3File( ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_s3 = false); -/// Tries to copy file using native copy (copyS3FileNative()), if this is not -/// possible it will fallback to read-write copy (copyDataToS3File()) -void copyS3File( - const CreateReadBuffer & create_read_buffer, - const std::shared_ptr & s3_client, - const String & src_bucket, - const String & src_key, - size_t src_offset, - size_t src_size, - const String & dest_bucket, - const String & dest_key, - const S3Settings::RequestSettings & settings, - const std::optional> & object_metadata = std::nullopt, - ThreadPoolCallbackRunner schedule_ = {}, - bool for_disk_s3 = false); - } #endif From 559d3281782c22fa380e85e188d2a15e404a4c19 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 17:16:02 +0200 Subject: [PATCH 0465/2047] Rename BACKUP setting native_copy to allow_s3_native_copy Signed-off-by: Azat Khuzhin --- src/Backups/BackupFactory.h | 2 +- src/Backups/BackupIO_S3.cpp | 8 ++++---- src/Backups/BackupIO_S3.h | 4 ++-- src/Backups/BackupSettings.cpp | 2 +- src/Backups/BackupSettings.h | 2 +- src/Backups/BackupsWorker.cpp | 4 ++-- src/Backups/RestoreSettings.cpp | 2 +- src/Backups/RestoreSettings.h | 2 +- src/Backups/registerBackupEngineS3.cpp | 4 ++-- .../0_stateless/02801_backup_native_copy.reference | 8 ++++---- tests/queries/0_stateless/02801_backup_native_copy.sh | 8 ++++---- 11 files changed, 23 insertions(+), 23 deletions(-) diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index 642f5cb07b9..e95aeddb086 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -35,7 +35,7 @@ public: std::shared_ptr backup_coordination; std::optional backup_uuid; bool deduplicate_files = true; - bool native_copy = true; + bool allow_s3_native_copy = true; }; static BackupFactory & instance(); diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 6531948c872..60fea9e2008 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -101,7 +101,7 @@ namespace BackupReaderS3::BackupReaderS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_) : BackupReaderDefault(&Poco::Logger::get("BackupReaderS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) @@ -110,7 +110,7 @@ BackupReaderS3::BackupReaderS3( { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint - request_settings.allow_native_copy = native_copy; + request_settings.allow_native_copy = allow_s3_native_copy; } BackupReaderS3::~BackupReaderS3() = default; @@ -184,7 +184,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s BackupWriterS3::BackupWriterS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_) : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) @@ -193,7 +193,7 @@ BackupWriterS3::BackupWriterS3( { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint - request_settings.allow_native_copy = native_copy; + request_settings.allow_native_copy = allow_s3_native_copy; } void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 16b2abfea3d..a93d6119786 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -17,7 +17,7 @@ namespace DB class BackupReaderS3 : public BackupReaderDefault { public: - BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); + BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_); ~BackupReaderS3() override; bool fileExists(const String & file_name) override; @@ -38,7 +38,7 @@ private: class BackupWriterS3 : public BackupWriterDefault { public: - BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); + BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_); ~BackupWriterS3() override; bool fileExists(const String & file_name) override; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 8e9fe7956f9..b6d776d0347 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes M(Bool, async) \ M(Bool, decrypt_files_from_encrypted_disks) \ M(Bool, deduplicate_files) \ - M(Bool, native_copy) \ + M(Bool, allow_s3_native_copy) \ M(UInt64, shard_num) \ M(UInt64, replica_num) \ M(Bool, internal) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index e21b70ee25f..7cec2d9693d 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -39,7 +39,7 @@ struct BackupSettings bool deduplicate_files = true; /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) - bool native_copy = true; + bool allow_s3_native_copy = true; /// 1-based shard index to store in the backup. 0 means all shards. /// Can only be used with BACKUP ON CLUSTER. diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index fddd4f34bb6..c08b110075e 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -348,7 +348,7 @@ void BackupsWorker::doBackup( backup_create_params.backup_coordination = backup_coordination; backup_create_params.backup_uuid = backup_settings.backup_uuid; backup_create_params.deduplicate_files = backup_settings.deduplicate_files; - backup_create_params.native_copy = backup_settings.native_copy; + backup_create_params.allow_s3_native_copy = backup_settings.allow_s3_native_copy; BackupMutablePtr backup = BackupFactory::instance().createBackup(backup_create_params); /// Write the backup. @@ -648,7 +648,7 @@ void BackupsWorker::doRestore( backup_open_params.backup_info = backup_info; backup_open_params.base_backup_info = restore_settings.base_backup_info; backup_open_params.password = restore_settings.password; - backup_open_params.native_copy = restore_settings.native_copy; + backup_open_params.allow_s3_native_copy = restore_settings.allow_s3_native_copy; BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); String current_database = context->getCurrentDatabase(); diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 4dd75911a91..2009ca4c1ff 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -161,7 +161,7 @@ namespace M(RestoreAccessCreationMode, create_access) \ M(Bool, allow_unresolved_access_dependencies) \ M(RestoreUDFCreationMode, create_function) \ - M(Bool, native_copy) \ + M(Bool, allow_s3_native_copy) \ M(Bool, internal) \ M(String, host_id) \ M(OptionalUUID, restore_uuid) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 59d73c83d69..1861e219dba 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -108,7 +108,7 @@ struct RestoreSettings RestoreUDFCreationMode create_function = RestoreUDFCreationMode::kCreateIfNotExists; /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) - bool native_copy = true; + bool allow_s3_native_copy = true; /// Internal, should not be specified by user. bool internal = false; diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index ef8ced94590..bd705e4d70f 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -107,12 +107,12 @@ void registerBackupEngineS3(BackupFactory & factory) if (params.open_mode == IBackup::OpenMode::READ) { - auto reader = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.native_copy, params.context); + auto reader = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.allow_s3_native_copy, params.context); return std::make_unique(backup_name_for_logging, archive_params, params.base_backup_info, reader, params.context); } else { - auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.native_copy, params.context); + auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.allow_s3_native_copy, params.context); return std::make_unique( backup_name_for_logging, archive_params, diff --git a/tests/queries/0_stateless/02801_backup_native_copy.reference b/tests/queries/0_stateless/02801_backup_native_copy.reference index 659df5e9b25..f9b008cde2e 100644 --- a/tests/queries/0_stateless/02801_backup_native_copy.reference +++ b/tests/queries/0_stateless/02801_backup_native_copy.reference @@ -1,4 +1,4 @@ -BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 -BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 -RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 -RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS allow_s3_native_copy = 1 1 +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS allow_s3_native_copy = 0 0 +RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS allow_s3_native_copy = 1 1 +RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS allow_s3_native_copy = 0 0 diff --git a/tests/queries/0_stateless/02801_backup_native_copy.sh b/tests/queries/0_stateless/02801_backup_native_copy.sh index 966d7ae9ce8..015dcb19b82 100755 --- a/tests/queries/0_stateless/02801_backup_native_copy.sh +++ b/tests/queries/0_stateless/02801_backup_native_copy.sh @@ -15,28 +15,28 @@ $CLICKHOUSE_CLIENT -nm -q " " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS allow_s3_native_copy=true" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS allow_s3_native_copy=false" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS allow_s3_native_copy=true" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS allow_s3_native_copy=false" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' From 1844ac37d76ac1a660681acb6b79af8af860d5ff Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 19:12:52 +0200 Subject: [PATCH 0466/2047] Remove create_read_buffer argument for copyS3File() Signed-off-by: Azat Khuzhin --- src/Backups/BackupIO_S3.cpp | 15 --------------- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 14 ++------------ src/IO/S3/copyS3File.cpp | 7 ++++++- src/IO/S3/copyS3File.h | 6 +++--- 4 files changed, 11 insertions(+), 31 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 60fea9e2008..d487ec6e80e 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -152,13 +152,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s "Blob writing function called with unexpected blob_path.size={} or mode={}", blob_path.size(), mode); - auto create_read_buffer = [this, path_in_backup] - { - return readFile(path_in_backup); - }; - copyS3File( - create_read_buffer, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, @@ -208,17 +202,8 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src /// In this case we can't use the native copy. if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) { - auto create_read_buffer = [src_disk, src_path, copy_encrypted, settings = read_settings.adjustBufferSize(start_pos + length)] - { - if (copy_encrypted) - return src_disk->readEncryptedFile(src_path, settings); - else - return src_disk->readFile(src_path, settings); - }; - LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName()); copyS3File( - create_read_buffer, client, /* src_bucket */ blob_path[1], /* src_key= */ blob_path[0], diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 3c19af188dc..e46ca3d0828 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -435,12 +435,7 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - auto create_read_buffer = [this, object_from] - { - return readObject(object_from); - }; - - copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, + copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } else @@ -456,12 +451,7 @@ void S3ObjectStorage::copyObject( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - auto create_read_buffer = [this, object_from] - { - return readObject(object_from); - }; - - copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, + copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 2c6557d97e7..2de2ccd0f9f 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -810,7 +810,6 @@ void copyDataToS3File( void copyS3File( - const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -829,7 +828,13 @@ void copyS3File( helper.performCopy(); } else + { + auto create_read_buffer = [&] + { + return std::make_unique(s3_client, src_bucket, src_key, "", settings, Context::getGlobalContextInstance()->getReadSettings()); + }; copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + } } } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 2c848076e9b..5d35e5ebe2d 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -23,10 +23,10 @@ using CreateReadBuffer = std::function()>; /// The parameters `src_offset` and `src_size` specify a part in the source to copy. /// /// Note, that it tries to copy file using native copy (CopyObject), but if it -/// has been disabled (with settings.allow_native_copy) it is fallbacks to -/// read-write copy (copyDataToS3File()). +/// has been disabled (with settings.allow_native_copy) or request failed +/// because it is a known issue, it is fallbacks to read-write copy +/// (copyDataToS3File()). void copyS3File( - const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, From b95836363085160a20bddfceaaf0709a0e721870 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 8 Jul 2023 14:32:34 +0200 Subject: [PATCH 0467/2047] tests: temporary fix the 02802_clickhouse_disks_s3_copy In #51135 the behavior of the `clickhouse-disks copy` had been changed, let's temporary update the test (and continue discussion about this change in that PR). Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh index f879b7a5621..33321607728 100755 --- a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh @@ -19,6 +19,8 @@ function run_test_for_disk() grep -o -e "Single part upload has completed." -e "Single operation copy has completed." } clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test + # NOTE: this is due to "copy" does works like "cp -R from to/" instead of "cp from to" + clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test.copy/test clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test.copy } From ac972661f9718b9d15e5bb49c63b2dff7d296fe3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 9 Jul 2023 08:21:18 +0200 Subject: [PATCH 0468/2047] Add exclusion for "API mode: {}" from S3 in 00002_log_and_exception_messages_formatting Signed-off-by: Azat Khuzhin --- .../0_stateless/00002_log_and_exception_messages_formatting.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index acb6117f937..86fe01dc0e3 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -36,7 +36,7 @@ create temporary table known_short_messages (s String) as select * from (select 'Database {} doesn''t exist', 'Dictionary ({}) not found', 'Unknown table function {}', 'Unknown format {}', 'Unknown explain kind ''{}''', 'Unknown setting {}', 'Unknown input format {}', 'Unknown identifier: ''{}''', 'User name is empty', 'Expected function, got: {}', -'Attempt to read after eof', 'String size is too big ({}), maximum: {}' +'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'API mode: {}' ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. From 3c18a181c997f1f43e759d72eeadcc5d4f35142d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 9 Jul 2023 13:54:18 +0200 Subject: [PATCH 0469/2047] Fix using of pools from the main thread Otherwise it is not possible to use clickhouse-disks with S3: $ clickhouse-disks -C /src/ch/clickhouse/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml --log-level test --disk s3_plain_native_copy copy default/test default/test.copy Failed to make request to: http://localhost:11111/test?list-type=2&max-keys=1&prefix=clickhouse-disks%2Fdefault%2Ftest.copy: Code: 49. DB::Exception: current_thread is not initialized. (LOGICAL_ERROR), Stack trace (when copying this message, always include the lines below): 0. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/exception:134: Poco::Exception::Exception(String const&, int) @ 0x000000001ad7c872 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 1. ./.cmake-llvm16/./src/Common/Exception.cpp:94: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x0000000011e2c4b7 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 2. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/string:1499: DB::Exception::Exception(int, char const (&) [34]) @ 0x000000000d341e58 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 3. ./.cmake-llvm16/./src/Common/MemoryTrackerSwitcher.h:19: DB::(anonymous namespace)::SingleEndpointHTTPSessionPool::allocObject() @ 0x0000000012010e5a in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 4. ./.cmake-llvm16/./src/Common/PoolBase.h:174: PoolBase::get(long) @ 0x0000000012011a6f in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 5. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/variant:797: DB::makePooledHTTPSession(Poco::URI const&, Poco::URI const&, DB::ConnectionTimeouts const&, unsigned long, bool, bool) @ 0x000000001200ec69 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 6. ./.cmake-llvm16/./src/IO/HTTPCommon.cpp:0: DB::makePooledHTTPSession(Poco::URI const&, DB::ConnectionTimeouts const&, unsigned long, bool, bool) @ 0x000000001200d909 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 7. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/variant:797: void DB::S3::PocoHTTPClient::makeRequestInternalImpl(Aws::Http::HttpRequest&, DB::S3::ClientConfigurationPerRequest const&, std::shared_ptr&, Aws::Utils::RateLimits::RateLimiterInterface*, Aws::Utils::RateLimits::RateLimiterInterface*) const @ 0x00000000163f5157 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 8. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/string:1499: DB::S3::PocoHTTPClient::makeRequestInternal(Aws::Http::HttpRequest&, std::shared_ptr&, Aws::Utils::RateLimits::RateLimiterInterface*, Aws::Utils::RateLimits::RateLimiterInterface*) const @ 0x00000000163f465d in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 9. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:622: DB::S3::PocoHTTPClient::MakeRequest(std::shared_ptr const&, Aws::Utils::RateLimits::RateLimiterInterface*, Aws::Utils::RateLimits::RateLimiterInterface*) const @ 0x00000000163f4454 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 10. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/source/client/AWSClient.cpp:506: Aws::Client::AWSClient::AttemptOneRequest(std::shared_ptr const&, Aws::AmazonWebServiceRequest const&, char const*, char const*, char const*) const @ 0x000000001ae2a922 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 11. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:612: Aws::Client::AWSClient::AttemptExhaustively(Aws::Http::URI const&, Aws::AmazonWebServiceRequest const&, Aws::Http::HttpMethod, char const*, char const*, char const*) const @ 0x000000001ae28299 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 12. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/include/aws/core/utils/Outcome.h:160: Aws::Client::AWSXMLClient::MakeRequest(Aws::Http::URI const&, Aws::AmazonWebServiceRequest const&, Aws::Http::HttpMethod, char const*, char const*, char const*) const @ 0x000000001ae3c9ed in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 13. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/source/client/AWSXmlClient.cpp:66: Aws::Client::AWSXMLClient::MakeRequest(Aws::AmazonWebServiceRequest const&, Aws::Endpoint::AWSEndpoint const&, Aws::Http::HttpMethod, char const*, char const*, char const*) const @ 0x000000001ae3c995 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 14. ./.cmake-llvm16/./contrib/aws/generated/src/aws-cpp-sdk-s3/source/S3Client.cpp:0: Aws::S3::S3Client::ListObjectsV2(Aws::S3::Model::ListObjectsV2Request const&) const @ 0x000000001aee6666 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 15. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/include/aws/core/utils/Outcome.h:160: DB::S3::Client::ListObjectsV2(DB::S3::ExtendedRequest const&) const @ 0x00000000163cee42 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 16. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/include/aws/core/utils/Outcome.h:120: DB::S3ObjectStorage::listObjects(String const&, std::vector>&, int) const @ 0x0000000016b582e2 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 17. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/vector:543: DB::IObjectStorage::existsOrHasAnyChild(String const&) const @ 0x000000001644ebe9 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 18. ./.cmake-llvm16/./src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp:0: DB::MetadataStorageFromPlainObjectStorage::exists(String const&) const @ 0x0000000016b54a64 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 19. ./.cmake-llvm16/./src/Disks/IDisk.cpp:145: DB::IDisk::copyDirectoryContent(String const&, std::shared_ptr const&, String const&) @ 0x0000000016b38fa0 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 20. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/string:1499: DB::CommandCopy::execute(std::vector> const&, std::shared_ptr&, Poco::Util::LayeredConfiguration&) @ 0x0000000012050403 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 21. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/vector:434: DB::DisksApp::main(std::vector> const&) @ 0x000000001204bf02 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 22. ./.cmake-llvm16/./base/poco/Util/src/Application.cpp:0: Poco::Util::Application::run() @ 0x000000001ac7a666 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 23. ./.cmake-llvm16/./programs/disks/DisksApp.cpp:0: mainEntryClickHouseDisks(int, char**) @ 0x000000001204c550 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 24. ./.cmake-llvm16/./programs/main.cpp:0: main @ 0x000000000cfbadc4 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 25. ? @ 0x00007ffff7dc9850 in ? 26. __libc_start_main @ 0x00007ffff7dc990a in ? 27. _start @ 0x000000000cfba1ee in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse (version 23.7.1.1) AWSXmlClient: HTTP response code: -1 Resolved remote host IP address: Request ID: Exception name: Error message: Code: 49. DB::Exception: current_thread is not initialized. (LOGICAL_ERROR) (version 23.7.1.1) 0 response headers: If the signature check failed. This could be because of a time skew. Attempting to adjust the signer. Request failed, now waiting 1600 ms before attempting again. Signed-off-by: Azat Khuzhin --- src/Common/MemoryTrackerSwitcher.h | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/MemoryTrackerSwitcher.h b/src/Common/MemoryTrackerSwitcher.h index 0fefcbb280a..3c99fd12353 100644 --- a/src/Common/MemoryTrackerSwitcher.h +++ b/src/Common/MemoryTrackerSwitcher.h @@ -6,17 +6,13 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - struct MemoryTrackerSwitcher { explicit MemoryTrackerSwitcher(MemoryTracker * new_tracker) { + /// current_thread is not initialized for the main thread, so simply do not switch anything if (!current_thread) - throw Exception(ErrorCodes::LOGICAL_ERROR, "current_thread is not initialized"); + return; auto * thread_tracker = CurrentThread::getMemoryTracker(); prev_untracked_memory = current_thread->untracked_memory; @@ -28,6 +24,10 @@ struct MemoryTrackerSwitcher ~MemoryTrackerSwitcher() { + /// current_thread is not initialized for the main thread, so simply do not switch anything + if (!current_thread) + return; + CurrentThread::flushUntrackedMemory(); auto * thread_tracker = CurrentThread::getMemoryTracker(); @@ -35,6 +35,7 @@ struct MemoryTrackerSwitcher thread_tracker->setParent(prev_memory_tracker_parent); } +private: MemoryTracker * prev_memory_tracker_parent = nullptr; Int64 prev_untracked_memory = 0; }; From 790b438b7c19979d49bf35e97b518b4266d7c515 Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 8 Jul 2023 13:33:49 +0000 Subject: [PATCH 0470/2047] add test_delete_race_leftovers --- .../configs/config.d/storage_conf.xml | 19 +++- .../test_alter_moving_garbage/test.py | 91 ++++++++++++++++++- 2 files changed, 104 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml b/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml index 1450a459257..67c4cc2d489 100644 --- a/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml +++ b/tests/integration/test_alter_moving_garbage/configs/config.d/storage_conf.xml @@ -1,12 +1,18 @@ - + s3 http://minio1:9001/root/data/ minio minio123 - + + + s3 + http://minio1:9001/root/data2/ + minio + minio123 + @@ -15,10 +21,17 @@ default - s3 + s31 + + + + s32 + + + diff --git a/tests/integration/test_alter_moving_garbage/test.py b/tests/integration/test_alter_moving_garbage/test.py index 330df3ac490..af9fffbb74d 100644 --- a/tests/integration/test_alter_moving_garbage/test.py +++ b/tests/integration/test_alter_moving_garbage/test.py @@ -39,7 +39,7 @@ def cluster(): def create_table(node, table_name, replicated, additional_settings): settings = { "storage_policy": "two_disks", - "old_parts_lifetime": 1, + "old_parts_lifetime": 0, "index_granularity": 512, "temporary_directories_lifetime": 0, "merge_tree_clear_old_temporary_directories_interval_seconds": 1, @@ -73,9 +73,13 @@ def create_table(node, table_name, replicated, additional_settings): "allow_remote_fs_zero_copy_replication,replicated_engine", [(False, False), (False, True), (True, True)], ) -def test_create_table( +def test_alter_moving( cluster, allow_remote_fs_zero_copy_replication, replicated_engine ): + """ + Test that we correctly move parts during ALTER TABLE + """ + if replicated_engine: nodes = list(cluster.instances.values()) else: @@ -126,7 +130,7 @@ def test_create_table( partition = f"2021-01-{i:02d}" try: random.choice(nodes).query( - f"ALTER TABLE {table_name} MOVE PARTITION '{partition}' TO DISK 's3'", + f"ALTER TABLE {table_name} MOVE PARTITION '{partition}' TO DISK 's31'", ) except QueryRuntimeException as e: if "PART_IS_TEMPORARILY_LOCKED" in str(e): @@ -153,3 +157,84 @@ def test_create_table( ) assert data_digest == "1000\n" + + +def test_delete_race_leftovers(cluster): + """ + Test that we correctly delete outdated parts and do not leave any leftovers on s3 + """ + + node = cluster.instances["node1"] + + table_name = "test_delete_race_leftovers" + additional_settings = { + # use another disk not to interfere with other tests + "storage_policy": "one_disk", + # always remove parts in parallel + "concurrent_part_removal_threshold": 1, + } + + create_table( + node, table_name, replicated=True, additional_settings=additional_settings + ) + + # Stop merges to have several small parts in active set + node.query(f"SYSTEM STOP MERGES {table_name}") + + # Creare several small parts in one partition + for i in range(1, 11): + node.query( + f"INSERT INTO {table_name} SELECT toDate('2021-01-01'), number as id, toString(sipHash64(number, {i})) FROM numbers(10_000)" + ) + table_digest_query = f"SELECT count(), sum(sipHash64(id, data)) FROM {table_name}" + table_digest = node.query(table_digest_query) + + # Execute several noop deletes to have parts with updated mutation id without changes in data + # New parts will have symlinks to old parts + node.query(f"SYSTEM START MERGES {table_name}") + for i in range(10): + node.query(f"DELETE FROM {table_name} WHERE data = ''") + + # Make existing parts outdated + # Also we don't want have changing parts set, + # because it will be difficult match objects on s3 and in remote_data_paths to check correctness + node.query(f"OPTIMIZE TABLE {table_name} FINAL") + + inactive_parts_query = ( + f"SELECT count() FROM system.parts " + f"WHERE not active AND table = '{table_name}' AND database = 'default'" + ) + + # Try to wait for deletion of outdated parts + # However, we do not want to wait too long + # If some parts are not deleted after several iterations, we will just continue + for i in range(20): + inactive_parts_count = int(node.query(inactive_parts_query).strip()) + if inactive_parts_count == 0: + print(f"Inactive parts are deleted after {i} iterations") + break + + print(f"Inactive parts count: {inactive_parts_count}") + time.sleep(5) + + # Check that we correctly deleted all outdated parts and no leftovers on s3 + known_remote_paths = set( + node.query( + f"SELECT remote_path FROM system.remote_data_paths WHERE disk_name = 's32'" + ).splitlines() + ) + + all_remote_paths = set( + obj.object_name + for obj in cluster.minio_client.list_objects( + cluster.minio_bucket, "data2/", recursive=True + ) + ) + + # Some blobs can be deleted after we listed remote_data_paths + # It's alright, thus we check only that all remote paths are known + # (in other words, all remote paths is subset of known paths) + assert all_remote_paths == {p for p in known_remote_paths if p in all_remote_paths} + + # Check that we have all data + assert table_digest == node.query(table_digest_query) From ff4ec823424dfb7067b1f3db714c61a6a10869ee Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 10 Jul 2023 07:29:31 +0000 Subject: [PATCH 0471/2047] Added hasSubsequenceUTF8 impl + tests --- src/Functions/HasSubsequenceImpl.h | 56 +++++++++++++++++-- src/Functions/hasSubsequence.cpp | 2 + .../hasSubsequenceCaseInsensitive.cpp | 2 + .../hasSubsequenceCaseInsensitiveUTF8.cpp | 4 +- src/Functions/hasSubsequenceUTF8.cpp | 4 +- .../02809_has_subsequence.reference | 22 ++++++++ .../0_stateless/02809_has_subsequence.sql | 24 ++++++++ 7 files changed, 108 insertions(+), 6 deletions(-) diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h index 1396e64ade5..10fe6215280 100644 --- a/src/Functions/HasSubsequenceImpl.h +++ b/src/Functions/HasSubsequenceImpl.h @@ -96,11 +96,18 @@ private: auto haystack = std::string(reinterpret_cast(haystack_slice.data), haystack_slice.size); auto needle = std::string(reinterpret_cast(needle_slice.data), needle_slice.size); + + if constexpr (!Impl::is_utf8) + { + Impl::toLowerIfNeed(haystack); + Impl::toLowerIfNeed(needle); - Impl::toLowerIfNeed(haystack); - Impl::toLowerIfNeed(needle); - - res_data[row_num] = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + res_data[row_num] = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + } + else + { + res_data[row_num] = hasSubsequenceUTF8(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); + } haystacks.next(); needles.next(); ++row_num; @@ -115,6 +122,47 @@ private: ++j; return j == needle_size; } + + static UInt8 hasSubsequenceUTF8(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) + { + const auto * haystack_pos = haystack; + const auto * needle_pos = needle; + const auto * haystack_end = haystack + haystack_size; + const auto * needle_end = needle + needle_size; + + if (!needle_size) + { + return 1; + } + + auto haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos); + auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos); + if (!haystack_code_point || !needle_code_point) + { + return 0; + } + + while (true) + { + if (needle_code_point == haystack_code_point) + { + needle_pos += UTF8::seqLength(*needle_pos); + if (needle_pos == needle_end) + { + break; + } + needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos); + } + haystack_pos += UTF8::seqLength(*haystack_pos); + if (haystack_pos == haystack_end) + { + break; + } + haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos); + } + + return needle_pos == needle_end; + } }; } diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp index 900e80f5524..acc574c8207 100644 --- a/src/Functions/hasSubsequence.cpp +++ b/src/Functions/hasSubsequence.cpp @@ -10,6 +10,8 @@ namespace struct HasSubsequenceCaseSensitiveASCII { + static constexpr bool is_utf8 = false; + static void toLowerIfNeed(String & /*s*/) { } }; diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp index dbac62d7f09..68c510794c3 100644 --- a/src/Functions/hasSubsequenceCaseInsensitive.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -9,6 +9,8 @@ namespace struct HasSubsequenceCaseInsensitiveASCII { + static constexpr bool is_utf8 = false; + static void toLowerIfNeed(String & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } }; diff --git a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp index c104ff52857..d1fb2f5152a 100644 --- a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp @@ -9,7 +9,9 @@ namespace struct HasSubsequenceCaseInsensitiveUTF8 { - static void toLowerIfNeed(String & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } + static constexpr bool is_utf8 = true; + + //static void toLowerIfNeed(String & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } }; struct NameHasSubsequenceCaseInsensitiveUTF8 diff --git a/src/Functions/hasSubsequenceUTF8.cpp b/src/Functions/hasSubsequenceUTF8.cpp index c67ce7d9c74..fcdcd28e02b 100644 --- a/src/Functions/hasSubsequenceUTF8.cpp +++ b/src/Functions/hasSubsequenceUTF8.cpp @@ -10,7 +10,9 @@ namespace struct HasSubsequenceCaseSensitiveUTF8 { - static void toLowerIfNeed(String & /*s*/) { } + static constexpr bool is_utf8 = true; + + // static void toLowerIfNeed(String & /*s*/) { } }; struct NameHasSubsequenceUTF8 diff --git a/tests/queries/0_stateless/02809_has_subsequence.reference b/tests/queries/0_stateless/02809_has_subsequence.reference index d12c0ba9fb3..8437a7f4b74 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.reference +++ b/tests/queries/0_stateless/02809_has_subsequence.reference @@ -1,6 +1,7 @@ hasSubsequence / const / const 1 1 +0 1 1 1 @@ -24,4 +25,25 @@ hasSubsequence / string / string 1 0 hasSubsequenceCaseInsensitive / const / const +0 1 +1 +hasSubsequenceCaseInsensitive / string / string +0 +1 +1 +hasSubsequenceUTF8 / const / const +1 +1 +0 +1 +0 +1 +0 +1 +1 +0 +1 +0 +1 +0 diff --git a/tests/queries/0_stateless/02809_has_subsequence.sql b/tests/queries/0_stateless/02809_has_subsequence.sql index 64f3fd8dc77..b8d3280488c 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.sql +++ b/tests/queries/0_stateless/02809_has_subsequence.sql @@ -1,6 +1,7 @@ select 'hasSubsequence / const / const'; select hasSubsequence('garbage', ''); select hasSubsequence('garbage', 'g'); +select hasSubsequence('garbage', 'G'); select hasSubsequence('garbage', 'a'); select hasSubsequence('garbage', 'e'); select hasSubsequence('garbage', 'gr'); @@ -31,5 +32,28 @@ select hasSubsequence(materialize('garbage'), materialize('garbage1')); select 'hasSubsequenceCaseInsensitive / const / const'; +select hasSubsequenceCaseInsensitive('garbage', 'w'); select hasSubsequenceCaseInsensitive('garbage', 'ARG'); +select hasSubsequenceCaseInsensitive('GARGAGE', 'arg'); +select 'hasSubsequenceCaseInsensitive / string / string'; +select hasSubsequenceCaseInsensitive(materialize('garbage'), materialize('w')); +select hasSubsequenceCaseInsensitive(materialize('garbage'), materialize('ARG')); +select hasSubsequenceCaseInsensitive(materialize('GARGAGE'), materialize('arg')); + +select 'hasSubsequenceUTF8 / const / const'; +select hasSubsequence('ClickHouse - столбцовая система управления базами данных', ''); +select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'C'); -- eng +select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'С'); -- cyrilic +select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'House'); +select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'house'); +select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'система'); +select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'Система'); +select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'ссубд'); + +select hasSubsequence(materialize('ClickHouse - столбцовая система управления базами данных'), 'субд'); +select hasSubsequence(materialize('ClickHouse - столбцовая система управления базами данных'), 'суббд'); +select hasSubsequence('ClickHouse - столбцовая система управления базами данных', materialize('стул')); +select hasSubsequence('ClickHouse - столбцовая система управления базами данных', materialize('два стула')); +select hasSubsequence(materialize('ClickHouse - столбцовая система управления базами данных'), materialize('орех')); +select hasSubsequence(materialize('ClickHouse - столбцовая система управления базами данных'), materialize('два ореха')); \ No newline at end of file From b2522d81c42cd19e2b6c76e18ee36ba7a0624fe7 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 21 Jun 2023 10:33:11 +0800 Subject: [PATCH 0472/2047] rewrite uniq to count --- .../RewriteUinqToCountVisitor.cpp | 88 +++++++++++++++++++ src/Interpreters/RewriteUinqToCountVisitor.h | 29 ++++++ 2 files changed, 117 insertions(+) create mode 100644 src/Interpreters/RewriteUinqToCountVisitor.cpp create mode 100644 src/Interpreters/RewriteUinqToCountVisitor.h diff --git a/src/Interpreters/RewriteUinqToCountVisitor.cpp b/src/Interpreters/RewriteUinqToCountVisitor.cpp new file mode 100644 index 00000000000..f491bb08c88 --- /dev/null +++ b/src/Interpreters/RewriteUinqToCountVisitor.cpp @@ -0,0 +1,88 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + +static bool matchFnUniq(String func_name) +{ + auto name = Poco::toLower(func_name); + return name == "uniq" || name == "" +} + +} + +/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to 'SELECT count() FROM (SELECT DISTINCT x ...)' +void RewriteUinqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) +{ + auto * selectq = ast->as(); + if (!selectq || !selectq->tables() || selectq->tables()->children.size() != 1) + return; + auto expr_list = selectq->select(); + if (!expr_list || expr_list->children.size() != 1) + return; + auto * func = expr_list->children[0]->as(); + if (!func || (Poco::toLower(func->name) != "uniq" && Poco::toLower(func->name) != "uniqexact")) + return; + auto arg = func->arguments->children; + if (arg.size() != 1) + return; + if (!arg[0]->as()) + return; + if (selectq->tables()->as()->children[0]->as()->children.size() != 1) + return; + auto * table_expr = selectq->tables()->as()->children[0]->as()->children[0]->as(); + if (!table_expr || table_expr->size() != 1 || !table_expr->database_and_table_name) + return; + // Check done, we now rewrite the AST + auto cloned_select_query = selectq->clone(); + expr_list->children[0] = makeASTFunction("count"); + + auto table_name = table_expr->database_and_table_name->as()->name(); + table_expr->children.clear(); + table_expr->children.emplace_back(std::make_shared()); + table_expr->database_and_table_name = nullptr; + table_expr->table_function = nullptr; + table_expr->subquery = table_expr->children[0]; + + auto column_name = arg[0]->as()->name(); + // Form AST for subquery + { + auto * select_ptr = cloned_select_query->as(); + select_ptr->refSelect()->children.clear(); + select_ptr->refSelect()->children.emplace_back(std::make_shared(column_name)); + auto exprlist = std::make_shared(); + exprlist->children.emplace_back(std::make_shared(column_name)); + cloned_select_query->as()->setExpression(ASTSelectQuery::Expression::GROUP_BY, exprlist); + + auto expr = std::make_shared(); + expr->children.emplace_back(cloned_select_query); + auto select_with_union = std::make_shared(); + select_with_union->union_mode = SelectUnionMode::UNION_DEFAULT; + select_with_union->is_normalized = false; + select_with_union->list_of_modes.clear(); + select_with_union->set_of_modes.clear(); + select_with_union->children.emplace_back(expr); + select_with_union->list_of_selects = expr; + table_expr->children[0]->as()->children.emplace_back(select_with_union); + } +} + +} diff --git a/src/Interpreters/RewriteUinqToCountVisitor.h b/src/Interpreters/RewriteUinqToCountVisitor.h new file mode 100644 index 00000000000..d7aa745352b --- /dev/null +++ b/src/Interpreters/RewriteUinqToCountVisitor.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include "Interpreters/TreeRewriter.h" + +namespace DB +{ + +class ASTFunction; + +/// Simple rewrite: +/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to +/// 'SELECT count() FROM (SELECT DISTINCT x ...)' +/// +/// 'SELECT uniq() FROM (SELECT x ... GROUP BY x)' to +/// 'SELECT count() FROM (SELECT x ... GROUP BY x)' +/// +/// Note we can rewrite all uniq variants except uniqUpTo. +class RewriteUinqToCountMatcher +{ +public: + struct Data {}; + static void visit(ASTPtr & ast, Data &); + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } +}; + +using RewriteUinqToCountVisitor = InDepthNodeVisitor; +} From bcfc0db9f1bf1e6f8943687dcf1b122e94a86edd Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Sun, 25 Jun 2023 14:43:39 +0800 Subject: [PATCH 0473/2047] add settiings --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 7 ++ .../RewriteUinqToCountVisitor.cpp | 88 ----------------- .../RewriteUniqToCountVisitor.cpp | 99 +++++++++++++++++++ ...tVisitor.h => RewriteUniqToCountVisitor.h} | 4 +- 5 files changed, 109 insertions(+), 90 deletions(-) delete mode 100644 src/Interpreters/RewriteUinqToCountVisitor.cpp create mode 100644 src/Interpreters/RewriteUniqToCountVisitor.cpp rename src/Interpreters/{RewriteUinqToCountVisitor.h => RewriteUniqToCountVisitor.h} (87%) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3250bdf0750..f5ab71a3b0d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -774,6 +774,7 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone if empty.", 0) \ + M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and other variants(except uniqUpTo) to distinct to to count, it is a RBO based optimization.", 0) // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6ea15312ec4..ebde5df2533 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include @@ -426,6 +427,12 @@ InterpreterSelectQuery::InterpreterSelectQuery( RewriteCountDistinctFunctionVisitor(data_rewrite_countdistinct).visit(query_ptr); } + if (settings.optimize_uniq_to_count) + { + RewriteUniqToCountMatcher::Data data_rewrite_uniq_count; + RewriteUniqToCountVisitor(data_rewrite_uniq_count).visit(query_ptr); + } + JoinedTables joined_tables(getSubqueryContext(context), getSelectQuery(), options.with_all_cols, options_.is_create_parameterized_view); bool got_storage_from_query = false; diff --git a/src/Interpreters/RewriteUinqToCountVisitor.cpp b/src/Interpreters/RewriteUinqToCountVisitor.cpp deleted file mode 100644 index f491bb08c88..00000000000 --- a/src/Interpreters/RewriteUinqToCountVisitor.cpp +++ /dev/null @@ -1,88 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace -{ - -static bool matchFnUniq(String func_name) -{ - auto name = Poco::toLower(func_name); - return name == "uniq" || name == "" -} - -} - -/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to 'SELECT count() FROM (SELECT DISTINCT x ...)' -void RewriteUinqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) -{ - auto * selectq = ast->as(); - if (!selectq || !selectq->tables() || selectq->tables()->children.size() != 1) - return; - auto expr_list = selectq->select(); - if (!expr_list || expr_list->children.size() != 1) - return; - auto * func = expr_list->children[0]->as(); - if (!func || (Poco::toLower(func->name) != "uniq" && Poco::toLower(func->name) != "uniqexact")) - return; - auto arg = func->arguments->children; - if (arg.size() != 1) - return; - if (!arg[0]->as()) - return; - if (selectq->tables()->as()->children[0]->as()->children.size() != 1) - return; - auto * table_expr = selectq->tables()->as()->children[0]->as()->children[0]->as(); - if (!table_expr || table_expr->size() != 1 || !table_expr->database_and_table_name) - return; - // Check done, we now rewrite the AST - auto cloned_select_query = selectq->clone(); - expr_list->children[0] = makeASTFunction("count"); - - auto table_name = table_expr->database_and_table_name->as()->name(); - table_expr->children.clear(); - table_expr->children.emplace_back(std::make_shared()); - table_expr->database_and_table_name = nullptr; - table_expr->table_function = nullptr; - table_expr->subquery = table_expr->children[0]; - - auto column_name = arg[0]->as()->name(); - // Form AST for subquery - { - auto * select_ptr = cloned_select_query->as(); - select_ptr->refSelect()->children.clear(); - select_ptr->refSelect()->children.emplace_back(std::make_shared(column_name)); - auto exprlist = std::make_shared(); - exprlist->children.emplace_back(std::make_shared(column_name)); - cloned_select_query->as()->setExpression(ASTSelectQuery::Expression::GROUP_BY, exprlist); - - auto expr = std::make_shared(); - expr->children.emplace_back(cloned_select_query); - auto select_with_union = std::make_shared(); - select_with_union->union_mode = SelectUnionMode::UNION_DEFAULT; - select_with_union->is_normalized = false; - select_with_union->list_of_modes.clear(); - select_with_union->set_of_modes.clear(); - select_with_union->children.emplace_back(expr); - select_with_union->list_of_selects = expr; - table_expr->children[0]->as()->children.emplace_back(select_with_union); - } -} - -} diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp new file mode 100644 index 00000000000..c3a9fb7547c --- /dev/null +++ b/src/Interpreters/RewriteUniqToCountVisitor.cpp @@ -0,0 +1,99 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + +static bool matchFnUniq(String func_name) +{ + auto name = Poco::toLower(func_name); + return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta"; +} + +class PrintTreeMatcher +{ +public: + struct Data {String prefix;}; + static void visit(ASTPtr & ast, Data &) + { + ast->getID(); + } + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } +}; + +using PrintTreeVisitor = InDepthNodeVisitor; + +} + +/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to 'SELECT count() FROM (SELECT DISTINCT x ...)' +void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) +{ + auto * selectq = ast->as(); + if (!selectq || !selectq->tables() || selectq->tables()->children.size() != 1) + return; + auto expr_list = selectq->select(); + if (!expr_list || expr_list->children.size() != 1) + return; + auto * func = expr_list->children[0]->as(); + if (!func || !matchFnUniq(func->name)) + return; + if (selectq->tables()->as()->children[0]->as()->children.size() != 1) + return; + auto * table_expr = selectq->tables()->as()->children[0]->as()->children[0]->as(); + if (!table_expr || table_expr->children.size() != 1 || !table_expr->subquery) + return; + auto * subquery = table_expr->subquery->as(); + subquery->formatForLogging(0); + + // Check done, we now rewrite the AST + auto cloned_select_query = selectq->clone(); + expr_list->children[0] = makeASTFunction("count"); + +// auto table_name = table_expr->database_and_table_name->as()->name(); + table_expr->children.clear(); + table_expr->children.emplace_back(std::make_shared()); + table_expr->database_and_table_name = nullptr; + table_expr->table_function = nullptr; + table_expr->subquery = table_expr->children[0]; + +// auto column_name = arg[0]->as()->name(); +// // Form AST for subquery +// { +// auto * select_ptr = cloned_select_query->as(); +// select_ptr->refSelect()->children.clear(); +// select_ptr->refSelect()->children.emplace_back(std::make_shared(column_name)); +// auto exprlist = std::make_shared(); +// exprlist->children.emplace_back(std::make_shared(column_name)); +// cloned_select_query->as()->setExpression(ASTSelectQuery::Expression::GROUP_BY, exprlist); +// +// auto expr = std::make_shared(); +// expr->children.emplace_back(cloned_select_query); +// auto select_with_union = std::make_shared(); +// select_with_union->union_mode = SelectUnionMode::UNION_DEFAULT; +// select_with_union->is_normalized = false; +// select_with_union->list_of_modes.clear(); +// select_with_union->set_of_modes.clear(); +// select_with_union->children.emplace_back(expr); +// select_with_union->list_of_selects = expr; +// table_expr->children[0]->as()->children.emplace_back(select_with_union); +// } +} + +} diff --git a/src/Interpreters/RewriteUinqToCountVisitor.h b/src/Interpreters/RewriteUniqToCountVisitor.h similarity index 87% rename from src/Interpreters/RewriteUinqToCountVisitor.h rename to src/Interpreters/RewriteUniqToCountVisitor.h index d7aa745352b..f59206101d4 100644 --- a/src/Interpreters/RewriteUinqToCountVisitor.h +++ b/src/Interpreters/RewriteUniqToCountVisitor.h @@ -17,7 +17,7 @@ class ASTFunction; /// 'SELECT count() FROM (SELECT x ... GROUP BY x)' /// /// Note we can rewrite all uniq variants except uniqUpTo. -class RewriteUinqToCountMatcher +class RewriteUniqToCountMatcher { public: struct Data {}; @@ -25,5 +25,5 @@ public: static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } }; -using RewriteUinqToCountVisitor = InDepthNodeVisitor; +using RewriteUniqToCountVisitor = InDepthNodeVisitor; } From 0dc1fdd67f8609564cda4ca61d17913d05d318c2 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 28 Jun 2023 14:12:24 +0800 Subject: [PATCH 0474/2047] add some comments --- .../RewriteUniqToCountVisitor.cpp | 111 ++++++++++-------- src/Interpreters/RewriteUniqToCountVisitor.h | 6 +- 2 files changed, 68 insertions(+), 49 deletions(-) diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp index c3a9fb7547c..d608b6dfae4 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.cpp +++ b/src/Interpreters/RewriteUniqToCountVisitor.cpp @@ -4,15 +4,11 @@ #include #include #include -#include #include #include #include #include -#include -#include #include -#include namespace DB @@ -21,28 +17,48 @@ namespace DB namespace { -static bool matchFnUniq(String func_name) +bool matchFnUniq(String func_name) { auto name = Poco::toLower(func_name); - return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta"; + return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" || name == "uniqCombined64"; } -class PrintTreeMatcher +bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs) { -public: - struct Data {String prefix;}; - static void visit(ASTPtr & ast, Data &) + if (!lhs || !rhs) + return false; + if (lhs->children.size() != rhs->children.size()) + return false; + for (size_t i = 0; i < lhs->children.size(); i++) { - ast->getID(); + if (lhs->children[i]->formatForLogging() != rhs->children[i]->formatForLogging()) // TODO not a elegant way + return false; } - static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } -}; + return true; +} -using PrintTreeVisitor = InDepthNodeVisitor; +/// Test whether lhs contains all expr in rhs. +bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs) +{ + if (!lhs || !rhs) + return false; + if (lhs->children.size() < rhs->children.size()) + return false; + std::vector lhs_strs; + for (const auto & le : lhs->children) + { + lhs_strs.emplace_back(le->formatForLogging()); + } + for (const auto & re : rhs->children) + { + if (std::find(lhs_strs.begin(), lhs_strs.end(), re->formatForLogging()) != lhs_strs.end()) + return false; + } + return true; +} } -/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to 'SELECT count() FROM (SELECT DISTINCT x ...)' void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) { auto * selectq = ast->as(); @@ -60,40 +76,43 @@ void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) if (!table_expr || table_expr->children.size() != 1 || !table_expr->subquery) return; auto * subquery = table_expr->subquery->as(); - subquery->formatForLogging(0); + if (!subquery) + return; + auto * sub_selectq = subquery->children[0]->as()->children[0]->as()->children[0]->as(); + if (!sub_selectq) + return; - // Check done, we now rewrite the AST - auto cloned_select_query = selectq->clone(); - expr_list->children[0] = makeASTFunction("count"); + auto match_distinct = [&]() -> bool + { + if (!sub_selectq->distinct) + return false; + auto sub_expr_list = sub_selectq->select(); + if (!sub_expr_list) + return false; + /// uniq expression list == subquery group by expression list + if (!expressionListEquals(func->children[0]->as(), sub_expr_list->as())) + return false; + return true; + }; -// auto table_name = table_expr->database_and_table_name->as()->name(); - table_expr->children.clear(); - table_expr->children.emplace_back(std::make_shared()); - table_expr->database_and_table_name = nullptr; - table_expr->table_function = nullptr; - table_expr->subquery = table_expr->children[0]; + auto match_group_by = [&]() -> bool + { + auto group_by = sub_selectq->groupBy(); + if (!group_by) + return false; + auto sub_expr_list = sub_selectq->select(); + if (!sub_expr_list) + return false; + /// uniq expression list == subquery group by expression list + if (!expressionListEquals(func->children[0]->as(), group_by->as())) + return false; + /// subquery select expression list must contain all columns in uniq expression list + expressionListContainsAll(sub_expr_list->as(), func->children[0]->as()); + return true; + }; -// auto column_name = arg[0]->as()->name(); -// // Form AST for subquery -// { -// auto * select_ptr = cloned_select_query->as(); -// select_ptr->refSelect()->children.clear(); -// select_ptr->refSelect()->children.emplace_back(std::make_shared(column_name)); -// auto exprlist = std::make_shared(); -// exprlist->children.emplace_back(std::make_shared(column_name)); -// cloned_select_query->as()->setExpression(ASTSelectQuery::Expression::GROUP_BY, exprlist); -// -// auto expr = std::make_shared(); -// expr->children.emplace_back(cloned_select_query); -// auto select_with_union = std::make_shared(); -// select_with_union->union_mode = SelectUnionMode::UNION_DEFAULT; -// select_with_union->is_normalized = false; -// select_with_union->list_of_modes.clear(); -// select_with_union->set_of_modes.clear(); -// select_with_union->children.emplace_back(expr); -// select_with_union->list_of_selects = expr; -// table_expr->children[0]->as()->children.emplace_back(select_with_union); -// } + if (match_distinct() || match_group_by()) + expr_list->children[0] = makeASTFunction("count"); } } diff --git a/src/Interpreters/RewriteUniqToCountVisitor.h b/src/Interpreters/RewriteUniqToCountVisitor.h index f59206101d4..42a86049bb9 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.h +++ b/src/Interpreters/RewriteUniqToCountVisitor.h @@ -10,11 +10,11 @@ namespace DB class ASTFunction; /// Simple rewrite: -/// 'SELECT uniq(x) FROM (SELECT DISTINCT x ...)' to +/// 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to /// 'SELECT count() FROM (SELECT DISTINCT x ...)' /// -/// 'SELECT uniq() FROM (SELECT x ... GROUP BY x)' to -/// 'SELECT count() FROM (SELECT x ... GROUP BY x)' +/// 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to +/// 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' /// /// Note we can rewrite all uniq variants except uniqUpTo. class RewriteUniqToCountMatcher From ca6905b308972f207b19e041149621d185913b3c Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 7 Jul 2023 17:03:37 +0800 Subject: [PATCH 0475/2047] rewrite support alias --- src/Core/Settings.h | 2 +- .../RewriteUniqToCountVisitor.cpp | 97 +++++++++++++------ 2 files changed, 71 insertions(+), 28 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f5ab71a3b0d..29ea7fe5cb9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -774,7 +774,7 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone if empty.", 0) \ - M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and other variants(except uniqUpTo) to distinct to to count, it is a RBO based optimization.", 0) + M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause, it is a RBO based optimization.", 0) // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp index d608b6dfae4..587a905e4c5 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.cpp +++ b/src/Interpreters/RewriteUniqToCountVisitor.cpp @@ -14,16 +14,49 @@ namespace DB { +using Aliases = std::unordered_map; + namespace { bool matchFnUniq(String func_name) { auto name = Poco::toLower(func_name); - return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" || name == "uniqCombined64"; + return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" + || name == "uniqCombined64"; } -bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs) +bool expressionEquals(const ASTPtr & lhs, const ASTPtr & rhs, Aliases & alias) +{ + if (lhs->getTreeHash() == rhs->getTreeHash()) + { + return true; + } + else + { + auto * lhs_idf = lhs->as(); + auto * rhs_idf = rhs->as(); + if (lhs_idf && rhs_idf) + { + /// compound identifiers, such as: + if (lhs_idf->shortName() == rhs_idf->shortName()) + return true; + + /// translate alias + if (alias.find(lhs_idf->shortName()) != alias.end()) + lhs_idf = alias.find(lhs_idf->shortName())->second->as(); + + if (alias.find(rhs_idf->shortName()) != alias.end()) + rhs_idf = alias.find(rhs_idf->shortName())->second->as(); + + if (lhs_idf->shortName() == rhs_idf->shortName()) + return true; + } + } + return false; +} + +bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs, Aliases & alias) { if (!lhs || !rhs) return false; @@ -31,27 +64,23 @@ bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs) return false; for (size_t i = 0; i < lhs->children.size(); i++) { - if (lhs->children[i]->formatForLogging() != rhs->children[i]->formatForLogging()) // TODO not a elegant way + if (!expressionEquals(lhs->children[i], rhs->children[i], alias)) return false; } return true; } -/// Test whether lhs contains all expr in rhs. -bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs) +/// Test whether lhs contains all expressions in rhs. +bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs, Aliases alias) { if (!lhs || !rhs) return false; if (lhs->children.size() < rhs->children.size()) return false; - std::vector lhs_strs; - for (const auto & le : lhs->children) - { - lhs_strs.emplace_back(le->formatForLogging()); - } for (const auto & re : rhs->children) { - if (std::find(lhs_strs.begin(), lhs_strs.end(), re->formatForLogging()) != lhs_strs.end()) + auto predicate = [&re, &alias](ASTPtr & le) { return expressionEquals(le, re, alias); }; + if (std::find_if(lhs->children.begin(), lhs->children.end(), predicate) == lhs->children.end()) return false; } return true; @@ -72,46 +101,60 @@ void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) return; if (selectq->tables()->as()->children[0]->as()->children.size() != 1) return; - auto * table_expr = selectq->tables()->as()->children[0]->as()->children[0]->as(); + auto * table_expr = selectq->tables() + ->as() + ->children[0] + ->as() + ->children[0] + ->as(); if (!table_expr || table_expr->children.size() != 1 || !table_expr->subquery) return; auto * subquery = table_expr->subquery->as(); if (!subquery) return; - auto * sub_selectq = subquery->children[0]->as()->children[0]->as()->children[0]->as(); + auto * sub_selectq = subquery->children[0] + ->as()->children[0] + ->as()->children[0] + ->as(); if (!sub_selectq) return; + auto sub_expr_list = sub_selectq->select(); + if (!sub_expr_list) + return; - auto match_distinct = [&]() -> bool + /// collect subquery select expressions alias + std::unordered_map alias; + for (auto expr : sub_expr_list->children) + { + if (!expr->tryGetAlias().empty()) + alias.insert({expr->tryGetAlias(), expr}); + } + + auto match_subquery_with_distinct = [&]() -> bool { if (!sub_selectq->distinct) return false; - auto sub_expr_list = sub_selectq->select(); - if (!sub_expr_list) - return false; /// uniq expression list == subquery group by expression list - if (!expressionListEquals(func->children[0]->as(), sub_expr_list->as())) + if (!expressionListEquals(func->children[0]->as(), sub_expr_list->as(), alias)) return false; return true; }; - auto match_group_by = [&]() -> bool + auto match_subquery_with_group_by = [&]() -> bool { - auto group_by = sub_selectq->groupBy(); + auto group_by = sub_selectq->groupBy(); // TODO group by type if (!group_by) return false; - auto sub_expr_list = sub_selectq->select(); - if (!sub_expr_list) - return false; - /// uniq expression list == subquery group by expression list - if (!expressionListEquals(func->children[0]->as(), group_by->as())) + /// uniq expression list == subquery group by expression list + if (!expressionListEquals(func->children[0]->as(), group_by->as(), alias)) return false; /// subquery select expression list must contain all columns in uniq expression list - expressionListContainsAll(sub_expr_list->as(), func->children[0]->as()); + if (!expressionListContainsAll(sub_expr_list->as(), func->children[0]->as(), alias)) + return false; return true; }; - if (match_distinct() || match_group_by()) + if (match_subquery_with_distinct() || match_subquery_with_group_by()) expr_list->children[0] = makeASTFunction("count"); } From e0b223aa7eac0e780a4048a2e302f67406ace0aa Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Fri, 7 Jul 2023 18:04:15 +0800 Subject: [PATCH 0476/2047] add tests for uniq to count rewrite --- .../test_rewrite_uniq_to_count/__init__.py | 0 .../test_rewrite_uniq_to_count/test.py | 81 +++++++++++++++++++ 2 files changed, 81 insertions(+) create mode 100644 tests/integration/test_rewrite_uniq_to_count/__init__.py create mode 100644 tests/integration/test_rewrite_uniq_to_count/test.py diff --git a/tests/integration/test_rewrite_uniq_to_count/__init__.py b/tests/integration/test_rewrite_uniq_to_count/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py new file mode 100644 index 00000000000..2e21a8a50a5 --- /dev/null +++ b/tests/integration/test_rewrite_uniq_to_count/test.py @@ -0,0 +1,81 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node") + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + prepare() + yield cluster + finally: + shutdown() + cluster.shutdown() + + +def prepare(): + node.query( + """ + CREATE TABLE IF NOT EXISTS test_rewrite_uniq_to_count + ( + `a` UInt8, + `b` UInt8, + `c` UInt8 + ) + ENGINE = MergeTree + ORDER BY `a` + """ + ) + node.query("INSERT INTO test_rewrite_uniq_to_count values ('1', '1', '1'), ('1', '1', '1')") + node.query("INSERT INTO test_rewrite_uniq_to_count values ('2', '2', '2'), ('2', '2', '2')") + node.query("INSERT INTO test_rewrite_uniq_to_count values ('3', '3', '3'), ('3', '3', '3')") + + +def shutdown(): + node.query("DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC") + + +def check(query, result): + # old analyzer + query = query + " settings optimize_uniq_to_count = 1" + assert node.query(query) == f"{result}\n" + assert "count" in node.query("EXPLAIN SYNTAX " + query) + + # # new analyzer + # query = query + ", allow_experimental_analyzer = 1" + # assert node.query(query) == f"{result}\n" + # assert "count" in node.query("EXPLAIN QUERY_TREE " + query) + + +def test_rewrite_distinct(started_cluster): + check("SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count)", + 3) + + check("SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t", + 3) + + check("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t", + 3) + + check("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as n FROM test_rewrite_uniq_to_count) t", + 3) + + +def test_rewrite_group_by(started_cluster): + check("SELECT uniq(a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a)", + 3) + + check("SELECT uniq(t.a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", + 3) + + check("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", + 3) + + check("SELECT uniq(t.a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + 3) + + check("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + 3) From 8004f37e972c9efc7cfc9d66a1157b020e7b6d89 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Mon, 10 Jul 2023 17:09:05 +0800 Subject: [PATCH 0477/2047] rewrite uniq to count in new analyzer --- src/Analyzer/Passes/QueryAnalysisPass.h | 1 + src/Analyzer/Passes/UniqToCountPass.cpp | 166 ++++++++++++++++++ src/Analyzer/Passes/UniqToCountPass.h | 30 ++++ src/Analyzer/QueryTreePassManager.cpp | 2 + src/Core/Settings.h | 2 +- .../RewriteUniqToCountVisitor.cpp | 12 +- src/Interpreters/RewriteUniqToCountVisitor.h | 17 +- .../test_rewrite_uniq_to_count/test.py | 39 +++- 8 files changed, 246 insertions(+), 23 deletions(-) create mode 100644 src/Analyzer/Passes/UniqToCountPass.cpp create mode 100644 src/Analyzer/Passes/UniqToCountPass.h diff --git a/src/Analyzer/Passes/QueryAnalysisPass.h b/src/Analyzer/Passes/QueryAnalysisPass.h index fa8778ebf76..ea845f26bd9 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.h +++ b/src/Analyzer/Passes/QueryAnalysisPass.h @@ -51,6 +51,7 @@ namespace DB * Function `arrayJoin` is handled properly. * For functions `dictGet` and its variations and for function `joinGet` identifier as first argument is handled properly. * Replace `countDistinct` and `countIfDistinct` aggregate functions using setting count_distinct_implementation. + * Replace `uniq` and `uniq` and its variants(except uniqUpTo) into `count` aggregate functions using setting optimize_uniq_to_count. * Add -OrNull suffix to aggregate functions if setting aggregate_functions_null_for_empty is true. * Function `exists` is converted into `in`. * Functions `in`, `notIn`, `globalIn`, `globalNotIn` converted into `nullIn`, `notNullIn`, `globalNullIn`, `globalNotNullIn` if setting transform_null_in is true. diff --git a/src/Analyzer/Passes/UniqToCountPass.cpp b/src/Analyzer/Passes/UniqToCountPass.cpp new file mode 100644 index 00000000000..1ffb83a6e36 --- /dev/null +++ b/src/Analyzer/Passes/UniqToCountPass.cpp @@ -0,0 +1,166 @@ +#include "UniqToCountPass.h" + +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +using Aliases = std::unordered_map; + +namespace +{ + + bool matchFnUniq(String func_name) + { + auto name = Poco::toLower(func_name); + return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" + || name == "uniqCombined64"; + } + + bool nodeEquals(const QueryTreeNodePtr & lhs, const QueryTreeNodePtr & rhs, const Aliases & alias) + { + auto * lhs_node = lhs->as(); + auto * rhs_node = rhs->as(); + + if (lhs_node && rhs_node) + { + if (lhs_node->getColumn() == rhs_node->getColumn()) + return true; + + /// translate alias + if (lhs->hasAlias() && alias.find(lhs->getAlias()) != alias.end()) + lhs_node = alias.find(lhs->getAlias())->second->as(); + + if (rhs->hasAlias() && alias.find(rhs->getAlias()) != alias.end()) + rhs_node = alias.find(rhs->getAlias())->second->as(); + + if (lhs_node && rhs_node && lhs_node == rhs_node) + return true; + } + return false; + } + + bool nodeListEquals(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs, const Aliases & alias) + { + if (lhs.size() != rhs.size()) + return false; + for (size_t i = 0; i < lhs.size(); i++) + { + if (!nodeEquals(lhs[i], rhs[i], alias)) + return false; + } + return true; + } + + bool nodeListContainsAll(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs, const Aliases & alias) + { + if (lhs.size() < rhs.size()) + return false; + for (const auto & re : rhs) + { + auto predicate = [&](const QueryTreeNodePtr & le) { return nodeEquals(le, re, alias); }; + if (std::find_if(lhs.begin(), lhs.end(), predicate) == lhs.end()) + return false; + } + return true; + } + +} + +class UniqToCountVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + void visitImpl(QueryTreeNodePtr & node) + { + if (!getSettings().optimize_uniq_to_count) + return; + + auto * query_node = node->as(); + if (!query_node) + return; + + /// Check that query has only single table expression which is subquery + auto * subquery_node = query_node->getJoinTree()->as(); + if (!subquery_node) + return; + + /// Check that query has only single node in projection + auto & projection_nodes = query_node->getProjection().getNodes(); + if (projection_nodes.size() != 1) + return; + + /// Check that projection_node is a function + auto & projection_node = projection_nodes[0]; + auto * function_node = projection_node->as(); + if (!function_node) + return; + + /// Check that query single projection node is `uniq` or its variants + if (!matchFnUniq(function_node->getFunctionName())) + return; + + /// collect subquery select expressions alias. + /// TODO new analyzer will lose alias info, so we will collect nothing and we can not rewrite SQL with alias. + Aliases alias; + for (auto & subquery_projection_node : subquery_node->getProjection().getNodes()) + { + if (subquery_projection_node->hasAlias()) + alias.insert({subquery_projection_node->getAlias(), subquery_projection_node}); + } + + auto & uniq_arguments_nodes = function_node->getArguments().getNodes(); + + /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' + auto match_subquery_with_distinct = [&]() -> bool + { + if (!subquery_node->isDistinct()) + return false; + /// uniq expression list == subquery group by expression list + if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjection().getNodes(), alias)) + return false; + return true; + }; + + /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' + auto match_subquery_with_group_by = [&]() -> bool + { + if (!subquery_node->hasGroupBy()) + return false; + /// uniq argument node list == subquery group by node list + if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getGroupByNode()->getChildren(), alias)) + return false; + /// subquery select node list must contain all columns in uniq argument node list + if (!nodeListContainsAll(subquery_node->getProjection().getNodes(), uniq_arguments_nodes, alias)) + return false; + return true; + }; + + /// Replace uniq of initial query to count + if (match_subquery_with_distinct() || match_subquery_with_group_by()) + { + AggregateFunctionProperties properties; + auto aggregate_function = AggregateFunctionFactory::instance().get("count", {}, {}, properties); + function_node->resolveAsAggregateFunction(std::move(aggregate_function)); + function_node->getArguments().getNodes().clear(); + query_node->resolveProjectionColumns({{"count()", function_node->getResultType()}}); + } + } +}; + + +void UniqToCountPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + UniqToCountVisitor visitor(std::move(context)); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/UniqToCountPass.h b/src/Analyzer/Passes/UniqToCountPass.h new file mode 100644 index 00000000000..4992d524e5e --- /dev/null +++ b/src/Analyzer/Passes/UniqToCountPass.h @@ -0,0 +1,30 @@ +#pragma once + +#include + +namespace DB +{ + +/** Optimize `uniq` and its variants(except uniqUpTo) into `count` over subquery. + * Example: 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to + * Result: 'SELECT count() FROM (SELECT DISTINCT x ...)' + * + * Example: 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to + * Result: 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' + * + * Note that we can rewrite all uniq variants except uniqUpTo. + */ +class UniqToCountPass final : public IQueryTreePass +{ +public: + String getName() override { return "UniqToCount"; } + + String getDescription() override + { + return "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause."; + } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; +}; + +} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index a6da2a66615..dd75b0f586d 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -246,6 +247,7 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 29ea7fe5cb9..f71346be7b8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -774,7 +774,7 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone if empty.", 0) \ - M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause, it is a RBO based optimization.", 0) + M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause.", 0) // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp index 587a905e4c5..ac42a8a82da 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.cpp +++ b/src/Interpreters/RewriteUniqToCountVisitor.cpp @@ -26,7 +26,7 @@ bool matchFnUniq(String func_name) || name == "uniqCombined64"; } -bool expressionEquals(const ASTPtr & lhs, const ASTPtr & rhs, Aliases & alias) +bool expressionEquals(const ASTPtr & lhs, const ASTPtr & rhs, const Aliases & alias) { if (lhs->getTreeHash() == rhs->getTreeHash()) { @@ -56,7 +56,7 @@ bool expressionEquals(const ASTPtr & lhs, const ASTPtr & rhs, Aliases & alias) return false; } -bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs, Aliases & alias) +bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs, const Aliases & alias) { if (!lhs || !rhs) return false; @@ -71,7 +71,7 @@ bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs, Alia } /// Test whether lhs contains all expressions in rhs. -bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs, Aliases alias) +bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs, const Aliases & alias) { if (!lhs || !rhs) return false; @@ -123,13 +123,14 @@ void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) return; /// collect subquery select expressions alias - std::unordered_map alias; + Aliases alias; for (auto expr : sub_expr_list->children) { if (!expr->tryGetAlias().empty()) alias.insert({expr->tryGetAlias(), expr}); } + /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' auto match_subquery_with_distinct = [&]() -> bool { if (!sub_selectq->distinct) @@ -140,9 +141,10 @@ void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) return true; }; + /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' auto match_subquery_with_group_by = [&]() -> bool { - auto group_by = sub_selectq->groupBy(); // TODO group by type + auto group_by = sub_selectq->groupBy(); if (!group_by) return false; /// uniq expression list == subquery group by expression list diff --git a/src/Interpreters/RewriteUniqToCountVisitor.h b/src/Interpreters/RewriteUniqToCountVisitor.h index 42a86049bb9..94528ccf2ee 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.h +++ b/src/Interpreters/RewriteUniqToCountVisitor.h @@ -9,14 +9,15 @@ namespace DB class ASTFunction; -/// Simple rewrite: -/// 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to -/// 'SELECT count() FROM (SELECT DISTINCT x ...)' -/// -/// 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to -/// 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' -/// -/// Note we can rewrite all uniq variants except uniqUpTo. +/** Optimize `uniq` into `count` over subquery. + * Example: 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to + * Result: 'SELECT count() FROM (SELECT DISTINCT x ...)' + * + * Example: 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to + * Result: 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' + * + * Note that we can rewrite all uniq variants except uniqUpTo. + */ class RewriteUniqToCountMatcher { public: diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py index 2e21a8a50a5..af0c4f09117 100644 --- a/tests/integration/test_rewrite_uniq_to_count/test.py +++ b/tests/integration/test_rewrite_uniq_to_count/test.py @@ -42,40 +42,61 @@ def check(query, result): # old analyzer query = query + " settings optimize_uniq_to_count = 1" assert node.query(query) == f"{result}\n" - assert "count" in node.query("EXPLAIN SYNTAX " + query) + assert "count()" in node.query("EXPLAIN SYNTAX " + query) - # # new analyzer - # query = query + ", allow_experimental_analyzer = 1" - # assert node.query(query) == f"{result}\n" - # assert "count" in node.query("EXPLAIN QUERY_TREE " + query) + # new analyzer + query = query + ", allow_experimental_analyzer = 1" + assert node.query(query) == f"{result}\n" + assert "count()" in node.query("EXPLAIN QUERY TREE " + query) + + +# For new analyzer loses alias info, we can not rewrite SQL with alias. +def check_by_old_analyzer(query, result): + # only old analyzer + query = query + " settings optimize_uniq_to_count = 1" + assert node.query(query) == f"{result}\n" + assert "count()" in node.query("EXPLAIN SYNTAX " + query) def test_rewrite_distinct(started_cluster): + # simple test check("SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count)", 3) + # test subquery alias check("SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t", 3) + # test table.column check("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t", 3) - check("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as n FROM test_rewrite_uniq_to_count) t", + # test select expression alias + check_by_old_analyzer("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", + 3) + + # test select expression alias + check_by_old_analyzer("SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t", 3) def test_rewrite_group_by(started_cluster): + # simple test check("SELECT uniq(a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a)", 3) + # test subquery alias check("SELECT uniq(t.a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", 3) - check("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", + # test select expression alias + check_by_old_analyzer("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", 3) - check("SELECT uniq(t.a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + # test select expression alias + check_by_old_analyzer("SELECT uniq(t.a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", 3) - check("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + # test select expression alias + check_by_old_analyzer("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", 3) From 17891ca1ebb198785d9f8de7bfcef4203bc37d9e Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 10 Jul 2023 09:18:09 +0000 Subject: [PATCH 0478/2047] Add case ins utf8 impl + tests --- .../functions/string-search-functions.md | 2 +- .../functions/string-search-functions.md | 2 +- src/Functions/HasSubsequenceImpl.h | 66 ++++++++----------- src/Functions/hasSubsequence.cpp | 5 +- .../hasSubsequenceCaseInsensitive.cpp | 5 +- .../hasSubsequenceCaseInsensitiveUTF8.cpp | 7 +- src/Functions/hasSubsequenceUTF8.cpp | 5 +- .../02809_has_subsequence.reference | 20 ++++-- .../0_stateless/02809_has_subsequence.sql | 31 +++++---- .../aspell-ignore/en/aspell-dict.txt | 5 ++ 10 files changed, 71 insertions(+), 77 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 04ad6474310..c10a1036677 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -647,7 +647,7 @@ hasSubsequence(haystack, needle) **Arguments** - `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — Subsequence to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal). **Returned values** diff --git a/docs/ru/sql-reference/functions/string-search-functions.md b/docs/ru/sql-reference/functions/string-search-functions.md index 21989e882b6..6e3830869cd 100644 --- a/docs/ru/sql-reference/functions/string-search-functions.md +++ b/docs/ru/sql-reference/functions/string-search-functions.md @@ -816,7 +816,7 @@ hasSubsequence(haystack, needle) **Аргументы** - `haystack` — строка, по которой выполняется поиск. [Строка](../syntax.md#syntax-string-literal). -- `needle` — подстрока, которую необходимо найти. [Строка](../syntax.md#syntax-string-literal). +- `needle` — подпоследовательность, которую необходимо найти. [Строка](../syntax.md#syntax-string-literal). **Возвращаемые значения** diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h index 10fe6215280..ea1826e1e33 100644 --- a/src/Functions/HasSubsequenceImpl.h +++ b/src/Functions/HasSubsequenceImpl.h @@ -3,22 +3,27 @@ #include #include #include -#include -#include + namespace DB { + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; +} namespace { using namespace GatherUtils; template -class FunctionsHasSubsequenceImpl : public IFunction +class HasSubsequenceImpl : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -72,8 +77,9 @@ public: else throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {}, first argument of function {} must be a string", + "Illegal columns {} and {} of arguments of function {}", arguments[0].column->getName(), + arguments[1].column->getName(), getName()); return col_res; @@ -87,43 +93,32 @@ private: SourceNeedle && needles, PaddedPODArray & res_data) const { - size_t row_num = 0; - while (!haystacks.isEnd()) { - [[maybe_unused]] auto haystack_slice = haystacks.getWhole(); - [[maybe_unused]] auto needle_slice = needles.getWhole(); + auto haystack_slice = haystacks.getWhole(); + auto needle_slice = needles.getWhole(); + size_t row_num = haystacks.rowNum(); - auto haystack = std::string(reinterpret_cast(haystack_slice.data), haystack_slice.size); - auto needle = std::string(reinterpret_cast(needle_slice.data), needle_slice.size); - if constexpr (!Impl::is_utf8) - { - Impl::toLowerIfNeed(haystack); - Impl::toLowerIfNeed(needle); - - res_data[row_num] = hasSubsequence(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); - } + res_data[row_num] = hasSubsequence(haystack_slice.data, haystack_slice.size, needle_slice.data, needle_slice.size); else - { - res_data[row_num] = hasSubsequenceUTF8(haystack.c_str(), haystack.size(), needle.c_str(), needle.size()); - } + res_data[row_num] = hasSubsequenceUTF8(haystack_slice.data, haystack_slice.size, needle_slice.data, needle_slice.size); + haystacks.next(); needles.next(); - ++row_num; } } - static UInt8 hasSubsequence(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) + static UInt8 hasSubsequence(const UInt8 * haystack, size_t haystack_size, const UInt8 * needle, size_t needle_size) { size_t j = 0; for (size_t i = 0; (i < haystack_size) && (j < needle_size); i++) - if (needle[j] == haystack[i]) + if (Impl::toLowerIfNeed(needle[j]) == Impl::toLowerIfNeed(haystack[i])) ++j; return j == needle_size; } - static UInt8 hasSubsequenceUTF8(const char * haystack, size_t haystack_size, const char * needle, size_t needle_size) + static UInt8 hasSubsequenceUTF8(const UInt8 * haystack, size_t haystack_size, const UInt8 * needle, size_t needle_size) { const auto * haystack_pos = haystack; const auto * needle_pos = needle; @@ -131,36 +126,27 @@ private: const auto * needle_end = needle + needle_size; if (!needle_size) - { return 1; - } auto haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos); auto needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos); if (!haystack_code_point || !needle_code_point) - { return 0; - } - - while (true) - { - if (needle_code_point == haystack_code_point) + + while (haystack_code_point && needle_code_point) + { + if (Impl::toLowerIfNeed(*needle_code_point) == Impl::toLowerIfNeed(*haystack_code_point)) { needle_pos += UTF8::seqLength(*needle_pos); - if (needle_pos == needle_end) - { + if (needle_pos >= needle_end) break; - } needle_code_point = UTF8::convertUTF8ToCodePoint(needle_pos, needle_end - needle_pos); } haystack_pos += UTF8::seqLength(*haystack_pos); - if (haystack_pos == haystack_end) - { + if (haystack_pos >= haystack_end) break; - } haystack_code_point = UTF8::convertUTF8ToCodePoint(haystack_pos, haystack_end - haystack_pos); } - return needle_pos == needle_end; } }; diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp index acc574c8207..4bcce53b4db 100644 --- a/src/Functions/hasSubsequence.cpp +++ b/src/Functions/hasSubsequence.cpp @@ -1,5 +1,4 @@ #include -#include #include @@ -12,7 +11,7 @@ struct HasSubsequenceCaseSensitiveASCII { static constexpr bool is_utf8 = false; - static void toLowerIfNeed(String & /*s*/) { } + static int toLowerIfNeed(int c) { return c; } }; struct NameHasSubsequence @@ -20,7 +19,7 @@ struct NameHasSubsequence static constexpr auto name = "hasSubsequence"; }; -using FunctionHasSubsequence = FunctionsHasSubsequenceImpl; +using FunctionHasSubsequence = HasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequence) diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp index 68c510794c3..c93bbead58c 100644 --- a/src/Functions/hasSubsequenceCaseInsensitive.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -1,5 +1,4 @@ #include -#include #include namespace DB @@ -11,7 +10,7 @@ struct HasSubsequenceCaseInsensitiveASCII { static constexpr bool is_utf8 = false; - static void toLowerIfNeed(String & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } + static int toLowerIfNeed(int c) { return std::tolower(c); } }; struct NameHasSubsequenceCaseInsensitive @@ -19,7 +18,7 @@ struct NameHasSubsequenceCaseInsensitive static constexpr auto name = "hasSubsequenceCaseInsensitive"; }; -using FunctionHasSubsequenceCaseInsensitive = FunctionsHasSubsequenceImpl; +using FunctionHasSubsequenceCaseInsensitive = HasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceCaseInsensitive) diff --git a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp index d1fb2f5152a..18438bc8b16 100644 --- a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp @@ -1,7 +1,8 @@ #include -#include #include +#include "Poco/Unicode.h" + namespace DB { namespace @@ -11,7 +12,7 @@ struct HasSubsequenceCaseInsensitiveUTF8 { static constexpr bool is_utf8 = true; - //static void toLowerIfNeed(String & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); } + static int toLowerIfNeed(int code_point) { return Poco::Unicode::toLower(code_point); } }; struct NameHasSubsequenceCaseInsensitiveUTF8 @@ -19,7 +20,7 @@ struct NameHasSubsequenceCaseInsensitiveUTF8 static constexpr auto name = "hasSubsequenceCaseInsensitiveUTF8"; }; -using FunctionHasSubsequenceCaseInsensitiveUTF8 = FunctionsHasSubsequenceImpl; +using FunctionHasSubsequenceCaseInsensitiveUTF8 = HasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceCaseInsensitiveUTF8) diff --git a/src/Functions/hasSubsequenceUTF8.cpp b/src/Functions/hasSubsequenceUTF8.cpp index fcdcd28e02b..7a22211eb8c 100644 --- a/src/Functions/hasSubsequenceUTF8.cpp +++ b/src/Functions/hasSubsequenceUTF8.cpp @@ -1,5 +1,4 @@ #include -#include #include @@ -12,7 +11,7 @@ struct HasSubsequenceCaseSensitiveUTF8 { static constexpr bool is_utf8 = true; - // static void toLowerIfNeed(String & /*s*/) { } + static int toLowerIfNeed(int code_point) { return code_point; } }; struct NameHasSubsequenceUTF8 @@ -20,7 +19,7 @@ struct NameHasSubsequenceUTF8 static constexpr auto name = "hasSubsequenceUTF8"; }; -using FunctionHasSubsequenceUTF8 = FunctionsHasSubsequenceImpl; +using FunctionHasSubsequenceUTF8 = HasSubsequenceImpl; } REGISTER_FUNCTION(hasSubsequenceUTF8) diff --git a/tests/queries/0_stateless/02809_has_subsequence.reference b/tests/queries/0_stateless/02809_has_subsequence.reference index 8437a7f4b74..0bf8e4e3a36 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.reference +++ b/tests/queries/0_stateless/02809_has_subsequence.reference @@ -1,4 +1,4 @@ -hasSubsequence / const / const +hasSubsequence 1 1 0 @@ -12,27 +12,23 @@ hasSubsequence / const / const 0 0 0 -hasSubsequence / const / string 1 1 0 -hasSubsequence / string / const 1 1 0 -hasSubsequence / string / string 1 1 0 -hasSubsequenceCaseInsensitive / const / const +hasSubsequenceCaseInsensitive 0 1 1 -hasSubsequenceCaseInsensitive / string / string 0 1 1 -hasSubsequenceUTF8 / const / const +hasSubsequenceUTF8 1 1 0 @@ -47,3 +43,13 @@ hasSubsequenceUTF8 / const / const 0 1 0 +hasSubsequenceCaseInsensitiveUTF8 +0 +1 +1 +1 +0 +1 +0 +1 +0 diff --git a/tests/queries/0_stateless/02809_has_subsequence.sql b/tests/queries/0_stateless/02809_has_subsequence.sql index b8d3280488c..6715d901309 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.sql +++ b/tests/queries/0_stateless/02809_has_subsequence.sql @@ -1,4 +1,4 @@ -select 'hasSubsequence / const / const'; +select 'hasSubsequence'; select hasSubsequence('garbage', ''); select hasSubsequence('garbage', 'g'); select hasSubsequence('garbage', 'G'); @@ -9,39 +9,28 @@ select hasSubsequence('garbage', 'ab'); select hasSubsequence('garbage', 'be'); select hasSubsequence('garbage', 'arg'); select hasSubsequence('garbage', 'garbage'); - select hasSubsequence('garbage', 'garbage1'); select hasSubsequence('garbage', 'arbw'); select hasSubsequence('garbage', 'ARG'); - -select 'hasSubsequence / const / string'; select hasSubsequence('garbage', materialize('')); select hasSubsequence('garbage', materialize('arg')); select hasSubsequence('garbage', materialize('arbw')); - -select 'hasSubsequence / string / const'; select hasSubsequence(materialize('garbage'), ''); select hasSubsequence(materialize('garbage'), 'arg'); select hasSubsequence(materialize('garbage'), 'arbw'); - -select 'hasSubsequence / string / string'; - select hasSubsequence(materialize('garbage'), materialize('')); select hasSubsequence(materialize('garbage'), materialize('arg')); select hasSubsequence(materialize('garbage'), materialize('garbage1')); -select 'hasSubsequenceCaseInsensitive / const / const'; - +select 'hasSubsequenceCaseInsensitive'; select hasSubsequenceCaseInsensitive('garbage', 'w'); select hasSubsequenceCaseInsensitive('garbage', 'ARG'); select hasSubsequenceCaseInsensitive('GARGAGE', 'arg'); - -select 'hasSubsequenceCaseInsensitive / string / string'; select hasSubsequenceCaseInsensitive(materialize('garbage'), materialize('w')); select hasSubsequenceCaseInsensitive(materialize('garbage'), materialize('ARG')); select hasSubsequenceCaseInsensitive(materialize('GARGAGE'), materialize('arg')); -select 'hasSubsequenceUTF8 / const / const'; +select 'hasSubsequenceUTF8'; select hasSubsequence('ClickHouse - столбцовая система управления базами данных', ''); select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'C'); -- eng select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'С'); -- cyrilic @@ -50,10 +39,20 @@ select hasSubsequence('ClickHouse - столбцовая система упра select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'система'); select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'Система'); select hasSubsequence('ClickHouse - столбцовая система управления базами данных', 'ссубд'); - select hasSubsequence(materialize('ClickHouse - столбцовая система управления базами данных'), 'субд'); select hasSubsequence(materialize('ClickHouse - столбцовая система управления базами данных'), 'суббд'); select hasSubsequence('ClickHouse - столбцовая система управления базами данных', materialize('стул')); select hasSubsequence('ClickHouse - столбцовая система управления базами данных', materialize('два стула')); select hasSubsequence(materialize('ClickHouse - столбцовая система управления базами данных'), materialize('орех')); -select hasSubsequence(materialize('ClickHouse - столбцовая система управления базами данных'), materialize('два ореха')); \ No newline at end of file +select hasSubsequence(materialize('ClickHouse - столбцовая система управления базами данных'), materialize('два ореха')); + +select 'hasSubsequenceCaseInsensitiveUTF8'; +select hasSubsequenceCaseInsensitiveUTF8('для онлайн обработки аналитических запросов (OLAP)', 'oltp'); +select hasSubsequenceCaseInsensitiveUTF8('для онлайн обработки аналитических запросов (OLAP)', 'оОоОоO'); +select hasSubsequenceCaseInsensitiveUTF8('для онлайн обработки аналитических запросов (OLAP)', 'я раб'); +select hasSubsequenceCaseInsensitiveUTF8(materialize('для онлайн обработки аналитических запросов (OLAP)'), 'работа'); +select hasSubsequenceCaseInsensitiveUTF8(materialize('для онлайн обработки аналитических запросов (OLAP)'), 'work'); +select hasSubsequenceCaseInsensitiveUTF8('для онлайн обработки аналитических запросов (OLAP)', materialize('добро)')); +select hasSubsequenceCaseInsensitiveUTF8('для онлайн обработки аналитических запросов (OLAP)', materialize('зло()')); +select hasSubsequenceCaseInsensitiveUTF8(materialize('для онлайн обработки аналитических запросов (OLAP)'), materialize('аналитика')); +select hasSubsequenceCaseInsensitiveUTF8(materialize('для онлайн обработки аналитических запросов (OLAP)'), materialize('аналитика для аналитиков')); \ No newline at end of file diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2802e52c288..270e486586e 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1534,6 +1534,10 @@ hadoop halfMD halfday hardlinks +hasSubsequence +hasSubsequenceCaseInsensitive +hasSubsequenceCaseInsensitiveUTF +hasSubsequenceUTF hasAll hasAny hasColumnInTable @@ -2238,6 +2242,7 @@ subquery subranges subreddits subseconds +subsequence substring substringUTF substrings From 4cd12a505333e55c7773163e0c6ea5b296600175 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 10 Jul 2023 09:33:53 +0000 Subject: [PATCH 0479/2047] Remove trailing whitespace --- src/Functions/HasSubsequenceImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h index ea1826e1e33..fda29820298 100644 --- a/src/Functions/HasSubsequenceImpl.h +++ b/src/Functions/HasSubsequenceImpl.h @@ -134,7 +134,7 @@ private: return 0; while (haystack_code_point && needle_code_point) - { + { if (Impl::toLowerIfNeed(*needle_code_point) == Impl::toLowerIfNeed(*haystack_code_point)) { needle_pos += UTF8::seqLength(*needle_pos); From 33405e70f2867e72e498abaeaa1bc8b7317e0284 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 10 Jul 2023 10:01:26 +0000 Subject: [PATCH 0480/2047] Try to fix build --- src/Functions/HasSubsequenceImpl.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h index fda29820298..afbf53d45f9 100644 --- a/src/Functions/HasSubsequenceImpl.h +++ b/src/Functions/HasSubsequenceImpl.h @@ -1,6 +1,9 @@ #pragma once #include +#include +#include +#include #include #include From 66e759ec9651cb04e66dc8aeb6f561a0f97812ab Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 10 Jul 2023 10:18:00 +0000 Subject: [PATCH 0481/2047] try to fix build again --- src/Functions/HasSubsequenceImpl.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/HasSubsequenceImpl.h b/src/Functions/HasSubsequenceImpl.h index afbf53d45f9..17955746aa2 100644 --- a/src/Functions/HasSubsequenceImpl.h +++ b/src/Functions/HasSubsequenceImpl.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include From 2c3ba033799c409cd6f7e65057a85e6c09605670 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 10 Jul 2023 12:11:34 +0000 Subject: [PATCH 0482/2047] Fix 02415_all_new_functions_must_be_documented --- .../02415_all_new_functions_must_be_documented.reference | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index b5c133988e6..d241e2f0d28 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -343,6 +343,10 @@ has hasAll hasAny hasColumnInTable +hasSubsequence +hasSubsequenceCaseInsensitive +hasSubsequenceCaseInsensitiveUTF8 +hasSubsequenceUTF8 hasSubstr hasThreadFuzzer hashid From 522b9ebf8c62cc564f9a2bcef5802d739a208318 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 26 Jan 2023 05:52:12 -0800 Subject: [PATCH 0483/2047] Implement KQL-style formatting for Interval --- .../operations/settings/settings-formats.md | 20 +++++++ src/Common/IntervalKind.cpp | 21 +++++++ src/Common/IntervalKind.h | 4 ++ src/Core/Settings.h | 1 + src/Core/SettingsEnums.cpp | 4 ++ src/Core/SettingsEnums.h | 2 + src/DataTypes/DataTypeInterval.cpp | 9 ++- src/DataTypes/DataTypeInterval.h | 1 + .../Serializations/SerializationInterval.cpp | 55 +++++++++++++++++++ .../Serializations/SerializationInterval.h | 20 +++++++ src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 11 ++++ src/Parsers/Kusto/ParserKQLTimespan.cpp | 27 +++++++++ src/Parsers/Kusto/ParserKQLTimespan.h | 14 +++++ ...02366_kql_native_interval_format.reference | 20 +++++++ .../02366_kql_native_interval_format.sql | 22 ++++++++ 16 files changed, 227 insertions(+), 5 deletions(-) create mode 100644 src/DataTypes/Serializations/SerializationInterval.cpp create mode 100644 src/DataTypes/Serializations/SerializationInterval.h create mode 100644 src/Parsers/Kusto/ParserKQLTimespan.cpp create mode 100644 src/Parsers/Kusto/ParserKQLTimespan.h create mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.reference create mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.sql diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 1b22a6d1223..7ab234399b9 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -242,6 +242,26 @@ See also: - [DateTime data type.](../../sql-reference/data-types/datetime.md) - [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) +## interval_format {#interval_format} + +Allows choosing different output formats of the text representation of interval types. + +Possible values: + +- `kql` - KQL-style output format. + + ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. + +- `numeric` - Numeric output format. + + ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. + +Default value: `numeric`. + +See also: + +- [Interval](../../sql-reference/data-types/special-data-types/interval.md) + ## input_format_ipv4_default_on_conversion_error {#input_format_ipv4_default_on_conversion_error} Deserialization of IPv4 will use default values instead of throwing exception on conversion error. diff --git a/src/Common/IntervalKind.cpp b/src/Common/IntervalKind.cpp index 411fc2886e7..9c653eefedb 100644 --- a/src/Common/IntervalKind.cpp +++ b/src/Common/IntervalKind.cpp @@ -10,6 +10,27 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +Int64 IntervalKind::toAvgNanoseconds() const +{ + static constexpr Int64 NANOSECONDS_PER_MICROSECOND = 1000; + static constexpr auto NANOSECONDS_PER_MILLISECOND = NANOSECONDS_PER_MICROSECOND * 1000; + static constexpr auto NANOSECONDS_PER_SECOND = NANOSECONDS_PER_MILLISECOND * 1000; + + switch (kind) + { + case IntervalKind::Millisecond: + return NANOSECONDS_PER_MILLISECOND; + case IntervalKind::Microsecond: + return NANOSECONDS_PER_MICROSECOND; + case IntervalKind::Nanosecond: + return 1; + default: + return toAvgSeconds() * NANOSECONDS_PER_SECOND; + } + + UNREACHABLE(); +} + Int32 IntervalKind::toAvgSeconds() const { switch (kind) diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index b46805655b1..6893286f196 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -29,6 +29,10 @@ struct IntervalKind constexpr std::string_view toString() const { return magic_enum::enum_name(kind); } + /// Returns number of nanoseconds in one interval. + /// For `Month`, `Quarter` and `Year` the function returns an average number of nanoseconds. + Int64 toAvgNanoseconds() const; + /// Returns number of seconds in one interval. /// For `Month`, `Quarter` and `Year` the function returns an average number of seconds. Int32 toAvgSeconds() const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3250bdf0750..a2d030102ad 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -906,6 +906,7 @@ class IColumn; \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ + M(IntervalFormat, interval_format, FormatSettings::IntervalFormat::Numeric, "Textual representation of Interval. Possible values: 'kql', 'numeric'.", 0) \ \ M(Bool, input_format_ipv4_default_on_conversion_error, false, "Deserialization of IPv4 will use default values instead of throwing exception on conversion error.", 0) \ M(Bool, input_format_ipv6_default_on_conversion_error, false, "Deserialization of IPV6 will use default values instead of throwing exception on conversion error.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 612f8689480..48528081258 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -79,6 +79,10 @@ IMPLEMENT_SETTING_ENUM(DateTimeOutputFormat, ErrorCodes::BAD_ARGUMENTS, {"iso", FormatSettings::DateTimeOutputFormat::ISO}, {"unix_timestamp", FormatSettings::DateTimeOutputFormat::UnixTimestamp}}) +IMPLEMENT_SETTING_ENUM_WITH_RENAME(IntervalFormat, ErrorCodes::BAD_ARGUMENTS, + {{"kql", FormatSettings::IntervalFormat::KQL}, + {"numeric", FormatSettings::IntervalFormat::Numeric}}) + IMPLEMENT_SETTING_AUTO_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS) IMPLEMENT_SETTING_AUTO_ENUM(LogQueriesType, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 1c5be910ef7..94dc4f9aee5 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -72,6 +72,8 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeIn DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOutputFormat, FormatSettings::DateTimeOutputFormat) +DECLARE_SETTING_ENUM_WITH_RENAME(IntervalFormat, FormatSettings::IntervalFormat) + DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion) enum class LogsLevel diff --git a/src/DataTypes/DataTypeInterval.cpp b/src/DataTypes/DataTypeInterval.cpp index 9faf0cec2d8..1649cb591cf 100644 --- a/src/DataTypes/DataTypeInterval.cpp +++ b/src/DataTypes/DataTypeInterval.cpp @@ -1,16 +1,16 @@ -#include -#include - +#include "DataTypeInterval.h" +#include "DataTypeFactory.h" +#include "Serializations/SerializationInterval.h" namespace DB { +SerializationPtr DataTypeInterval::doGetDefaultSerialization() const { return std::make_shared(kind); } bool DataTypeInterval::equals(const IDataType & rhs) const { return typeid(rhs) == typeid(*this) && kind == static_cast(rhs).kind; } - void registerDataTypeInterval(DataTypeFactory & factory) { factory.registerSimpleDataType("IntervalNanosecond", [] { return DataTypePtr(std::make_shared(IntervalKind::Nanosecond)); }); @@ -25,5 +25,4 @@ void registerDataTypeInterval(DataTypeFactory & factory) factory.registerSimpleDataType("IntervalQuarter", [] { return DataTypePtr(std::make_shared(IntervalKind::Quarter)); }); factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared(IntervalKind::Year)); }); } - } diff --git a/src/DataTypes/DataTypeInterval.h b/src/DataTypes/DataTypeInterval.h index 7de56c13b56..c398a54268e 100644 --- a/src/DataTypes/DataTypeInterval.h +++ b/src/DataTypes/DataTypeInterval.h @@ -24,6 +24,7 @@ public: explicit DataTypeInterval(IntervalKind kind_) : kind(kind_) {} + SerializationPtr doGetDefaultSerialization() const override; std::string doGetName() const override { return fmt::format("Interval{}", kind.toString()); } const char * getFamilyName() const override { return "Interval"; } String getSQLCompatibleName() const override { return "TEXT"; } diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp new file mode 100644 index 00000000000..cccb64ed390 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -0,0 +1,55 @@ +#include "SerializationInterval.h" + +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int NOT_IMPLEMENTED; +} + +using ColumnInterval = DataTypeInterval::ColumnType; + +SerializationInterval::SerializationInterval(IntervalKind kind_) + : SerializationCustomSimpleText(DataTypeFactory::instance().get("Int64")->getDefaultSerialization()), kind(std::move(kind_)) +{ +} + +void SerializationInterval::serializeText( + const IColumn & column, const size_t row_num, WriteBuffer & ostr, const FormatSettings & format_settings) const +{ + const auto * interval_column = checkAndGetColumn(column); + if (!interval_column) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); + + if (const auto & format = format_settings.interval.format; format == FormatSettings::IntervalFormat::Numeric) + nested_serialization->serializeText(column, row_num, ostr, format_settings); + else if (format == FormatSettings::IntervalFormat::KQL) + { + const auto & value = interval_column->getData()[row_num]; + const auto ticks = kind.toAvgNanoseconds() * value / 100; + const auto interval_as_string = ParserKQLTimespan::compose(ticks); + ostr.write(interval_as_string.c_str(), interval_as_string.length()); + } + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); +} + +void SerializationInterval::deserializeText( + [[maybe_unused]] IColumn & column, + [[maybe_unused]] ReadBuffer & istr, + [[maybe_unused]] const FormatSettings & format_settings, + [[maybe_unused]] const bool whole) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); +} +} diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h new file mode 100644 index 00000000000..be5f914235f --- /dev/null +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -0,0 +1,20 @@ +#pragma once + +#include "SerializationCustomSimpleText.h" + +#include + +namespace DB +{ +class SerializationInterval : public SerializationCustomSimpleText +{ +public: + explicit SerializationInterval(IntervalKind kind_); + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; + +private: + IntervalKind kind; +}; +} diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index ad991aa0335..fef15f0a56d 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -86,6 +86,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.skip_trailing_empty_lines = settings.input_format_custom_skip_trailing_empty_lines; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; + format_settings.interval.format = settings.interval_format; format_settings.input_format_ipv4_default_on_conversion_error = settings.input_format_ipv4_default_on_conversion_error; format_settings.input_format_ipv6_default_on_conversion_error = settings.input_format_ipv6_default_on_conversion_error; format_settings.bool_true_representation = settings.bool_true_representation; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 07d4a7ede4a..bc2e6aa7297 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -77,6 +77,17 @@ struct FormatSettings DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; + enum class IntervalFormat + { + KQL, + Numeric + }; + + struct + { + IntervalFormat format = IntervalFormat::Numeric; + } interval; + bool input_format_ipv4_default_on_conversion_error = false; bool input_format_ipv6_default_on_conversion_error = false; diff --git a/src/Parsers/Kusto/ParserKQLTimespan.cpp b/src/Parsers/Kusto/ParserKQLTimespan.cpp new file mode 100644 index 00000000000..d8ef4ed0227 --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLTimespan.cpp @@ -0,0 +1,27 @@ +#include "ParserKQLTimespan.h" + +#include + +namespace DB +{ +std::string ParserKQLTimespan::compose(const Int64 ticks) +{ + static constexpr Int64 TICKS_PER_SECOND = 10000000; + static constexpr auto TICKS_PER_MINUTE = TICKS_PER_SECOND * 60; + static constexpr auto TICKS_PER_HOUR = TICKS_PER_MINUTE * 60; + static constexpr auto TICKS_PER_DAY = TICKS_PER_HOUR * 24; + + const auto abs_ticks = std::abs(ticks); + std::string result = ticks < 0 ? "-" : ""; + if (abs_ticks >= TICKS_PER_DAY) + result.append(std::format("{}.", abs_ticks / TICKS_PER_DAY)); + + result.append(std::format( + "{:02}:{:02}:{:02}", (abs_ticks / TICKS_PER_HOUR) % 24, (abs_ticks / TICKS_PER_MINUTE) % 60, (abs_ticks / TICKS_PER_SECOND) % 60)); + + if (const auto fractional_second = abs_ticks % TICKS_PER_SECOND) + result.append(std::format(".{:07}", fractional_second)); + + return result; +} +} diff --git a/src/Parsers/Kusto/ParserKQLTimespan.h b/src/Parsers/Kusto/ParserKQLTimespan.h new file mode 100644 index 00000000000..8251823253c --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLTimespan.h @@ -0,0 +1,14 @@ +#pragma once + +#include + +#include + +namespace DB +{ +class ParserKQLTimespan +{ +public: + static std::string compose(Int64 ticks); +}; +} diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference new file mode 100644 index 00000000000..fa319341791 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -0,0 +1,20 @@ +default format +99 +100 +1 +42 +66 +66 +5 +1 +2 +kql format +00:00:00 +00:00:00.0000001 +00:00:00.0010000 +00:00:42 +01:06:00 +2.18:00:00 +5.00:00:00 +7.00:00:00 +14.00:00:00 diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql b/tests/queries/0_stateless/02366_kql_native_interval_format.sql new file mode 100644 index 00000000000..db987265bb7 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql @@ -0,0 +1,22 @@ +select 'default format'; +select toIntervalNanosecond(99); +select toIntervalNanosecond(100); +select toIntervalMillisecond(1); +select toIntervalSecond(42); +select toIntervalMinute(66); +select toIntervalHour(66); +select toIntervalDay(5); +select toIntervalWeek(1); +select toIntervalWeek(2); + +select 'kql format'; +set interval_format = 'kql'; +select toIntervalNanosecond(99); +select toIntervalNanosecond(100); +select toIntervalMillisecond(1); +select toIntervalSecond(42); +select toIntervalMinute(66); +select toIntervalHour(66); +select toIntervalDay(5); +select toIntervalWeek(1); +select toIntervalWeek(2); From 748c7fe04eef101988c555a064155a5878ba221c Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 27 Jan 2023 06:20:40 -0800 Subject: [PATCH 0484/2047] Refactor functional tests using Jinja2 --- ...02366_kql_native_interval_format.reference | 25 +++++++++++-------- .../02366_kql_native_interval_format.sql | 22 ---------------- .../02366_kql_native_interval_format.sql.j2 | 16 ++++++++++++ 3 files changed, 30 insertions(+), 33 deletions(-) delete mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.sql create mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index fa319341791..6588ca95e2a 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -1,14 +1,5 @@ -default format -99 -100 -1 -42 -66 -66 -5 -1 -2 -kql format +numeric +kql 00:00:00 00:00:00.0000001 00:00:00.0010000 @@ -18,3 +9,15 @@ kql format 5.00:00:00 7.00:00:00 14.00:00:00 +('00:01:12', '21.00:00:00', '00:00:00.0000002') +numeric +99 +100 +1 +42 +66 +66 +5 +1 +2 +('72', '3', '200') diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql b/tests/queries/0_stateless/02366_kql_native_interval_format.sql deleted file mode 100644 index db987265bb7..00000000000 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.sql +++ /dev/null @@ -1,22 +0,0 @@ -select 'default format'; -select toIntervalNanosecond(99); -select toIntervalNanosecond(100); -select toIntervalMillisecond(1); -select toIntervalSecond(42); -select toIntervalMinute(66); -select toIntervalHour(66); -select toIntervalDay(5); -select toIntervalWeek(1); -select toIntervalWeek(2); - -select 'kql format'; -set interval_format = 'kql'; -select toIntervalNanosecond(99); -select toIntervalNanosecond(100); -select toIntervalMillisecond(1); -select toIntervalSecond(42); -select toIntervalMinute(66); -select toIntervalHour(66); -select toIntervalDay(5); -select toIntervalWeek(1); -select toIntervalWeek(2); diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 new file mode 100644 index 00000000000..6f12dfd96db --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 @@ -0,0 +1,16 @@ +select value from system.settings where name = 'interval_format'; + +{% for format in ['kql', 'numeric'] -%} +select '{{ format }}'; +set interval_format = '{{ format }}'; +select toIntervalNanosecond(99); +select toIntervalNanosecond(100); +select toIntervalMillisecond(1); +select toIntervalSecond(42); +select toIntervalMinute(66); +select toIntervalHour(66); +select toIntervalDay(5); +select toIntervalWeek(1); +select toIntervalWeek(2); +select toIntervalSecond(72) + toIntervalWeek(3) + toIntervalNanosecond(200); +{% endfor -%} From b5cef61ef31fe094d4c756a5032ba2a25cc3b25d Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 2 Feb 2023 10:17:24 -0800 Subject: [PATCH 0485/2047] Refactor solution to improve extensibility --- .../Serializations/SerializationInterval.cpp | 270 ++++++++++++++++-- .../Serializations/SerializationInterval.h | 44 ++- ...02366_kql_native_interval_format.reference | 4 +- 3 files changed, 285 insertions(+), 33 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index cccb64ed390..0cd34a78f8e 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -1,5 +1,7 @@ #include "SerializationInterval.h" +#include "SerializationCustomSimpleText.h" + #include #include #include @@ -9,47 +11,263 @@ #include +#include +#include + namespace DB { +using ColumnInterval = DataTypeInterval::ColumnType; + namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int NOT_IMPLEMENTED; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; +} } -using ColumnInterval = DataTypeInterval::ColumnType; - -SerializationInterval::SerializationInterval(IntervalKind kind_) - : SerializationCustomSimpleText(DataTypeFactory::instance().get("Int64")->getDefaultSerialization()), kind(std::move(kind_)) +namespace { -} - -void SerializationInterval::serializeText( - const IColumn & column, const size_t row_num, WriteBuffer & ostr, const FormatSettings & format_settings) const +class SerializationKQLInterval : public DB::SerializationCustomSimpleText { - const auto * interval_column = checkAndGetColumn(column); +public: + explicit SerializationKQLInterval(DB::IntervalKind kind_) : SerializationCustomSimpleText(nullptr), kind(kind_) { } + + void serializeText(const DB::IColumn & column, size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings & settings) const override; + void deserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, bool whole) const override; + +private: + DB::IntervalKind kind; +}; + +void SerializationKQLInterval::serializeText( + const DB::IColumn & column, const size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings &) const +{ + const auto * interval_column = checkAndGetColumn(column); if (!interval_column) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); + throw DB::Exception(DB::ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); - if (const auto & format = format_settings.interval.format; format == FormatSettings::IntervalFormat::Numeric) - nested_serialization->serializeText(column, row_num, ostr, format_settings); - else if (format == FormatSettings::IntervalFormat::KQL) - { - const auto & value = interval_column->getData()[row_num]; - const auto ticks = kind.toAvgNanoseconds() * value / 100; - const auto interval_as_string = ParserKQLTimespan::compose(ticks); - ostr.write(interval_as_string.c_str(), interval_as_string.length()); - } - else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); + const auto & value = interval_column->getData()[row]; + const auto ticks = kind.toAvgNanoseconds() * value / 100; + const auto interval_as_string = DB::ParserKQLTimespan::compose(ticks); + ostr.write(interval_as_string.c_str(), interval_as_string.length()); } -void SerializationInterval::deserializeText( - [[maybe_unused]] IColumn & column, - [[maybe_unused]] ReadBuffer & istr, - [[maybe_unused]] const FormatSettings & format_settings, +void SerializationKQLInterval::deserializeText( + [[maybe_unused]] DB::IColumn & column, + [[maybe_unused]] DB::ReadBuffer & istr, + [[maybe_unused]] const DB::FormatSettings & settings, [[maybe_unused]] const bool whole) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); + throw DB::Exception( + DB::ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); +} + +template Method> +void dispatch( + std::span> serializations, + const Method method, + const DB::FormatSettings::IntervalFormat format, + Args &&... args) +{ + const auto format_index = magic_enum::enum_index(format); + if (!format_index) + throw DB::Exception(DB::ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "No such format exists"); + + const auto & serialization = serializations[*format_index]; + if (!serialization) + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); + + (serialization.get()->*method)(std::forward(args)...); +} +} + +namespace DB +{ +SerializationInterval::SerializationInterval(IntervalKind kind_) +{ + serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::KQL).value()) + = std::make_unique(std::move(kind_)); + serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::Numeric).value()) + = std::make_unique>(); +} + +void SerializationInterval::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast(&ISerialization::deserializeBinary), + settings.interval.format, + field, + istr, + settings); +} + +void SerializationInterval::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast(&ISerialization::deserializeBinary), + settings.interval.format, + column, + istr, + settings); +} + +void SerializationInterval::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const +{ + dispatch( + serializations, + &ISerialization::deserializeBinaryBulk, + FormatSettings::IntervalFormat::Numeric, + column, + istr, + limit, + avg_value_size_hint); +} + +void SerializationInterval::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const +{ + dispatch(serializations, &ISerialization::deserializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, settings, state); +} + + +void SerializationInterval::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const +{ + dispatch( + serializations, + &ISerialization::deserializeBinaryBulkWithMultipleStreams, + FormatSettings::IntervalFormat::Numeric, + column, + limit, + settings, + state, + cache); +} + + +void SerializationInterval::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextCSV, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextEscaped, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextJSON, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextQuoted, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextRaw, settings.interval.format, column, istr, settings); +} + + +void SerializationInterval::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeWholeText, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast(&ISerialization::serializeBinary), + settings.interval.format, + field, + ostr, + settings); +} + +void SerializationInterval::serializeBinary(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast( + &ISerialization::serializeBinary), + settings.interval.format, + column, + row, + ostr, + settings); +} + +void SerializationInterval::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const +{ + dispatch(serializations, &ISerialization::serializeBinaryBulk, FormatSettings::IntervalFormat::Numeric, column, ostr, offset, limit); +} + +void SerializationInterval::serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const +{ + dispatch( + serializations, &ISerialization::serializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, column, settings, state); +} + +void SerializationInterval::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const +{ + dispatch(serializations, &ISerialization::serializeBinaryBulkStateSuffix, FormatSettings::IntervalFormat::Numeric, settings, state); +} + +void SerializationInterval::serializeBinaryBulkWithMultipleStreams( + const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const +{ + dispatch( + serializations, + &ISerialization::serializeBinaryBulkWithMultipleStreams, + FormatSettings::IntervalFormat::Numeric, + column, + offset, + limit, + settings, + state); +} + +void SerializationInterval::serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeText, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextCSV(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextCSV, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextEscaped( + const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextEscaped, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextJSON(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextJSON, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextQuoted( + const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextQuoted, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextRaw, settings.interval.format, column, row, ostr, settings); } } diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h index be5f914235f..5cd13991471 100644 --- a/src/DataTypes/Serializations/SerializationInterval.h +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -1,20 +1,54 @@ #pragma once -#include "SerializationCustomSimpleText.h" +#include "ISerialization.h" +#include #include namespace DB { -class SerializationInterval : public SerializationCustomSimpleText +class SerializationInterval : public ISerialization { public: explicit SerializationInterval(IntervalKind kind_); - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; + void deserializeBinaryBulkStatePrefix(DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; + void deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeBinary(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; + void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; + void serializeBinaryBulkStateSuffix(SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + void serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextCSV(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextEscaped(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextJSON(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextQuoted(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; private: - IntervalKind kind; + std::array, magic_enum::enum_count()> serializations; }; } diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index 6588ca95e2a..9d8043f2556 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -9,7 +9,7 @@ kql 5.00:00:00 7.00:00:00 14.00:00:00 -('00:01:12', '21.00:00:00', '00:00:00.0000002') +('00:01:12','21.00:00:00','00:00:00.0000002') numeric 99 100 @@ -20,4 +20,4 @@ numeric 5 1 2 -('72', '3', '200') +(72,3,200) From b673aa8e6b736b339b3644a3bb91bf6d6b2ccf2d Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 27 Feb 2023 12:47:21 -0800 Subject: [PATCH 0486/2047] Use the dialect configuration --- .../operations/settings/settings-formats.md | 20 --------------- docs/en/operations/settings/settings.md | 20 +++++++++++++++ src/Core/Settings.h | 1 - src/Core/SettingsEnums.cpp | 4 --- src/Core/SettingsEnums.h | 2 -- src/Formats/FormatFactory.cpp | 11 +++++++- ...02366_kql_native_interval_format.reference | 25 +++++++++---------- .../02366_kql_native_interval_format.sql.j2 | 8 +++--- 8 files changed, 45 insertions(+), 46 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 7ab234399b9..1b22a6d1223 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -242,26 +242,6 @@ See also: - [DateTime data type.](../../sql-reference/data-types/datetime.md) - [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) -## interval_format {#interval_format} - -Allows choosing different output formats of the text representation of interval types. - -Possible values: - -- `kql` - KQL-style output format. - - ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. - -- `numeric` - Numeric output format. - - ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. - -Default value: `numeric`. - -See also: - -- [Interval](../../sql-reference/data-types/special-data-types/interval.md) - ## input_format_ipv4_default_on_conversion_error {#input_format_ipv4_default_on_conversion_error} Deserialization of IPv4 will use default values instead of throwing exception on conversion error. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8b969f87a4d..4d5de1ace94 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -98,6 +98,26 @@ Default value: 0. ``` +## dialect {#dialect} + +Allows choosing support for different query languages. + +Possible values: + +- `clickhouse` - ClickHouse SQL. +- `kusto` - Microsoft KQL. +- `kusto_auto` - Tries ClickHouse SQL first, then Microsoft KQL. + +In mode `clickhouse`, ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. + +In modes `kusto` and `kusto_auto`, ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. + +Default value: `clickhouse`. + +See also: + +- [Interval](../../sql-reference/data-types/special-data-types/interval.md) + ## distributed_product_mode {#distributed-product-mode} Changes the behaviour of [distributed subqueries](../../sql-reference/operators/in.md). diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a2d030102ad..3250bdf0750 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -906,7 +906,6 @@ class IColumn; \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ - M(IntervalFormat, interval_format, FormatSettings::IntervalFormat::Numeric, "Textual representation of Interval. Possible values: 'kql', 'numeric'.", 0) \ \ M(Bool, input_format_ipv4_default_on_conversion_error, false, "Deserialization of IPv4 will use default values instead of throwing exception on conversion error.", 0) \ M(Bool, input_format_ipv6_default_on_conversion_error, false, "Deserialization of IPV6 will use default values instead of throwing exception on conversion error.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 48528081258..612f8689480 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -79,10 +79,6 @@ IMPLEMENT_SETTING_ENUM(DateTimeOutputFormat, ErrorCodes::BAD_ARGUMENTS, {"iso", FormatSettings::DateTimeOutputFormat::ISO}, {"unix_timestamp", FormatSettings::DateTimeOutputFormat::UnixTimestamp}}) -IMPLEMENT_SETTING_ENUM_WITH_RENAME(IntervalFormat, ErrorCodes::BAD_ARGUMENTS, - {{"kql", FormatSettings::IntervalFormat::KQL}, - {"numeric", FormatSettings::IntervalFormat::Numeric}}) - IMPLEMENT_SETTING_AUTO_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS) IMPLEMENT_SETTING_AUTO_ENUM(LogQueriesType, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 94dc4f9aee5..1c5be910ef7 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -72,8 +72,6 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeIn DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOutputFormat, FormatSettings::DateTimeOutputFormat) -DECLARE_SETTING_ENUM_WITH_RENAME(IntervalFormat, FormatSettings::IntervalFormat) - DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion) enum class LogsLevel diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index fef15f0a56d..f26de67169a 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -19,6 +19,15 @@ #include +namespace +{ +DB::FormatSettings::IntervalFormat convert(const DB::Dialect dialect) +{ + return dialect == DB::Dialect::kusto || dialect == DB::Dialect::kusto_auto ? DB::FormatSettings::IntervalFormat::KQL + : DB::FormatSettings::IntervalFormat::Numeric; +} +} + namespace DB { @@ -86,7 +95,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.skip_trailing_empty_lines = settings.input_format_custom_skip_trailing_empty_lines; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; - format_settings.interval.format = settings.interval_format; + format_settings.interval.format = convert(context->getSettingsRef().dialect); format_settings.input_format_ipv4_default_on_conversion_error = settings.input_format_ipv4_default_on_conversion_error; format_settings.input_format_ipv6_default_on_conversion_error = settings.input_format_ipv6_default_on_conversion_error; format_settings.bool_true_representation = settings.bool_true_representation; diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index 9d8043f2556..06603a2fec6 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -1,16 +1,4 @@ -numeric -kql -00:00:00 -00:00:00.0000001 -00:00:00.0010000 -00:00:42 -01:06:00 -2.18:00:00 -5.00:00:00 -7.00:00:00 -14.00:00:00 -('00:01:12','21.00:00:00','00:00:00.0000002') -numeric +clickhouse 99 100 1 @@ -21,3 +9,14 @@ numeric 1 2 (72,3,200) +kusto_auto +00:00:00 +00:00:00.0000001 +00:00:00.0010000 +00:00:42 +01:06:00 +2.18:00:00 +5.00:00:00 +7.00:00:00 +14.00:00:00 +('00:01:12','21.00:00:00','00:00:00.0000002') diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 index 6f12dfd96db..a6b4535cc39 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 @@ -1,8 +1,6 @@ -select value from system.settings where name = 'interval_format'; - -{% for format in ['kql', 'numeric'] -%} -select '{{ format }}'; -set interval_format = '{{ format }}'; +{% for dialect in ['clickhouse', 'kusto_auto'] -%} +select '{{ dialect }}'; +set dialect = '{{ dialect }}'; select toIntervalNanosecond(99); select toIntervalNanosecond(100); select toIntervalMillisecond(1); From 2d2debe3ce7b1698b85a5771afad077daa6eae4b Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 26 Jun 2023 11:15:19 -0700 Subject: [PATCH 0487/2047] Introduce a separate setting for interval output formatting --- .../operations/settings/settings-formats.md | 20 +++ src/Core/Settings.h | 1 + src/Core/SettingsEnums.cpp | 4 + src/Core/SettingsEnums.h | 2 + .../Serializations/SerializationInterval.cpp | 135 +++++------------- .../Serializations/SerializationInterval.h | 33 ++++- src/Formats/FormatFactory.cpp | 11 +- src/Formats/FormatSettings.h | 6 +- ...02366_kql_native_interval_format.reference | 25 ++-- .../02366_kql_native_interval_format.sql.j2 | 8 +- 10 files changed, 117 insertions(+), 128 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 1b22a6d1223..53fd9f8ba5b 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -242,6 +242,26 @@ See also: - [DateTime data type.](../../sql-reference/data-types/datetime.md) - [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) +## interval_output_format {#interval_output_format} + +Allows choosing different output formats of the text representation of interval types. + +Possible values: + +- `kusto` - KQL-style output format. + + ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. + +- `numeric` - Numeric output format. + + ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. + +Default value: `numeric`. + +See also: + +- [Interval](../../sql-reference/data-types/special-data-types/interval.md) + ## input_format_ipv4_default_on_conversion_error {#input_format_ipv4_default_on_conversion_error} Deserialization of IPv4 will use default values instead of throwing exception on conversion error. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3250bdf0750..2a78fda03b5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -906,6 +906,7 @@ class IColumn; \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ + M(IntervalOutputFormat, interval_output_format, FormatSettings::IntervalOutputFormat::Numeric, "Textual representation of Interval. Possible values: 'kusto', 'numeric'.", 0) \ \ M(Bool, input_format_ipv4_default_on_conversion_error, false, "Deserialization of IPv4 will use default values instead of throwing exception on conversion error.", 0) \ M(Bool, input_format_ipv6_default_on_conversion_error, false, "Deserialization of IPV6 will use default values instead of throwing exception on conversion error.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 612f8689480..1e2cbce9309 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -79,6 +79,10 @@ IMPLEMENT_SETTING_ENUM(DateTimeOutputFormat, ErrorCodes::BAD_ARGUMENTS, {"iso", FormatSettings::DateTimeOutputFormat::ISO}, {"unix_timestamp", FormatSettings::DateTimeOutputFormat::UnixTimestamp}}) +IMPLEMENT_SETTING_ENUM(IntervalOutputFormat, ErrorCodes::BAD_ARGUMENTS, + {{"kusto", FormatSettings::IntervalOutputFormat::Kusto}, + {"numeric", FormatSettings::IntervalOutputFormat::Numeric}}) + IMPLEMENT_SETTING_AUTO_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS) IMPLEMENT_SETTING_AUTO_ENUM(LogQueriesType, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 1c5be910ef7..c2783447441 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -72,6 +72,8 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeIn DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOutputFormat, FormatSettings::DateTimeOutputFormat) +DECLARE_SETTING_ENUM_WITH_RENAME(IntervalOutputFormat, FormatSettings::IntervalOutputFormat) + DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion) enum class LogsLevel diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index 0cd34a78f8e..9ffa2b31ec3 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -1,19 +1,9 @@ #include "SerializationInterval.h" -#include "SerializationCustomSimpleText.h" - -#include -#include -#include -#include +#include #include #include -#include - -#include -#include - namespace DB { using ColumnInterval = DataTypeInterval::ColumnType; @@ -24,80 +14,39 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; } -} -namespace +void SerializationKustoInterval::serializeText( + const IColumn & column, const size_t row, WriteBuffer & ostr, const FormatSettings &) const { -class SerializationKQLInterval : public DB::SerializationCustomSimpleText -{ -public: - explicit SerializationKQLInterval(DB::IntervalKind kind_) : SerializationCustomSimpleText(nullptr), kind(kind_) { } - - void serializeText(const DB::IColumn & column, size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings & settings) const override; - void deserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, bool whole) const override; - -private: - DB::IntervalKind kind; -}; - -void SerializationKQLInterval::serializeText( - const DB::IColumn & column, const size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings &) const -{ - const auto * interval_column = checkAndGetColumn(column); + const auto * interval_column = checkAndGetColumn(column); if (!interval_column) - throw DB::Exception(DB::ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); const auto & value = interval_column->getData()[row]; const auto ticks = kind.toAvgNanoseconds() * value / 100; - const auto interval_as_string = DB::ParserKQLTimespan::compose(ticks); + const auto interval_as_string = ParserKQLTimespan::compose(ticks); ostr.write(interval_as_string.c_str(), interval_as_string.length()); } -void SerializationKQLInterval::deserializeText( - [[maybe_unused]] DB::IColumn & column, - [[maybe_unused]] DB::ReadBuffer & istr, - [[maybe_unused]] const DB::FormatSettings & settings, +void SerializationKustoInterval::deserializeText( + [[maybe_unused]] IColumn & column, + [[maybe_unused]] ReadBuffer & istr, + [[maybe_unused]] const FormatSettings & settings, [[maybe_unused]] const bool whole) const { - throw DB::Exception( - DB::ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); } -template Method> -void dispatch( - std::span> serializations, - const Method method, - const DB::FormatSettings::IntervalFormat format, - Args &&... args) +SerializationInterval::SerializationInterval(IntervalKind interval_kind_) : interval_kind(std::move(interval_kind_)) { - const auto format_index = magic_enum::enum_index(format); - if (!format_index) - throw DB::Exception(DB::ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "No such format exists"); - - const auto & serialization = serializations[*format_index]; - if (!serialization) - throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); - - (serialization.get()->*method)(std::forward(args)...); -} -} - -namespace DB -{ -SerializationInterval::SerializationInterval(IntervalKind kind_) -{ - serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::KQL).value()) - = std::make_unique(std::move(kind_)); - serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::Numeric).value()) - = std::make_unique>(); } void SerializationInterval::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { dispatch( - serializations, static_cast(&ISerialization::deserializeBinary), - settings.interval.format, + settings.interval.output_format, field, istr, settings); @@ -106,9 +55,8 @@ void SerializationInterval::deserializeBinary(Field & field, ReadBuffer & istr, void SerializationInterval::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { dispatch( - serializations, static_cast(&ISerialization::deserializeBinary), - settings.interval.format, + settings.interval.output_format, column, istr, settings); @@ -117,19 +65,13 @@ void SerializationInterval::deserializeBinary(IColumn & column, ReadBuffer & ist void SerializationInterval::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const { dispatch( - serializations, - &ISerialization::deserializeBinaryBulk, - FormatSettings::IntervalFormat::Numeric, - column, - istr, - limit, - avg_value_size_hint); + &ISerialization::deserializeBinaryBulk, FormatSettings::IntervalOutputFormat::Numeric, column, istr, limit, avg_value_size_hint); } void SerializationInterval::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { - dispatch(serializations, &ISerialization::deserializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, settings, state); + dispatch(&ISerialization::deserializeBinaryBulkStatePrefix, FormatSettings::IntervalOutputFormat::Numeric, settings, state); } @@ -141,9 +83,8 @@ void SerializationInterval::deserializeBinaryBulkWithMultipleStreams( SubstreamsCache * cache) const { dispatch( - serializations, &ISerialization::deserializeBinaryBulkWithMultipleStreams, - FormatSettings::IntervalFormat::Numeric, + FormatSettings::IntervalOutputFormat::Numeric, column, limit, settings, @@ -154,41 +95,40 @@ void SerializationInterval::deserializeBinaryBulkWithMultipleStreams( void SerializationInterval::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextCSV, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextCSV, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextEscaped, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextEscaped, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextJSON, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextJSON, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextQuoted, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextQuoted, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextRaw, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextRaw, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeWholeText, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeWholeText, settings.interval.output_format, column, istr, settings); } void SerializationInterval::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { dispatch( - serializations, static_cast(&ISerialization::serializeBinary), - settings.interval.format, + settings.interval.output_format, field, ostr, settings); @@ -197,10 +137,9 @@ void SerializationInterval::serializeBinary(const Field & field, WriteBuffer & o void SerializationInterval::serializeBinary(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { dispatch( - serializations, static_cast( &ISerialization::serializeBinary), - settings.interval.format, + settings.interval.output_format, column, row, ostr, @@ -209,29 +148,27 @@ void SerializationInterval::serializeBinary(const IColumn & column, size_t row, void SerializationInterval::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const { - dispatch(serializations, &ISerialization::serializeBinaryBulk, FormatSettings::IntervalFormat::Numeric, column, ostr, offset, limit); + dispatch(&ISerialization::serializeBinaryBulk, FormatSettings::IntervalOutputFormat::Numeric, column, ostr, offset, limit); } void SerializationInterval::serializeBinaryBulkStatePrefix( const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - dispatch( - serializations, &ISerialization::serializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, column, settings, state); + dispatch(&ISerialization::serializeBinaryBulkStatePrefix, FormatSettings::IntervalOutputFormat::Numeric, column, settings, state); } void SerializationInterval::serializeBinaryBulkStateSuffix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - dispatch(serializations, &ISerialization::serializeBinaryBulkStateSuffix, FormatSettings::IntervalFormat::Numeric, settings, state); + dispatch(&ISerialization::serializeBinaryBulkStateSuffix, FormatSettings::IntervalOutputFormat::Numeric, settings, state); } void SerializationInterval::serializeBinaryBulkWithMultipleStreams( const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { dispatch( - serializations, &ISerialization::serializeBinaryBulkWithMultipleStreams, - FormatSettings::IntervalFormat::Numeric, + FormatSettings::IntervalOutputFormat::Numeric, column, offset, limit, @@ -241,33 +178,33 @@ void SerializationInterval::serializeBinaryBulkWithMultipleStreams( void SerializationInterval::serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeText, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeText, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextCSV(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextCSV, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextCSV, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextEscaped( const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextEscaped, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextEscaped, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextJSON(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextJSON, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextJSON, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextQuoted( const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextQuoted, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextQuoted, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextRaw, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextRaw, settings.interval.output_format, column, row, ostr, settings); } } diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h index 5cd13991471..e1f0aac0e78 100644 --- a/src/DataTypes/Serializations/SerializationInterval.h +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -1,12 +1,26 @@ #pragma once #include "ISerialization.h" +#include "SerializationCustomSimpleText.h" +#include #include #include namespace DB { +class SerializationKustoInterval : public SerializationCustomSimpleText +{ +public: + explicit SerializationKustoInterval(IntervalKind kind_) : SerializationCustomSimpleText(nullptr), kind(kind_) { } + + void serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; + +private: + IntervalKind kind; +}; + class SerializationInterval : public ISerialization { public: @@ -49,6 +63,23 @@ public: void serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; private: - std::array, magic_enum::enum_count()> serializations; + template Method> + void dispatch(const Method method, const FormatSettings::IntervalOutputFormat format, Args &&... args) const + { + const ISerialization * serialization = nullptr; + if (format == FormatSettings::IntervalOutputFormat::Kusto) + serialization = &serialization_kusto; + else if (format == FormatSettings::IntervalOutputFormat::Numeric) + serialization = &serialization_numeric; + + if (!serialization) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); + + (serialization->*method)(std::forward(args)...); + } + + IntervalKind interval_kind; + SerializationKustoInterval serialization_kusto{interval_kind}; + SerializationNumber serialization_numeric; }; } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index f26de67169a..7f33643366d 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -19,15 +19,6 @@ #include -namespace -{ -DB::FormatSettings::IntervalFormat convert(const DB::Dialect dialect) -{ - return dialect == DB::Dialect::kusto || dialect == DB::Dialect::kusto_auto ? DB::FormatSettings::IntervalFormat::KQL - : DB::FormatSettings::IntervalFormat::Numeric; -} -} - namespace DB { @@ -95,7 +86,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.skip_trailing_empty_lines = settings.input_format_custom_skip_trailing_empty_lines; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; - format_settings.interval.format = convert(context->getSettingsRef().dialect); + format_settings.interval.output_format = settings.interval_output_format; format_settings.input_format_ipv4_default_on_conversion_error = settings.input_format_ipv4_default_on_conversion_error; format_settings.input_format_ipv6_default_on_conversion_error = settings.input_format_ipv6_default_on_conversion_error; format_settings.bool_true_representation = settings.bool_true_representation; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index bc2e6aa7297..e870c61a476 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -77,15 +77,15 @@ struct FormatSettings DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; - enum class IntervalFormat + enum class IntervalOutputFormat { - KQL, + Kusto, Numeric }; struct { - IntervalFormat format = IntervalFormat::Numeric; + IntervalOutputFormat output_format = IntervalOutputFormat::Numeric; } interval; bool input_format_ipv4_default_on_conversion_error = false; diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index 06603a2fec6..8a12c6885c4 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -1,15 +1,5 @@ -clickhouse -99 -100 -1 -42 -66 -66 -5 -1 -2 -(72,3,200) -kusto_auto +numeric +kusto 00:00:00 00:00:00.0000001 00:00:00.0010000 @@ -20,3 +10,14 @@ kusto_auto 7.00:00:00 14.00:00:00 ('00:01:12','21.00:00:00','00:00:00.0000002') +numeric +99 +100 +1 +42 +66 +66 +5 +1 +2 +(72,3,200) diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 index a6b4535cc39..0731687222d 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 @@ -1,6 +1,8 @@ -{% for dialect in ['clickhouse', 'kusto_auto'] -%} -select '{{ dialect }}'; -set dialect = '{{ dialect }}'; +select value from system.settings where name = 'interval_output_format'; + +{% for format in ['kusto', 'numeric'] -%} +select '{{ format }}'; +set interval_output_format = '{{ format }}'; select toIntervalNanosecond(99); select toIntervalNanosecond(100); select toIntervalMillisecond(1); From 54c7e6504fc0d01ad813cea44be5ebced5c91798 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 26 Jun 2023 11:55:23 -0700 Subject: [PATCH 0488/2047] Fix style issues --- src/DataTypes/Serializations/SerializationInterval.cpp | 1 - src/DataTypes/Serializations/SerializationInterval.h | 5 +++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index 9ffa2b31ec3..a6d22481b88 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -12,7 +12,6 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int NOT_IMPLEMENTED; - extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; } void SerializationKustoInterval::serializeText( diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h index e1f0aac0e78..a4e6c204e4f 100644 --- a/src/DataTypes/Serializations/SerializationInterval.h +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + class SerializationKustoInterval : public SerializationCustomSimpleText { public: From c2514279e4fdda66491ade958cb334f284d19fba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 01:31:30 +0300 Subject: [PATCH 0489/2047] Update settings.md --- docs/en/operations/settings/settings.md | 20 -------------------- 1 file changed, 20 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4d5de1ace94..8b969f87a4d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -98,26 +98,6 @@ Default value: 0. ``` -## dialect {#dialect} - -Allows choosing support for different query languages. - -Possible values: - -- `clickhouse` - ClickHouse SQL. -- `kusto` - Microsoft KQL. -- `kusto_auto` - Tries ClickHouse SQL first, then Microsoft KQL. - -In mode `clickhouse`, ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. - -In modes `kusto` and `kusto_auto`, ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. - -Default value: `clickhouse`. - -See also: - -- [Interval](../../sql-reference/data-types/special-data-types/interval.md) - ## distributed_product_mode {#distributed-product-mode} Changes the behaviour of [distributed subqueries](../../sql-reference/operators/in.md). From 8b3848affed8dfbe61a805f7ceeca6689798e4cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 01:32:15 +0300 Subject: [PATCH 0490/2047] Update DataTypeInterval.cpp --- src/DataTypes/DataTypeInterval.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/DataTypeInterval.cpp b/src/DataTypes/DataTypeInterval.cpp index 1649cb591cf..b8b7640300c 100644 --- a/src/DataTypes/DataTypeInterval.cpp +++ b/src/DataTypes/DataTypeInterval.cpp @@ -1,6 +1,6 @@ -#include "DataTypeInterval.h" -#include "DataTypeFactory.h" -#include "Serializations/SerializationInterval.h" +#include +#include +#include namespace DB { From ba267f05c9dd16c7d02f3f2ec249f72d11014712 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 01:32:42 +0300 Subject: [PATCH 0491/2047] Update DataTypeInterval.cpp --- src/DataTypes/DataTypeInterval.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/DataTypes/DataTypeInterval.cpp b/src/DataTypes/DataTypeInterval.cpp index b8b7640300c..f8fe8bb3b4b 100644 --- a/src/DataTypes/DataTypeInterval.cpp +++ b/src/DataTypes/DataTypeInterval.cpp @@ -2,8 +2,10 @@ #include #include + namespace DB { + SerializationPtr DataTypeInterval::doGetDefaultSerialization() const { return std::make_shared(kind); } bool DataTypeInterval::equals(const IDataType & rhs) const @@ -25,4 +27,5 @@ void registerDataTypeInterval(DataTypeFactory & factory) factory.registerSimpleDataType("IntervalQuarter", [] { return DataTypePtr(std::make_shared(IntervalKind::Quarter)); }); factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared(IntervalKind::Year)); }); } + } From 9a74b971846c74f566d3ad8a2cc466961a374d56 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 6 Jul 2023 07:50:29 -0700 Subject: [PATCH 0492/2047] Improve naming --- .../Serializations/SerializationInterval.cpp | 4 ++-- .../{ParserKQLTimespan.cpp => Formatters.cpp} | 4 ++-- src/Parsers/Kusto/Formatters.h | 10 ++++++++++ src/Parsers/Kusto/ParserKQLTimespan.h | 14 -------------- 4 files changed, 14 insertions(+), 18 deletions(-) rename src/Parsers/Kusto/{ParserKQLTimespan.cpp => Formatters.cpp} (90%) create mode 100644 src/Parsers/Kusto/Formatters.h delete mode 100644 src/Parsers/Kusto/ParserKQLTimespan.h diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index a6d22481b88..59086d8aef3 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB { @@ -23,7 +23,7 @@ void SerializationKustoInterval::serializeText( const auto & value = interval_column->getData()[row]; const auto ticks = kind.toAvgNanoseconds() * value / 100; - const auto interval_as_string = ParserKQLTimespan::compose(ticks); + const auto interval_as_string = formatKQLTimespan(ticks); ostr.write(interval_as_string.c_str(), interval_as_string.length()); } diff --git a/src/Parsers/Kusto/ParserKQLTimespan.cpp b/src/Parsers/Kusto/Formatters.cpp similarity index 90% rename from src/Parsers/Kusto/ParserKQLTimespan.cpp rename to src/Parsers/Kusto/Formatters.cpp index d8ef4ed0227..f12af479445 100644 --- a/src/Parsers/Kusto/ParserKQLTimespan.cpp +++ b/src/Parsers/Kusto/Formatters.cpp @@ -1,10 +1,10 @@ -#include "ParserKQLTimespan.h" +#include "Formatters.h" #include namespace DB { -std::string ParserKQLTimespan::compose(const Int64 ticks) +std::string formatKQLTimespan(const Int64 ticks) { static constexpr Int64 TICKS_PER_SECOND = 10000000; static constexpr auto TICKS_PER_MINUTE = TICKS_PER_SECOND * 60; diff --git a/src/Parsers/Kusto/Formatters.h b/src/Parsers/Kusto/Formatters.h new file mode 100644 index 00000000000..16f52baf941 --- /dev/null +++ b/src/Parsers/Kusto/Formatters.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +#include + +namespace DB +{ +std::string formatKQLTimespan(Int64 ticks); +} diff --git a/src/Parsers/Kusto/ParserKQLTimespan.h b/src/Parsers/Kusto/ParserKQLTimespan.h deleted file mode 100644 index 8251823253c..00000000000 --- a/src/Parsers/Kusto/ParserKQLTimespan.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include - -#include - -namespace DB -{ -class ParserKQLTimespan -{ -public: - static std::string compose(Int64 ticks); -}; -} From bea58a2780d030e5d84dd8a6d29a36fe40ce75c2 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 10 Jul 2023 17:21:37 -0400 Subject: [PATCH 0493/2047] Docs: added two methods for user profile --- docs/en/operations/settings/index.md | 91 +++++++++++++++++++++++++++- 1 file changed, 88 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index eb1d5db5676..022b50925c4 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -11,9 +11,9 @@ There are multiple ways to define ClickHouse settings. Settings are configured i The order of priority for defining a setting is: -1. Settings in the `users.xml` server configuration file - - - Set in the element ``. +1. Settings in user profiles. Set user profiles either by: + - SQL (recommended) + - adding one or more XML or YAML files to `/etc/clickhouse-server/users.d` 2. Session settings @@ -28,6 +28,91 @@ The order of priority for defining a setting is: View the [Settings](./settings.md) page for a description of the ClickHouse settings. +## Examples + +These examples all set the value of the `async_insert` setting to `1`, and show how to examine +the settings in a running system. + +### Using SQL to create a user profile + +```sql +CREATE +SETTINGS PROFILE log_ingest SETTINGS async_insert = 1 +``` + +```sql +CREATE USER ingester +IDENTIFIED WITH sha256_hash BY '7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3' +# highlight-next-line +SETTINGS PROFILE log_ingest +``` + +#### Examine the user profile and assignment + +```sql +SHOW ACCESS +``` + +```response +┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ +│ CREATE USER default IDENTIFIED WITH sha256_password │ +# highlight-next-line +│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ +│ CREATE SETTINGS PROFILE default │ +# highlight-next-line +│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ +│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ +│ ... │ +└────────────────────────────────────────────────────────────────────────────────────┘ +``` + +### Using XML to create a user profile + +```xml title=/etc/clickhouse-server/users.d/users.xml + +# highlight-start + + + 1 + + +# highlight-end + + + + 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 +# highlight-start + log_ingest +# highlight-end + + + 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 + 1 + 1 + + + +``` + +#### Examine the user profile and assignment + +```sql +SHOW ACCESS +``` + +```response +┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ +│ CREATE USER default IDENTIFIED WITH sha256_password │ +# highlight-next-line +│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ +│ CREATE SETTINGS PROFILE default │ +# highlight-next-line +│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ +│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ +│ ... │ +└────────────────────────────────────────────────────────────────────────────────────┘ +``` + ## Converting a Setting to its Default Value If you change a setting and would like to revert it back to its default value, set the value to `DEFAULT`. The syntax looks like: From 77e45e5669eae8ae05c5acdc5b0cbf6647d1f50a Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 10 Jul 2023 20:23:03 -0400 Subject: [PATCH 0494/2047] Add `DROP ROLE ... FROM` and `CREATE ROLE ... AT` syntax --- src/Access/AccessControl.cpp | 2 +- src/Access/DiskAccessStorage.h | 2 +- src/Access/MultipleAccessStorage.cpp | 35 +++++++++ src/Access/MultipleAccessStorage.h | 5 ++ src/Access/UsersConfigAccessStorage.h | 2 +- .../Access/InterpreterCreateRoleQuery.cpp | 42 +++++++++-- .../InterpreterDropAccessEntityQuery.cpp | 18 +++-- src/Parsers/Access/ASTCreateRoleQuery.cpp | 6 ++ src/Parsers/Access/ASTCreateRoleQuery.h | 1 + .../Access/ASTDropAccessEntityQuery.cpp | 5 ++ src/Parsers/Access/ASTDropAccessEntityQuery.h | 1 + src/Parsers/Access/ParserCreateRoleQuery.cpp | 5 ++ .../Access/ParserDropAccessEntityQuery.cpp | 5 ++ src/Parsers/Access/parseUserName.h | 5 ++ .../System/StorageSystemRoleGrants.cpp | 7 +- .../__init__.py | 0 .../configs/roles.xml | 13 ++++ .../test.py | 72 +++++++++++++++++++ 18 files changed, 211 insertions(+), 15 deletions(-) create mode 100644 tests/integration/test_multi_access_storage_role_management/__init__.py create mode 100644 tests/integration/test_multi_access_storage_role_management/configs/roles.xml create mode 100644 tests/integration/test_multi_access_storage_role_management/test.py diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 6179c823b56..0c61a9e3e1b 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -415,7 +415,7 @@ void AccessControl::addStoragesFromUserDirectoriesConfig( String type = key_in_user_directories; if (size_t bracket_pos = type.find('['); bracket_pos != String::npos) type.resize(bracket_pos); - if ((type == "users_xml") || (type == "users_config")) + if ((type == "users.xml") || (type == "users_config")) type = UsersConfigAccessStorage::STORAGE_TYPE; else if ((type == "local") || (type == "local_directory")) type = DiskAccessStorage::STORAGE_TYPE; diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 069a966c8e9..e3bd32c0292 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -13,7 +13,7 @@ class AccessChangesNotifier; class DiskAccessStorage : public IAccessStorage { public: - static constexpr char STORAGE_TYPE[] = "local directory"; + static constexpr char STORAGE_TYPE[] = "local_directory"; DiskAccessStorage(const String & storage_name_, const String & directory_path_, AccessChangesNotifier & changes_notifier_, bool readonly_, bool allow_backup_); ~DiskAccessStorage() override; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index f36a670ddbf..f36beb3f80b 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes { extern const int ACCESS_ENTITY_ALREADY_EXISTS; extern const int ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND; + extern const int ACCESS_ENTITY_NOT_FOUND; } using Storage = IAccessStorage; @@ -178,6 +179,40 @@ ConstStoragePtr MultipleAccessStorage::getStorage(const UUID & id) const return const_cast(this)->getStorage(id); } +StoragePtr MultipleAccessStorage::findStorageByName(const DB::String & storage_name) +{ + auto storages = getStoragesInternal(); + for (const auto & storage : *storages) + { + if (storage->getStorageName() == storage_name) + return storage; + } + + return nullptr; +} + + +ConstStoragePtr MultipleAccessStorage::findStorageByName(const DB::String & storage_name) const +{ + return const_cast(this)->findStorageByName(storage_name); +} + + +StoragePtr MultipleAccessStorage::getStorageByName(const DB::String & storage_name) +{ + auto storage = findStorageByName(storage_name); + if (storage) + return storage; + + throw Exception(ErrorCodes::ACCESS_ENTITY_NOT_FOUND, "Access storage with name {} is not found", storage_name); +} + + +ConstStoragePtr MultipleAccessStorage::getStorageByName(const DB::String & storage_name) const +{ + return const_cast(this)->getStorageByName(storage_name); +} + AccessEntityPtr MultipleAccessStorage::readImpl(const UUID & id, bool throw_if_not_exists) const { if (auto storage = findStorage(id)) diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 6a0c1bdfc02..1898c6ba5d3 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -41,6 +41,11 @@ public: ConstStoragePtr getStorage(const UUID & id) const; StoragePtr getStorage(const UUID & id); + ConstStoragePtr findStorageByName(const String & storage_name) const; + StoragePtr findStorageByName(const String & storage_name); + ConstStoragePtr getStorageByName(const String & storage_name) const; + StoragePtr getStorageByName(const String & storage_name); + bool exists(const UUID & id) const override; bool isBackupAllowed() const override; diff --git a/src/Access/UsersConfigAccessStorage.h b/src/Access/UsersConfigAccessStorage.h index b533ccbf200..d8ddf07ff40 100644 --- a/src/Access/UsersConfigAccessStorage.h +++ b/src/Access/UsersConfigAccessStorage.h @@ -20,7 +20,7 @@ class UsersConfigAccessStorage : public IAccessStorage { public: - static constexpr char STORAGE_TYPE[] = "users.xml"; + static constexpr char STORAGE_TYPE[] = "users_xml"; UsersConfigAccessStorage(const String & storage_name_, AccessControl & access_control_, bool allow_backup_); ~UsersConfigAccessStorage() override; diff --git a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index 3386dfb8792..87abe002d3d 100644 --- a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -8,6 +8,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateRoleFromQueryImpl( @@ -52,6 +58,15 @@ BlockIO InterpreterCreateRoleQuery::execute() if (!query.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext()); + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -62,11 +77,11 @@ BlockIO InterpreterCreateRoleQuery::execute() }; if (query.if_exists) { - auto ids = access_control.find(query.names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(query.names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(query.names), update_func); + storage->update(storage->getIDs(query.names), update_func); } else { @@ -78,12 +93,27 @@ BlockIO InterpreterCreateRoleQuery::execute() new_roles.emplace_back(std::move(new_role)); } + if (!query.storage_name.empty()) + { + for (const auto & name : query.names) + { + auto id = access_control.find(name); + + if (!id) + continue; + + auto another_storage_ptr = access_control.findStorage(*id); + if (another_storage_ptr != storage_ptr) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Role {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_roles); + storage->tryInsert(new_roles); else if (query.or_replace) - access_control.insertOrReplace(new_roles); + storage->insertOrReplace(new_roles); else - access_control.insert(new_roles); + storage->insert(new_roles); } return {}; diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index d4c37064065..54e3b95226c 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -26,18 +26,26 @@ BlockIO InterpreterDropAccessEntityQuery::execute() query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); - auto do_drop = [&](const Strings & names) + auto do_drop = [&](const Strings & names, const String & storage_name) { + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + if (!storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(storage_name); + storage = storage_ptr.get(); + } + if (query.if_exists) - access_control.tryRemove(access_control.find(query.type, names)); + storage->tryRemove(storage->find(query.type, names)); else - access_control.remove(access_control.getIDs(query.type, names)); + storage->remove(storage->getIDs(query.type, names)); }; if (query.type == AccessEntityType::ROW_POLICY) - do_drop(query.row_policy_names->toStrings()); + do_drop(query.row_policy_names->toStrings(), query.storage_name); else - do_drop(query.names); + do_drop(query.names, query.storage_name); return {}; } diff --git a/src/Parsers/Access/ASTCreateRoleQuery.cpp b/src/Parsers/Access/ASTCreateRoleQuery.cpp index d624b9a9157..14946f2d3cd 100644 --- a/src/Parsers/Access/ASTCreateRoleQuery.cpp +++ b/src/Parsers/Access/ASTCreateRoleQuery.cpp @@ -71,6 +71,12 @@ void ASTCreateRoleQuery::formatImpl(const FormatSettings & format, FormatState & format.ostr << (format.hilite ? hilite_keyword : "") << " OR REPLACE" << (format.hilite ? hilite_none : ""); formatNames(names, format); + + if (!storage_name.empty()) + format.ostr << (format.hilite ? IAST::hilite_keyword : "") + << " AT " << (format.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(format); if (!new_name.empty()) diff --git a/src/Parsers/Access/ASTCreateRoleQuery.h b/src/Parsers/Access/ASTCreateRoleQuery.h index 42d1a4031b6..4e465553164 100644 --- a/src/Parsers/Access/ASTCreateRoleQuery.h +++ b/src/Parsers/Access/ASTCreateRoleQuery.h @@ -28,6 +28,7 @@ public: Strings names; String new_name; + String storage_name; std::shared_ptr settings; diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp index 88f2d7bce63..02b29f3fd2c 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp @@ -53,6 +53,11 @@ void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, Forma else formatNames(names, settings); + if (!storage_name.empty()) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") + << " FROM " << (settings.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(settings); } diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.h b/src/Parsers/Access/ASTDropAccessEntityQuery.h index f3a065c50df..32f4a8f8047 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.h +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.h @@ -21,6 +21,7 @@ public: AccessEntityType type; bool if_exists = false; Strings names; + String storage_name; std::shared_ptr row_policy_names; String getID(char) const override; diff --git a/src/Parsers/Access/ParserCreateRoleQuery.cpp b/src/Parsers/Access/ParserCreateRoleQuery.cpp index da9749958ee..de9e5baed18 100644 --- a/src/Parsers/Access/ParserCreateRoleQuery.cpp +++ b/src/Parsers/Access/ParserCreateRoleQuery.cpp @@ -91,6 +91,10 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec String new_name; std::shared_ptr settings; String cluster; + String storage_name; + + if (ParserKeyword{"AT"}.ignore(pos, expected)) + parseStorageName(pos, expected, storage_name); while (true) { @@ -125,6 +129,7 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->names = std::move(names); query->new_name = std::move(new_name); query->settings = std::move(settings); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp index d21164bc1a2..d5ecdf27449 100644 --- a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp @@ -53,6 +53,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & Strings names; std::shared_ptr row_policy_names; + String storage_name; String cluster; if ((type == AccessEntityType::USER) || (type == AccessEntityType::ROLE)) @@ -76,6 +77,9 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & return false; } + if (ParserKeyword{"FROM"}.ignore(pos, expected)) + parseStorageName(pos, expected, storage_name); + if (cluster.empty()) parseOnCluster(pos, expected, cluster); @@ -87,6 +91,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->cluster = std::move(cluster); query->names = std::move(names); query->row_policy_names = std::move(row_policy_names); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/parseUserName.h b/src/Parsers/Access/parseUserName.h index fbad78e7bce..717911bf2f1 100644 --- a/src/Parsers/Access/parseUserName.h +++ b/src/Parsers/Access/parseUserName.h @@ -34,4 +34,9 @@ inline bool parseRoleNames(IParser::Pos & pos, Expected & expected, Strings & ro return parseUserNames(pos, expected, role_names); } +inline bool parseStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) +{ + return parseUserName(pos, expected, storage_name); +} + } diff --git a/src/Storages/System/StorageSystemRoleGrants.cpp b/src/Storages/System/StorageSystemRoleGrants.cpp index cf5a24f88cd..8fcd0fd7cf5 100644 --- a/src/Storages/System/StorageSystemRoleGrants.cpp +++ b/src/Storages/System/StorageSystemRoleGrants.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -22,6 +23,7 @@ NamesAndTypesList StorageSystemRoleGrants::getNamesAndTypes() {"user_name", std::make_shared(std::make_shared())}, {"role_name", std::make_shared(std::make_shared())}, {"granted_role_name", std::make_shared()}, + {"granted_role_id", std::make_shared()}, {"granted_role_is_default", std::make_shared()}, {"with_admin_option", std::make_shared()}, }; @@ -45,12 +47,14 @@ void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr auto & column_role_name = assert_cast(assert_cast(*res_columns[column_index]).getNestedColumn()); auto & column_role_name_null_map = assert_cast(*res_columns[column_index++]).getNullMapData(); auto & column_granted_role_name = assert_cast(*res_columns[column_index++]); + auto & column_granted_role_id = assert_cast(*res_columns[column_index++]).getData(); auto & column_is_default = assert_cast(*res_columns[column_index++]).getData(); auto & column_admin_option = assert_cast(*res_columns[column_index++]).getData(); auto add_row = [&](const String & grantee_name, AccessEntityType grantee_type, const String & granted_role_name, + const UUID & granted_role_id, bool is_default, bool with_admin_option) { @@ -72,6 +76,7 @@ void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr assert(false); column_granted_role_name.insertData(granted_role_name.data(), granted_role_name.length()); + column_granted_role_id.push_back(granted_role_id.toUnderType()); column_is_default.push_back(is_default); column_admin_option.push_back(with_admin_option); }; @@ -90,7 +95,7 @@ void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr continue; bool is_default = !default_roles || default_roles->match(role_id); - add_row(grantee_name, grantee_type, *role_name, is_default, element.admin_option); + add_row(grantee_name, grantee_type, *role_name, role_id, is_default, element.admin_option); } } }; diff --git a/tests/integration/test_multi_access_storage_role_management/__init__.py b/tests/integration/test_multi_access_storage_role_management/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_multi_access_storage_role_management/configs/roles.xml b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml new file mode 100644 index 00000000000..672c7e24728 --- /dev/null +++ b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml @@ -0,0 +1,13 @@ + + + + + + + + + GRANT ALL ON *.* WITH GRANT OPTION + + + + diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py new file mode 100644 index 00000000000..473a37695e9 --- /dev/null +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -0,0 +1,72 @@ +import pytest +import os +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + stay_alive=True, +) + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + node.query("CREATE DATABASE mydb") + node.query("CREATE TABLE mydb.table1(x UInt32) ENGINE=Log") + + node.query("CREATE USER test_user") + node.query("CREATE USER test_user2") + yield cluster + + finally: + cluster.shutdown() + + +def test_role_from_different_storages(): + node.query("CREATE ROLE default_role") + node.query("GRANT SELECT ON system.* TO default_role") + + assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT SELECT ON system.* TO default_role"]) + assert node.query("SHOW ROLES") == TSV(["default_role"]) + + node.query("GRANT default_role TO test_user") + + node.copy_file_to_container( + os.path.join(SCRIPT_DIR, "configs/roles.xml"), + "/etc/clickhouse-server/users.d/roles.xml", + ) + + node.restart_clickhouse() + + assert node.query("SELECT name, storage FROM system.roles") == TSV( + [ + ["default_role", "users_xml"], + ["default_role", "local_directory"] + ] + ) + + # Role from users.xml will have priority + assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT ALL ON *.* TO default_role WITH GRANT OPTION"]) + + node.query("GRANT default_role TO test_user") + node.query("GRANT default_role TO test_user2") + assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user2'") == TSV( + ["62bedbf3-7fb1-94cb-3a35-e479693223b3"] # roles from users.xml have deterministic ids + ) + + node.query("DROP ROLE default_role FROM local_directory") + assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user'") == TSV(["62bedbf3-7fb1-94cb-3a35-e479693223b3"]) + + # Already exists + with pytest.raises(QueryRuntimeException): + node.query("CREATE ROLE default_role AT memory") + + node.query("CREATE ROLE other_role AT memory") + + assert node.query("SELECT storage FROM system.roles WHERE name = 'other_role'") == TSV(["memory"]) From ee591ef7f52f3f53dd8cb59a6acf33a734561f65 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 11 Jul 2023 00:33:34 +0000 Subject: [PATCH 0495/2047] Automatic style fix --- .../test.py | 29 ++++++++++++------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index 473a37695e9..d9a4a73c56b 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -32,7 +32,9 @@ def test_role_from_different_storages(): node.query("CREATE ROLE default_role") node.query("GRANT SELECT ON system.* TO default_role") - assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT SELECT ON system.* TO default_role"]) + assert node.query("SHOW GRANTS FOR default_role") == TSV( + ["GRANT SELECT ON system.* TO default_role"] + ) assert node.query("SHOW ROLES") == TSV(["default_role"]) node.query("GRANT default_role TO test_user") @@ -45,23 +47,28 @@ def test_role_from_different_storages(): node.restart_clickhouse() assert node.query("SELECT name, storage FROM system.roles") == TSV( - [ - ["default_role", "users_xml"], - ["default_role", "local_directory"] - ] + [["default_role", "users_xml"], ["default_role", "local_directory"]] ) # Role from users.xml will have priority - assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT ALL ON *.* TO default_role WITH GRANT OPTION"]) + assert node.query("SHOW GRANTS FOR default_role") == TSV( + ["GRANT ALL ON *.* TO default_role WITH GRANT OPTION"] + ) node.query("GRANT default_role TO test_user") node.query("GRANT default_role TO test_user2") - assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user2'") == TSV( - ["62bedbf3-7fb1-94cb-3a35-e479693223b3"] # roles from users.xml have deterministic ids + assert node.query( + "SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user2'" + ) == TSV( + [ + "62bedbf3-7fb1-94cb-3a35-e479693223b3" + ] # roles from users.xml have deterministic ids ) node.query("DROP ROLE default_role FROM local_directory") - assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user'") == TSV(["62bedbf3-7fb1-94cb-3a35-e479693223b3"]) + assert node.query( + "SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user'" + ) == TSV(["62bedbf3-7fb1-94cb-3a35-e479693223b3"]) # Already exists with pytest.raises(QueryRuntimeException): @@ -69,4 +76,6 @@ def test_role_from_different_storages(): node.query("CREATE ROLE other_role AT memory") - assert node.query("SELECT storage FROM system.roles WHERE name = 'other_role'") == TSV(["memory"]) + assert node.query( + "SELECT storage FROM system.roles WHERE name = 'other_role'" + ) == TSV(["memory"]) From a01a6834482cb8af82dd528fa557f3157b60eea9 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 11 Jul 2023 09:51:52 +0800 Subject: [PATCH 0496/2047] fix style --- .../test_rewrite_uniq_to_count/test.py | 79 +++++++++++++------ 1 file changed, 53 insertions(+), 26 deletions(-) diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py index af0c4f09117..ec9dc6d9b7f 100644 --- a/tests/integration/test_rewrite_uniq_to_count/test.py +++ b/tests/integration/test_rewrite_uniq_to_count/test.py @@ -29,13 +29,21 @@ def prepare(): ORDER BY `a` """ ) - node.query("INSERT INTO test_rewrite_uniq_to_count values ('1', '1', '1'), ('1', '1', '1')") - node.query("INSERT INTO test_rewrite_uniq_to_count values ('2', '2', '2'), ('2', '2', '2')") - node.query("INSERT INTO test_rewrite_uniq_to_count values ('3', '3', '3'), ('3', '3', '3')") + node.query( + "INSERT INTO test_rewrite_uniq_to_count values ('1', '1', '1'), ('1', '1', '1')" + ) + node.query( + "INSERT INTO test_rewrite_uniq_to_count values ('2', '2', '2'), ('2', '2', '2')" + ) + node.query( + "INSERT INTO test_rewrite_uniq_to_count values ('3', '3', '3'), ('3', '3', '3')" + ) def shutdown(): - node.query("DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC") + node.query( + "DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC" + ) def check(query, result): @@ -50,7 +58,6 @@ def check(query, result): assert "count()" in node.query("EXPLAIN QUERY TREE " + query) -# For new analyzer loses alias info, we can not rewrite SQL with alias. def check_by_old_analyzer(query, result): # only old analyzer query = query + " settings optimize_uniq_to_count = 1" @@ -60,43 +67,63 @@ def check_by_old_analyzer(query, result): def test_rewrite_distinct(started_cluster): # simple test - check("SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count)", - 3) + check( + "SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count)", + 3, + ) # test subquery alias - check("SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t", - 3) + check( + "SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t", + 3, + ) - # test table.column - check("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t", - 3) + # test compound column name + check( + "SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t", + 3, + ) # test select expression alias - check_by_old_analyzer("SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", - 3) + check_by_old_analyzer( + "SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", + 3, + ) # test select expression alias - check_by_old_analyzer("SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t", - 3) + check_by_old_analyzer( + "SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t", + 3, + ) def test_rewrite_group_by(started_cluster): # simple test - check("SELECT uniq(a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a)", - 3) + check( + "SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a)", + 3, + ) # test subquery alias - check("SELECT uniq(t.a) FROM (SELECT a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3) + check( + "SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", + 3, + ) # test select expression alias - check_by_old_analyzer("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3) + check_by_old_analyzer( + "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", + 3, + ) # test select expression alias - check_by_old_analyzer("SELECT uniq(t.a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", - 3) + check_by_old_analyzer( + "SELECT uniq(t.a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + 3, + ) # test select expression alias - check_by_old_analyzer("SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, min(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", - 3) + check_by_old_analyzer( + "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + 3, + ) From e592c7c353fdc8ad7d0e1f23bfcf35fe52273490 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 11 Jul 2023 10:32:33 +0800 Subject: [PATCH 0497/2047] Fix integration tests - recover clickhouse_stay_alive_command lost by incomplete copy-paste --- tests/integration/helpers/cluster.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 3b2c5ff40bb..8b9f75e2545 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3238,6 +3238,9 @@ class ClickHouseInstance: self.clickhouse_start_command = clickhouse_start_command.replace( "{main_config_file}", self.main_config_name ) + self.clickhouse_stay_alive_command = "bash -c \"trap 'pkill tail' INT TERM; {} --daemon; coproc tail -f /dev/null; wait $$!\"".format( + clickhouse_start_command + ) self.path = p.join(self.cluster.instances_dir, name) self.docker_compose_path = p.join(self.path, "docker-compose.yml") @@ -4327,7 +4330,7 @@ class ClickHouseInstance: entrypoint_cmd = self.clickhouse_start_command if self.stay_alive: - entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND.replace( + entrypoint_cmd = self.clickhouse_stay_alive_command.replace( "{main_config_file}", self.main_config_name ) else: From cd1111b17b07cdaade0e909a4139205763701d24 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 11 Jul 2023 10:11:21 +0800 Subject: [PATCH 0498/2047] ignore alias for UniqToCountPass --- src/Analyzer/Passes/QueryAnalysisPass.h | 1 - src/Analyzer/Passes/UniqToCountPass.cpp | 90 +++++++------------ .../test_rewrite_uniq_to_count/test.py | 6 +- 3 files changed, 35 insertions(+), 62 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.h b/src/Analyzer/Passes/QueryAnalysisPass.h index ea845f26bd9..fa8778ebf76 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.h +++ b/src/Analyzer/Passes/QueryAnalysisPass.h @@ -51,7 +51,6 @@ namespace DB * Function `arrayJoin` is handled properly. * For functions `dictGet` and its variations and for function `joinGet` identifier as first argument is handled properly. * Replace `countDistinct` and `countIfDistinct` aggregate functions using setting count_distinct_implementation. - * Replace `uniq` and `uniq` and its variants(except uniqUpTo) into `count` aggregate functions using setting optimize_uniq_to_count. * Add -OrNull suffix to aggregate functions if setting aggregate_functions_null_for_empty is true. * Function `exists` is converted into `in`. * Functions `in`, `notIn`, `globalIn`, `globalNotIn` converted into `nullIn`, `notNullIn`, `globalNullIn`, `globalNotNullIn` if setting transform_null_in is true. diff --git a/src/Analyzer/Passes/UniqToCountPass.cpp b/src/Analyzer/Passes/UniqToCountPass.cpp index 1ffb83a6e36..ae7952051e7 100644 --- a/src/Analyzer/Passes/UniqToCountPass.cpp +++ b/src/Analyzer/Passes/UniqToCountPass.cpp @@ -11,65 +11,50 @@ namespace DB { -using Aliases = std::unordered_map; - namespace { - bool matchFnUniq(String func_name) - { - auto name = Poco::toLower(func_name); - return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" - || name == "uniqCombined64"; - } +bool matchFnUniq(String func_name) +{ + auto name = Poco::toLower(func_name); + return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" + || name == "uniqCombined64"; +} - bool nodeEquals(const QueryTreeNodePtr & lhs, const QueryTreeNodePtr & rhs, const Aliases & alias) - { - auto * lhs_node = lhs->as(); - auto * rhs_node = rhs->as(); +bool nodeEquals(const QueryTreeNodePtr & lhs, const QueryTreeNodePtr & rhs) +{ + auto * lhs_node = lhs->as(); + auto * rhs_node = rhs->as(); - if (lhs_node && rhs_node) - { - if (lhs_node->getColumn() == rhs_node->getColumn()) - return true; + if (lhs_node && rhs_node && lhs_node->getColumn() == rhs_node->getColumn()) + return true; + return false; +} - /// translate alias - if (lhs->hasAlias() && alias.find(lhs->getAlias()) != alias.end()) - lhs_node = alias.find(lhs->getAlias())->second->as(); - - if (rhs->hasAlias() && alias.find(rhs->getAlias()) != alias.end()) - rhs_node = alias.find(rhs->getAlias())->second->as(); - - if (lhs_node && rhs_node && lhs_node == rhs_node) - return true; - } +bool nodeListEquals(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs) +{ + if (lhs.size() != rhs.size()) return false; - } - - bool nodeListEquals(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs, const Aliases & alias) + for (size_t i = 0; i < lhs.size(); i++) { - if (lhs.size() != rhs.size()) + if (!nodeEquals(lhs[i], rhs[i])) return false; - for (size_t i = 0; i < lhs.size(); i++) - { - if (!nodeEquals(lhs[i], rhs[i], alias)) - return false; - } - return true; } + return true; +} - bool nodeListContainsAll(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs, const Aliases & alias) +bool nodeListContainsAll(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs) +{ + if (lhs.size() < rhs.size()) + return false; + for (const auto & re : rhs) { - if (lhs.size() < rhs.size()) + auto predicate = [&](const QueryTreeNodePtr & le) { return nodeEquals(le, re); }; + if (std::find_if(lhs.begin(), lhs.end(), predicate) == lhs.end()) return false; - for (const auto & re : rhs) - { - auto predicate = [&](const QueryTreeNodePtr & le) { return nodeEquals(le, re, alias); }; - if (std::find_if(lhs.begin(), lhs.end(), predicate) == lhs.end()) - return false; - } - return true; } + return true; +} } @@ -108,15 +93,6 @@ public: if (!matchFnUniq(function_node->getFunctionName())) return; - /// collect subquery select expressions alias. - /// TODO new analyzer will lose alias info, so we will collect nothing and we can not rewrite SQL with alias. - Aliases alias; - for (auto & subquery_projection_node : subquery_node->getProjection().getNodes()) - { - if (subquery_projection_node->hasAlias()) - alias.insert({subquery_projection_node->getAlias(), subquery_projection_node}); - } - auto & uniq_arguments_nodes = function_node->getArguments().getNodes(); /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' @@ -125,7 +101,7 @@ public: if (!subquery_node->isDistinct()) return false; /// uniq expression list == subquery group by expression list - if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjection().getNodes(), alias)) + if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjection().getNodes())) return false; return true; }; @@ -136,10 +112,10 @@ public: if (!subquery_node->hasGroupBy()) return false; /// uniq argument node list == subquery group by node list - if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getGroupByNode()->getChildren(), alias)) + if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getGroupByNode()->getChildren())) return false; /// subquery select node list must contain all columns in uniq argument node list - if (!nodeListContainsAll(subquery_node->getProjection().getNodes(), uniq_arguments_nodes, alias)) + if (!nodeListContainsAll(subquery_node->getProjection().getNodes(), uniq_arguments_nodes)) return false; return true; }; diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py index ec9dc6d9b7f..d7fa9f39441 100644 --- a/tests/integration/test_rewrite_uniq_to_count/test.py +++ b/tests/integration/test_rewrite_uniq_to_count/test.py @@ -41,9 +41,7 @@ def prepare(): def shutdown(): - node.query( - "DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC" - ) + node.query("DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC") def check(query, result): @@ -107,7 +105,7 @@ def test_rewrite_group_by(started_cluster): # test subquery alias check( "SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3, + 3, ) # test select expression alias From 063eebc16b8250b42f3f39b7cf00d8dcb578a702 Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 11 Jul 2023 12:59:34 +0800 Subject: [PATCH 0499/2047] fix clang-tidy checking --- src/Interpreters/RewriteUniqToCountVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp index ac42a8a82da..7445068207a 100644 --- a/src/Interpreters/RewriteUniqToCountVisitor.cpp +++ b/src/Interpreters/RewriteUniqToCountVisitor.cpp @@ -124,7 +124,7 @@ void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) /// collect subquery select expressions alias Aliases alias; - for (auto expr : sub_expr_list->children) + for (const auto & expr : sub_expr_list->children) { if (!expr->tryGetAlias().empty()) alias.insert({expr->tryGetAlias(), expr}); From bd5f39351502d67b35ef5e02f166be2d31b7e6fe Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 11 Jul 2023 05:14:39 +0000 Subject: [PATCH 0500/2047] Always remove blobs in MetadataStorageFromPlainObjectStorageTransaction::unlinkMetadata --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 3650c7eaac8..022ff86df50 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -138,7 +138,8 @@ void MetadataStorageFromPlainObjectStorageTransaction::addBlobToMetadata( UnlinkMetadataFileOperationOutcomePtr MetadataStorageFromPlainObjectStorageTransaction::unlinkMetadata(const std::string &) { - return nullptr; + /// No hardlinks, so will always remove file. + return std::make_shared(UnlinkMetadataFileOperationOutcome{0}); } } From bc16531876b79623ef05f94623407b187fb89575 Mon Sep 17 00:00:00 2001 From: xiao <821008736@qq.com> Date: Tue, 11 Jul 2023 17:01:56 +0800 Subject: [PATCH 0501/2047] Update CompletedPipelineExecutor.cpp --- src/Processors/Executors/CompletedPipelineExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index 2964d9b6aa2..b0f842dec1b 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -115,7 +115,7 @@ CompletedPipelineExecutor::~CompletedPipelineExecutor() } catch (...) { - tryLogCurrentException("PullingAsyncPipelineExecutor"); + tryLogCurrentException("CompletedPipelineExecutor"); } } From 062b1c464c2961672dc23cc85c6226431f1c44f3 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 11 Jul 2023 10:04:43 +0000 Subject: [PATCH 0502/2047] watch for certificate file updates in configreloader --- programs/keeper/Keeper.cpp | 315 ++++++++++++------------ programs/server/Server.cpp | 9 +- src/Access/UsersConfigAccessStorage.cpp | 2 +- src/Common/Config/ConfigReloader.cpp | 28 ++- src/Common/Config/ConfigReloader.h | 26 +- 5 files changed, 197 insertions(+), 183 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index db35e30bd30..2a54ddb952f 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -1,28 +1,28 @@ #include "Keeper.h" -#include -#include -#include -#include -#include -#include #include -#include +#include +#include #include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include +#include +#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include @@ -30,13 +30,13 @@ #include #include -#include #include +#include #include "Core/Defines.h" #include "config.h" -#include "config_version.h" #include "config_tools.h" +#include "config_version.h" #if USE_SSL @@ -44,8 +44,8 @@ # include #endif -#include #include +#include #include @@ -70,9 +70,9 @@ int mainEntryClickHouseKeeper(int argc, char ** argv) // Weak symbols don't work correctly on Darwin // so we have a stub implementation to avoid linker errors -void collectCrashLog( - Int32, UInt64, const String &, const StackTrace &) -{} +void collectCrashLog(Int32, UInt64, const String &, const StackTrace &) +{ +} #endif @@ -87,7 +87,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -Poco::Net::SocketAddress Keeper::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) const +Poco::Net::SocketAddress +Keeper::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) const { auto address = makeSocketAddress(host, port, &logger()); socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config().getBool("listen_reuse_port", false)); @@ -113,7 +114,9 @@ void Keeper::createServer(const std::string & listen_host, const char * port_nam if (listen_try) { - LOG_WARNING(&logger(), "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " + LOG_WARNING( + &logger(), + "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " "specify not disabled IPv4 or IPv6 address to listen in element of configuration " "file. Example for disabled IPv6: 0.0.0.0 ." " Example for disabled IPv4: ::", @@ -137,12 +140,13 @@ int Keeper::run() if (config().hasOption("help")) { Poco::Util::HelpFormatter help_formatter(Keeper::options()); - auto header_str = fmt::format("{0} [OPTION] [-- [ARG]...]\n" + auto header_str = fmt::format( + "{0} [OPTION] [-- [ARG]...]\n" #if ENABLE_CLICKHOUSE_KEEPER_CLIENT - "{0} client [OPTION]\n" + "{0} client [OPTION]\n" #endif - "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010", - commandName()); + "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010", + commandName()); help_formatter.setHeader(header_str); help_formatter.format(std::cout); return 0; @@ -161,7 +165,9 @@ void Keeper::initialize(Poco::Util::Application & self) BaseDaemon::initialize(self); logger().information("starting up"); - LOG_INFO(&logger(), "OS Name = {}, OS Version = {}, OS Architecture = {}", + LOG_INFO( + &logger(), + "OS Name = {}, OS Version = {}, OS Architecture = {}", Poco::Environment::osName(), Poco::Environment::osVersion(), Poco::Environment::osArchitecture()); @@ -186,82 +192,64 @@ void Keeper::handleCustomArguments(const std::string & arg, [[maybe_unused]] con void Keeper::defineOptions(Poco::Util::OptionSet & options) { + options.addOption(Poco::Util::Option("help", "h", "show help and exit").required(false).repeatable(false).binding("help")); + options.addOption(Poco::Util::Option("version", "V", "show version and exit").required(false).repeatable(false).binding("version")); options.addOption( - Poco::Util::Option("help", "h", "show help and exit") + Poco::Util::Option( + "force-recovery", "force-recovery", "Force recovery mode allowing Keeper to overwrite cluster configuration without quorum") .required(false) .repeatable(false) - .binding("help")); - options.addOption( - Poco::Util::Option("version", "V", "show version and exit") - .required(false) - .repeatable(false) - .binding("version")); - options.addOption( - Poco::Util::Option("force-recovery", "force-recovery", "Force recovery mode allowing Keeper to overwrite cluster configuration without quorum") - .required(false) - .repeatable(false) - .noArgument() - .callback(Poco::Util::OptionCallback(this, &Keeper::handleCustomArguments))); + .noArgument() + .callback(Poco::Util::OptionCallback(this, &Keeper::handleCustomArguments))); BaseDaemon::defineOptions(options); } namespace { -struct KeeperHTTPContext : public IHTTPContext -{ - explicit KeeperHTTPContext(ContextPtr context_) - : context(std::move(context_)) - {} - - uint64_t getMaxHstsAge() const override + struct KeeperHTTPContext : public IHTTPContext { - return context->getConfigRef().getUInt64("keeper_server.hsts_max_age", 0); - } + explicit KeeperHTTPContext(ContextPtr context_) : context(std::move(context_)) { } - uint64_t getMaxUriSize() const override + uint64_t getMaxHstsAge() const override { return context->getConfigRef().getUInt64("keeper_server.hsts_max_age", 0); } + + uint64_t getMaxUriSize() const override { return context->getConfigRef().getUInt64("keeper_server.http_max_uri_size", 1048576); } + + uint64_t getMaxFields() const override { return context->getConfigRef().getUInt64("keeper_server.http_max_fields", 1000000); } + + uint64_t getMaxFieldNameSize() const override + { + return context->getConfigRef().getUInt64("keeper_server.http_max_field_name_size", 128 * 1024); + } + + uint64_t getMaxFieldValueSize() const override + { + return context->getConfigRef().getUInt64("keeper_server.http_max_field_value_size", 128 * 1024); + } + + uint64_t getMaxChunkSize() const override + { + return context->getConfigRef().getUInt64("keeper_server.http_max_chunk_size", 100_GiB); + } + + Poco::Timespan getReceiveTimeout() const override + { + return {context->getConfigRef().getInt64("keeper_server.http_receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0}; + } + + Poco::Timespan getSendTimeout() const override + { + return {context->getConfigRef().getInt64("keeper_server.http_send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0}; + } + + ContextPtr context; + }; + + HTTPContextPtr httpContext() { - return context->getConfigRef().getUInt64("keeper_server.http_max_uri_size", 1048576); + return std::make_shared(Context::getGlobalContextInstance()); } - uint64_t getMaxFields() const override - { - return context->getConfigRef().getUInt64("keeper_server.http_max_fields", 1000000); - } - - uint64_t getMaxFieldNameSize() const override - { - return context->getConfigRef().getUInt64("keeper_server.http_max_field_name_size", 128 * 1024); - } - - uint64_t getMaxFieldValueSize() const override - { - return context->getConfigRef().getUInt64("keeper_server.http_max_field_value_size", 128 * 1024); - } - - uint64_t getMaxChunkSize() const override - { - return context->getConfigRef().getUInt64("keeper_server.http_max_chunk_size", 100_GiB); - } - - Poco::Timespan getReceiveTimeout() const override - { - return {context->getConfigRef().getInt64("keeper_server.http_receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0}; - } - - Poco::Timespan getSendTimeout() const override - { - return {context->getConfigRef().getInt64("keeper_server.http_send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0}; - } - - ContextPtr context; -}; - -HTTPContextPtr httpContext() -{ - return std::make_shared(Context::getGlobalContextInstance()); -} - } int Keeper::main(const std::vector & /*args*/) @@ -298,7 +286,7 @@ try std::filesystem::create_directories(path); /// Check that the process user id matches the owner of the data. - assertProcessUserMatchesDataOwner(path, [&](const std::string & message){ LOG_WARNING(log, fmt::runtime(message)); }); + assertProcessUserMatchesDataOwner(path, [&](const std::string & message) { LOG_WARNING(log, fmt::runtime(message)); }); DB::ServerUUID::load(path + "/uuid", log); @@ -307,8 +295,7 @@ try GlobalThreadPool::initialize( config().getUInt("max_thread_pool_size", 100), config().getUInt("max_thread_pool_free_size", 1000), - config().getUInt("thread_pool_queue_size", 10000) - ); + config().getUInt("thread_pool_queue_size", 10000)); static ServerErrorHandler error_handler; Poco::ErrorHandler::set(&error_handler); @@ -350,8 +337,7 @@ try for (const auto & server : *servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); return metrics; - } - ); + }); std::vector listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host"); @@ -367,10 +353,7 @@ try global_context->initializeKeeperDispatcher(/* start_async = */ true); FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher()); - auto config_getter = [&] () -> const Poco::Util::AbstractConfiguration & - { - return global_context->getConfigRef(); - }; + auto config_getter = [&]() -> const Poco::Util::AbstractConfiguration & { return global_context->getConfigRef(); }; auto tcp_receive_timeout = config().getInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC); auto tcp_send_timeout = config().getInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC); @@ -379,43 +362,55 @@ try { /// TCP Keeper const char * port_name = "keeper_server.tcp_port"; - createServer(listen_host, port_name, listen_try, [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port); - socket.setReceiveTimeout(Poco::Timespan{tcp_receive_timeout, 0}); - socket.setSendTimeout(Poco::Timespan{tcp_send_timeout, 0}); - servers->emplace_back( - listen_host, - port_name, - "Keeper (tcp): " + address.toString(), - std::make_unique( - new KeeperTCPHandlerFactory( - config_getter, global_context->getKeeperDispatcher(), - tcp_receive_timeout, tcp_send_timeout, false), server_pool, socket)); - }); + createServer( + listen_host, + port_name, + listen_try, + [&](UInt16 port) + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(Poco::Timespan{tcp_receive_timeout, 0}); + socket.setSendTimeout(Poco::Timespan{tcp_send_timeout, 0}); + servers->emplace_back( + listen_host, + port_name, + "Keeper (tcp): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory( + config_getter, global_context->getKeeperDispatcher(), tcp_receive_timeout, tcp_send_timeout, false), + server_pool, + socket)); + }); const char * secure_port_name = "keeper_server.tcp_port_secure"; - createServer(listen_host, secure_port_name, listen_try, [&](UInt16 port) - { + createServer( + listen_host, + secure_port_name, + listen_try, + [&](UInt16 port) + { #if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan{tcp_receive_timeout, 0}); - socket.setSendTimeout(Poco::Timespan{tcp_send_timeout, 0}); - servers->emplace_back( - listen_host, - secure_port_name, - "Keeper with secure protocol (tcp_secure): " + address.toString(), - std::make_unique( - new KeeperTCPHandlerFactory( - config_getter, global_context->getKeeperDispatcher(), - tcp_receive_timeout, tcp_send_timeout, true), server_pool, socket)); + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan{tcp_receive_timeout, 0}); + socket.setSendTimeout(Poco::Timespan{tcp_send_timeout, 0}); + servers->emplace_back( + listen_host, + secure_port_name, + "Keeper with secure protocol (tcp_secure): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory( + config_getter, global_context->getKeeperDispatcher(), tcp_receive_timeout, tcp_send_timeout, true), + server_pool, + socket)); #else - UNUSED(port); - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); + UNUSED(port); + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); #endif - }); + }); const auto & config = config_getter(); auto http_context = httpContext(); @@ -426,19 +421,27 @@ try /// Prometheus (if defined and not setup yet with http_port) port_name = "prometheus.port"; - createServer(listen_host, port_name, listen_try, [&, my_http_context = std::move(http_context)](UInt16 port) mutable - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port); - socket.setReceiveTimeout(my_http_context->getReceiveTimeout()); - socket.setSendTimeout(my_http_context->getSendTimeout()); - servers->emplace_back( - listen_host, - port_name, - "Prometheus: http://" + address.toString(), - std::make_unique( - std::move(my_http_context), createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); - }); + createServer( + listen_host, + port_name, + listen_try, + [&, my_http_context = std::move(http_context)](UInt16 port) mutable + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(my_http_context->getReceiveTimeout()); + socket.setSendTimeout(my_http_context->getSendTimeout()); + servers->emplace_back( + listen_host, + port_name, + "Prometheus: http://" + address.toString(), + std::make_unique( + std::move(my_http_context), + createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), + server_pool, + socket, + http_params)); + }); } for (auto & server : *servers) @@ -454,7 +457,7 @@ try /// ConfigReloader have to strict parameters which are redundant in our case auto main_config_reloader = std::make_unique( config_path, - include_from_path, + std::vector{{include_from_path}}, config().getString("path", ""), std::move(unused_cache), unused_event, @@ -463,7 +466,7 @@ try if (config->has("keeper_server")) global_context->updateKeeperConfiguration(*config); }, - /* already_loaded = */ false); /// Reload it right now (initial loading) + /* already_loaded = */ false); /// Reload it right now (initial loading) SCOPE_EXIT({ LOG_INFO(log, "Shutting down."); @@ -488,7 +491,10 @@ try current_connections = waitServersToFinish(*servers, servers_lock, config().getInt("shutdown_wait_unfinished", 5)); if (current_connections) - LOG_INFO(log, "Closed connections to Keeper. But {} remain. Probably some users cannot finish their connections after context shutdown.", current_connections); + LOG_INFO( + log, + "Closed connections to Keeper. But {} remain. Probably some users cannot finish their connections after context shutdown.", + current_connections); else LOG_INFO(log, "Closed connections to Keeper."); @@ -526,11 +532,10 @@ catch (...) void Keeper::logRevision() const { - Poco::Logger::root().information("Starting ClickHouse Keeper " + std::string{VERSION_STRING} - + "(revision : " + std::to_string(ClickHouseRevision::getVersionRevision()) - + ", git hash: " + (git_hash.empty() ? "" : git_hash) - + ", build id: " + (build_id.empty() ? "" : build_id) + ")" - + ", PID " + std::to_string(getpid())); + Poco::Logger::root().information( + "Starting ClickHouse Keeper " + std::string{VERSION_STRING} + "(revision : " + + std::to_string(ClickHouseRevision::getVersionRevision()) + ", git hash: " + (git_hash.empty() ? "" : git_hash) + + ", build id: " + (build_id.empty() ? "" : build_id) + ")" + ", PID " + std::to_string(getpid())); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 071f7d3177e..af9abc1024f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1100,9 +1100,16 @@ try SensitiveDataMasker::setInstance(std::make_unique(config(), "query_masking_rules")); } + const std::string cert_path = config().getString("openSSL.server.certificateFile", ""); + const std::string key_path = config().getString("openSSL.server.privateKeyFile", ""); + + std::vector extra_paths = {include_from_path}; + if (!cert_path.empty()) extra_paths.emplace_back(cert_path); + if (!key_path.empty()) extra_paths.emplace_back(key_path); + auto main_config_reloader = std::make_unique( config_path, - include_from_path, + extra_paths, config().getString("path", ""), std::move(main_config_zk_node_cache), main_config_zk_changed_event, diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 15765045c97..bb7d9dfd4f7 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -807,7 +807,7 @@ void UsersConfigAccessStorage::load( config_reloader.reset(); config_reloader = std::make_unique( users_config_path, - include_from_path, + std::vector{{include_from_path}}, preprocessed_dir, zkutil::ZooKeeperNodeCache(get_zookeeper_function), std::make_shared(), diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index de7011b67bf..5ff2fcbf502 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -14,14 +14,15 @@ namespace DB { ConfigReloader::ConfigReloader( - const std::string & path_, - const std::string & include_from_path_, + std::string_view config_path_, + const std::vector& extra_paths_, const std::string & preprocessed_dir_, zkutil::ZooKeeperNodeCache && zk_node_cache_, const zkutil::EventPtr & zk_changed_event_, Updater && updater_, bool already_loaded) - : path(path_), include_from_path(include_from_path_) + : config_path(config_path_) + , extra_paths(extra_paths_) , preprocessed_dir(preprocessed_dir_) , zk_node_cache(std::move(zk_node_cache_)) , zk_changed_event(zk_changed_event_) @@ -98,10 +99,10 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac FilesChangesTracker new_files = getNewFileList(); if (force || need_reload_from_zk || new_files.isDifferOrNewerThan(files)) { - ConfigProcessor config_processor(path); + ConfigProcessor config_processor(config_path); ConfigProcessor::LoadedConfig loaded_config; - LOG_DEBUG(log, "Loading config '{}'", path); + LOG_DEBUG(log, "Loading config '{}'", config_path); try { @@ -118,7 +119,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac if (throw_on_error) throw; - tryLogCurrentException(log, "ZooKeeper error when loading config from '" + path + "'"); + tryLogCurrentException(log, "ZooKeeper error when loading config from '" + config_path + "'"); return; } catch (...) @@ -126,7 +127,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac if (throw_on_error) throw; - tryLogCurrentException(log, "Error loading config from '" + path + "'"); + tryLogCurrentException(log, "Error loading config from '" + config_path + "'"); return; } config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); @@ -142,7 +143,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac need_reload_from_zk = false; } - LOG_DEBUG(log, "Loaded config '{}', performing update on configuration", path); + LOG_DEBUG(log, "Loaded config '{}', performing update on configuration", config_path); try { @@ -152,11 +153,11 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac { if (throw_on_error) throw; - tryLogCurrentException(log, "Error updating configuration from '" + path + "' config."); + tryLogCurrentException(log, "Error updating configuration from '" + config_path + "' config."); return; } - LOG_DEBUG(log, "Loaded config '{}', performed update on configuration", path); + LOG_DEBUG(log, "Loaded config '{}', performed update on configuration", config_path); } } @@ -196,10 +197,11 @@ ConfigReloader::FilesChangesTracker ConfigReloader::getNewFileList() const { FilesChangesTracker file_list; - file_list.addIfExists(path); - file_list.addIfExists(include_from_path); + file_list.addIfExists(config_path); + for (const std::string& path : extra_paths) + file_list.addIfExists(path); - for (const auto & merge_path : ConfigProcessor::getConfigMergeFiles(path)) + for (const auto & merge_path : ConfigProcessor::getConfigMergeFiles(config_path)) file_list.addIfExists(merge_path); return file_list; diff --git a/src/Common/Config/ConfigReloader.h b/src/Common/Config/ConfigReloader.h index 982e21c91e2..c5a18524318 100644 --- a/src/Common/Config/ConfigReloader.h +++ b/src/Common/Config/ConfigReloader.h @@ -4,7 +4,8 @@ #include #include #include -#include +#include +#include #include #include #include @@ -22,23 +23,21 @@ class Context; /** Every two seconds checks configuration files for update. * If configuration is changed, then config will be reloaded by ConfigProcessor * and the reloaded config will be applied via Updater functor. - * It doesn't take into account changes of --config-file, and parameters. + * It doesn't take into account changes of --config-file and . */ class ConfigReloader { public: using Updater = std::function; - /** include_from_path is usually /etc/metrika.xml (i.e. value of tag) - */ ConfigReloader( - const std::string & path, - const std::string & include_from_path, - const std::string & preprocessed_dir, - zkutil::ZooKeeperNodeCache && zk_node_cache, - const zkutil::EventPtr & zk_changed_event, - Updater && updater, - bool already_loaded); + std::string_view path_, + const std::vector& extra_paths_, + const std::string & preprocessed_dir, + zkutil::ZooKeeperNodeCache && zk_node_cache, + const zkutil::EventPtr & zk_changed_event, + Updater && updater, + bool already_loaded); ~ConfigReloader(); @@ -73,8 +72,9 @@ private: Poco::Logger * log = &Poco::Logger::get("ConfigReloader"); - std::string path; - std::string include_from_path; + std::string config_path; + std::vector extra_paths; + std::string preprocessed_dir; FilesChangesTracker files; zkutil::ZooKeeperNodeCache zk_node_cache; From 23e3c57a6ad61074a3bb76f3e419faee9313f727 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 11 Jul 2023 10:13:30 +0000 Subject: [PATCH 0503/2047] reload certificates for Keeper --- programs/keeper/Keeper.cpp | 330 +++++++++++++++++++------------------ 1 file changed, 168 insertions(+), 162 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 2a54ddb952f..f4bb08676d2 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -1,28 +1,29 @@ #include "Keeper.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include +#include #include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include @@ -30,13 +31,13 @@ #include #include -#include #include +#include #include "Core/Defines.h" #include "config.h" -#include "config_tools.h" #include "config_version.h" +#include "config_tools.h" #if USE_SSL @@ -44,8 +45,8 @@ # include #endif -#include #include +#include #include @@ -70,9 +71,9 @@ int mainEntryClickHouseKeeper(int argc, char ** argv) // Weak symbols don't work correctly on Darwin // so we have a stub implementation to avoid linker errors -void collectCrashLog(Int32, UInt64, const String &, const StackTrace &) -{ -} +void collectCrashLog( + Int32, UInt64, const String &, const StackTrace &) +{} #endif @@ -87,8 +88,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -Poco::Net::SocketAddress -Keeper::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) const +Poco::Net::SocketAddress Keeper::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) const { auto address = makeSocketAddress(host, port, &logger()); socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config().getBool("listen_reuse_port", false)); @@ -114,9 +114,7 @@ void Keeper::createServer(const std::string & listen_host, const char * port_nam if (listen_try) { - LOG_WARNING( - &logger(), - "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " + LOG_WARNING(&logger(), "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " "specify not disabled IPv4 or IPv6 address to listen in element of configuration " "file. Example for disabled IPv6: 0.0.0.0 ." " Example for disabled IPv4: ::", @@ -140,13 +138,12 @@ int Keeper::run() if (config().hasOption("help")) { Poco::Util::HelpFormatter help_formatter(Keeper::options()); - auto header_str = fmt::format( - "{0} [OPTION] [-- [ARG]...]\n" + auto header_str = fmt::format("{0} [OPTION] [-- [ARG]...]\n" #if ENABLE_CLICKHOUSE_KEEPER_CLIENT - "{0} client [OPTION]\n" + "{0} client [OPTION]\n" #endif - "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010", - commandName()); + "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010", + commandName()); help_formatter.setHeader(header_str); help_formatter.format(std::cout); return 0; @@ -165,9 +162,7 @@ void Keeper::initialize(Poco::Util::Application & self) BaseDaemon::initialize(self); logger().information("starting up"); - LOG_INFO( - &logger(), - "OS Name = {}, OS Version = {}, OS Architecture = {}", + LOG_INFO(&logger(), "OS Name = {}, OS Version = {}, OS Architecture = {}", Poco::Environment::osName(), Poco::Environment::osVersion(), Poco::Environment::osArchitecture()); @@ -192,64 +187,82 @@ void Keeper::handleCustomArguments(const std::string & arg, [[maybe_unused]] con void Keeper::defineOptions(Poco::Util::OptionSet & options) { - options.addOption(Poco::Util::Option("help", "h", "show help and exit").required(false).repeatable(false).binding("help")); - options.addOption(Poco::Util::Option("version", "V", "show version and exit").required(false).repeatable(false).binding("version")); options.addOption( - Poco::Util::Option( - "force-recovery", "force-recovery", "Force recovery mode allowing Keeper to overwrite cluster configuration without quorum") + Poco::Util::Option("help", "h", "show help and exit") .required(false) .repeatable(false) - .noArgument() - .callback(Poco::Util::OptionCallback(this, &Keeper::handleCustomArguments))); + .binding("help")); + options.addOption( + Poco::Util::Option("version", "V", "show version and exit") + .required(false) + .repeatable(false) + .binding("version")); + options.addOption( + Poco::Util::Option("force-recovery", "force-recovery", "Force recovery mode allowing Keeper to overwrite cluster configuration without quorum") + .required(false) + .repeatable(false) + .noArgument() + .callback(Poco::Util::OptionCallback(this, &Keeper::handleCustomArguments))); BaseDaemon::defineOptions(options); } namespace { - struct KeeperHTTPContext : public IHTTPContext +struct KeeperHTTPContext : public IHTTPContext +{ + explicit KeeperHTTPContext(ContextPtr context_) + : context(std::move(context_)) + {} + + uint64_t getMaxHstsAge() const override { - explicit KeeperHTTPContext(ContextPtr context_) : context(std::move(context_)) { } - - uint64_t getMaxHstsAge() const override { return context->getConfigRef().getUInt64("keeper_server.hsts_max_age", 0); } - - uint64_t getMaxUriSize() const override { return context->getConfigRef().getUInt64("keeper_server.http_max_uri_size", 1048576); } - - uint64_t getMaxFields() const override { return context->getConfigRef().getUInt64("keeper_server.http_max_fields", 1000000); } - - uint64_t getMaxFieldNameSize() const override - { - return context->getConfigRef().getUInt64("keeper_server.http_max_field_name_size", 128 * 1024); - } - - uint64_t getMaxFieldValueSize() const override - { - return context->getConfigRef().getUInt64("keeper_server.http_max_field_value_size", 128 * 1024); - } - - uint64_t getMaxChunkSize() const override - { - return context->getConfigRef().getUInt64("keeper_server.http_max_chunk_size", 100_GiB); - } - - Poco::Timespan getReceiveTimeout() const override - { - return {context->getConfigRef().getInt64("keeper_server.http_receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0}; - } - - Poco::Timespan getSendTimeout() const override - { - return {context->getConfigRef().getInt64("keeper_server.http_send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0}; - } - - ContextPtr context; - }; - - HTTPContextPtr httpContext() - { - return std::make_shared(Context::getGlobalContextInstance()); + return context->getConfigRef().getUInt64("keeper_server.hsts_max_age", 0); } + uint64_t getMaxUriSize() const override + { + return context->getConfigRef().getUInt64("keeper_server.http_max_uri_size", 1048576); + } + + uint64_t getMaxFields() const override + { + return context->getConfigRef().getUInt64("keeper_server.http_max_fields", 1000000); + } + + uint64_t getMaxFieldNameSize() const override + { + return context->getConfigRef().getUInt64("keeper_server.http_max_field_name_size", 128 * 1024); + } + + uint64_t getMaxFieldValueSize() const override + { + return context->getConfigRef().getUInt64("keeper_server.http_max_field_value_size", 128 * 1024); + } + + uint64_t getMaxChunkSize() const override + { + return context->getConfigRef().getUInt64("keeper_server.http_max_chunk_size", 100_GiB); + } + + Poco::Timespan getReceiveTimeout() const override + { + return {context->getConfigRef().getInt64("keeper_server.http_receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0}; + } + + Poco::Timespan getSendTimeout() const override + { + return {context->getConfigRef().getInt64("keeper_server.http_send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0}; + } + + ContextPtr context; +}; + +HTTPContextPtr httpContext() +{ + return std::make_shared(Context::getGlobalContextInstance()); +} + } int Keeper::main(const std::vector & /*args*/) @@ -286,7 +299,7 @@ try std::filesystem::create_directories(path); /// Check that the process user id matches the owner of the data. - assertProcessUserMatchesDataOwner(path, [&](const std::string & message) { LOG_WARNING(log, fmt::runtime(message)); }); + assertProcessUserMatchesDataOwner(path, [&](const std::string & message){ LOG_WARNING(log, fmt::runtime(message)); }); DB::ServerUUID::load(path + "/uuid", log); @@ -295,7 +308,8 @@ try GlobalThreadPool::initialize( config().getUInt("max_thread_pool_size", 100), config().getUInt("max_thread_pool_free_size", 1000), - config().getUInt("thread_pool_queue_size", 10000)); + config().getUInt("thread_pool_queue_size", 10000) + ); static ServerErrorHandler error_handler; Poco::ErrorHandler::set(&error_handler); @@ -337,7 +351,8 @@ try for (const auto & server : *servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); return metrics; - }); + } + ); std::vector listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host"); @@ -353,7 +368,10 @@ try global_context->initializeKeeperDispatcher(/* start_async = */ true); FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher()); - auto config_getter = [&]() -> const Poco::Util::AbstractConfiguration & { return global_context->getConfigRef(); }; + auto config_getter = [&] () -> const Poco::Util::AbstractConfiguration & + { + return global_context->getConfigRef(); + }; auto tcp_receive_timeout = config().getInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC); auto tcp_send_timeout = config().getInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC); @@ -362,55 +380,43 @@ try { /// TCP Keeper const char * port_name = "keeper_server.tcp_port"; - createServer( - listen_host, - port_name, - listen_try, - [&](UInt16 port) - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port); - socket.setReceiveTimeout(Poco::Timespan{tcp_receive_timeout, 0}); - socket.setSendTimeout(Poco::Timespan{tcp_send_timeout, 0}); - servers->emplace_back( - listen_host, - port_name, - "Keeper (tcp): " + address.toString(), - std::make_unique( - new KeeperTCPHandlerFactory( - config_getter, global_context->getKeeperDispatcher(), tcp_receive_timeout, tcp_send_timeout, false), - server_pool, - socket)); - }); + createServer(listen_host, port_name, listen_try, [&](UInt16 port) + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(Poco::Timespan{tcp_receive_timeout, 0}); + socket.setSendTimeout(Poco::Timespan{tcp_send_timeout, 0}); + servers->emplace_back( + listen_host, + port_name, + "Keeper (tcp): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory( + config_getter, global_context->getKeeperDispatcher(), + tcp_receive_timeout, tcp_send_timeout, false), server_pool, socket)); + }); const char * secure_port_name = "keeper_server.tcp_port_secure"; - createServer( - listen_host, - secure_port_name, - listen_try, - [&](UInt16 port) - { + createServer(listen_host, secure_port_name, listen_try, [&](UInt16 port) + { #if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan{tcp_receive_timeout, 0}); - socket.setSendTimeout(Poco::Timespan{tcp_send_timeout, 0}); - servers->emplace_back( - listen_host, - secure_port_name, - "Keeper with secure protocol (tcp_secure): " + address.toString(), - std::make_unique( - new KeeperTCPHandlerFactory( - config_getter, global_context->getKeeperDispatcher(), tcp_receive_timeout, tcp_send_timeout, true), - server_pool, - socket)); + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan{tcp_receive_timeout, 0}); + socket.setSendTimeout(Poco::Timespan{tcp_send_timeout, 0}); + servers->emplace_back( + listen_host, + secure_port_name, + "Keeper with secure protocol (tcp_secure): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory( + config_getter, global_context->getKeeperDispatcher(), + tcp_receive_timeout, tcp_send_timeout, true), server_pool, socket)); #else - UNUSED(port); - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); + UNUSED(port); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); #endif - }); + }); const auto & config = config_getter(); auto http_context = httpContext(); @@ -421,27 +427,19 @@ try /// Prometheus (if defined and not setup yet with http_port) port_name = "prometheus.port"; - createServer( - listen_host, - port_name, - listen_try, - [&, my_http_context = std::move(http_context)](UInt16 port) mutable - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, listen_host, port); - socket.setReceiveTimeout(my_http_context->getReceiveTimeout()); - socket.setSendTimeout(my_http_context->getSendTimeout()); - servers->emplace_back( - listen_host, - port_name, - "Prometheus: http://" + address.toString(), - std::make_unique( - std::move(my_http_context), - createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), - server_pool, - socket, - http_params)); - }); + createServer(listen_host, port_name, listen_try, [&, my_http_context = std::move(http_context)](UInt16 port) mutable + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(socket, listen_host, port); + socket.setReceiveTimeout(my_http_context->getReceiveTimeout()); + socket.setSendTimeout(my_http_context->getSendTimeout()); + servers->emplace_back( + listen_host, + port_name, + "Prometheus: http://" + address.toString(), + std::make_unique( + std::move(my_http_context), createPrometheusMainHandlerFactory(*this, config_getter(), async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); + }); } for (auto & server : *servers) @@ -454,10 +452,18 @@ try zkutil::EventPtr unused_event = std::make_shared(); zkutil::ZooKeeperNodeCache unused_cache([] { return nullptr; }); + + const std::string cert_path = config().getString("openSSL.server.certificateFile", ""); + const std::string key_path = config().getString("openSSL.server.privateKeyFile", ""); + + std::vector extra_paths = {include_from_path}; + if (!cert_path.empty()) extra_paths.emplace_back(cert_path); + if (!key_path.empty()) extra_paths.emplace_back(key_path); + /// ConfigReloader have to strict parameters which are redundant in our case auto main_config_reloader = std::make_unique( config_path, - std::vector{{include_from_path}}, + extra_paths, config().getString("path", ""), std::move(unused_cache), unused_event, @@ -465,8 +471,10 @@ try { if (config->has("keeper_server")) global_context->updateKeeperConfiguration(*config); + + CertificateReloader::instance().tryLoad(*config); }, - /* already_loaded = */ false); /// Reload it right now (initial loading) + /* already_loaded = */ false); /// Reload it right now (initial loading) SCOPE_EXIT({ LOG_INFO(log, "Shutting down."); @@ -491,10 +499,7 @@ try current_connections = waitServersToFinish(*servers, servers_lock, config().getInt("shutdown_wait_unfinished", 5)); if (current_connections) - LOG_INFO( - log, - "Closed connections to Keeper. But {} remain. Probably some users cannot finish their connections after context shutdown.", - current_connections); + LOG_INFO(log, "Closed connections to Keeper. But {} remain. Probably some users cannot finish their connections after context shutdown.", current_connections); else LOG_INFO(log, "Closed connections to Keeper."); @@ -532,10 +537,11 @@ catch (...) void Keeper::logRevision() const { - Poco::Logger::root().information( - "Starting ClickHouse Keeper " + std::string{VERSION_STRING} + "(revision : " - + std::to_string(ClickHouseRevision::getVersionRevision()) + ", git hash: " + (git_hash.empty() ? "" : git_hash) - + ", build id: " + (build_id.empty() ? "" : build_id) + ")" + ", PID " + std::to_string(getpid())); + Poco::Logger::root().information("Starting ClickHouse Keeper " + std::string{VERSION_STRING} + + "(revision : " + std::to_string(ClickHouseRevision::getVersionRevision()) + + ", git hash: " + (git_hash.empty() ? "" : git_hash) + + ", build id: " + (build_id.empty() ? "" : build_id) + ")" + + ", PID " + std::to_string(getpid())); } From 6d712f595af9096b20865c5cea216f05c791784c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 4 Jul 2023 23:45:49 +0200 Subject: [PATCH 0504/2047] Make sure that remote query has non-zero columns --- src/QueryPipeline/RemoteQueryExecutor.cpp | 14 +++++++++++++- src/QueryPipeline/RemoteQueryExecutor.h | 4 ++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index cd6f65b7b43..428d39cbc68 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -48,7 +48,9 @@ RemoteQueryExecutor::RemoteQueryExecutor( : header(header_), query(query_), context(context_), scalars(scalars_) , external_tables(external_tables_), stage(stage_) , extension(extension_) -{} +{ + assertHeaderIsNotEmpty(header); +} RemoteQueryExecutor::RemoteQueryExecutor( Connection & connection, @@ -91,6 +93,8 @@ RemoteQueryExecutor::RemoteQueryExecutor( , scalars(scalars_), external_tables(external_tables_), stage(stage_) , extension(extension_) { + assertHeaderIsNotEmpty(header); + create_connections = [this, connections_, throttler, extension_](AsyncCallback) mutable { auto res = std::make_unique(std::move(connections_), context->getSettingsRef(), throttler); if (extension_ && extension_->replica_info) @@ -108,6 +112,8 @@ RemoteQueryExecutor::RemoteQueryExecutor( , scalars(scalars_), external_tables(external_tables_), stage(stage_) , extension(extension_) { + assertHeaderIsNotEmpty(header); + create_connections = [this, pool, throttler, extension_](AsyncCallback async_callback)->std::unique_ptr { const Settings & current_settings = context->getSettingsRef(); @@ -754,4 +760,10 @@ bool RemoteQueryExecutor::hasThrownException() const return got_exception_from_replica || got_unknown_packet_from_replica; } +void RemoteQueryExecutor::assertHeaderIsNotEmpty(const Block & header) +{ + if (header.columns() == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote query executor must not have header without columns"); +} + } diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index a843ce520de..054ea26d280 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -304,6 +304,10 @@ private: /// Reads packet by packet Block readPackets(); + + /// Block does not support passing 0 columns but non-zero rows. + /// We need to ensure that this is not happening. + static void assertHeaderIsNotEmpty(const Block & header); }; } From 9757e272b931d40d60d4bfa43b9999907a06cdf0 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 6 Jul 2023 15:41:02 +0200 Subject: [PATCH 0505/2047] Check number of rows in the reader instead --- src/Formats/NativeReader.cpp | 6 ++++++ src/QueryPipeline/RemoteQueryExecutor.cpp | 14 +------------- src/QueryPipeline/RemoteQueryExecutor.h | 4 ---- 3 files changed, 7 insertions(+), 17 deletions(-) diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index ff75ffb3c02..4c25460eb63 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -149,6 +149,9 @@ Block NativeReader::read() rows = index_block_it->num_rows; } + if (columns == 0 && !header && rows != 0) + throw Exception(ErrorCodes::INCORRECT_DATA, "Zero columns but {} rows in Native format.", rows); + for (size_t i = 0; i < columns; ++i) { if (use_index) @@ -290,6 +293,9 @@ Block NativeReader::read() res.swap(tmp_res); } + if (res.rows() != rows) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Row count mismatch after desirialization, got: {}, expected: {}", res.rows(), rows); + return res; } diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 428d39cbc68..cd6f65b7b43 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -48,9 +48,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( : header(header_), query(query_), context(context_), scalars(scalars_) , external_tables(external_tables_), stage(stage_) , extension(extension_) -{ - assertHeaderIsNotEmpty(header); -} +{} RemoteQueryExecutor::RemoteQueryExecutor( Connection & connection, @@ -93,8 +91,6 @@ RemoteQueryExecutor::RemoteQueryExecutor( , scalars(scalars_), external_tables(external_tables_), stage(stage_) , extension(extension_) { - assertHeaderIsNotEmpty(header); - create_connections = [this, connections_, throttler, extension_](AsyncCallback) mutable { auto res = std::make_unique(std::move(connections_), context->getSettingsRef(), throttler); if (extension_ && extension_->replica_info) @@ -112,8 +108,6 @@ RemoteQueryExecutor::RemoteQueryExecutor( , scalars(scalars_), external_tables(external_tables_), stage(stage_) , extension(extension_) { - assertHeaderIsNotEmpty(header); - create_connections = [this, pool, throttler, extension_](AsyncCallback async_callback)->std::unique_ptr { const Settings & current_settings = context->getSettingsRef(); @@ -760,10 +754,4 @@ bool RemoteQueryExecutor::hasThrownException() const return got_exception_from_replica || got_unknown_packet_from_replica; } -void RemoteQueryExecutor::assertHeaderIsNotEmpty(const Block & header) -{ - if (header.columns() == 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Remote query executor must not have header without columns"); -} - } diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 054ea26d280..a843ce520de 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -304,10 +304,6 @@ private: /// Reads packet by packet Block readPackets(); - - /// Block does not support passing 0 columns but non-zero rows. - /// We need to ensure that this is not happening. - static void assertHeaderIsNotEmpty(const Block & header); }; } From ce13131de47b28d61b013c386697bf011b13054e Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 11 Jul 2023 10:29:41 +0000 Subject: [PATCH 0506/2047] Fix integration tests --- src/Common/Config/ConfigReloader.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index 45192d2d281..12a449882d7 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -6,6 +6,7 @@ #include "ConfigProcessor.h" #include #include +#include namespace fs = std::filesystem; @@ -132,7 +133,8 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); #if USE_SSL - config_processor.decryptConfig(loaded_config); + if (endsWith(path, "config.xml")) + config_processor.decryptConfig(loaded_config); #endif /** We should remember last modification time if and only if config was successfully loaded From 471e5a056dc14251136372b3e22925df5a178f94 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 11 Jul 2023 10:40:55 +0000 Subject: [PATCH 0507/2047] fix --- programs/keeper/Keeper.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index f4bb08676d2..4c0f08eb314 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -472,7 +472,9 @@ try if (config->has("keeper_server")) global_context->updateKeeperConfiguration(*config); +#if USE_SSL CertificateReloader::instance().tryLoad(*config); +#endif }, /* already_loaded = */ false); /// Reload it right now (initial loading) From 3e8906e5a231c714020d180bb40edfea6838d404 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 11 Jul 2023 12:23:10 +0000 Subject: [PATCH 0508/2047] fix build for non-ssl systems --- programs/keeper/Keeper.cpp | 2 +- programs/server/Server.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 4c0f08eb314..6034d63a016 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -21,7 +21,6 @@ #include #include #include -#include #include #include @@ -43,6 +42,7 @@ #if USE_SSL # include # include +# include #endif #include diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index af9abc1024f..c7a7ba71e83 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -88,7 +88,6 @@ #include #include #include -#include #include #include #include @@ -109,6 +108,7 @@ #if USE_SSL # include +# include #endif #if USE_GRPC From 49c1beb8705399dfb40ae5213ebca70ba31e3852 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 11 Jul 2023 22:25:30 +1000 Subject: [PATCH 0509/2047] Used timeout function instead of undefined clickhouse_client_loop_timeout in functional tests (#51923) * removed undefined clickhouse_client_loop_timeout * test fix and improvement * Changes after review iteration --- .../0_stateless/02242_delete_user_race.sh | 27 ++++++++++++------- .../0_stateless/02243_drop_user_grant_race.sh | 13 ++++----- 2 files changed, 24 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02242_delete_user_race.sh b/tests/queries/0_stateless/02242_delete_user_race.sh index f22b7796bd4..8f387333c33 100755 --- a/tests/queries/0_stateless/02242_delete_user_race.sh +++ b/tests/queries/0_stateless/02242_delete_user_race.sh @@ -22,18 +22,27 @@ $CLICKHOUSE_CLIENT -nm -q " function delete_user() { - $CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS test_user_02242" ||: + while true; do + $CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS test_user_02242" ||: + sleep 0.$RANDOM; + done } function create_and_login_user() { - $CLICKHOUSE_CLIENT -q "CREATE USER IF NOT EXISTS test_user_02242" ||: - $CLICKHOUSE_CLIENT -u "test_user_02242" -q "SELECT COUNT(*) FROM system.session_log WHERE user == 'test_user_02242'" > /dev/null ||: + while true; do + $CLICKHOUSE_CLIENT -q "CREATE USER IF NOT EXISTS test_user_02242" ||: + $CLICKHOUSE_CLIENT -u "test_user_02242" -q "SELECT COUNT(*) FROM system.session_log WHERE user == 'test_user_02242'" > /dev/null ||: + sleep 0.$RANDOM; + done } function set_role() { - $CLICKHOUSE_CLIENT -q "SET ROLE test_role_02242 TO test_user_02242" ||: + while true; do + $CLICKHOUSE_CLIENT -q "SET DEFAULT ROLE test_role_02242 TO test_user_02242" ||: + sleep 0.$RANDOM; + done } export -f delete_user @@ -42,12 +51,10 @@ export -f set_role TIMEOUT=10 -for (( i = 0 ; i < 100; ++i )) -do - clickhouse_client_loop_timeout $TIMEOUT create_and_login_user 2> /dev/null & - clickhouse_client_loop_timeout $TIMEOUT delete_user 2> /dev/null & - clickhouse_client_loop_timeout $TIMEOUT set_role 2> /dev/null & -done + +timeout $TIMEOUT bash -c create_and_login_user 2> /dev/null & +timeout $TIMEOUT bash -c delete_user 2> /dev/null & +timeout $TIMEOUT bash -c set_role 2> /dev/null & wait diff --git a/tests/queries/0_stateless/02243_drop_user_grant_race.sh b/tests/queries/0_stateless/02243_drop_user_grant_race.sh index e36be96aa02..46ad776006e 100755 --- a/tests/queries/0_stateless/02243_drop_user_grant_race.sh +++ b/tests/queries/0_stateless/02243_drop_user_grant_race.sh @@ -19,17 +19,18 @@ $CLICKHOUSE_CLIENT -nm -q " function create_drop_grant() { - $CLICKHOUSE_CLIENT -q "CREATE USER IF NOT EXISTS test_user_02243 GRANTEES NONE" ||: - $CLICKHOUSE_CLIENT -q "GRANT ALL ON *.* TO test_user_02243 WITH GRANT OPTION" ||: - $CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS test_user_02243" & - $CLICKHOUSE_CLIENT --user test_user_02243 -q "GRANT ALL ON *.* TO kek_02243" & - wait + while true; do + $CLICKHOUSE_CLIENT -q "CREATE USER IF NOT EXISTS test_user_02243 GRANTEES NONE" ||: + $CLICKHOUSE_CLIENT -q "GRANT ALL ON *.* TO test_user_02243 WITH GRANT OPTION" ||: + $CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS test_user_02243" & + $CLICKHOUSE_CLIENT --user test_user_02243 -q "GRANT ALL ON *.* TO kek_02243" & + done } export -f create_drop_grant TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT create_drop_grant 2> /dev/null & +timeout $TIMEOUT bash -c create_drop_grant 2> /dev/null & wait $CLICKHOUSE_CLIENT --user kek_02243 -q "SELECT * FROM test" 2>&1| grep -Fa "Exception: " | grep -Eo ACCESS_DENIED | uniq From da3de04470e6305dc7d0e6a8713791b479b378d4 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 11 Jul 2023 12:27:23 +0000 Subject: [PATCH 0510/2047] fix --- src/Common/Config/ConfigReloader.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/Config/ConfigReloader.h b/src/Common/Config/ConfigReloader.h index c5a18524318..2529c7a5236 100644 --- a/src/Common/Config/ConfigReloader.h +++ b/src/Common/Config/ConfigReloader.h @@ -4,8 +4,7 @@ #include #include #include -#include -#include +#include #include #include #include From 1739bb306abe11210127fd384c5a278c3e59556d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 10 May 2023 02:26:32 +0800 Subject: [PATCH 0511/2047] Projection & optimize_aggregators_of_group_by_keys Fix projection with optimize_aggregators_of_group_by_keys = true --- src/Parsers/ASTProjectionSelectQuery.cpp | 7 +++++++ ...jection_optimize_aggregators_of_group_by_keys.reference | 1 + ...10_projection_optimize_aggregators_of_group_by_keys.sql | 7 +++++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference create mode 100644 tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql diff --git a/src/Parsers/ASTProjectionSelectQuery.cpp b/src/Parsers/ASTProjectionSelectQuery.cpp index da3d9286f0a..7b5889753fa 100644 --- a/src/Parsers/ASTProjectionSelectQuery.cpp +++ b/src/Parsers/ASTProjectionSelectQuery.cpp @@ -142,6 +142,13 @@ ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const } if (groupBy()) select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, groupBy()->clone()); + + auto settings_query = std::make_shared(); + SettingsChanges settings_changes; + settings_changes.insertSetting("optimize_aggregators_of_group_by_keys", false); + settings_query->changes = std::move(settings_changes); + settings_query->is_standalone = false; + select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(settings_query)); return node; } diff --git a/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference new file mode 100644 index 00000000000..738b2b9b918 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference @@ -0,0 +1 @@ +2012-10-23 diff --git a/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql new file mode 100644 index 00000000000..ef80dcd7143 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql @@ -0,0 +1,7 @@ +drop table if exists proj; + +CREATE TABLE proj(date Date, PROJECTION maxdate( SELECT max(date) GROUP BY date )) ENGINE = MergeTree ORDER BY tuple() as select toDate('2012-10-24')-number%100 from numbers(1e2); + +SELECT max(date) FROM proj PREWHERE date != '2012-10-24'; + +drop table proj; From a33b90c648e6797c9272a2a31b689fc58bd680f1 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 16 Jun 2023 10:35:55 +0800 Subject: [PATCH 0512/2047] Projection & optimize_group_by_function_keys --- src/Parsers/ASTProjectionSelectQuery.cpp | 1 + ..._optimize_group_by_function_keys.reference | 0 ...ection_optimize_group_by_function_keys.sql | 31 +++++++++++++++++++ 3 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.reference create mode 100644 tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql diff --git a/src/Parsers/ASTProjectionSelectQuery.cpp b/src/Parsers/ASTProjectionSelectQuery.cpp index 7b5889753fa..0cfdc3762a1 100644 --- a/src/Parsers/ASTProjectionSelectQuery.cpp +++ b/src/Parsers/ASTProjectionSelectQuery.cpp @@ -146,6 +146,7 @@ ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const auto settings_query = std::make_shared(); SettingsChanges settings_changes; settings_changes.insertSetting("optimize_aggregators_of_group_by_keys", false); + settings_changes.insertSetting("optimize_group_by_function_keys", false); settings_query->changes = std::move(settings_changes); settings_query->is_standalone = false; select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(settings_query)); diff --git a/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.reference b/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql b/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql new file mode 100644 index 00000000000..bc9b3151900 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql @@ -0,0 +1,31 @@ +drop table if exists proj; + +create table proj ( + bool_value UInt8, + zero_integer_value Int32, + integer_value Int32, + float_value Float32, + datetime_value DateTime, + string_value String, + projection test_projection ( + select + toStartOfDay (toDateTime (datetime_value)) as Day, + datetime_value, + float_value, + count( + distinct if(zero_integer_value = 1, string_value, NULL) + ) + group by + Day, + datetime_value, + float_value + ) + ) engine MergeTree +partition by + toDate (datetime_value) +order by + bool_value; + +insert into proj values (1, 1, 1, 1, '2012-10-24 21:30:00', 'ab'); + +drop table proj; From a73dca1c2ffd07c962801c6983bafa3a39d34317 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 11 Jul 2023 13:27:11 +0000 Subject: [PATCH 0513/2047] Move getEncryptionMethod to CompressionCodecEncrypted.h --- src/Common/Config/ConfigProcessor.cpp | 37 +++++-------------- src/Compression/CompressionCodecEncrypted.cpp | 12 +++++- src/Compression/CompressionCodecEncrypted.h | 3 ++ 3 files changed, 24 insertions(+), 28 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 11b45977322..9c2100b1c15 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -52,25 +52,6 @@ namespace ErrorCodes #endif } -#if USE_SSL -namespace -{ - -/// Get method for string name. Throw exception for wrong name -EncryptionMethod getEncryptionMethod(const std::string & name) -{ - if (name == "AES_128_GCM_SIV") - return AES_128_GCM_SIV; - else if (name == "AES_256_GCM_SIV") - return AES_256_GCM_SIV; - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", name); -} - -} - -#endif - /// For cutting preprocessed path to this base static std::string main_config_path; @@ -204,11 +185,12 @@ static void mergeAttributes(Element & config_element, Element & with_element) std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) { - auto codec = DB::CompressionCodecEncrypted(getEncryptionMethod(codec_name)); + EncryptionMethod method = getEncryptionMethod(codec_name); + CompressionCodecEncrypted codec(method); - DB::Memory<> memory; - memory.resize(codec.getCompressedReserveSize(static_cast(value.size()))); - auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory.data()); + Memory<> memory; + memory.resize(codec.getCompressedReserveSize(static_cast(value.size()))); + auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory.data()); auto encrypted_value = std::string(memory.data(), bytes_written); std::string hex_value; boost::algorithm::hex(encrypted_value.begin(), encrypted_value.end(), std::back_inserter(hex_value)); @@ -217,9 +199,10 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) { - auto codec = DB::CompressionCodecEncrypted(getEncryptionMethod(codec_name)); + EncryptionMethod method = getEncryptionMethod(codec_name); + CompressionCodecEncrypted codec(method); - DB::Memory<> memory; + Memory<> memory; std::string encrypted_value; try @@ -232,7 +215,7 @@ std::string ConfigProcessor::decryptValue(const std::string & codec_name, const } memory.resize(codec.readDecompressedBlockSize(encrypted_value.data())); - codec.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); + codec.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); std::string decrypted_value = std::string(memory.data(), memory.size()); return decrypted_value; } @@ -797,7 +780,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) { - DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); + CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); Node * config_root = getRootNode(loaded_config.preprocessed_xml.get()); decryptRecursive(config_root); loaded_config.configuration = new Poco::Util::XMLConfiguration(loaded_config.preprocessed_xml); diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 022bbd583e4..01bf6281a0e 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -28,6 +28,17 @@ namespace DB namespace ErrorCodes { extern const int OPENSSL_ERROR; + extern const int BAD_ARGUMENTS; +} + +EncryptionMethod getEncryptionMethod(const std::string & name) +{ + if (name == "AES_128_GCM_SIV") + return AES_128_GCM_SIV; + else if (name == "AES_256_GCM_SIV") + return AES_256_GCM_SIV; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", name); } namespace @@ -79,7 +90,6 @@ namespace ErrorCodes { extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; extern const int INCORRECT_DATA; } diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h index 0f680dbcb09..a8faf3d0b6c 100644 --- a/src/Compression/CompressionCodecEncrypted.h +++ b/src/Compression/CompressionCodecEncrypted.h @@ -18,6 +18,9 @@ enum EncryptionMethod MAX_ENCRYPTION_METHOD }; +/// Get method for string name. Throw exception for wrong name +EncryptionMethod getEncryptionMethod(const std::string & name); + /** This codec encrypts and decrypts blocks with AES-128 in * GCM-SIV mode (RFC-8452), which is the only cipher currently * supported. Although it is implemented as a compression codec From d0f81fb1cd10f45ad4fe9679ad9b55841b59b7ba Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Tue, 11 Jul 2023 13:33:07 +0000 Subject: [PATCH 0514/2047] fix --- programs/keeper/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 18bdc8f317c..54c39f5709f 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -73,6 +73,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/TCPServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/NotFoundHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/ProtocolServerAdapter.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/CertificateReloader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusMetricsWriter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/waitServersToFinish.cpp From 3b8ecb172f81bcaa7538f257a3431cd4e0787645 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 11 Jul 2023 14:43:25 +0000 Subject: [PATCH 0515/2047] Move descryption code to savePreprocessedConfig --- src/Common/Config/ConfigProcessor.cpp | 13 +++++++++---- src/Common/Config/ConfigProcessor.h | 12 ++++++------ src/Common/Config/ConfigReloader.cpp | 6 ------ src/Daemon/BaseDaemon.cpp | 4 ---- 4 files changed, 15 insertions(+), 20 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 9c2100b1c15..0d40f782be2 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -183,7 +183,7 @@ static void mergeAttributes(Element & config_element, Element & with_element) #if USE_SSL -std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) +std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) const { EncryptionMethod method = getEncryptionMethod(codec_name); CompressionCodecEncrypted codec(method); @@ -197,7 +197,7 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const return hex_value; } -std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) +std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) const { EncryptionMethod method = getEncryptionMethod(codec_name); CompressionCodecEncrypted codec(method); @@ -778,7 +778,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( #if USE_SSL -void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) +void ConfigProcessor::decryptEncryptedElements(LoadedConfig & loaded_config) { CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); Node * config_root = getRootNode(loaded_config.preprocessed_xml.get()); @@ -788,7 +788,7 @@ void ConfigProcessor::decryptConfig(LoadedConfig & loaded_config) #endif -void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir) +void ConfigProcessor::savePreprocessedConfig(LoadedConfig & loaded_config, std::string preprocessed_dir) { try { @@ -843,6 +843,11 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config, { LOG_WARNING(log, "Couldn't save preprocessed config to {}: {}", preprocessed_path, e.displayText()); } + +#if USE_SSL + if (fs::path(preprocessed_path).filename() == "config.xml") + decryptEncryptedElements(loaded_config); +#endif } void ConfigProcessor::setConfigPath(const std::string & config_path) diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 8d7caa9e9c8..0da304bc0d2 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -94,18 +94,15 @@ public: #if USE_SSL /// Encrypt text value - std::string encryptValue(const std::string & codec_name, const std::string & value); + std::string encryptValue(const std::string & codec_name, const std::string & value) const; /// Decrypt value - std::string decryptValue(const std::string & codec_name, const std::string & value); - - /// Decrypt nodes in config with specified encryption attributes - void decryptConfig(LoadedConfig & loaded_config); + std::string decryptValue(const std::string & codec_name, const std::string & value) const; #endif /// Save preprocessed config to specified directory. /// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/ - void savePreprocessedConfig(const LoadedConfig & loaded_config, std::string preprocessed_dir); + void savePreprocessedConfig(LoadedConfig & loaded_config, std::string preprocessed_dir); /// Set path of main config.xml. It will be cut from all configs placed to preprocessed_configs/ static void setConfigPath(const std::string & config_path); @@ -137,6 +134,9 @@ private: #if USE_SSL void decryptRecursive(Poco::XML::Node * config_root); + + /// Decrypt elements in config with specified encryption attributes + void decryptEncryptedElements(LoadedConfig & loaded_config); #endif void mergeRecursive(XMLDocumentPtr config, Poco::XML::Node * config_root, const Poco::XML::Node * with_root); diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index 12a449882d7..de7011b67bf 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -6,7 +6,6 @@ #include "ConfigProcessor.h" #include #include -#include namespace fs = std::filesystem; @@ -132,11 +131,6 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac } config_processor.savePreprocessedConfig(loaded_config, preprocessed_dir); -#if USE_SSL - if (endsWith(path, "config.xml")) - config_processor.decryptConfig(loaded_config); -#endif - /** We should remember last modification time if and only if config was successfully loaded * Otherwise a race condition could occur during config files update: * File is contain raw (and non-valid) data, therefore config is not applied. diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 3e31ed167c0..319d2bc8b5b 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -763,10 +763,6 @@ void BaseDaemon::initialize(Application & self) ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, ""); -#if USE_SSL - DB::ConfigProcessor(config_path).decryptConfig(loaded_config); -#endif - /// Write core dump on crash. { struct rlimit rlim; From b9adb2039bfdd90eb5b05b3f76a4aebd5e0b1757 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 11 Jul 2023 15:13:47 +0000 Subject: [PATCH 0516/2047] Update MD docs --- docs/en/operations/configuration-files.md | 10 ++++------ docs/ru/operations/configuration-files.md | 10 ++++------ 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index f8f249598cc..8cd34cc6c30 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -65,9 +65,11 @@ XML substitution example: Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. -## Decryption {#decryption} +## Encrypting Configuration {#encryption} -Elements with text nodes may be encrypted with [encryption codecs](../sql-reference/statements/create/table.md#encryption-codecs). In this case `` section should be included in configuration file and each element node with encrypted text should have `encryption_codec` attribute with name of codec. +You can use symmetric encryption to encrypt a configuration element, for example, a password field. To do so, first configure the [encryption codec](../sql-reference/statements/create/table.md#encryption-codecs), then add attribute `encryption_codec`` with the name of the encryption codec as value to the element to encrypt. + +Unlike attributes `from_zk`, `from_env` and `incl`` (or element `include``), no substitution, i.e. decryption of the encrypted value, is performed in the preprocessed file. Decryption happens only at runtime in the server process. Example: @@ -97,10 +99,6 @@ Example: 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 ``` -:::note -The decryption is executed after creation of preprocessed configuration file. It means that elements with `encryption_codec` attribute in the preprocessed configuration file are encrypted. But the values of corresponding parameters in server's memory are decrypted. -::: - ## User Settings {#user-settings} The `config.xml` file can specify a separate config with user settings, profiles, and quotas. The relative path to this config is set in the `users_config` element. By default, it is `users.xml`. If `users_config` is omitted, the user settings, profiles, and quotas are specified directly in `config.xml`. diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index df50d900919..91877bdef95 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -85,9 +85,11 @@ $ cat /etc/clickhouse-server/users.d/alice.xml Сервер следит за изменениями конфигурационных файлов, а также файлов и ZooKeeper-узлов, которые были использованы при выполнении подстановок и переопределений, и перезагружает настройки пользователей и кластеров на лету. То есть, можно изменять кластера, пользователей и их настройки без перезапуска сервера. -## Расшифровка {#decryption} +## Шифрование {#encryption} -Элементы с текстовыми узлами могут быть зашифрован с помощью [кодеков шифрования](../sql-reference/statements/create/table.md#create-query-encryption-codecs). В этом случае секция `` должна быть включена в конфигурационный файл и каждый элемент с зашифрованным текстом должен иметь аттрибут `encryption_codec` с именем кодека. +Вы можете использовать симметричное шифрование для зашифровки элемента конфигурации, например, поля password. Чтобы это сделать, сначала настройте [кодек шифрования](../sql-reference/statements/create/table.md#encryption-codecs), затем добавьте аттибут`encryption_codec`` с именем кодека шифрования как значение к элементу, который надо зашифровать. + +В отличии от аттрибутов `from_zk`, `from_env` и `incl`` (или элемента `include``), подстановка, т.е. расшифровка зашифрованного значения, не выподняется в файле предобработки. Расшифровка происходит только во время исполнения в серверном процессе. Пример: @@ -117,10 +119,6 @@ $ cat /etc/clickhouse-server/users.d/alice.xml 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 ``` -:::note -Расшифровка выполняется после создания конфигурационного файла предобработки. Это означает что элементы с аттрибутом `encryption_codec` в конфигурационном файле предобработки зашифрованы. Но значения соответствующих параметров в памяти сервера расшифрованы. -::: - ## Примеры записи конфигурации на YAML {#example} Здесь можно рассмотреть пример реальной конфигурации записанной на YAML: [config.yaml.example](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/config.yaml.example). From b0fb96c8b67a4e55900e5bad58af0c15b0ff93b5 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Tue, 11 Jul 2023 21:51:05 +0800 Subject: [PATCH 0517/2047] Inline SipHash::get128(char *) The SipHash-128 values of the strings are calculated and stored in a set when the UniqExact aggregate function is applied to a Column- String, as the following snippet shows: StringRef value = column.getDataAt(row_num); UInt128 key; SipHash hash; hash.update(value.data, value.size); hash.get128(key); The implementation of SipHash requires an internal 8-byte buffer, current_word (or current_bytes), for holding the input data in the cross function calls of SipHash::update and SipHash::get128. But, in the above situation, where the update and get128 only take place once, the copy of input data to the buffer is unnecessary. With get128 inlined, the compiler could optimize the code as a whole and reduce the unexpected memory operations, and as a result, the query performance is improved. --- src/Common/SipHash.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 9e6479d81c1..cdec00d4bcc 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -188,7 +188,7 @@ public: /// Get the result in some form. This can only be done once! - void get128(char * out) + ALWAYS_INLINE void get128(char * out) { finalize(); #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ From 2eccead2d421833857222bbe05e084512653386c Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 11 Jul 2023 16:27:28 +0000 Subject: [PATCH 0518/2047] Fix const(Nullable) case, add test --- src/Functions/array/range.cpp | 8 ++++++-- tests/queries/0_stateless/02797_range_nullable.reference | 1 + tests/queries/0_stateless/02797_range_nullable.sql | 2 ++ 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/range.cpp b/src/Functions/array/range.cpp index b14f2baca15..57679ccb180 100644 --- a/src/Functions/array/range.cpp +++ b/src/Functions/array/range.cpp @@ -404,8 +404,12 @@ private: { if (!col.type->isNullable()) return; - const auto & nullable_col = assert_cast(*col.column); - const auto & null_map = nullable_col.getNullMapData(); + const ColumnNullable * nullable_col = checkAndGetColumn(*col.column); + if (!nullable_col) + nullable_col = checkAndGetColumnConstData(col.column.get()); + if (!nullable_col) + return; + const auto & null_map = nullable_col->getNullMapData(); if (!memoryIsZero(null_map.data(), 0, null_map.size())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal (null) value column {} of argument of function {}", col.column->getName(), getName()); }; diff --git a/tests/queries/0_stateless/02797_range_nullable.reference b/tests/queries/0_stateless/02797_range_nullable.reference index a082a71f4ec..0e01ba9984c 100644 --- a/tests/queries/0_stateless/02797_range_nullable.reference +++ b/tests/queries/0_stateless/02797_range_nullable.reference @@ -5,3 +5,4 @@ [0] [0,2,4,6,8] [0,2,4,6,8] +[0,2,4,6,8] diff --git a/tests/queries/0_stateless/02797_range_nullable.sql b/tests/queries/0_stateless/02797_range_nullable.sql index 2b0fe69b123..ae35eb6fb57 100644 --- a/tests/queries/0_stateless/02797_range_nullable.sql +++ b/tests/queries/0_stateless/02797_range_nullable.sql @@ -5,6 +5,8 @@ select range('string', Null); SELECT range(toNullable(1)); SELECT range(0::Nullable(UInt64), 10::Nullable(UInt64), 2::Nullable(UInt64)); SELECT range(0::Nullable(Int64), 10::Nullable(Int64), 2::Nullable(Int64)); +SELECT range(materialize(0), 10::Nullable(UInt64), 2::Nullable(UInt64)); SELECT range(Null::Nullable(UInt64), 10::Nullable(UInt64), 2::Nullable(UInt64)); -- { serverError BAD_ARGUMENTS } SELECT range(0::Nullable(UInt64), Null::Nullable(UInt64), 2::Nullable(UInt64)); -- { serverError BAD_ARGUMENTS } SELECT range(0::Nullable(UInt64), 10::Nullable(UInt64), Null::Nullable(UInt64)); -- { serverError BAD_ARGUMENTS } +SELECT range(Null::Nullable(UInt8), materialize(1)); -- { serverError BAD_ARGUMENTS } From 326d51c3fbb90c1e94c6fef647247434cce2347e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 11 Jul 2023 18:29:21 +0200 Subject: [PATCH 0519/2047] Prohibit send_metadata for s3_plain disks send_metadata changes the filenames (includes revision), while s3_plain do not care about this, and expect that the file name will not be changed. So during initialization s3_plain will do some basic checks and will write file clickhouse_access_check_e0f72fcc-d6f8-44fe-8997-96cb87767515, but instead r0000000000000000000000000000000000000000000000000000000000000001-file-clickhouse_access_check_e0f72fcc-d6f8-44fe-8997-96cb87767515 will be written, later read will not find the file and it will fail: Application: Code: 33. DB::Exception: Cannot read all data. Bytes read: 0. Bytes expected: 4.: While checking access for disk s3_plain: Cannot attach table `system`.`crash_log` ... And besides, send_metadata does not make sense for s3_plain. Signed-off-by: Azat Khuzhin --- src/Disks/ObjectStorages/S3/registerDiskS3.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index fb125ae8517..d1264affaea 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -122,6 +122,14 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check) auto client = getClient(config, config_prefix, context, *settings); if (type == "s3_plain") { + /// send_metadata changes the filenames (includes revision), while + /// s3_plain do not care about this, and expect that the file name + /// will not be changed. + /// + /// And besides, send_metadata does not make sense for s3_plain. + if (config.getBool(config_prefix + ".send_metadata", false)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "s3_plain does not supports send_metadata"); + s3_storage = std::make_shared(std::move(client), std::move(settings), uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); metadata_storage = std::make_shared(s3_storage, uri.key); } From 7c7a447fd274887492cf2817378d4d50c16ddac5 Mon Sep 17 00:00:00 2001 From: John Spurlock Date: Tue, 11 Jul 2023 12:16:55 -0500 Subject: [PATCH 0520/2047] Fix a few typos in docker readme. --- docker/server/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/server/README.md b/docker/server/README.md index 67646a262f5..6200acbd30c 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -97,8 +97,8 @@ docker run -d \ You may also want to mount: -* `/etc/clickhouse-server/config.d/*.xml` - files with server configuration adjustmenets -* `/etc/clickhouse-server/users.d/*.xml` - files with user settings adjustmenets +* `/etc/clickhouse-server/config.d/*.xml` - files with server configuration adjustments +* `/etc/clickhouse-server/users.d/*.xml` - files with user settings adjustments * `/docker-entrypoint-initdb.d/` - folder with database initialization scripts (see below). ### Linux capabilities From 0e9f0e6837cbbb9c7fa784fc9139451ce9779c57 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 11 Jul 2023 11:09:01 -0700 Subject: [PATCH 0521/2047] Fix ForEach Aggregate state for s390x --- src/AggregateFunctions/AggregateFunctionForEach.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionForEach.h b/src/AggregateFunctions/AggregateFunctionForEach.h index 480b4cc690e..ec59fd2e616 100644 --- a/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/src/AggregateFunctions/AggregateFunctionForEach.h @@ -240,7 +240,7 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { const AggregateFunctionForEachData & state = data(place); - writeBinary(state.dynamic_array_size, buf); + writeBinaryLittleEndian(state.dynamic_array_size, buf); const char * nested_state = state.array_of_aggregate_datas; for (size_t i = 0; i < state.dynamic_array_size; ++i) @@ -255,7 +255,7 @@ public: AggregateFunctionForEachData & state = data(place); size_t new_size = 0; - readBinary(new_size, buf); + readBinaryLittleEndian(new_size, buf); ensureAggregateData(place, new_size, *arena); From 1b403ab3e6ac52b0f56f071388fc07fae74379d6 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 30 May 2023 12:31:22 -0700 Subject: [PATCH 0522/2047] Fix crc32(WeakHash) issue for s390x --- src/Common/HashTable/Hash.h | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Common/HashTable/Hash.h b/src/Common/HashTable/Hash.h index 189603fcbf5..87107fa9f82 100644 --- a/src/Common/HashTable/Hash.h +++ b/src/Common/HashTable/Hash.h @@ -57,28 +57,25 @@ inline DB::UInt64 intHash64(DB::UInt64 x) inline uint32_t s390x_crc32_u8(uint32_t crc, uint8_t v) { - return crc32_be(crc, reinterpret_cast(&v), sizeof(v)); + return crc32c_le_vx(crc, reinterpret_cast(&v), sizeof(v)); } inline uint32_t s390x_crc32_u16(uint32_t crc, uint16_t v) { - return crc32_be(crc, reinterpret_cast(&v), sizeof(v)); + v = std::byteswap(v); + return crc32c_le_vx(crc, reinterpret_cast(&v), sizeof(v)); } inline uint32_t s390x_crc32_u32(uint32_t crc, uint32_t v) { - return crc32_be(crc, reinterpret_cast(&v), sizeof(v)); + v = std::byteswap(v); + return crc32c_le_vx(crc, reinterpret_cast(&v), sizeof(v)); } inline uint64_t s390x_crc32(uint64_t crc, uint64_t v) { - uint64_t _crc = crc; - uint32_t value_h, value_l; - value_h = (v >> 32) & 0xffffffff; - value_l = v & 0xffffffff; - _crc = crc32_be(static_cast(_crc), reinterpret_cast(&value_h), sizeof(uint32_t)); - _crc = crc32_be(static_cast(_crc), reinterpret_cast(&value_l), sizeof(uint32_t)); - return _crc; + v = std::byteswap(v); + return crc32c_le_vx(static_cast(crc), reinterpret_cast(&v), sizeof(uint64_t)); } #endif From 6086787957ce27de5510c964eff8f4c24a6bf088 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 11 Jul 2023 16:59:16 -0400 Subject: [PATCH 0523/2047] add overview --- .../settings.md | 4 +- docs/en/operations/settings/index.md | 175 +--------------- .../settings/settings-query-level.md | 188 ++++++++++++++++++ 3 files changed, 198 insertions(+), 169 deletions(-) create mode 100644 docs/en/operations/settings/settings-query-level.md diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index bad7e388377..8bb7c5cb037 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1,11 +1,11 @@ --- slug: /en/operations/server-configuration-parameters/settings sidebar_position: 57 -sidebar_label: Server Settings +sidebar_label: Global Server Settings description: This section contains descriptions of server settings that cannot be changed at the session or query level. --- -# Server Settings +# Global Server Settings This section contains descriptions of server settings that cannot be changed at the session or query level. diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index 022b50925c4..6863d7f3191 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -7,175 +7,16 @@ pagination_next: en/operations/settings/settings # Settings Overview -There are multiple ways to define ClickHouse settings. Settings are configured in layers, and each subsequent layer redefines the previous values of a setting. +There are two main groups of ClickHouse settings: -The order of priority for defining a setting is: +- Global server settings +- Query-level settings -1. Settings in user profiles. Set user profiles either by: - - SQL (recommended) - - adding one or more XML or YAML files to `/etc/clickhouse-server/users.d` +The main distinction between global server settings and query-level settings is that +global server settings must be set in configuration files while query-level settings +can be set in configuration files or with SQL queries. -2. Session settings +Read about [global server settings](/docs/en/operations/server-configuration-parameters/settings.md) to learn more about configuring your ClickHouse server at the global server level. - - Send `SET setting=value` from the ClickHouse console client in interactive mode. - Similarly, you can use ClickHouse sessions in the HTTP protocol. To do this, you need to specify the `session_id` HTTP parameter. +Read about [query-level settings](/docs/en/operations/settings/settings-query-level.md) to learn more about configuring your ClickHouse server at the query-level. -3. Query settings - - - When starting the ClickHouse console client in non-interactive mode, set the startup parameter `--setting=value`. - - When using the HTTP API, pass CGI parameters (`URL?setting_1=value&setting_2=value...`). - - Define settings in the [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select-query) clause of the SELECT query. The setting value is applied only to that query and is reset to the default or previous value after the query is executed. - -View the [Settings](./settings.md) page for a description of the ClickHouse settings. - -## Examples - -These examples all set the value of the `async_insert` setting to `1`, and show how to examine -the settings in a running system. - -### Using SQL to create a user profile - -```sql -CREATE -SETTINGS PROFILE log_ingest SETTINGS async_insert = 1 -``` - -```sql -CREATE USER ingester -IDENTIFIED WITH sha256_hash BY '7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3' -# highlight-next-line -SETTINGS PROFILE log_ingest -``` - -#### Examine the user profile and assignment - -```sql -SHOW ACCESS -``` - -```response -┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ -│ CREATE USER default IDENTIFIED WITH sha256_password │ -# highlight-next-line -│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ -│ CREATE SETTINGS PROFILE default │ -# highlight-next-line -│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ -│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ -│ ... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -``` - -### Using XML to create a user profile - -```xml title=/etc/clickhouse-server/users.d/users.xml - -# highlight-start - - - 1 - - -# highlight-end - - - - 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 -# highlight-start - log_ingest -# highlight-end - - - 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 - 1 - 1 - - - -``` - -#### Examine the user profile and assignment - -```sql -SHOW ACCESS -``` - -```response -┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ -│ CREATE USER default IDENTIFIED WITH sha256_password │ -# highlight-next-line -│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ -│ CREATE SETTINGS PROFILE default │ -# highlight-next-line -│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ -│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ -│ ... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -``` - -## Converting a Setting to its Default Value - -If you change a setting and would like to revert it back to its default value, set the value to `DEFAULT`. The syntax looks like: - -```sql -SET setting_name = DEFAULT -``` - -For example, the default value of `max_insert_block_size` is 1048449. Suppose you change its value to 100000: - -```sql -SET max_insert_block_size=100000; - -SELECT value FROM system.settings where name='max_insert_block_size'; -``` - -The response is: - -```response -┌─value──┐ -│ 100000 │ -└────────┘ -``` - -The following command sets its value back to 1048449: - -```sql -SET max_insert_block_size=DEFAULT; - -SELECT value FROM system.settings where name='max_insert_block_size'; -``` - -The setting is now back to its default: - -```response -┌─value───┐ -│ 1048449 │ -└─────────┘ -``` - - -## Custom Settings {#custom_settings} - -In addition to the common [settings](../../operations/settings/settings.md), users can define custom settings. - -A custom setting name must begin with one of predefined prefixes. The list of these prefixes must be declared in the [custom_settings_prefixes](../../operations/server-configuration-parameters/settings.md#custom_settings_prefixes) parameter in the server configuration file. - -```xml -custom_ -``` - -To define a custom setting use `SET` command: - -```sql -SET custom_a = 123; -``` - -To get the current value of a custom setting use `getSetting()` function: - -```sql -SELECT getSetting('custom_a'); -``` - -**See Also** - -- [Server Configuration Settings](../../operations/server-configuration-parameters/settings.md) diff --git a/docs/en/operations/settings/settings-query-level.md b/docs/en/operations/settings/settings-query-level.md new file mode 100644 index 00000000000..99b18e5e49f --- /dev/null +++ b/docs/en/operations/settings/settings-query-level.md @@ -0,0 +1,188 @@ +--- +sidebar_label: Query-level Settings +title: Query-level Settings +slug: /en/operations/settings/query-level +--- + +There are multiple ways to set ClickHouse query-level settings. Settings are configured in layers, and each subsequent layer redefines the previous values of a setting. + +The order of priority for defining a setting is: + +1. Settings in user profiles. Set user profiles either by: + + - SQL (recommended) + - adding one or more XML or YAML files to `/etc/clickhouse-server/users.d` + +2. Session settings + + - Send `SET setting=value` from the ClickHouse Cloud SQL console or + `clickhouse client` in interactive mode. Similarly, you can use ClickHouse + sessions in the HTTP protocol. To do this, you need to specify the + `session_id` HTTP parameter. + +3. Query settings + + - When starting `clickhouse client` in non-interactive mode, set the startup + parameter `--setting=value`. + - When using the HTTP API, pass CGI parameters (`URL?setting_1=value&setting_2=value...`). + - Define settings in the + [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select-query) + clause of the SELECT query. The setting value is applied only to that query + and is reset to the default or previous value after the query is executed. + +## Examples + +These examples all set the value of the `async_insert` setting to `1`, and +show how to examine the settings in a running system. + +### Using SQL to create a settings profile + +This creates the profile `log_ingest` with the setting `async_inset = 1`: + +```sql +CREATE +SETTINGS PROFILE log_ingest SETTINGS async_insert = 1 +``` + +This creates the user `ingester` and assigns the user the settings profile `log_ingest`: + +```sql +CREATE USER ingester +IDENTIFIED WITH sha256_hash BY '7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3' +# highlight-next-line +SETTINGS PROFILE log_ingest +``` + +#### Examine the settings profile and assignment + +```sql +SHOW ACCESS +``` + +```response +┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ +│ CREATE USER default IDENTIFIED WITH sha256_password │ +# highlight-next-line +│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ +│ CREATE SETTINGS PROFILE default │ +# highlight-next-line +│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ +│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ +│ ... │ +└────────────────────────────────────────────────────────────────────────────────────┘ +``` + +### Using XML to create a settings profile + +```xml title=/etc/clickhouse-server/users.d/users.xml + +# highlight-start + + + 1 + + +# highlight-end + + + + 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 +# highlight-start + log_ingest +# highlight-end + + + 7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3 + 1 + 1 + + + +``` + +#### Examine the settings profile and assignment + +```sql +SHOW ACCESS +``` + +```response +┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ +│ CREATE USER default IDENTIFIED WITH sha256_password │ +# highlight-next-line +│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ +│ CREATE SETTINGS PROFILE default │ +# highlight-next-line +│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ +│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ +│ ... │ +└────────────────────────────────────────────────────────────────────────────────────┘ +``` + +## Converting a Setting to its Default Value + +If you change a setting and would like to revert it back to its default value, set the value to `DEFAULT`. The syntax looks like: + +```sql +SET setting_name = DEFAULT +``` + +For example, the default value of `max_insert_block_size` is 1048449. Suppose you change its value to 100000: + +```sql +SET max_insert_block_size=100000; + +SELECT value FROM system.settings where name='max_insert_block_size'; +``` + +The response is: + +```response +┌─value──┐ +│ 100000 │ +└────────┘ +``` + +The following command sets its value back to 1048449: + +```sql +SET max_insert_block_size=DEFAULT; + +SELECT value FROM system.settings where name='max_insert_block_size'; +``` + +The setting is now back to its default: + +```response +┌─value───┐ +│ 1048449 │ +└─────────┘ +``` + + +## Custom Settings {#custom_settings} + +In addition to the common [settings](../../operations/settings/settings.md), users can define custom settings. + +A custom setting name must begin with one of predefined prefixes. The list of these prefixes must be declared in the [custom_settings_prefixes](../../operations/server-configuration-parameters/settings.md#custom_settings_prefixes) parameter in the server configuration file. + +```xml +custom_ +``` + +To define a custom setting use `SET` command: + +```sql +SET custom_a = 123; +``` + +To get the current value of a custom setting use `getSetting()` function: + +```sql +SELECT getSetting('custom_a'); +``` + +**See Also** + +- View the [Settings](./settings.md) page for a description of the ClickHouse settings. +- [Global server settings](../../operations/server-configuration-parameters/settings.md) From 46fd88dca4bbcf6b83843780c4579de4212e0222 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 11 Jul 2023 17:50:52 -0400 Subject: [PATCH 0524/2047] add session setting --- .../settings/settings-query-level.md | 89 ++++++++++++------- 1 file changed, 59 insertions(+), 30 deletions(-) diff --git a/docs/en/operations/settings/settings-query-level.md b/docs/en/operations/settings/settings-query-level.md index 99b18e5e49f..81cc2294a4c 100644 --- a/docs/en/operations/settings/settings-query-level.md +++ b/docs/en/operations/settings/settings-query-level.md @@ -8,7 +8,7 @@ There are multiple ways to set ClickHouse query-level settings. Settings are con The order of priority for defining a setting is: -1. Settings in user profiles. Set user profiles either by: +1. Applying a setting to a user directly, or within a settings profile - SQL (recommended) - adding one or more XML or YAML files to `/etc/clickhouse-server/users.d` @@ -35,7 +35,32 @@ The order of priority for defining a setting is: These examples all set the value of the `async_insert` setting to `1`, and show how to examine the settings in a running system. -### Using SQL to create a settings profile +### Using SQL to apply a setting to a user directly + +This creates the user `ingester` with the setting `async_inset = 1`: + +```sql +CREATE USER ingester +IDENTIFIED WITH sha256_hash BY '7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f281276aae20f86dc3' +# highlight-next-line +SETTINGS async_insert = 1 +``` + +#### Examine the settings profile and assignment + +```sql +SHOW ACCESS +``` + +```response +┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ +│ ... │ +# highlight-next-line +│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS async_insert = true │ +│ ... │ +└────────────────────────────────────────────────────────────────────────────────────┘ +``` +### Using SQL to create a settings profile and assign to a user This creates the profile `log_ingest` with the setting `async_inset = 1`: @@ -53,26 +78,8 @@ IDENTIFIED WITH sha256_hash BY '7e099f39b84ea79559b3e85ea046804e63725fd1f46b37f2 SETTINGS PROFILE log_ingest ``` -#### Examine the settings profile and assignment -```sql -SHOW ACCESS -``` - -```response -┌─ACCESS─────────────────────────────────────────────────────────────────────────────┐ -│ CREATE USER default IDENTIFIED WITH sha256_password │ -# highlight-next-line -│ CREATE USER ingester IDENTIFIED WITH sha256_password SETTINGS PROFILE log_ingest │ -│ CREATE SETTINGS PROFILE default │ -# highlight-next-line -│ CREATE SETTINGS PROFILE log_ingest SETTINGS async_insert = true │ -│ CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1 │ -│ ... │ -└────────────────────────────────────────────────────────────────────────────────────┘ -``` - -### Using XML to create a settings profile +### Using XML to create a settings profile and user ```xml title=/etc/clickhouse-server/users.d/users.xml @@ -119,6 +126,29 @@ SHOW ACCESS └────────────────────────────────────────────────────────────────────────────────────┘ ``` +### Assign a setting to a session + +```sql +SET async_insert =1; +SELECT value FROM system.settings where name='async_insert'; +``` + +```response +┌─value──┐ +│ 1 │ +└────────┘ +``` + +### Assign a setting during a query + +```sql +INSERT INTO YourTable +# highlight-next-line +SETTINGS async_insert=1 +VALUES (...) +``` + + ## Converting a Setting to its Default Value If you change a setting and would like to revert it back to its default value, set the value to `DEFAULT`. The syntax looks like: @@ -127,39 +157,38 @@ If you change a setting and would like to revert it back to its default value, s SET setting_name = DEFAULT ``` -For example, the default value of `max_insert_block_size` is 1048449. Suppose you change its value to 100000: +For example, the default value of `async_insert` is `0`. Suppose you change its value to `1`: ```sql -SET max_insert_block_size=100000; +SET async_insert = 1; -SELECT value FROM system.settings where name='max_insert_block_size'; +SELECT value FROM system.settings where name='async_insert'; ``` The response is: ```response ┌─value──┐ -│ 100000 │ +│ 1 │ └────────┘ ``` -The following command sets its value back to 1048449: +The following command sets its value back to 0: ```sql -SET max_insert_block_size=DEFAULT; +SET async_insert = DEFAULT; -SELECT value FROM system.settings where name='max_insert_block_size'; +SELECT value FROM system.settings where name='async_insert'; ``` The setting is now back to its default: ```response ┌─value───┐ -│ 1048449 │ +│ 0 │ └─────────┘ ``` - ## Custom Settings {#custom_settings} In addition to the common [settings](../../operations/settings/settings.md), users can define custom settings. From 1a4d48324e3ffe1f8c0c05a337b590ce8aa9ff33 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 11 Jul 2023 19:58:28 -0400 Subject: [PATCH 0525/2047] Fix fast tests --- .../queries/0_stateless/01293_create_role.reference | 2 +- .../01294_create_settings_profile.reference | 12 ++++++------ .../0_stateless/01295_create_row_policy.reference | 6 +++--- .../queries/0_stateless/01297_create_quota.reference | 8 ++++---- .../02117_show_create_table_system.reference | 1 + 5 files changed, 15 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/01293_create_role.reference b/tests/queries/0_stateless/01293_create_role.reference index 9b3c4eabd47..42f091bddd5 100644 --- a/tests/queries/0_stateless/01293_create_role.reference +++ b/tests/queries/0_stateless/01293_create_role.reference @@ -28,7 +28,7 @@ CREATE ROLE r2_01293 CREATE ROLE r1_01293 SETTINGS readonly = 1 CREATE ROLE r2_01293 SETTINGS readonly = 1 -- system.roles -r1_01293 local directory +r1_01293 local_directory -- system.settings_profile_elements \N \N r1_01293 0 readonly 1 \N \N \N \N \N \N r2_01293 0 \N \N \N \N \N default diff --git a/tests/queries/0_stateless/01294_create_settings_profile.reference b/tests/queries/0_stateless/01294_create_settings_profile.reference index dbb73bca851..a10d5758752 100644 --- a/tests/queries/0_stateless/01294_create_settings_profile.reference +++ b/tests/queries/0_stateless/01294_create_settings_profile.reference @@ -53,12 +53,12 @@ CREATE SETTINGS PROFILE s4_01294 SETTINGS INHERIT readonly, INHERIT readonly CREATE SETTINGS PROFILE s5_01294 SETTINGS INHERIT readonly, readonly = 1 CREATE SETTINGS PROFILE s6_01294 SETTINGS INHERIT readonly, readonly CONST -- system.settings_profiles -s1_01294 local directory 0 0 [] [] -s2_01294 local directory 1 0 ['r1_01294'] [] -s3_01294 local directory 1 0 ['r1_01294'] [] -s4_01294 local directory 1 0 ['r1_01294'] [] -s5_01294 local directory 3 0 ['u1_01294'] [] -s6_01294 local directory 0 1 [] ['r1_01294','u1_01294'] +s1_01294 local_directory 0 0 [] [] +s2_01294 local_directory 1 0 ['r1_01294'] [] +s3_01294 local_directory 1 0 ['r1_01294'] [] +s4_01294 local_directory 1 0 ['r1_01294'] [] +s5_01294 local_directory 3 0 ['u1_01294'] [] +s6_01294 local_directory 0 1 [] ['r1_01294','u1_01294'] -- system.settings_profile_elements s2_01294 \N \N 0 readonly 0 \N \N \N \N s3_01294 \N \N 0 max_memory_usage 5000000 4000000 6000000 CONST \N diff --git a/tests/queries/0_stateless/01295_create_row_policy.reference b/tests/queries/0_stateless/01295_create_row_policy.reference index 6e3169b7fec..d73d9752bc1 100644 --- a/tests/queries/0_stateless/01295_create_row_policy.reference +++ b/tests/queries/0_stateless/01295_create_row_policy.reference @@ -30,6 +30,6 @@ CREATE ROW POLICY p5_01295 ON db2.table2 FOR SELECT USING a = b CREATE ROW POLICY p1_01295 ON db.table FOR SELECT USING 1 TO ALL CREATE ROW POLICY p2_01295 ON db.table FOR SELECT USING 1 TO ALL -- system.row_policies -p1_01295 ON db.table p1_01295 db table local directory (a < b) AND (c > d) 0 0 [] [] -p2_01295 ON db.table p2_01295 db table local directory id = currentUser() 1 0 ['u1_01295'] [] -p3_01295 ON db.table p3_01295 db table local directory 1 0 1 [] ['r1_01295'] +p1_01295 ON db.table p1_01295 db table local_directory (a < b) AND (c > d) 0 0 [] [] +p2_01295 ON db.table p2_01295 db table local_directory id = currentUser() 1 0 ['u1_01295'] [] +p3_01295 ON db.table p3_01295 db table local_directory 1 0 1 [] ['r1_01295'] diff --git a/tests/queries/0_stateless/01297_create_quota.reference b/tests/queries/0_stateless/01297_create_quota.reference index 6b844e0d2f6..308bbf79024 100644 --- a/tests/queries/0_stateless/01297_create_quota.reference +++ b/tests/queries/0_stateless/01297_create_quota.reference @@ -52,10 +52,10 @@ CREATE QUOTA q2_01297 FOR INTERVAL 1 day MAX errors = 5 CREATE QUOTA q1_01297 FOR INTERVAL 1 day TRACKING ONLY TO r1_01297 CREATE QUOTA q2_01297 FOR INTERVAL 1 day TRACKING ONLY TO r1_01297 -- system.quotas -q1_01297 local directory ['user_name'] [] 0 ['r1_01297'] [] -q2_01297 local directory [] [5259492] 0 ['r1_01297','u1_01297'] [] -q3_01297 local directory ['client_key','user_name'] [5259492,15778476] 0 [] [] -q4_01297 local directory [] [604800] 1 [] ['u1_01297'] +q1_01297 local_directory ['user_name'] [] 0 ['r1_01297'] [] +q2_01297 local_directory [] [5259492] 0 ['r1_01297','u1_01297'] [] +q3_01297 local_directory ['client_key','user_name'] [5259492,15778476] 0 [] [] +q4_01297 local_directory [] [604800] 1 [] ['u1_01297'] -- system.quota_limits q2_01297 5259492 0 100 \N \N 11 1000 10000 1001 10001 2.5 \N q3_01297 5259492 0 \N \N \N \N 1002 \N \N \N \N \N diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 3834b05601f..aaff6d92d32 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -955,6 +955,7 @@ CREATE TABLE system.role_grants `user_name` Nullable(String), `role_name` Nullable(String), `granted_role_name` String, + `granted_role_id` UUID, `granted_role_is_default` UInt8, `with_admin_option` UInt8 ) From 52246ab5b544a3fa5c9482871f90e5a2517c2710 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 11 Jul 2023 23:23:15 +0800 Subject: [PATCH 0526/2047] Ignore setting constraints for projection ASTs --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/InterpreterSetQuery.cpp | 5 +- src/Interpreters/InterpreterSetQuery.h | 2 +- src/Interpreters/SelectQueryOptions.h | 8 +++ .../optimizeUseAggregateProjection.cpp | 2 +- src/Storages/ProjectionsDescription.cpp | 50 ++++++++++++------- 6 files changed, 47 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6ea15312ec4..32812151b59 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3181,7 +3181,7 @@ void InterpreterSelectQuery::initSettings() { auto & query = getSelectQuery(); if (query.settings()) - InterpreterSetQuery(query.settings(), context).executeForCurrentContext(); + InterpreterSetQuery(query.settings(), context).executeForCurrentContext(options.ignore_setting_constraints); auto & client_info = context->getClientInfo(); auto min_major = DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index c25de7c55ea..6db57a4f950 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -24,10 +24,11 @@ BlockIO InterpreterSetQuery::execute() } -void InterpreterSetQuery::executeForCurrentContext() +void InterpreterSetQuery::executeForCurrentContext(bool ignore_setting_constraints) { const auto & ast = query_ptr->as(); - getContext()->checkSettingsConstraints(ast.changes); + if (!ignore_setting_constraints) + getContext()->checkSettingsConstraints(ast.changes); getContext()->applySettingsChanges(ast.changes); getContext()->resetSettingsToDefaultValue(ast.default_settings); } diff --git a/src/Interpreters/InterpreterSetQuery.h b/src/Interpreters/InterpreterSetQuery.h index bcd4022f9bb..2438762f347 100644 --- a/src/Interpreters/InterpreterSetQuery.h +++ b/src/Interpreters/InterpreterSetQuery.h @@ -23,7 +23,7 @@ public: /** Set setting for current context (query context). * It is used for interpretation of SETTINGS clause in SELECT query. */ - void executeForCurrentContext(); + void executeForCurrentContext(bool ignore_setting_constraints = false); bool supportsTransactions() const override { return true; } diff --git a/src/Interpreters/SelectQueryOptions.h b/src/Interpreters/SelectQueryOptions.h index e6895ed243b..c91329c869c 100644 --- a/src/Interpreters/SelectQueryOptions.h +++ b/src/Interpreters/SelectQueryOptions.h @@ -51,6 +51,8 @@ struct SelectQueryOptions bool settings_limit_offset_done = false; bool is_explain = false; /// The value is true if it's explain statement. bool is_create_parameterized_view = false; + /// Bypass setting constraints for some internal queries such as projection ASTs. + bool ignore_setting_constraints = false; /// These two fields are used to evaluate shardNum() and shardCount() function when /// prefer_localhost_replica == 1 and local instance is selected. They are needed because local @@ -141,6 +143,12 @@ struct SelectQueryOptions return *this; } + SelectQueryOptions & ignoreSettingConstraints(bool value = true) + { + ignore_setting_constraints = value; + return *this; + } + SelectQueryOptions & setInternal(bool value = false) { is_internal = value; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 2959178b2e5..6e76d6e67a7 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -69,7 +69,7 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( projection.query_ast, context, Pipe(std::make_shared(metadata_snapshot->getSampleBlock())), - SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreASTOptimizations()); + SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreASTOptimizations().ignoreSettingConstraints()); const auto & analysis_result = interpreter.getAnalysisResult(); const auto & query_analyzer = interpreter.getQueryAnalyzer(); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index e568fba0495..73fb279d51c 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -7,18 +7,18 @@ #include #include #include +#include #include #include -#include #include #include -#include -#include -#include -#include #include #include +#include +#include +#include +#include #include @@ -109,9 +109,16 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const auto external_storage_holder = std::make_shared(query_context, columns, ConstraintsDescription{}); StoragePtr storage = external_storage_holder->getTable(); InterpreterSelectQuery select( - result.query_ast, query_context, storage, {}, + result.query_ast, + query_context, + storage, + {}, /// Here we ignore ast optimizations because otherwise aggregation keys may be removed from result header as constants. - SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizations()); + SelectQueryOptions{QueryProcessingStage::WithMergeableState} + .modify() + .ignoreAlias() + .ignoreASTOptimizations() + .ignoreSettingConstraints()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); @@ -220,9 +227,16 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( auto external_storage_holder = std::make_shared(query_context, columns, ConstraintsDescription{}); StoragePtr storage = external_storage_holder->getTable(); InterpreterSelectQuery select( - result.query_ast, query_context, storage, {}, + result.query_ast, + query_context, + storage, + {}, /// Here we ignore ast optimizations because otherwise aggregation keys may be removed from result header as constants. - SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizations()); + SelectQueryOptions{QueryProcessingStage::WithMergeableState} + .modify() + .ignoreAlias() + .ignoreASTOptimizations() + .ignoreSettingConstraints()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); @@ -241,7 +255,8 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( result.sample_block_for_keys.insert({nullptr, key.type, key.name}); auto it = partition_column_name_to_value_index.find(key.name); if (it == partition_column_name_to_value_index.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "minmax_count projection can only have keys about partition columns. It's a bug"); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "minmax_count projection can only have keys about partition columns. It's a bug"); result.partition_value_indices.push_back(it->second); } } @@ -282,7 +297,8 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) Pipe(std::make_shared(block)), SelectQueryOptions{ type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns - : QueryProcessingStage::WithMergeableState}) + : QueryProcessingStage::WithMergeableState} + .ignoreSettingConstraints()) .buildQueryPipeline(); builder.resize(1); // Generate aggregated blocks with rows less or equal than the original block. @@ -353,8 +369,8 @@ void ProjectionsDescription::add(ProjectionDescription && projection, const Stri { if (if_not_exists) return; - throw Exception(ErrorCodes::ILLEGAL_PROJECTION, "Cannot add projection {}: projection with this name already exists", - projection.name); + throw Exception( + ErrorCodes::ILLEGAL_PROJECTION, "Cannot add projection {}: projection with this name already exists", projection.name); } auto insert_it = projections.cend(); @@ -363,10 +379,10 @@ void ProjectionsDescription::add(ProjectionDescription && projection, const Stri insert_it = projections.cbegin(); else if (!after_projection.empty()) { - auto it = std::find_if(projections.cbegin(), projections.cend(), [&after_projection](const auto & projection_) - { - return projection_.name == after_projection; - }); + auto it = std::find_if( + projections.cbegin(), + projections.cend(), + [&after_projection](const auto & projection_) { return projection_.name == after_projection; }); if (it != projections.cend()) ++it; insert_it = it; From c9a754dc4b916bbde1d5d11e84fcafac4ed787cb Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 12 Jul 2023 08:47:37 +0300 Subject: [PATCH 0527/2047] Add more tests --- tests/queries/0_stateless/02809_has_subsequence.reference | 2 ++ tests/queries/0_stateless/02809_has_subsequence.sql | 4 +++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02809_has_subsequence.reference b/tests/queries/0_stateless/02809_has_subsequence.reference index 0bf8e4e3a36..5f533d7f5bb 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.reference +++ b/tests/queries/0_stateless/02809_has_subsequence.reference @@ -10,6 +10,8 @@ hasSubsequence 1 1 0 +1 +0 0 0 1 diff --git a/tests/queries/0_stateless/02809_has_subsequence.sql b/tests/queries/0_stateless/02809_has_subsequence.sql index 6715d901309..dea05369a0e 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.sql +++ b/tests/queries/0_stateless/02809_has_subsequence.sql @@ -8,6 +8,8 @@ select hasSubsequence('garbage', 'gr'); select hasSubsequence('garbage', 'ab'); select hasSubsequence('garbage', 'be'); select hasSubsequence('garbage', 'arg'); +select hasSubsequence('garbage', 'gra'); +select hasSubsequence('garbage', 'rga');s select hasSubsequence('garbage', 'garbage'); select hasSubsequence('garbage', 'garbage1'); select hasSubsequence('garbage', 'arbw'); @@ -55,4 +57,4 @@ select hasSubsequenceCaseInsensitiveUTF8(materialize('для онлайн обр select hasSubsequenceCaseInsensitiveUTF8('для онлайн обработки аналитических запросов (OLAP)', materialize('добро)')); select hasSubsequenceCaseInsensitiveUTF8('для онлайн обработки аналитических запросов (OLAP)', materialize('зло()')); select hasSubsequenceCaseInsensitiveUTF8(materialize('для онлайн обработки аналитических запросов (OLAP)'), materialize('аналитика')); -select hasSubsequenceCaseInsensitiveUTF8(materialize('для онлайн обработки аналитических запросов (OLAP)'), materialize('аналитика для аналитиков')); \ No newline at end of file +select hasSubsequenceCaseInsensitiveUTF8(materialize('для онлайн обработки аналитических запросов (OLAP)'), materialize('аналитика для аналитиков')); From 2eef451f77616ada1974568ef658724636fe382b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 12 Jul 2023 08:48:58 +0300 Subject: [PATCH 0528/2047] Remove trash --- tests/queries/0_stateless/02809_has_subsequence.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02809_has_subsequence.sql b/tests/queries/0_stateless/02809_has_subsequence.sql index dea05369a0e..bcc491a95fe 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.sql +++ b/tests/queries/0_stateless/02809_has_subsequence.sql @@ -9,7 +9,7 @@ select hasSubsequence('garbage', 'ab'); select hasSubsequence('garbage', 'be'); select hasSubsequence('garbage', 'arg'); select hasSubsequence('garbage', 'gra'); -select hasSubsequence('garbage', 'rga');s +select hasSubsequence('garbage', 'rga'); select hasSubsequence('garbage', 'garbage'); select hasSubsequence('garbage', 'garbage1'); select hasSubsequence('garbage', 'arbw'); From 8133a5960ef10df111ca6b431a896514e6137885 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 12 Jul 2023 08:51:24 +0000 Subject: [PATCH 0529/2047] Update tests/analyzer_integration_broken_tests.txt --- tests/analyzer_integration_broken_tests.txt | 62 ++++++++++++++++++++- 1 file changed, 61 insertions(+), 1 deletion(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 3aa3b0dff2a..46b9ade43de 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -135,4 +135,64 @@ test_quota/test.py::test_reload_users_xml_by_timer test_quota/test.py::test_simpliest_quota test_quota/test.py::test_tracking_quota test_quota/test.py::test_users_xml_is_readonly -test_replicated_merge_tree_compatibility/test.py::test_replicated_merge_tree_defaults_compatibili \ No newline at end of file +test_replicated_merge_tree_compatibility/test.py::test_replicated_merge_tree_defaults_compatibility +test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_wide-Wide] +test_old_versions/test.py::test_client_is_older_than_server +test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive +test_old_versions/test.py::test_server_is_older_than_client +test_polymorphic_parts/test.py::test_compact_parts_only +test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_compact-Compact] +test_polymorphic_parts/test.py::test_polymorphic_parts_index +test_old_versions/test.py::test_distributed_query_initiator_is_older_than_shard +test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1] +test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0] +test_ttl_replicated/test.py::test_ttl_table[DELETE] +test_ttl_replicated/test.py::test_ttl_columns +test_ttl_replicated/test.py::test_ttl_compatibility[node_left2-node_right2-2] +test_ttl_replicated/test.py::test_ttl_table[] +test_version_update/test.py::test_aggregate_function_versioning_server_upgrade +test_version_update/test.py::test_aggregate_function_versioning_fetch_data_from_old_to_new_server +test_ttl_replicated/test.py::test_ttl_double_delete_rule_returns_error +test_ttl_replicated/test.py::test_ttl_alter_delete[test_ttl_alter_delete] +test_ttl_replicated/test.py::test_ttl_alter_delete[test_ttl_alter_delete_replicated] +test_ttl_replicated/test.py::test_ttl_compatibility[node_left0-node_right0-0] +test_version_update/test.py::test_modulo_partition_key_issue_23508 +test_ttl_replicated/test.py::test_ttl_many_columns +test_ttl_replicated/test.py::test_modify_column_ttl +test_ttl_replicated/test.py::test_merge_with_ttl_timeout +test_ttl_replicated/test.py::test_ttl_empty_parts +test_ttl_replicated/test.py::test_ttl_compatibility[node_left1-node_right1-1] +test_version_update/test.py::test_aggregate_function_versioning_persisting_metadata +test_version_update/test.py::test_aggregate_function_versioning_issue_16587 +test_ttl_replicated/test.py::test_modify_ttl +test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database +test_profile_events_s3/test.py::test_profile_events +test_version_update_after_mutation/test.py::test_upgrade_while_mutation +test_version_update_after_mutation/test.py::test_mutate_and_upgrade +test_system_flush_logs/test.py::test_system_logs[system.text_log-0] +test_user_defined_object_persistence/test.py::test_persistence +test_settings_profile/test.py::test_show_profiles +test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster +test_select_access_rights/test_main.py::test_alias_columns +test_select_access_rights/test_main.py::test_select_count +test_select_access_rights/test_main.py::test_select_join +test_replicated_merge_tree_compatibility/test.py::test_replicated_merge_tree_defaults_compatibility +test_postgresql_protocol/test.py::test_python_client +test_quota/test.py::test_add_remove_interval +test_quota/test.py::test_add_remove_quota +test_quota/test.py::test_consumption_of_show_clusters +test_quota/test.py::test_consumption_of_show_databases +test_quota/test.py::test_consumption_of_show_privileges +test_quota/test.py::test_consumption_of_show_processlist +test_quota/test.py::test_consumption_of_show_tables +test_quota/test.py::test_dcl_introspection +test_quota/test.py::test_dcl_management +test_quota/test.py::test_exceed_quota +test_quota/test.py::test_query_inserts +test_quota/test.py::test_quota_from_users_xml +test_quota/test.py::test_reload_users_xml_by_timer +test_quota/test.py::test_simpliest_quota +test_quota/test.py::test_tracking_quota +test_quota/test.py::test_users_xml_is_readonly +test_replicating_constants/test.py::test_different_versions +test_merge_tree_s3/test.py::test_heavy_insert_select_check_memory[node] From b4723ca8dffd29f16695982f3344e3a154288ac2 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 12 Jul 2023 09:16:13 +0000 Subject: [PATCH 0530/2047] Try to fix report --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 6a6134d7204..c7466f00ac9 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -983,7 +983,7 @@ class ClickhouseIntegrationTestsRunner: result_state = "success" test_result = [] - for state in ("ERROR", "FAILED", "PASSED", "SKIPPED", "FLAKY"): + for state in ("ERROR", "FAILED", "PASSED", "SKIPPED", "FLAKY", "BROKEN", "NOT_FAILED"): if state == "PASSED": text_state = "OK" elif state == "FAILED": From 881a45ec83315de7d7d78f4683e765cfa53f9ae7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 12 Jul 2023 09:26:01 +0000 Subject: [PATCH 0531/2047] Automatic style fix --- tests/integration/ci-runner.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index c7466f00ac9..00bca60e7db 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -983,7 +983,15 @@ class ClickhouseIntegrationTestsRunner: result_state = "success" test_result = [] - for state in ("ERROR", "FAILED", "PASSED", "SKIPPED", "FLAKY", "BROKEN", "NOT_FAILED"): + for state in ( + "ERROR", + "FAILED", + "PASSED", + "SKIPPED", + "FLAKY", + "BROKEN", + "NOT_FAILED", + ): if state == "PASSED": text_state = "OK" elif state == "FAILED": From 3bd9c57a8cfc4210e421690b597afa7ba68a36d0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Jul 2023 09:29:36 +0000 Subject: [PATCH 0532/2047] allow to ignore CREATE INDEX query for better SQL syntax compatibility --- src/Core/Settings.h | 1 + .../InterpreterCreateIndexQuery.cpp | 17 ++++++++ src/Parsers/ASTIndexDeclaration.cpp | 41 +++++++++++-------- src/Parsers/ParserCreateIndexQuery.cpp | 29 +++++++++---- 4 files changed, 64 insertions(+), 24 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3250bdf0750..8f61c3973e3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -774,6 +774,7 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone if empty.", 0) \ + M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0)\ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp index 714bcd6d356..0e36fa22404 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.cpp +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -15,6 +15,8 @@ namespace DB namespace ErrorCodes { extern const int TABLE_IS_READ_ONLY; + extern const int LOGICAL_ERROR; + extern const int INCORRECT_QUERY; } @@ -23,6 +25,21 @@ BlockIO InterpreterCreateIndexQuery::execute() auto current_context = getContext(); const auto & create_index = query_ptr->as(); + // Noop if allow_create_index_without_type = true. throw otherwise + if (!create_index.index_decl->as()->type) + { + if (!current_context->getSettingsRef().allow_create_index_without_type) + { + throw Exception(ErrorCodes::INCORRECT_QUERY, "CREATE INDEX without TYPE is forbidden." + " SET allow_create_index_without_type=1 to ignore this statements."); + } + else + { + // Nothing to do + return {}; + } + } + AccessRightsElements required_access; required_access.emplace_back(AccessType::ALTER_ADD_INDEX, create_index.getDatabase(), create_index.getTable()); diff --git a/src/Parsers/ASTIndexDeclaration.cpp b/src/Parsers/ASTIndexDeclaration.cpp index d223661451e..e647f02f553 100644 --- a/src/Parsers/ASTIndexDeclaration.cpp +++ b/src/Parsers/ASTIndexDeclaration.cpp @@ -13,8 +13,8 @@ ASTPtr ASTIndexDeclaration::clone() const auto res = std::make_shared(); res->name = name; - res->granularity = granularity; - + if (granularity) + res->granularity = granularity; if (expr) res->set(res->expr, expr->clone()); if (type) @@ -25,23 +25,32 @@ ASTPtr ASTIndexDeclaration::clone() const void ASTIndexDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const { - if (part_of_create_index_query) + if (expr) { - s.ostr << "("; - expr->formatImpl(s, state, frame); - s.ostr << ")"; - } - else - { - s.ostr << backQuoteIfNeed(name); - s.ostr << " "; - expr->formatImpl(s, state, frame); + if (part_of_create_index_query) + { + s.ostr << "("; + expr->formatImpl(s, state, frame); + s.ostr << ")"; + } + else + { + s.ostr << backQuoteIfNeed(name); + s.ostr << " "; + expr->formatImpl(s, state, frame); + } } - s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : ""); - type->formatImpl(s, state, frame); - s.ostr << (s.hilite ? hilite_keyword : "") << " GRANULARITY " << (s.hilite ? hilite_none : ""); - s.ostr << granularity; + if (type) + { + s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : ""); + type->formatImpl(s, state, frame); + } + if (granularity) + { + s.ostr << (s.hilite ? hilite_keyword : "") << " GRANULARITY " << (s.hilite ? hilite_none : ""); + s.ostr << granularity; + } } } diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index f231573b920..5bcea50af87 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -17,24 +17,36 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected { ParserKeyword s_type("TYPE"); ParserKeyword s_granularity("GRANULARITY"); - + ParserToken open(TokenType::OpeningRoundBracket); + ParserToken close(TokenType::ClosingRoundBracket); + ParserOrderByExpressionList order_list; ParserDataType data_type_p; ParserExpression expression_p; ParserUnsignedInteger granularity_p; ASTPtr expr; + ASTPtr order; ASTPtr type; ASTPtr granularity; + if (open.ignore(pos, expected)) + { + if (!order_list.parse(pos, order, expected)) + return false; + + if (!close.ignore(pos, expected)) + return false; + } /// Skip name parser for SQL-standard CREATE INDEX - if (!expression_p.parse(pos, expr, expected)) + else if (!expression_p.parse(pos, expr, expected)) return false; - if (!s_type.ignore(pos, expected)) - return false; - if (!data_type_p.parse(pos, type, expected)) - return false; + if (s_type.ignore(pos, expected)) + { + if (!data_type_p.parse(pos, type, expected)) + return false; + } if (s_granularity.ignore(pos, expected)) { @@ -45,13 +57,14 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected auto index = std::make_shared(); index->part_of_create_index_query = true; index->set(index->expr, expr); - index->set(index->type, type); + if (type) + index->set(index->type, type); if (granularity) index->granularity = granularity->as().value.safeGet(); else { - if (index->type->name == "annoy") + if (index->type && index->type->name == "annoy") index->granularity = ASTIndexDeclaration::DEFAULT_ANNOY_INDEX_GRANULARITY; else index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY; From 31806d0b6c6726e6a2540587735ba9449c012062 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Jul 2023 11:14:49 +0000 Subject: [PATCH 0533/2047] no style --- src/Interpreters/InterpreterCreateIndexQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp index 0e36fa22404..752bc6200ce 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.cpp +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -15,7 +15,6 @@ namespace DB namespace ErrorCodes { extern const int TABLE_IS_READ_ONLY; - extern const int LOGICAL_ERROR; extern const int INCORRECT_QUERY; } From 798f31a612d63a64f3b237c29c597bccf71dffbc Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 12 Jul 2023 12:36:57 +0000 Subject: [PATCH 0534/2047] Update version_date.tsv and changelogs after v23.4.6.25-stable --- docs/changelogs/v23.4.6.25-stable.md | 26 ++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 27 insertions(+) create mode 100644 docs/changelogs/v23.4.6.25-stable.md diff --git a/docs/changelogs/v23.4.6.25-stable.md b/docs/changelogs/v23.4.6.25-stable.md new file mode 100644 index 00000000000..01a9c06f3e9 --- /dev/null +++ b/docs/changelogs/v23.4.6.25-stable.md @@ -0,0 +1,26 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.4.6.25-stable (a06848b1770) FIXME as compared to v23.4.5.22-stable (0ced5d6a8da) + +#### Improvement +* Backported in [#51234](https://github.com/ClickHouse/ClickHouse/issues/51234): Improve the progress bar for file/s3/hdfs/url table functions by using chunk size from source data and using incremental total size counting in each thread. Fix the progress bar for *Cluster functions. This closes [#47250](https://github.com/ClickHouse/ClickHouse/issues/47250). [#51088](https://github.com/ClickHouse/ClickHouse/pull/51088) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix backward compatibility for IP types hashing in aggregate functions [#50551](https://github.com/ClickHouse/ClickHouse/pull/50551) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix segfault in MathUnary [#51499](https://github.com/ClickHouse/ClickHouse/pull/51499) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix for moving 'IN' conditions to PREWHERE [#51610](https://github.com/ClickHouse/ClickHouse/pull/51610) ([Alexander Gololobov](https://github.com/davenger)). +* Fix reading from empty column in `parseSipHashKey` [#51804](https://github.com/ClickHouse/ClickHouse/pull/51804) ([Nikita Taranov](https://github.com/nickitat)). +* Allow parametric UDFs [#51964](https://github.com/ClickHouse/ClickHouse/pull/51964) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Decoupled commits from [#51180](https://github.com/ClickHouse/ClickHouse/issues/51180) for backports [#51561](https://github.com/ClickHouse/ClickHouse/pull/51561) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix MergeTreeMarksLoader segfaulting if marks file is longer than expected [#51636](https://github.com/ClickHouse/ClickHouse/pull/51636) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix source image for sqllogic [#51728](https://github.com/ClickHouse/ClickHouse/pull/51728) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index dd46f6103d0..6573e3d7683 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -4,6 +4,7 @@ v23.5.4.25-stable 2023-06-29 v23.5.3.24-stable 2023-06-17 v23.5.2.7-stable 2023-06-10 v23.5.1.3174-stable 2023-06-09 +v23.4.6.25-stable 2023-07-12 v23.4.5.22-stable 2023-06-29 v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 From 2ee7a9846f0b31b7a792f877ae2c8c02cb5bad5a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 12 Jul 2023 12:47:24 +0000 Subject: [PATCH 0535/2047] fix --- src/Parsers/ParserCreateIndexQuery.cpp | 10 +++++----- src/Storages/IndicesDescription.cpp | 13 +++++++++++-- 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index 5bcea50af87..d2ae7f972b7 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -29,7 +29,11 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected ASTPtr type; ASTPtr granularity; - if (open.ignore(pos, expected)) + /// Skip name parser for SQL-standard CREATE INDEX + if (expression_p.parse(pos, expr, expected)) + { + } + else if (open.ignore(pos, expected)) { if (!order_list.parse(pos, order, expected)) return false; @@ -37,10 +41,6 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (!close.ignore(pos, expected)) return false; } - /// Skip name parser for SQL-standard CREATE INDEX - else if (!expression_p.parse(pos, expr, expected)) - return false; - if (s_type.ignore(pos, expected)) { diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index a93ac248c98..c7aeaf8e4ef 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -11,6 +11,7 @@ #include #include +#include "Common/Exception.h" namespace DB @@ -89,8 +90,16 @@ IndexDescription IndexDescription::getIndexFromAST(const ASTPtr & definition_ast result.type = Poco::toLower(index_definition->type->name); result.granularity = index_definition->granularity; - ASTPtr expr_list = extractKeyExpressionList(index_definition->expr->clone()); - result.expression_list_ast = expr_list->clone(); + ASTPtr expr_list; + if (index_definition->expr) + { + expr_list = extractKeyExpressionList(index_definition->expr->clone()); + result.expression_list_ast = expr_list->clone(); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expression is not set"); + } auto syntax = TreeRewriter(context).analyze(expr_list, columns.getAllPhysical()); result.expression = ExpressionAnalyzer(expr_list, syntax, context).getActions(true); From d891c4c9a218784d1717710a676b3a132339b14b Mon Sep 17 00:00:00 2001 From: Lars Eidnes Date: Wed, 12 Jul 2023 15:06:01 +0200 Subject: [PATCH 0536/2047] MaterializedMySQL: Delay disconnection of mysqlxx::Pool::Entry until refcount==0 If Entry::disconnect() is called, it is removed from its pool and underlying mysqlxx::Connection is disconnected. Since Entry objects are copyable and some of them could be alive when disconnect() is called which would lead to undefined behavior. Suggesting to disconnect it when the last object is being destroyed. --- src/Common/mysqlxx/Pool.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index d10889d1f97..64a69c48e1d 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -25,8 +25,6 @@ void Pool::Entry::incrementRefCount() /// First reference, initialize thread if (data->ref_count.fetch_add(1) == 0) mysql_thread_init(); - - chassert(!data->removed_from_pool); } @@ -43,7 +41,10 @@ void Pool::Entry::decrementRefCount() /// In Pool::Entry::disconnect() we remove connection from the list of pool's connections. /// So now we must deallocate the memory. if (data->removed_from_pool) + { + data->conn.disconnect(); ::delete data; + } } } @@ -230,8 +231,6 @@ void Pool::removeConnection(Connection* connection) std::lock_guard lock(mutex); if (connection) { - if (!connection->removed_from_pool) - connection->conn.disconnect(); connections.remove(connection); connection->removed_from_pool = true; } @@ -240,6 +239,7 @@ void Pool::removeConnection(Connection* connection) void Pool::Entry::disconnect() { + // Remove the Entry from the Pool. Actual disconnection is delayed until refcount == 0. pool->removeConnection(data); } From 771b1f8f475993b331749af9d7edff5fc2e5866c Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 12 Jul 2023 15:48:09 +0000 Subject: [PATCH 0537/2047] Prototype (working) --- programs/server/Server.cpp | 8 +---- src/Common/SystemLogBase.cpp | 54 ++++++++++++++++++++--------- src/Common/SystemLogBase.h | 61 ++++++++++++++++++++++++--------- src/Interpreters/SystemLog.cpp | 26 +++++++------- src/Interpreters/SystemLog.h | 17 ++++----- src/Interpreters/TextLog.cpp | 2 +- src/Interpreters/TextLog.h | 8 +++++ src/Loggers/Loggers.cpp | 18 ++++------ src/Loggers/Loggers.h | 13 ------- src/Loggers/OwnSplitChannel.cpp | 6 ++-- src/Loggers/OwnSplitChannel.h | 8 +++-- 11 files changed, 129 insertions(+), 92 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c7a7ba71e83..79a28558d93 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1599,13 +1599,7 @@ try global_context->setSystemZooKeeperLogAfterInitializationIfNeeded(); /// Build loggers before tables startup to make log messages from tables /// attach available in system.text_log - { - String level_str = config().getString("text_log.level", ""); - int level = level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(level_str); - setTextLog(global_context->getTextLog(), level); - - buildLoggers(config(), logger()); - } + buildLoggers(config(), logger()); /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) attachSystemTablesServer(global_context, *database_catalog.getSystemDatabase(), has_zookeeper); attachInformationSchema(global_context, *database_catalog.getDatabase(DatabaseCatalog::INFORMATION_SCHEMA)); diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 86adcbbd31b..28dc5ec2666 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -38,10 +38,20 @@ namespace ISystemLog::~ISystemLog() = default; -void ISystemLog::stopFlushThread() +template +SystemLogBase::SystemLogBase(std::shared_ptr> ex_queue) +{ + if (ex_queue) + queue = ex_queue; + else + queue = std::make_shared>(); +} + +template +void SystemLogBase::stopFlushThread() { { - std::lock_guard lock(mutex); + std::lock_guard lock(queue->mutex); if (!saving_thread || !saving_thread->joinable()) return; @@ -52,22 +62,26 @@ void ISystemLog::stopFlushThread() is_shutdown = true; /// Tell thread to shutdown. - flush_event.notify_all(); + queue->flush_event.notify_all(); } saving_thread->join(); } -void ISystemLog::startup() +template +void SystemLogBase::startup() { - std::lock_guard lock(mutex); - saving_thread = std::make_unique([this] { savingThreadFunction(); }); + std::lock_guard lock(queue->mutex); + std::cout << "void ISystemLog::startup()" << std::endl; + saving_thread = std::make_unique([this] { + std::cout << "void ISystemLog::ThreadFromGlobalPool()" << std::endl; + savingThreadFunction(); }); } static thread_local bool recursive_add_call = false; template -void SystemLogBase::add(const LogElement & element) +void SystemLogQueue::add(const LogElement & element) { /// It is possible that the method will be called recursively. /// Better to drop these events to avoid complications. @@ -75,21 +89,21 @@ void SystemLogBase::add(const LogElement & element) return; recursive_add_call = true; SCOPE_EXIT({ recursive_add_call = false; }); - - /// Memory can be allocated while resizing on queue.push_back. + /// Memory can be allocated while resizing on queue.push_back. /// The size of allocation can be in order of a few megabytes. /// But this should not be accounted for query memory usage. /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; + /// Should not log messages under mutex. bool queue_is_half_full = false; { std::unique_lock lock(mutex); - if (is_shutdown) - return; + // if (queue.is_shutdown) + // return; // TODO if (queue.size() == DBMS_SYSTEM_LOG_QUEUE_SIZE / 2) { @@ -134,25 +148,31 @@ void SystemLogBase::add(const LogElement & element) LOG_INFO(log, "Queue is half full for system log '{}'.", demangle(typeid(*this).name())); } +template +void SystemLogBase::add(const LogElement & element) +{ + queue->add(element); +} + template void SystemLogBase::flush(bool force) { uint64_t this_thread_requested_offset; { - std::lock_guard lock(mutex); + std::lock_guard lock(queue->mutex); if (is_shutdown) return; - this_thread_requested_offset = queue_front_index + queue.size(); + this_thread_requested_offset = queue->queue_front_index + queue->size(); // Publish our flush request, taking care not to overwrite the requests // made by other threads. is_force_prepare_tables |= force; - requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); + queue->requested_flush_up_to = std::max(queue->requested_flush_up_to, this_thread_requested_offset); - flush_event.notify_all(); + queue->flush_event.notify_all(); } LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); @@ -161,8 +181,8 @@ void SystemLogBase::flush(bool force) // too fast for our parallel functional tests, probably because they // heavily load the disk. const int timeout_seconds = 180; - std::unique_lock lock(mutex); - bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] + std::unique_lock lock(queue->mutex); + bool result = queue->flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] { return flushed_up_to >= this_thread_requested_offset && !is_force_prepare_tables; }); diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index f8febd8b159..cb6003c94ef 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -55,25 +55,52 @@ public: virtual void prepareTable() = 0; /// Start the background thread. - virtual void startup(); + virtual void startup() = 0; /// Stop the background flush thread before destructor. No more data will be written. virtual void shutdown() = 0; + virtual void stopFlushThread() = 0; + virtual ~ISystemLog(); virtual void savingThreadFunction() = 0; protected: - std::unique_ptr saving_thread; + std::unique_ptr saving_thread; +}; + +template +class SystemLogQueue +{ +public: + + void add(const LogElement & element); + size_t size() const { return queue.size(); } + void push_back(const LogElement & element) + { + queue.push_back(element); + } + + // Queue is bounded. But its size is quite large to not block in all normal cases. + std::vector queue; + // An always-incrementing index of the first message currently in the queue. + // We use it to give a global sequential index to every message, so that we + // can wait until a particular message is flushed. This is used to implement + // synchronous log flushing for SYSTEM FLUSH LOGS. + uint64_t queue_front_index = 0; /// Data shared between callers of add()/flush()/shutdown(), and the saving thread std::mutex mutex; - - bool is_shutdown = false; std::condition_variable flush_event; - void stopFlushThread(); + // Requested to flush logs up to this index, exclusive + uint64_t requested_flush_up_to = 0; + + // Logged overflow message at this queue front index + uint64_t logged_queue_full_at_index = -1; + + Poco::Logger * log; }; template @@ -82,6 +109,8 @@ class SystemLogBase : public ISystemLog public: using Self = SystemLogBase; + SystemLogBase(std::shared_ptr> ex_queue = nullptr); + /** Append a record into log. * Writing to table will be done asynchronously and in case of failure, record could be lost. */ @@ -90,6 +119,10 @@ public: /// Flush data in the buffer to disk void flush(bool force) override; + void startup() override; + + void stopFlushThread() override; + String getName() const override { return LogElement::name(); } static const char * getDefaultOrderBy() { return "event_date, event_time"; } @@ -97,21 +130,17 @@ public: protected: Poco::Logger * log; - // Queue is bounded. But its size is quite large to not block in all normal cases. - std::vector queue; - // An always-incrementing index of the first message currently in the queue. - // We use it to give a global sequential index to every message, so that we - // can wait until a particular message is flushed. This is used to implement - // synchronous log flushing for SYSTEM FLUSH LOGS. - uint64_t queue_front_index = 0; + std::shared_ptr> queue; + // A flag that says we must create the tables even if the queue is empty. bool is_force_prepare_tables = false; - // Requested to flush logs up to this index, exclusive - uint64_t requested_flush_up_to = 0; + // Flushed log up to this index, exclusive uint64_t flushed_up_to = 0; - // Logged overflow message at this queue front index - uint64_t logged_queue_full_at_index = -1; + + + bool is_shutdown = false; + }; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 3fd0297f5b8..cbf355d020a 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -332,8 +332,10 @@ SystemLog::SystemLog( const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_) - : WithContext(context_) + size_t flush_interval_milliseconds_, + std::shared_ptr> ex_queue) + : Base(ex_queue) + , WithContext(context_) , table_id(database_name_, table_name_) , storage_def(storage_def_) , create_query(serializeAST(*getCreateTableQuery())) @@ -371,21 +373,21 @@ void SystemLog::savingThreadFunction() bool should_prepare_tables_anyway = false; { - std::unique_lock lock(mutex); - flush_event.wait_for(lock, + std::unique_lock lock(queue->mutex); + queue->flush_event.wait_for(lock, std::chrono::milliseconds(flush_interval_milliseconds), [&] () { - return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables; + return queue->requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables; } ); - queue_front_index += queue.size(); - to_flush_end = queue_front_index; + queue->queue_front_index += queue->size(); + to_flush_end = queue->queue_front_index; // Swap with existing array from previous flush, to save memory // allocations. to_flush.resize(0); - queue.swap(to_flush); + queue->queue.swap(to_flush); should_prepare_tables_anyway = is_force_prepare_tables; @@ -399,9 +401,9 @@ void SystemLog::savingThreadFunction() prepareTable(); LOG_TRACE(log, "Table created (force)"); - std::lock_guard lock(mutex); + std::lock_guard lock(queue->mutex); is_force_prepare_tables = false; - flush_event.notify_all(); + queue->flush_event.notify_all(); } } else @@ -474,10 +476,10 @@ void SystemLog::flushImpl(const std::vector & to_flush, } { - std::lock_guard lock(mutex); + std::lock_guard lock(queue->mutex); flushed_up_to = to_flush_end; is_force_prepare_tables = false; - flush_event.notify_all(); + queue->flush_event.notify_all(); } LOG_TRACE(log, "Flushed system log up to offset {}", to_flush_end); diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 84b70c67e2a..bf41ff12d2a 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -108,22 +108,23 @@ public: const String & database_name_, const String & table_name_, const String & storage_def_, - size_t flush_interval_milliseconds_); + size_t flush_interval_milliseconds_, + std::shared_ptr> ex_queue = nullptr); void shutdown() override; protected: - using ISystemLog::mutex; - using ISystemLog::is_shutdown; - using ISystemLog::flush_event; - using ISystemLog::stopFlushThread; + //using ISystemLog::mutex; + using Base::is_shutdown; + // using ISystemLog::flush_event; + using Base::stopFlushThread; using Base::log; using Base::queue; - using Base::queue_front_index; + // using Base::queue_front_index; using Base::is_force_prepare_tables; - using Base::requested_flush_up_to; + //using Base::requested_flush_up_to; using Base::flushed_up_to; - using Base::logged_queue_full_at_index; + // using Base::logged_queue_full_at_index; private: diff --git a/src/Interpreters/TextLog.cpp b/src/Interpreters/TextLog.cpp index 45d5a7b2344..a66092c1c2b 100644 --- a/src/Interpreters/TextLog.cpp +++ b/src/Interpreters/TextLog.cpp @@ -84,7 +84,7 @@ TextLog::TextLog(ContextPtr context_, const String & database_name_, const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_) : SystemLog(context_, database_name_, table_name_, - storage_def_, flush_interval_milliseconds_) + storage_def_, flush_interval_milliseconds_, getLogQueue()) { // SystemLog methods may write text logs, so we disable logging for the text // log table to avoid recursion. diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 6efc1c906d4..9c761f9d865 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -46,6 +46,14 @@ public: const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_); + + static std::shared_ptr> getLogQueue() + { + static std::shared_ptr> queue = std::make_shared>(); + return queue; + } }; + + } diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 0c3a7bd615d..96793ee5e0a 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -34,21 +34,9 @@ static std::string createDirectory(const std::string & file) return path; } -#ifndef WITHOUT_TEXT_LOG -void Loggers::setTextLog(std::shared_ptr log, int max_priority) -{ - text_log = log; - text_log_max_priority = max_priority; -} -#endif void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger /*_root*/, const std::string & cmd_name) { -#ifndef WITHOUT_TEXT_LOG - if (split) - if (auto log = text_log.lock()) - split->addTextLog(log, text_log_max_priority); -#endif auto current_logger = config.getString("logger", ""); if (config_logger.has_value() && *config_logger == current_logger) @@ -62,6 +50,12 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log /// Use extended interface of Channel for more comprehensive logging. split = new DB::OwnSplitChannel(); +#ifndef WITHOUT_TEXT_LOG + String text_log_level_str = config.getString("text_log.level", ""); + int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); + split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); +#endif + auto log_level_string = config.getString("logger.level", "trace"); /// different channels (log, console, syslog) may have different loglevels configured diff --git a/src/Loggers/Loggers.h b/src/Loggers/Loggers.h index ebc10954b94..9eff731a4c5 100644 --- a/src/Loggers/Loggers.h +++ b/src/Loggers/Loggers.h @@ -7,12 +7,6 @@ #include #include "OwnSplitChannel.h" -#ifndef WITHOUT_TEXT_LOG -namespace DB -{ - class TextLog; -} -#endif namespace Poco::Util { @@ -29,9 +23,6 @@ public: /// Close log files. On next log write files will be reopened. void closeLogs(Poco::Logger & logger); -#ifndef WITHOUT_TEXT_LOG - void setTextLog(std::shared_ptr log, int max_priority); -#endif private: Poco::AutoPtr log_file; @@ -41,10 +32,6 @@ private: /// Previous value of logger element in config. It is used to reinitialize loggers whenever the value changed. std::optional config_logger; -#ifndef WITHOUT_TEXT_LOG - std::weak_ptr text_log; - int text_log_max_priority = -1; -#endif Poco::AutoPtr split; }; diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 03db198c305..7661996ea33 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -135,7 +135,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) elem.source_line = msg.getSourceLine(); elem.message_format_string = msg.getFormatString(); - std::shared_ptr text_log_locked{}; + std::shared_ptr> text_log_locked{}; { std::lock_guard lock(text_log_mutex); text_log_locked = text_log.lock(); @@ -153,10 +153,10 @@ void OwnSplitChannel::addChannel(Poco::AutoPtr channel, const std } #ifndef WITHOUT_TEXT_LOG -void OwnSplitChannel::addTextLog(std::shared_ptr log, int max_priority) +void OwnSplitChannel::addTextLog(std::shared_ptr> log_queue, int max_priority) { std::lock_guard lock(text_log_mutex); - text_log = log; + text_log = log_queue; text_log_max_priority.store(max_priority, std::memory_order_relaxed); } #endif diff --git a/src/Loggers/OwnSplitChannel.h b/src/Loggers/OwnSplitChannel.h index 80305c1ccee..ab86c32f85a 100644 --- a/src/Loggers/OwnSplitChannel.h +++ b/src/Loggers/OwnSplitChannel.h @@ -10,7 +10,9 @@ #ifndef WITHOUT_TEXT_LOG namespace DB { - class TextLog; + template class SystemLogQueue; + struct TextLogElement; + using FooBar = SystemLogQueue; } #endif @@ -31,7 +33,7 @@ public: void addChannel(Poco::AutoPtr channel, const std::string & name); #ifndef WITHOUT_TEXT_LOG - void addTextLog(std::shared_ptr log, int max_priority); + void addTextLog(std::shared_ptr log_queue, int max_priority); #endif void setLevel(const std::string & name, int level); @@ -48,7 +50,7 @@ private: std::mutex text_log_mutex; #ifndef WITHOUT_TEXT_LOG - std::weak_ptr text_log; + std::weak_ptr text_log; std::atomic text_log_max_priority = -1; #endif }; From 4e70272da1098a475eea2f0be170418536d3aff8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 7 Jul 2023 21:51:22 +0200 Subject: [PATCH 0538/2047] impl --- src/Interpreters/ActionsVisitor.cpp | 5 +++++ .../0_stateless/02701_non_parametric_function.reference | 1 + .../0_stateless/02701_non_parametric_function.sql | 9 +++++++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02701_non_parametric_function.reference create mode 100644 tests/queries/0_stateless/02701_non_parametric_function.sql diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 01f2d4cf22e..8cf3eb9b45e 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -78,6 +78,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; + extern const int FUNCTION_CANNOT_HAVE_PARAMETERS; } static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols) @@ -1103,6 +1104,10 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & e.addMessage("Or unknown aggregate function " + node.name + ". Maybe you meant: " + toString(hints)); throw; } + + /// Normal functions are not parametric for now. + if (node.parameters) + throw Exception(ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS, "Function {} is not parametric", node.name); } Names argument_names; diff --git a/tests/queries/0_stateless/02701_non_parametric_function.reference b/tests/queries/0_stateless/02701_non_parametric_function.reference new file mode 100644 index 00000000000..00750edc07d --- /dev/null +++ b/tests/queries/0_stateless/02701_non_parametric_function.reference @@ -0,0 +1 @@ +3 diff --git a/tests/queries/0_stateless/02701_non_parametric_function.sql b/tests/queries/0_stateless/02701_non_parametric_function.sql new file mode 100644 index 00000000000..11c03372a73 --- /dev/null +++ b/tests/queries/0_stateless/02701_non_parametric_function.sql @@ -0,0 +1,9 @@ +-- Tags: no-parallel + +SELECT * FROM system.numbers WHERE number > toUInt64(10)(number) LIMIT 10; -- { serverError 309 } + +CREATE FUNCTION IF NOT EXISTS sum_udf as (x, y) -> (x + y); + +SELECT sum_udf(1)(1, 2); + +DROP FUNCTION IF EXISTS sum_udf; From 3649f104444076f7f5cc232fae8940cde8025d46 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 13 Jul 2023 05:19:18 +0000 Subject: [PATCH 0539/2047] Remove spaces --- src/Common/SystemLogBase.cpp | 4 +--- src/Common/SystemLogBase.h | 2 +- src/Interpreters/TextLog.h | 4 +--- 3 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 28dc5ec2666..2d2c3e2053f 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -73,9 +73,7 @@ void SystemLogBase::startup() { std::lock_guard lock(queue->mutex); std::cout << "void ISystemLog::startup()" << std::endl; - saving_thread = std::make_unique([this] { - std::cout << "void ISystemLog::ThreadFromGlobalPool()" << std::endl; - savingThreadFunction(); }); + saving_thread = std::make_unique([this] { savingThreadFunction(); }); } static thread_local bool recursive_add_call = false; diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index cb6003c94ef..221d9946b48 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -67,7 +67,7 @@ public: virtual void savingThreadFunction() = 0; protected: - std::unique_ptr saving_thread; + std::unique_ptr saving_thread; }; template diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 9c761f9d865..33c38da2f8f 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -46,7 +46,7 @@ public: const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_); - + static std::shared_ptr> getLogQueue() { static std::shared_ptr> queue = std::make_shared>(); @@ -54,6 +54,4 @@ public: } }; - - } From 220c0255abea1c4fbb5dd66a4fda350be9aa9023 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 13 Jul 2023 08:15:50 +0000 Subject: [PATCH 0540/2047] Remove mutex --- src/Common/SystemLogBase.cpp | 1 - src/Loggers/Loggers.cpp | 10 ++++++---- src/Loggers/OwnSplitChannel.cpp | 6 +----- src/Loggers/OwnSplitChannel.h | 2 -- 4 files changed, 7 insertions(+), 12 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 2d2c3e2053f..09f4c050f15 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -72,7 +72,6 @@ template void SystemLogBase::startup() { std::lock_guard lock(queue->mutex); - std::cout << "void ISystemLog::startup()" << std::endl; saving_thread = std::make_unique([this] { savingThreadFunction(); }); } diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 96793ee5e0a..2fda836e7b7 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -37,7 +37,6 @@ static std::string createDirectory(const std::string & file) void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger /*_root*/, const std::string & cmd_name) { - auto current_logger = config.getString("logger", ""); if (config_logger.has_value() && *config_logger == current_logger) return; @@ -51,9 +50,12 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log split = new DB::OwnSplitChannel(); #ifndef WITHOUT_TEXT_LOG - String text_log_level_str = config.getString("text_log.level", ""); - int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); - split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); + if (config.has("text_log")) + { + String text_log_level_str = config.getString("text_log.level", ""); + int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); + split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); + } #endif auto log_level_string = config.getString("logger.level", "trace"); diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 7661996ea33..17806153905 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -136,10 +136,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) elem.message_format_string = msg.getFormatString(); std::shared_ptr> text_log_locked{}; - { - std::lock_guard lock(text_log_mutex); - text_log_locked = text_log.lock(); - } + text_log_locked = text_log.lock(); if (text_log_locked) text_log_locked->add(elem); } @@ -155,7 +152,6 @@ void OwnSplitChannel::addChannel(Poco::AutoPtr channel, const std #ifndef WITHOUT_TEXT_LOG void OwnSplitChannel::addTextLog(std::shared_ptr> log_queue, int max_priority) { - std::lock_guard lock(text_log_mutex); text_log = log_queue; text_log_max_priority.store(max_priority, std::memory_order_relaxed); } diff --git a/src/Loggers/OwnSplitChannel.h b/src/Loggers/OwnSplitChannel.h index ab86c32f85a..155d0872465 100644 --- a/src/Loggers/OwnSplitChannel.h +++ b/src/Loggers/OwnSplitChannel.h @@ -47,8 +47,6 @@ private: using ExtendedChannelPtrPair = std::pair; std::map channels; - std::mutex text_log_mutex; - #ifndef WITHOUT_TEXT_LOG std::weak_ptr text_log; std::atomic text_log_max_priority = -1; From 8916e7d69b6e27a1077c01dd345f1268ee85f38f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 13 Jul 2023 08:49:48 +0000 Subject: [PATCH 0541/2047] Update version_date.tsv and changelogs after v23.3.8.21-lts --- docs/changelogs/v23.3.8.21-lts.md | 23 +++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 24 insertions(+) create mode 100644 docs/changelogs/v23.3.8.21-lts.md diff --git a/docs/changelogs/v23.3.8.21-lts.md b/docs/changelogs/v23.3.8.21-lts.md new file mode 100644 index 00000000000..83b5070ef52 --- /dev/null +++ b/docs/changelogs/v23.3.8.21-lts.md @@ -0,0 +1,23 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.3.8.21-lts (1675f2264f3) FIXME as compared to v23.3.7.5-lts (bc683c11c92) + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix backward compatibility for IP types hashing in aggregate functions [#50551](https://github.com/ClickHouse/ClickHouse/pull/50551) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix segfault in MathUnary [#51499](https://github.com/ClickHouse/ClickHouse/pull/51499) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix for moving 'IN' conditions to PREWHERE [#51610](https://github.com/ClickHouse/ClickHouse/pull/51610) ([Alexander Gololobov](https://github.com/davenger)). +* Fix reading from empty column in `parseSipHashKey` [#51804](https://github.com/ClickHouse/ClickHouse/pull/51804) ([Nikita Taranov](https://github.com/nickitat)). +* Check refcount in `RemoveManyObjectStorageOperation::finalize` instead of `execute` [#51954](https://github.com/ClickHouse/ClickHouse/pull/51954) ([vdimir](https://github.com/vdimir)). +* Allow parametric UDFs [#51964](https://github.com/ClickHouse/ClickHouse/pull/51964) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Decoupled commits from [#51180](https://github.com/ClickHouse/ClickHouse/issues/51180) for backports [#51561](https://github.com/ClickHouse/ClickHouse/pull/51561) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix MergeTreeMarksLoader segfaulting if marks file is longer than expected [#51636](https://github.com/ClickHouse/ClickHouse/pull/51636) ([Michael Kolupaev](https://github.com/al13n321)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 6573e3d7683..8b535e3d897 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -10,6 +10,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.8.21-lts 2023-07-13 v23.3.7.5-lts 2023-06-29 v23.3.6.7-lts 2023-06-28 v23.3.5.9-lts 2023-06-22 From bc392a607321093f8fa4e12b99289ae4c113fd80 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 13 Jul 2023 09:55:35 +0000 Subject: [PATCH 0542/2047] enable flag in sqllogic --- tests/sqllogic/connection.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/sqllogic/connection.py b/tests/sqllogic/connection.py index ca03839fc35..0033c29c41c 100644 --- a/tests/sqllogic/connection.py +++ b/tests/sqllogic/connection.py @@ -62,7 +62,7 @@ def default_clickhouse_odbc_conn_str(): return str( OdbcConnectingArgs.create_from_kw( dsn="ClickHouse DSN (ANSI)", - Url="http://localhost:8123/query?default_format=ODBCDriver2&default_table_engine=MergeTree&union_default_mode=DISTINCT&group_by_use_nulls=1&join_use_nulls=1", + Url="http://localhost:8123/query?default_format=ODBCDriver2&default_table_engine=MergeTree&union_default_mode=DISTINCT&group_by_use_nulls=1&join_use_nulls=1&allow_create_index_without_type=1", ) ) From 3a54182f778ca96cb107f7d934719553447a2733 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 13 Jul 2023 13:55:55 +0200 Subject: [PATCH 0543/2047] Fix test_extreme_deduplication flakiness This is part of **"Why my PR is not merged?"** patch set series. Since #50107 there one more tunable that can increase the cleanup period - max_cleanup_delay_period. Plus, increase the delay for the cleanup thread, otherwise it is too fragile. Signed-off-by: Azat Khuzhin --- .../test_extreme_deduplication/configs/conf.d/merge_tree.xml | 1 + tests/integration/test_extreme_deduplication/test.py | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml b/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml index d9062e8da20..6f1d05d125e 100644 --- a/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml +++ b/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml @@ -4,6 +4,7 @@ 1 1 0 + 1 1 diff --git a/tests/integration/test_extreme_deduplication/test.py b/tests/integration/test_extreme_deduplication/test.py index 71f783d37c9..9604809f0b5 100644 --- a/tests/integration/test_extreme_deduplication/test.py +++ b/tests/integration/test_extreme_deduplication/test.py @@ -50,7 +50,7 @@ def test_deduplication_window_in_seconds(started_cluster): assert TSV(node.query("SELECT count() FROM simple")) == TSV("2\n") # wait clean thread - time.sleep(2) + time.sleep(5) assert ( TSV.toMat( From ba57c84db3c778eb4d0fe4e5a19b7615ab0772be Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 3 Jul 2023 11:40:11 +0800 Subject: [PATCH 0544/2047] bug fix csv input field type mismatch --- docs/en/interfaces/formats.md | 2 + .../operations/settings/settings-formats.md | 29 ++++++++++++ src/Core/Settings.h | 2 + src/Formats/FormatFactory.cpp | 2 + src/Formats/FormatSettings.h | 2 + .../Formats/Impl/CSVRowInputFormat.cpp | 47 ++++++++++++++++--- ...11_csv_input_field_type_mismatch.reference | 3 ++ .../02811_csv_input_field_type_mismatch.sh | 13 +++++ .../data_csv/csv_with_diff_field_types.csv | 3 ++ 9 files changed, 97 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference create mode 100644 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh create mode 100644 tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ed2f010a632..ab6e514b4b2 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,6 +472,8 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. +- [input_format_csv_allow_check_deserialize](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_check_deserialize) - Allow to check whether the csv input field can be deserialized. Default value - `false`. +- [input_format_csv_allow_set_default_if_deserialize_failed](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_set_default_if_deserialize_failed) - Allow to set default value to column if the csv input field deserialize failed. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 3eea5ef4ad9..bb2159210d4 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -969,6 +969,35 @@ Result a b ``` +### input_format_csv_allow_check_deserialize_result {#input_format_csv_allow_check_deserialize_result} + +Allow to use whitespace or tab as field delimiter in CSV strings. + +Default value: `false`. + +### input_format_csv_allow_set_column_default_value_if_deserialize_failed {#input_format_csv_allow_set_column_default_value_if_deserialize_failed} + +Allow to set default value to column if the csv input field deserialize failed + +Default value: `false`. + +**Examples** + +Query + +```bash +echo 'a,b,c' > 1.txt +./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" +cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize_result=true, input_format_csv_allow_set_column_default_value_if_deserialize_failed=true FORMAT CSV" +./clickhouse local -q "select * from test_tbl" +``` + +Result + +```text +a 0 1971-01-01 +``` + ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f1a314fed37..b67745d1f59 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,6 +872,8 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ + M(Bool, input_format_csv_allow_check_deserialize_result, false, "Allow to check the csv input field deserialize whether success or not.", 0) \ + M(Bool, input_format_csv_allow_set_column_default_value_if_deserialize_failed, false, "All to set column default value if the input field deserialize failed.", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 182abc84ffe..8a2a7e97941 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,6 +73,8 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; + format_settings.csv.allow_check_deserialize = settings.input_format_csv_allow_check_deserialize; + format_settings.csv.allow_set_default_if_deserialize_failed = settings.input_format_csv_allow_set_default_if_deserialize_failed; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index dd4608227d0..50ebdc1c67f 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -141,6 +141,8 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; + bool allow_check_deserialize = false; + bool allow_set_default_if_deserialize_failed=false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 79ce2549b4d..8c4660fe4f9 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -315,14 +316,48 @@ bool CSVFormatReader::readField( return false; } - if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) + BufferBase::Position pos_start = buf->position(); + size_t col_size = column.size(); + try { - /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); + if (format_settings.csv.allow_check_deserialize_result) + { + std::string field; + readCSVField(field, *buf, format_settings.csv); + ReadBufferFromMemory tmp(field); + if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) + SerializationNullable::deserializeTextCSVImpl(column, tmp, format_settings, serialization); + else + serialization->deserializeTextCSV(column, tmp, format_settings); + if (column.size() == col_size + 1 && field.size() > 0 && tmp.position() != tmp.buffer().end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Text CSV deserialize field bytes logical error."); + } + else + { + if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) + { + /// If value is null but type is not nullable then use default value instead. + return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); + } + /// Read the column normally. + serialization->deserializeTextCSV(column, *buf, format_settings); + } + } + catch (Exception & e) + { + LOG_DEBUG(&Poco::Logger::get("CSVRowInputFormat"), "Failed to deserialize CSV column, exception message:{}", e.what()); + if (format_settings.csv.allow_set_column_default_value_if_deserialize_failed) + { + // Reset the column and buffer position, then skip the field and set column default value. + if (column.size() == col_size + 1) + column.popBack(1); + buf->position() = pos_start; + skipField(); + column.insertDefault(); + } + else + throw; } - - /// Read the column normally. - serialization->deserializeTextCSV(column, *buf, format_settings); return true; } diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference new file mode 100644 index 00000000000..c5ee611a230 --- /dev/null +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference @@ -0,0 +1,3 @@ +a 1 2023-03-14 +a 0 1970-01-01 +c 1 1970-01-01 diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh new file mode 100644 index 00000000000..246905a5b18 --- /dev/null +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +# NOTE: this sh wrapper is required because of shell_config + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" +$CLICKHOUSE_CLIENT -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" +cat $CURDIR/data_csv/csv_with_diff_field_types.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize_result=true, input_format_csv_allow_set_column_default_value_if_deserialize_failed=true FORMAT CSV" +$CLICKHOUSE_CLIENT -q "select * from test_tbl" +$CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv b/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv new file mode 100644 index 00000000000..464172c515c --- /dev/null +++ b/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv @@ -0,0 +1,3 @@ +a,1,2023-03-14 +a,b,c +c,1,a \ No newline at end of file From 2720e9177d9be212e469ee5339decd8dda05a58d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 13 Jul 2023 13:29:45 +0200 Subject: [PATCH 0545/2047] Fix flakiness of test_keeper_s3_snapshot This is part of "Why my PR is not merged?" patch set series. Signed-off-by: Azat Khuzhin --- docker/test/integration/runner/Dockerfile | 1 + .../test_keeper_s3_snapshot/test.py | 29 ++++++++++++------- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index d6c127c8421..0d1fa00b214 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -98,6 +98,7 @@ RUN python3 -m pip install --no-cache-dir \ redis \ requests-kerberos \ tzlocal==2.1 \ + retry \ urllib3 # Hudi supports only spark 3.3.*, not 3.4 diff --git a/tests/integration/test_keeper_s3_snapshot/test.py b/tests/integration/test_keeper_s3_snapshot/test.py index 3e19bc4822c..84ffc964621 100644 --- a/tests/integration/test_keeper_s3_snapshot/test.py +++ b/tests/integration/test_keeper_s3_snapshot/test.py @@ -1,6 +1,7 @@ import pytest from helpers.cluster import ClickHouseCluster from time import sleep +from retry import retry from kazoo.client import KazooClient @@ -88,15 +89,19 @@ def test_s3_upload(started_cluster): for obj in list(cluster.minio_client.list_objects("snapshots")) ] - saved_snapshots = get_saved_snapshots() - assert set(saved_snapshots) == set( - [ - "snapshot_50.bin.zstd", - "snapshot_100.bin.zstd", - "snapshot_150.bin.zstd", - "snapshot_200.bin.zstd", - ] - ) + # Keeper sends snapshots asynchornously, hence we need to retry. + @retry(AssertionError, tries=10, delay=2) + def _check_snapshots(): + assert set(get_saved_snapshots()) == set( + [ + "snapshot_50.bin.zstd", + "snapshot_100.bin.zstd", + "snapshot_150.bin.zstd", + "snapshot_200.bin.zstd", + ] + ) + + _check_snapshots() destroy_zk_client(node1_zk) node1.stop_clickhouse(kill=True) @@ -108,9 +113,11 @@ def test_s3_upload(started_cluster): for _ in range(200): node2_zk.create("/test", sequence=True) - saved_snapshots = get_saved_snapshots() + @retry(AssertionError, tries=10, delay=2) + def _check_snapshots_without_quorum(): + assert len(get_saved_snapshots()) > 4 - assert len(saved_snapshots) > 4 + _check_snapshots_without_quorum() success_upload_message = "Successfully uploaded" assert node2.contains_in_log(success_upload_message) or node3.contains_in_log( From b2665031dcbaabb2a16b36459eee2b961d8f81f7 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 5 Jul 2023 11:55:34 +0800 Subject: [PATCH 0546/2047] review fix --- docs/en/interfaces/formats.md | 4 ++-- docs/en/operations/settings/settings-formats.md | 6 +++--- src/Core/Settings.h | 4 ++-- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 4 ++-- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 4 ++-- .../0_stateless/02811_csv_input_field_type_mismatch.sh | 2 +- 7 files changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ab6e514b4b2..2207a6f0442 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,8 +472,8 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_allow_check_deserialize](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_check_deserialize) - Allow to check whether the csv input field can be deserialized. Default value - `false`. -- [input_format_csv_allow_set_default_if_deserialize_failed](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_set_default_if_deserialize_failed) - Allow to set default value to column if the csv input field deserialize failed. Default value - `false`. +- [input_format_csv_allow_check_deserialize](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_check_deserialize) - Allow to check whether the csv input field can be deserialized. Default value - `false`. +- [input_format_csv_set_default_if_deserialize_failed](/docs/en/operations/settings/settings-formats.md/#input_format_csv_set_default_if_deserialize_failed) - Set default value to column if the csv input field deserialize failed. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index bb2159210d4..b74504ac5fd 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -969,13 +969,13 @@ Result a b ``` -### input_format_csv_allow_check_deserialize_result {#input_format_csv_allow_check_deserialize_result} +### input_format_csv_allow_check_deserialize {#input_format_csv_allow_check_deserialize} Allow to use whitespace or tab as field delimiter in CSV strings. Default value: `false`. -### input_format_csv_allow_set_column_default_value_if_deserialize_failed {#input_format_csv_allow_set_column_default_value_if_deserialize_failed} +### input_format_csv_set_default_if_deserialize_failed {#input_format_csv_set_default_if_deserialize_failed} Allow to set default value to column if the csv input field deserialize failed @@ -988,7 +988,7 @@ Query ```bash echo 'a,b,c' > 1.txt ./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize_result=true, input_format_csv_allow_set_column_default_value_if_deserialize_failed=true FORMAT CSV" +cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize=true, input_format_csv_set_default_if_deserialize_failed=true FORMAT CSV" ./clickhouse local -q "select * from test_tbl" ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b67745d1f59..5ea355c61c2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,8 +872,8 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_allow_check_deserialize_result, false, "Allow to check the csv input field deserialize whether success or not.", 0) \ - M(Bool, input_format_csv_allow_set_column_default_value_if_deserialize_failed, false, "All to set column default value if the input field deserialize failed.", 0) \ + M(Bool, input_format_csv_allow_check_deserialize, false, "Allow to check the csv input field deserialize whether success or not.", 0) \ + M(Bool, input_format_csv_set_default_if_deserialize_failed, false, "All to set column default value if the input field deserialize failed.", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 8a2a7e97941..ac7ceb7e791 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -74,7 +74,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; format_settings.csv.allow_check_deserialize = settings.input_format_csv_allow_check_deserialize; - format_settings.csv.allow_set_default_if_deserialize_failed = settings.input_format_csv_allow_set_default_if_deserialize_failed; + format_settings.csv.set_default_if_deserialize_failed = settings.input_format_csv_set_default_if_deserialize_failed; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 50ebdc1c67f..e9de7a51108 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -141,8 +141,8 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; - bool allow_check_deserialize = false; - bool allow_set_default_if_deserialize_failed=false; + bool allow_check_deserialize=false; + bool set_default_if_deserialize_failed=false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 8c4660fe4f9..a0709121d13 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -320,7 +320,7 @@ bool CSVFormatReader::readField( size_t col_size = column.size(); try { - if (format_settings.csv.allow_check_deserialize_result) + if (format_settings.csv.allow_check_deserialize) { std::string field; readCSVField(field, *buf, format_settings.csv); @@ -329,7 +329,7 @@ bool CSVFormatReader::readField( SerializationNullable::deserializeTextCSVImpl(column, tmp, format_settings, serialization); else serialization->deserializeTextCSV(column, tmp, format_settings); - if (column.size() == col_size + 1 && field.size() > 0 && tmp.position() != tmp.buffer().end()) + if (column.size() == col_size + 1 && field.size() > 0 && !tm.eof()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Text CSV deserialize field bytes logical error."); } else diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh index 246905a5b18..df736ea6792 100644 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -8,6 +8,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" $CLICKHOUSE_CLIENT -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat $CURDIR/data_csv/csv_with_diff_field_types.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize_result=true, input_format_csv_allow_set_column_default_value_if_deserialize_failed=true FORMAT CSV" +cat $CURDIR/data_csv/csv_with_diff_field_types.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize=true, input_format_csv_set_default_if_deserialize_failed=true FORMAT CSV" $CLICKHOUSE_CLIENT -q "select * from test_tbl" $CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file From 4a10c7286e9fa95694284968be1f39fffcb29b6c Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 13 Jul 2023 12:41:17 +0000 Subject: [PATCH 0547/2047] Add name to logger in queue --- src/Common/SystemLogBase.cpp | 19 ++++++++++++++----- src/Common/SystemLogBase.h | 5 ++++- src/Interpreters/SystemLog.cpp | 4 ++-- src/Interpreters/TextLog.h | 2 +- src/Loggers/Loggers.cpp | 17 ++++++++--------- 5 files changed, 29 insertions(+), 18 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 09f4c050f15..8285f133d73 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -39,12 +39,14 @@ namespace ISystemLog::~ISystemLog() = default; template -SystemLogBase::SystemLogBase(std::shared_ptr> ex_queue) +SystemLogBase::SystemLogBase( + const String & name_, + std::shared_ptr> queue_) { - if (ex_queue) - queue = ex_queue; + if (queue_) + queue = queue_; else - queue = std::make_shared>(); + queue = std::make_shared>(name_); } template @@ -75,6 +77,11 @@ void SystemLogBase::startup() saving_thread = std::make_unique([this] { savingThreadFunction(); }); } +template +SystemLogQueue::SystemLogQueue(const String & name_) + : log(&Poco::Logger::get(name_)) +{} + static thread_local bool recursive_add_call = false; template @@ -92,7 +99,6 @@ void SystemLogQueue::add(const LogElement & element) /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; - /// Should not log messages under mutex. bool queue_is_half_full = false; @@ -194,4 +200,7 @@ void SystemLogBase::flush(bool force) #define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) +#define INSTANTIATE_SYSTEM_LOG_BASE2(ELEMENT) template class SystemLogQueue; +SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE2) + } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 221d9946b48..3b1b848369b 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -74,6 +74,7 @@ template class SystemLogQueue { public: + SystemLogQueue(const String & name_); void add(const LogElement & element); size_t size() const { return queue.size(); } @@ -109,7 +110,9 @@ class SystemLogBase : public ISystemLog public: using Self = SystemLogBase; - SystemLogBase(std::shared_ptr> ex_queue = nullptr); + SystemLogBase( + const String & name_, + std::shared_ptr> queue_ = nullptr); /** Append a record into log. * Writing to table will be done asynchronously and in case of failure, record could be lost. diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index cbf355d020a..b77cb2311d5 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -333,8 +333,8 @@ SystemLog::SystemLog( const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_, - std::shared_ptr> ex_queue) - : Base(ex_queue) + std::shared_ptr> queue_) + : Base(database_name_ + "." + table_name_, queue_) , WithContext(context_) , table_id(database_name_, table_name_) , storage_def(storage_def_) diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 33c38da2f8f..8390259e147 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -49,7 +49,7 @@ public: static std::shared_ptr> getLogQueue() { - static std::shared_ptr> queue = std::make_shared>(); + static std::shared_ptr> queue = std::make_shared>("text_log"); return queue; } }; diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 2fda836e7b7..fa143440cc2 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -49,15 +49,6 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log /// Use extended interface of Channel for more comprehensive logging. split = new DB::OwnSplitChannel(); -#ifndef WITHOUT_TEXT_LOG - if (config.has("text_log")) - { - String text_log_level_str = config.getString("text_log.level", ""); - int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); - split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); - } -#endif - auto log_level_string = config.getString("logger.level", "trace"); /// different channels (log, console, syslog) may have different loglevels configured @@ -258,6 +249,14 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log } } } +#ifndef WITHOUT_TEXT_LOG + if (config.has("text_log")) + { + String text_log_level_str = config.getString("text_log.level", ""); + int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); + split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); + } +#endif } void Loggers::updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger) From 96912b12bcd4e6ae453ebc7ad950db6f5e83a64b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 13 Jul 2023 13:08:36 +0000 Subject: [PATCH 0548/2047] forgot to commit --- .../02813_create_index_noop.reference | 0 .../0_stateless/02813_create_index_noop.sql | 1000 +++++++++++++++++ 2 files changed, 1000 insertions(+) create mode 100644 tests/queries/0_stateless/02813_create_index_noop.reference create mode 100644 tests/queries/0_stateless/02813_create_index_noop.sql diff --git a/tests/queries/0_stateless/02813_create_index_noop.reference b/tests/queries/0_stateless/02813_create_index_noop.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02813_create_index_noop.sql b/tests/queries/0_stateless/02813_create_index_noop.sql new file mode 100644 index 00000000000..3d65f81af9d --- /dev/null +++ b/tests/queries/0_stateless/02813_create_index_noop.sql @@ -0,0 +1,1000 @@ +SET allow_create_index_without_type=0; +CREATE INDEX idx_tab1_0 on tab1 (col0); -- { serverError INCORRECT_QUERY } +SET allow_create_index_without_type=1; +CREATE INDEX idx_tab1_0 on tab1 (col0); +CREATE INDEX idx_tab1_1 on tab1 (col1); +CREATE INDEX idx_tab1_3 on tab1 (col3); +CREATE INDEX idx_tab1_4 on tab1 (col4); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0 DESC,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col0); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col1 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col3,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col3,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col0,col4,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col0,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col0,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col3 DESC,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1 DESC,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col1); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col0,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col4 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col1,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col0,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col0,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col1,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col4 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col3 DESC,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col3); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col0 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col1 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col3,col4); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col0,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col0,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col1 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col3 DESC,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col4 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col4); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col0 DESC,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col0); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col1 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col1 DESC,col3); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col1); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col3 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_0 ON tab2 (col4,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col3 DESC,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col3,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0 DESC,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col0); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col1,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col3 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col3 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col3 DESC,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col0,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col0 DESC,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1 DESC,col4 DESC,col3,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col3 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col3,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col1,col4,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col0 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col0 DESC,col4 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col0,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col3,col4); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4 DESC,col3); +CREATE INDEX idx_tab2_1 ON tab2 (col4); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col0); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col1 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col1); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col3 DESC); +CREATE INDEX idx_tab2_1 ON tab2 (col4,col3 DESC,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col3,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0 DESC,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col0); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col1 DESC,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col4 DESC,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col0,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col0 DESC,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col0,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col0,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col3 DESC,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1 DESC,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col1); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col3,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col1,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col3 DESC,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col3); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col1 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col1 DESC,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col3,col4); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4 DESC,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col4); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col0,col3); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col1 DESC,col0); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col1); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col3 DESC); +CREATE INDEX idx_tab2_2 ON tab2 (col4,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col0 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col0 DESC,col4 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col0); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col1); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col4 DESC,col1 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col4 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col0,col4); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col0,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col4 DESC,col0); +CREATE INDEX idx_tab2_3 ON tab2 (col1 DESC,col4 DESC,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col1); +CREATE INDEX idx_tab2_3 ON tab2 (col1,col0 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1,col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1,col3); +CREATE INDEX idx_tab2_3 ON tab2 (col1,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col1,col4); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col0 DESC,col4); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col0); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col1,col4); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3 DESC,col4,col1); +CREATE INDEX idx_tab2_3 ON tab2 (col3); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col0 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col1); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col3,col4); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col0 DESC,col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col0,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_3 ON tab2 (col4 DESC,col3,col0); +CREATE INDEX idx_tab2_3 ON tab2 (col4); +CREATE INDEX idx_tab2_3 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4,col1 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4,col3 DESC); +CREATE INDEX idx_tab2_3 ON tab2 (col4,col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col1); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col4 DESC,col3); +CREATE INDEX idx_tab2_4 ON tab2 (col0 DESC,col4); +CREATE INDEX idx_tab2_4 ON tab2 (col0); +CREATE INDEX idx_tab2_4 ON tab2 (col0,col3 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col0,col3); +CREATE INDEX idx_tab2_4 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1); +CREATE INDEX idx_tab2_4 ON tab2 (col1,col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col1,col4); +CREATE INDEX idx_tab2_4 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_4 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col3 DESC,col4); +CREATE INDEX idx_tab2_4 ON tab2 (col3); +CREATE INDEX idx_tab2_4 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_4 ON tab2 (col3,col1); +CREATE INDEX idx_tab2_4 ON tab2 (col3,col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col3,col4); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col3); +CREATE INDEX idx_tab2_4 ON tab2 (col4 DESC,col3,col0 DESC,col1 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4); +CREATE INDEX idx_tab2_4 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4,col0); +CREATE INDEX idx_tab2_4 ON tab2 (col4,col1 DESC); +CREATE INDEX idx_tab2_4 ON tab2 (col4,col1); +CREATE INDEX idx_tab2_4 ON tab2 (col4,col3); +CREATE INDEX idx_tab2_5 ON tab2 (col0 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col0 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col0 DESC,col3); +CREATE INDEX idx_tab2_5 ON tab2 (col0); +CREATE INDEX idx_tab2_5 ON tab2 (col0,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col0,col1); +CREATE INDEX idx_tab2_5 ON tab2 (col0,col4 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col1 DESC,col3); +CREATE INDEX idx_tab2_5 ON tab2 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col1 DESC,col4); +CREATE INDEX idx_tab2_5 ON tab2 (col1); +CREATE INDEX idx_tab2_5 ON tab2 (col1,col0); +CREATE INDEX idx_tab2_5 ON tab2 (col1,col3 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col1,col3); +CREATE INDEX idx_tab2_5 ON tab2 (col1,col4 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC,col1); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3 DESC,col4,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col3); +CREATE INDEX idx_tab2_5 ON tab2 (col3,col0); +CREATE INDEX idx_tab2_5 ON tab2 (col4 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4 DESC,col1); +CREATE INDEX idx_tab2_5 ON tab2 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4 DESC,col3); +CREATE INDEX idx_tab2_5 ON tab2 (col4); +CREATE INDEX idx_tab2_5 ON tab2 (col4,col0 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4,col0); +CREATE INDEX idx_tab2_5 ON tab2 (col4,col0,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4,col1 DESC); +CREATE INDEX idx_tab2_5 ON tab2 (col4,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col1 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col1,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col1,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col3,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col4 DESC,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col4 DESC,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col0 DESC,col4,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col0); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col4 DESC,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col0,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col0 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col3,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col1 DESC,col4,col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col0,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col3 DESC,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col1,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col1,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col3); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col1 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col1,col0 DESC,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col3,col4); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col0,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col1,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col4 DESC,col3,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col0 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col1 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col1 DESC,col0); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col3 DESC); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col3); +CREATE INDEX idx_tab3_0 ON tab3 (col4,col3,col1,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col3,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col0); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col1,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col3); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col4 DESC,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col0,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col0,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col3); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col3,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col1); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col0 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col3 DESC,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col3); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col1,col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col0 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col0,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col3 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col3); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col0 DESC,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col0,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col3,col4); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col1,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col1 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col1,col0); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col3 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_1 ON tab3 (col4,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0 DESC,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col0); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col0,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1 DESC,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col1); +CREATE INDEX idx_tab3_2 ON tab3 (col1,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col1,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col1,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col1,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col1,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col1,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col4 DESC,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col3 DESC,col4,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col3); +CREATE INDEX idx_tab3_2 ON tab3 (col3,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3,col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col3,col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col3,col4); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col0 DESC,col3); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col0,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4 DESC,col3,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col4); +CREATE INDEX idx_tab3_2 ON tab3 (col4,col0 DESC,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_2 ON tab3 (col4,col1 DESC); +CREATE INDEX idx_tab3_2 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_2 ON tab3 (col4,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC,col3,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col0 DESC,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col0); +CREATE INDEX idx_tab3_3 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_3 ON tab3 (col0,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col0,col3); +CREATE INDEX idx_tab3_3 ON tab3 (col0,col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC,col3); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1 DESC,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col1); +CREATE INDEX idx_tab3_3 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_3 ON tab3 (col1,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1,col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1,col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col1,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col0); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col1 DESC,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col3 DESC,col4); +CREATE INDEX idx_tab3_3 ON tab3 (col3); +CREATE INDEX idx_tab3_3 ON tab3 (col3,col0 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col3,col0); +CREATE INDEX idx_tab3_3 ON tab3 (col3,col0,col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col3,col4,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4 DESC,col0); +CREATE INDEX idx_tab3_3 ON tab3 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4 DESC,col3); +CREATE INDEX idx_tab3_3 ON tab3 (col4); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col0 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col1 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col3 DESC); +CREATE INDEX idx_tab3_3 ON tab3 (col4,col3); +CREATE INDEX idx_tab3_4 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col0 DESC,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_4 ON tab3 (col0 DESC,col4); +CREATE INDEX idx_tab3_4 ON tab3 (col0); +CREATE INDEX idx_tab3_4 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col0,col3); +CREATE INDEX idx_tab3_4 ON tab3 (col0,col4 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col0,col4); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col4 DESC,col0); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col4); +CREATE INDEX idx_tab3_4 ON tab3 (col1 DESC,col4,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col1); +CREATE INDEX idx_tab3_4 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_4 ON tab3 (col1,col4 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col3 DESC,col1,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col3 DESC,col4,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col3); +CREATE INDEX idx_tab3_4 ON tab3 (col3,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col3,col4 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col4 DESC,col0 DESC,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col4 DESC,col0); +CREATE INDEX idx_tab3_4 ON tab3 (col4); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col0,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col1 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col1,col0 DESC); +CREATE INDEX idx_tab3_4 ON tab3 (col4,col3); +CREATE INDEX idx_tab3_5 ON tab3 (col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col0 DESC,col1); +CREATE INDEX idx_tab3_5 ON tab3 (col0 DESC,col3); +CREATE INDEX idx_tab3_5 ON tab3 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col0); +CREATE INDEX idx_tab3_5 ON tab3 (col0,col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col0,col1); +CREATE INDEX idx_tab3_5 ON tab3 (col0,col1,col3); +CREATE INDEX idx_tab3_5 ON tab3 (col0,col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col0,col4); +CREATE INDEX idx_tab3_5 ON tab3 (col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col1 DESC,col3); +CREATE INDEX idx_tab3_5 ON tab3 (col1); +CREATE INDEX idx_tab3_5 ON tab3 (col1,col0); +CREATE INDEX idx_tab3_5 ON tab3 (col1,col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col1,col4 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col1,col4); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC,col1); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC,col1,col0); +CREATE INDEX idx_tab3_5 ON tab3 (col3 DESC,col4); +CREATE INDEX idx_tab3_5 ON tab3 (col3); +CREATE INDEX idx_tab3_5 ON tab3 (col3,col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col3,col0 DESC,col4); +CREATE INDEX idx_tab3_5 ON tab3 (col3,col0); +CREATE INDEX idx_tab3_5 ON tab3 (col3,col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col3,col4); +CREATE INDEX idx_tab3_5 ON tab3 (col4 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4 DESC,col0 DESC,col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4); +CREATE INDEX idx_tab3_5 ON tab3 (col4,col0 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4,col0); +CREATE INDEX idx_tab3_5 ON tab3 (col4,col1); +CREATE INDEX idx_tab3_5 ON tab3 (col4,col3 DESC); +CREATE INDEX idx_tab3_5 ON tab3 (col4,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col1,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col1,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col3,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0 DESC,col4 DESC,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col0); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col1,col4 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col3,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col3,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col0,col4,col1,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col3,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col4 DESC,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col4 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col1 DESC,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col1); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col0 DESC,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col0,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col3 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col1,col4,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col1 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col4 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col3 DESC,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col3); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col0 DESC,col1,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col0,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col1 DESC,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col4); +CREATE INDEX idx_tab4_0 ON tab4 (col3,col4,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col0,col1 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col1 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col3 DESC,col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4 DESC,col3); +CREATE INDEX idx_tab4_0 ON tab4 (col4); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col0 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col0); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col0,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col1 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col1); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col3 DESC); +CREATE INDEX idx_tab4_0 ON tab4 (col4,col3); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC,col3); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0 DESC,col4); +CREATE INDEX idx_tab4_1 ON tab4 (col0); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col1,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col3); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col3,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col0,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col3 DESC,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1 DESC,col4); +CREATE INDEX idx_tab4_1 ON tab4 (col1); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col3); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col3,col4 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col4 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col4); +CREATE INDEX idx_tab4_1 ON tab4 (col1,col4,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col4); +CREATE INDEX idx_tab4_1 ON tab4 (col3 DESC,col4,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col3); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col1 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col3,col4); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col0 DESC,col3 DESC,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col0); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col0,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col1); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col3 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col3 DESC,col0,col1 DESC); +CREATE INDEX idx_tab4_1 ON tab4 (col4,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col0 DESC,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col0 DESC,col4); +CREATE INDEX idx_tab4_2 ON tab4 (col0); +CREATE INDEX idx_tab4_2 ON tab4 (col0,col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col0,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col0,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col0,col4); +CREATE INDEX idx_tab4_2 ON tab4 (col0,col4,col1 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1 DESC,col4 DESC,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col0,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col0,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col1,col4); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col0); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3 DESC,col4); +CREATE INDEX idx_tab4_2 ON tab4 (col3); +CREATE INDEX idx_tab4_2 ON tab4 (col3,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_2 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col3,col4); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col1); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col4 DESC,col3,col0 DESC); +CREATE INDEX idx_tab4_2 ON tab4 (col4); +CREATE INDEX idx_tab4_2 ON tab4 (col4,col0,col1); +CREATE INDEX idx_tab4_2 ON tab4 (col4,col0,col3); +CREATE INDEX idx_tab4_2 ON tab4 (col4,col1); +CREATE INDEX idx_tab4_2 ON tab4 (col4,col3 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0 DESC,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0 DESC,col3); +CREATE INDEX idx_tab4_3 ON tab4 (col0); +CREATE INDEX idx_tab4_3 ON tab4 (col0,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col0,col3 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0,col3,col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col0,col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col3); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col1 DESC,col4); +CREATE INDEX idx_tab4_3 ON tab4 (col1); +CREATE INDEX idx_tab4_3 ON tab4 (col1,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col1,col4); +CREATE INDEX idx_tab4_3 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col3 DESC,col1,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col3); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col0 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col3,col4); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col3); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col3,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col4 DESC,col3,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col4); +CREATE INDEX idx_tab4_3 ON tab4 (col4,col0 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col4,col0); +CREATE INDEX idx_tab4_3 ON tab4 (col4,col1 DESC); +CREATE INDEX idx_tab4_3 ON tab4 (col4,col1); +CREATE INDEX idx_tab4_3 ON tab4 (col4,col3); +CREATE INDEX idx_tab4_4 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col0 DESC,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col0 DESC,col3); +CREATE INDEX idx_tab4_4 ON tab4 (col0); +CREATE INDEX idx_tab4_4 ON tab4 (col0,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col0,col1,col3 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col0,col3); +CREATE INDEX idx_tab4_4 ON tab4 (col0,col4); +CREATE INDEX idx_tab4_4 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col1 DESC,col0); +CREATE INDEX idx_tab4_4 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col1); +CREATE INDEX idx_tab4_4 ON tab4 (col1,col0 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col1,col3); +CREATE INDEX idx_tab4_4 ON tab4 (col1,col4 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC,col0); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3 DESC,col4 DESC,col0 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3); +CREATE INDEX idx_tab4_4 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_4 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col3,col4 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col0); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col3 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col3 DESC,col0 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col3); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col3,col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4 DESC,col3,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col4); +CREATE INDEX idx_tab4_4 ON tab4 (col4,col0); +CREATE INDEX idx_tab4_4 ON tab4 (col4,col1); +CREATE INDEX idx_tab4_4 ON tab4 (col4,col3 DESC); +CREATE INDEX idx_tab4_4 ON tab4 (col4,col3 DESC,col0); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col1); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col3 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0 DESC,col4 DESC,col3); +CREATE INDEX idx_tab4_5 ON tab4 (col0); +CREATE INDEX idx_tab4_5 ON tab4 (col0,col3 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col0,col3); +CREATE INDEX idx_tab4_5 ON tab4 (col0,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC,col0); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC,col3 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC,col3); +CREATE INDEX idx_tab4_5 ON tab4 (col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1); +CREATE INDEX idx_tab4_5 ON tab4 (col1,col0 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1,col3 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col1,col4); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC,col1 DESC,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC,col1); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC,col1,col4); +CREATE INDEX idx_tab4_5 ON tab4 (col3 DESC,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col3); +CREATE INDEX idx_tab4_5 ON tab4 (col3,col0); +CREATE INDEX idx_tab4_5 ON tab4 (col3,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col3,col1); +CREATE INDEX idx_tab4_5 ON tab4 (col3,col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4 DESC,col1 DESC,col0 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4 DESC,col3); +CREATE INDEX idx_tab4_5 ON tab4 (col4); +CREATE INDEX idx_tab4_5 ON tab4 (col4,col0 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4,col1 DESC,col3); +CREATE INDEX idx_tab4_5 ON tab4 (col4,col3 DESC,col1 DESC); +CREATE INDEX idx_tab4_5 ON tab4 (col4,col3); \ No newline at end of file From 7e1b5d899744bbd1162e0bb4ba22851c576068ff Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 13 Jul 2023 17:21:23 +0200 Subject: [PATCH 0549/2047] updated docs for toYearWeek() --- .../functions/date-time-functions.md | 14 +++++++++----- .../functions/date-time-functions.md | 15 +++++++++------ 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 3f61e7a214d..4717813a56d 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -694,10 +694,14 @@ SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS we Returns year and week for a date. The year in the result may be different from the year in the date argument for the first and the last week of the year. -The mode argument works exactly like the mode argument to `toWeek()`. For the single-argument syntax, a mode value of 0 is used. +The mode argument works like the mode argument to `toWeek()`. For the single-argument syntax, a mode value of 0 is used. `toISOYear()` is a compatibility function that is equivalent to `intDiv(toYearWeek(date,3),100)`. +:::warning +The week number returned by `toYearWeek()` can be different from what the `toWeek()` returns. `toWeek()` always returns week number in the context of the given year, and in case `toWeek()` returns `0`, `toYearWeek()` returns the value corresponding to the last week of previous year. See `prev_yearWeek` in example below. + + **Syntax** ``` sql @@ -707,13 +711,13 @@ toYearWeek(t[, mode[, timezone]]) **Example** ``` sql -SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9; +SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9, toYearWeek(toDate('2022-01-01')) AS prev_yearWeek; ``` ``` text -┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┐ -│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ -└────────────┴───────────┴───────────┴───────────┘ +┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┬─prev_yearWeek─┐ +│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ 202152 │ +└────────────┴───────────┴───────────┴───────────┴───────────────┘ ``` ## age diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 17ab04b7799..d251163df8d 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -599,24 +599,27 @@ SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS we ## toYearWeek(date[,mode]) {#toyearweek} Возвращает год и неделю для даты. Год в результате может отличаться от года в аргументе даты для первой и последней недели года. -Аргумент mode работает точно так же, как аргумент mode [toWeek()](#toweek). Если mode не задан, используется режим 0. +Аргумент mode работает так же, как аргумент mode [toWeek()](#toweek), значение mode по умолчанию -- `0`. -`toISOYear() ` эквивалентно `intDiv(toYearWeek(date,3),100)`. +`toISOYear() ` эквивалентно `intDiv(toYearWeek(date,3),100)` + +:::warning +Однако, есть отличие в работе функций `toWeek()` и `toYearWeek()`. `toWeek()` возвращает номер недели в контексте заданного года, и в случае, когда `toWeek()` вернёт `0`, `toYearWeek()` вернёт значение, соответствующее последней неделе предыдущего года (см. `prev_yearWeek` в примере). **Пример** Запрос: ```sql -SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9; +SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9, toYearWeek(toDate('2022-01-01')) AS prev_yearWeek; ``` Результат: ```text -┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┐ -│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ -└────────────┴───────────┴───────────┴───────────┘ +┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┬─prev_yearWeek─┐ +│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ 202152 │ +└────────────┴───────────┴───────────┴───────────┴───────────────┘ ``` ## age From a87474cee1aeb924cb11c043308c36888f6a9a26 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 13 Jul 2023 17:25:49 +0200 Subject: [PATCH 0550/2047] rm \n --- docs/en/sql-reference/functions/date-time-functions.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4717813a56d..ddd6a367e31 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -701,7 +701,6 @@ The mode argument works like the mode argument to `toWeek()`. For the single-arg :::warning The week number returned by `toYearWeek()` can be different from what the `toWeek()` returns. `toWeek()` always returns week number in the context of the given year, and in case `toWeek()` returns `0`, `toYearWeek()` returns the value corresponding to the last week of previous year. See `prev_yearWeek` in example below. - **Syntax** ``` sql From 6c3b7f76d485205eacaa580a98259f13b435f8d1 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 13 Jul 2023 16:00:01 +0000 Subject: [PATCH 0551/2047] Add S3 storage class as 4th parameter --- src/Backups/BackupIO_S3.cpp | 3 ++- src/Backups/BackupIO_S3.h | 2 +- src/Backups/registerBackupEngineS3.cpp | 10 +++++++--- src/Storages/StorageS3Settings.h | 2 ++ 4 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 9a2a457e13e..14c568719c3 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -177,7 +177,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s BackupWriterS3::BackupWriterS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const String & storage_class_name, const ContextPtr & context_) : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) @@ -186,6 +186,7 @@ BackupWriterS3::BackupWriterS3( { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint + request_settings.setStorageClassName(storage_class_name); } void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index cca56bae6bc..3a84c853f2e 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -38,7 +38,7 @@ private: class BackupWriterS3 : public BackupWriterDefault { public: - BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_); + BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const String & storage_class_name, const ContextPtr & context_); ~BackupWriterS3() override; bool fileExists(const String & file_name) override; diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 8387b4627d5..4f5a0d3e865 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -52,6 +52,7 @@ void registerBackupEngineS3(BackupFactory & factory) const auto & args = params.backup_info.args; String s3_uri, access_key_id, secret_access_key; + String storage_class; if (!id_arg.empty()) { @@ -76,15 +77,17 @@ void registerBackupEngineS3(BackupFactory & factory) } else { - if ((args.size() != 1) && (args.size() != 3)) + if ((args.size() != 1) && (args.size() != 3) && (args.size() != 4)) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Backup S3 requires 1 or 3 arguments: url, [access_key_id, secret_access_key]"); + "Backup S3 requires 1, 3 or 4 arguments: url, [access_key_id, secret_access_key, [storage_class]]"); s3_uri = args[0].safeGet(); if (args.size() >= 3) { access_key_id = args[1].safeGet(); secret_access_key = args[2].safeGet(); + if (args.size() == 4) + storage_class = args[3].safeGet(); } } @@ -112,7 +115,8 @@ void registerBackupEngineS3(BackupFactory & factory) } else { - auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.context); + auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, storage_class, params.context); + return std::make_unique( backup_name_for_logging, archive_params, diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 991e323acb6..63a588023ea 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -76,6 +76,8 @@ struct S3Settings const PartUploadSettings & getUploadSettings() const { return upload_settings; } + void setStorageClassName(const String & storage_class_name) { upload_settings.storage_class_name = storage_class_name; } + RequestSettings() = default; explicit RequestSettings(const Settings & settings); explicit RequestSettings(const NamedCollection & collection); From 0de7fb5c620eb6d7ee010f2053737f95577b77f8 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 13 Jul 2023 20:12:59 +0300 Subject: [PATCH 0552/2047] typofix1 --- docs/en/sql-reference/functions/date-time-functions.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index fa5826fbdda..ce1a4f4d283 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -700,6 +700,7 @@ The mode argument works like the mode argument to `toWeek()`. For the single-arg :::warning The week number returned by `toYearWeek()` can be different from what the `toWeek()` returns. `toWeek()` always returns week number in the context of the given year, and in case `toWeek()` returns `0`, `toYearWeek()` returns the value corresponding to the last week of previous year. See `prev_yearWeek` in example below. +::: **Syntax** From 54723d4822438cae9dc009c5b28e4f5fbadd1741 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 13 Jul 2023 20:14:06 +0300 Subject: [PATCH 0553/2047] typofix2 --- docs/ru/sql-reference/functions/date-time-functions.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 8f31fcebf8b..4db8a1ec6f8 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -605,6 +605,7 @@ SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS we :::warning Однако, есть отличие в работе функций `toWeek()` и `toYearWeek()`. `toWeek()` возвращает номер недели в контексте заданного года, и в случае, когда `toWeek()` вернёт `0`, `toYearWeek()` вернёт значение, соответствующее последней неделе предыдущего года (см. `prev_yearWeek` в примере). +::: **Пример** From f7505f0c7749d06f53dbfaf345f00127ca5ad012 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 13 Jul 2023 19:41:45 +0200 Subject: [PATCH 0554/2047] update docs --- docs/en/sql-reference/data-types/datetime.md | 1 + docs/en/sql-reference/data-types/datetime64.md | 1 + .../sql-reference/functions/type-conversion-functions.md | 6 +++++- docs/ru/sql-reference/data-types/datetime.md | 1 + docs/ru/sql-reference/data-types/datetime64.md | 1 + .../sql-reference/functions/type-conversion-functions.md | 8 +++++++- 6 files changed, 16 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index 0da273e01ad..fe279edb709 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -143,5 +143,6 @@ Time shifts for multiple days. Some pacific islands changed their timezone offse - [The `date_time_input_format` setting](../../operations/settings/settings.md#settings-date_time_input_format) - [The `date_time_output_format` setting](../../operations/settings/settings.md#settings-date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [The `session_timezone` setting](../../operations/settings/settings.md#session_timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-datetime) - [The `Date` data type](../../sql-reference/data-types/date.md) diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index 793691850b1..3b80e8b1a8b 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -119,6 +119,7 @@ FROM dt; - [The `date_time_input_format` setting](../../operations/settings/settings-formats.md#date_time_input_format) - [The `date_time_output_format` setting](../../operations/settings/settings-formats.md#date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [The `session_timezone` setting](../../operations/settings/settings.md#session_timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-for-working-with-dates-and-times) - [`Date` data type](../../sql-reference/data-types/date.md) - [`DateTime` data type](../../sql-reference/data-types/datetime.md) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 214c885bc0e..36f40b37238 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -399,7 +399,11 @@ toDateTime(expr[, time_zone ]) - `expr` — The value. [String](/docs/en/sql-reference/data-types/string.md), [Int](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md). - `time_zone` — Time zone. [String](/docs/en/sql-reference/data-types/string.md). -If `expr` is a number, it is interpreted as the number of seconds since the beginning of the Unix Epoch (as Unix timestamp). +:::note +If `expr` is a number, it is interpreted as the number of seconds since the beginning of the Unix Epoch (as Unix timestamp). +If `expr` is a [String](/docs/en/sql-reference/data-types/string.md), it may be interpreted as a Unix timestamp or as a string representation of date / date with time. +Thus, parsing of short numbers' string representations (up to 4 digits) is explicitly disabled due to ambiguity, e.g. a string `'1999'` may be both a year (an incomplete string representation of Date / DateTime) or a unix timestamp. Longer numeric strings are allowed. +::: **Returned value** diff --git a/docs/ru/sql-reference/data-types/datetime.md b/docs/ru/sql-reference/data-types/datetime.md index e8d4a3ee9fd..80d844a1713 100644 --- a/docs/ru/sql-reference/data-types/datetime.md +++ b/docs/ru/sql-reference/data-types/datetime.md @@ -122,6 +122,7 @@ FROM dt - [Настройка `date_time_input_format`](../../operations/settings/index.md#settings-date_time_input_format) - [Настройка `date_time_output_format`](../../operations/settings/index.md) - [Конфигурационный параметр сервера `timezone`](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [Параметр `session_timezone`](../../operations/settings/settings.md#session_timezone) - [Операторы для работы с датой и временем](../../sql-reference/operators/index.md#operators-datetime) - [Тип данных `Date`](date.md) - [Тип данных `DateTime64`](datetime64.md) diff --git a/docs/ru/sql-reference/data-types/datetime64.md b/docs/ru/sql-reference/data-types/datetime64.md index da2f81f4828..78ad43e4764 100644 --- a/docs/ru/sql-reference/data-types/datetime64.md +++ b/docs/ru/sql-reference/data-types/datetime64.md @@ -102,6 +102,7 @@ FROM dt; - [Настройка `date_time_input_format`](../../operations/settings/settings.md#settings-date_time_input_format) - [Настройка `date_time_output_format`](../../operations/settings/settings.md) - [Конфигурационный параметр сервера `timezone`](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [Параметр `session_timezone`](../../operations/settings/settings.md#session_timezone) - [Операторы для работы с датой и временем](../../sql-reference/operators/index.md#operators-datetime) - [Тип данных `Date`](date.md) - [Тип данных `DateTime`](datetime.md) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index d5e6246fe9e..e53104d8d71 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -284,7 +284,13 @@ toDateTime(expr[, time_zone ]) - `expr` — Значение для преобразования. [String](/docs/ru/sql-reference/data-types/string.md), [Int](/docs/ru/sql-reference/data-types/int-uint.md), [Date](/docs/ru/sql-reference/data-types/date.md) или [DateTime](/docs/ru/sql-reference/data-types/datetime.md). - `time_zone` — Часовой пояс. [String](/docs/ru/sql-reference/data-types/string.md). -Если `expr` является числом, оно интерпретируется как количество секунд от начала unix эпохи. +:::note +Если `expr` является числом, то оно интерпретируется как число секунд с начала Unix-эпохи (Unix Timestamp). + +Если же `expr` -- [строка (String)](/docs/ru/sql-reference/data-types/string.md), то оно может быть интерпретировано и как Unix Timestamp, и как строковое представление даты / даты со временем. +Ввиду неоднозначности запрещён парсинг строк длиной 4 и меньше. Так, строка `'1999'` могла бы представлять собой как год (неполное строковое представление даты или даты со временем), так и Unix Timestamp. +Строки длиной 5 символов и более не несут неоднозначности, а следовательно, их парсинг разрешён. +::: **Возвращаемое значение** From 2f17208792363f73db8fb750aa6e3e6f907bac37 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 13 Jul 2023 20:38:17 +0200 Subject: [PATCH 0555/2047] Add missing --force for docker network prune (otherwise it is noop on CI) On CI there is no stdin attached, so this command simply do nothing, fix this. Fixes: https://github.com/ClickHouse/ClickHouse/pull/51969 Signed-off-by: Azat Khuzhin --- tests/integration/conftest.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 5933883f7b0..a4e25e653b3 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -45,7 +45,7 @@ def cleanup_environment(): logging.debug("Pruning Docker networks") run_and_check( - ["docker network prune"], + ["docker network prune --force"], shell=True, nothrow=True, ) From c6b8097090a8c418667c43997ba0be36b41acee4 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 14 Jul 2023 11:24:38 +0800 Subject: [PATCH 0556/2047] rebase main --- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index a0709121d13..ada9d78711a 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -329,7 +329,7 @@ bool CSVFormatReader::readField( SerializationNullable::deserializeTextCSVImpl(column, tmp, format_settings, serialization); else serialization->deserializeTextCSV(column, tmp, format_settings); - if (column.size() == col_size + 1 && field.size() > 0 && !tm.eof()) + if (column.size() == col_size + 1 && field.size() > 0 && !tmp.eof()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Text CSV deserialize field bytes logical error."); } else @@ -346,7 +346,7 @@ bool CSVFormatReader::readField( catch (Exception & e) { LOG_DEBUG(&Poco::Logger::get("CSVRowInputFormat"), "Failed to deserialize CSV column, exception message:{}", e.what()); - if (format_settings.csv.allow_set_column_default_value_if_deserialize_failed) + if (format_settings.csv.set_default_if_deserialize_failed) { // Reset the column and buffer position, then skip the field and set column default value. if (column.size() == col_size + 1) From 575de8d497673f7db12ccaa893921957d6d6725e Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 14 Jul 2023 07:04:17 +0200 Subject: [PATCH 0557/2047] Fix typo last_removal_attemp_time --- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/System/StorageSystemParts.cpp | 4 ++-- .../0_stateless/02117_show_create_table_system.reference | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 2c0cf37b3a5..52f4e385a0e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -500,7 +500,7 @@ public: mutable std::atomic removal_state = DataPartRemovalState::NOT_ATTEMPTED; - mutable std::atomic last_removal_attemp_time = 0; + mutable std::atomic last_removal_attempt_time = 0; protected: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ec8ce3f5e3d..e8dbf48c7a2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2187,7 +2187,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force) const DataPartPtr & part = *it; - part->last_removal_attemp_time.store(time_now, std::memory_order_relaxed); + part->last_removal_attempt_time.store(time_now, std::memory_order_relaxed); /// Do not remove outdated part if it may be visible for some transaction if (!part->version.canBeRemoved()) diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 7399bd789a7..576b38c1584 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -119,7 +119,7 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"has_lightweight_delete", std::make_shared()}, - {"last_removal_attemp_time", std::make_shared()}, + {"last_removal_attempt_time", std::make_shared()}, {"removal_state", std::make_shared()}, } ) @@ -343,7 +343,7 @@ void StorageSystemParts::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(part->hasLightweightDelete()); if (columns_mask[src_index++]) - columns[res_index++]->insert(static_cast(part->last_removal_attemp_time.load(std::memory_order_relaxed))); + columns[res_index++]->insert(static_cast(part->last_removal_attempt_time.load(std::memory_order_relaxed))); if (columns_mask[src_index++]) columns[res_index++]->insert(getRemovalStateDescription(part->removal_state.load(std::memory_order_relaxed))); diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 3c221d6a473..ed7fe3bac42 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -516,7 +516,7 @@ CREATE TABLE system.parts `creation_csn` UInt64, `removal_csn` UInt64, `has_lightweight_delete` UInt8, - `last_removal_attemp_time` DateTime, + `last_removal_attempt_time` DateTime, `removal_state` String, `bytes` UInt64, `marks_size` UInt64, From c27f0691a8a5e06c1a9bf89045f71090d511108e Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 14 Jul 2023 05:38:55 +0000 Subject: [PATCH 0558/2047] Clean extra whitespace --- src/Backups/BackupIO_S3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 14c568719c3..ecd88788782 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -177,7 +177,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s BackupWriterS3::BackupWriterS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const String & storage_class_name, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const String & storage_class_name, const ContextPtr & context_) : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) From c4dabd1aae40acf7df982f49b887483ff1e1c3b8 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 14 Jul 2023 13:53:29 +0800 Subject: [PATCH 0559/2047] alias Signed-off-by: Lloyd-Pottiger --- src/Functions/now.cpp | 1 + src/Functions/today.cpp | 2 ++ 2 files changed, 3 insertions(+) diff --git a/src/Functions/now.cpp b/src/Functions/now.cpp index d3a94379a61..827b800a243 100644 --- a/src/Functions/now.cpp +++ b/src/Functions/now.cpp @@ -138,6 +138,7 @@ private: REGISTER_FUNCTION(Now) { factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerAlias("current_timestamp", NowOverloadResolver::name, FunctionFactory::CaseInsensitive); } } diff --git a/src/Functions/today.cpp b/src/Functions/today.cpp index f106e3992a8..16a5b98d7ec 100644 --- a/src/Functions/today.cpp +++ b/src/Functions/today.cpp @@ -86,6 +86,8 @@ public: REGISTER_FUNCTION(Today) { factory.registerFunction(); + factory.registerAlias("current_date", TodayOverloadResolver::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("curdate", TodayOverloadResolver::name, FunctionFactory::CaseInsensitive); } } From e4cbece6dd7ce2dfbb4f6a57345a6a57d3716aa8 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Fri, 14 Jul 2023 14:03:19 +0800 Subject: [PATCH 0560/2047] update docs Signed-off-by: Lloyd-Pottiger --- docs/en/sql-reference/functions/date-time-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 40ede6e0b62..5c199dd6a72 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1130,7 +1130,7 @@ Result: └──────────────────────────────────────────────────────────────┘ ``` -## now +## now/current_timestamp Returns the current date and time at the moment of query analysis. The function is a constant expression. @@ -1259,7 +1259,7 @@ Result: └─────────────────────┴─────────────────────┴──────────┘ ``` -## today +## today/curdate/current_date Accepts zero arguments and returns the current date at one of the moments of query analysis. The same as ‘toDate(now())’. From 8ea3bf4ade73ef3188026d5ac64db48ab7fa09a5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 14 Jul 2023 16:09:22 +0800 Subject: [PATCH 0561/2047] improve ch to arrow --- src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index c3685e813d3..899b84cc132 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -208,7 +208,7 @@ namespace DB const String & column_name, ColumnPtr & column, const DataTypePtr & column_type, - const PaddedPODArray * null_bytemap, + const PaddedPODArray *, arrow::ArrayBuilder * array_builder, String format_name, size_t start, @@ -231,7 +231,9 @@ namespace DB /// Start new array. components_status = builder.Append(); checkStatus(components_status, nested_column->getName(), format_name); - fillArrowArray(column_name, nested_column, nested_type, null_bytemap, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); + + /// Pass null null_map, because fillArrowArray will decide whether nested_type is nullable, if nullable, it will create a new null_map from nested_column + fillArrowArray(column_name, nested_column, nested_type, nullptr, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } } From f153d6aa3c69b5d7a6e6df87ffdb84b1632e5abe Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 14 Jul 2023 10:02:18 +0000 Subject: [PATCH 0562/2047] Fix shutdown --- src/Common/SystemLogBase.cpp | 13 +++++++------ src/Common/SystemLogBase.h | 8 ++++---- src/Loggers/OwnSplitChannel.h | 6 +++--- 3 files changed, 14 insertions(+), 13 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 211bd457714..7373786e514 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -62,6 +62,7 @@ void SystemLogBase::stopFlushThread() return; is_shutdown = true; + queue->shutdown(); /// Tell thread to shutdown. queue->flush_event.notify_all(); @@ -105,8 +106,8 @@ void SystemLogQueue::add(const LogElement & element) { std::unique_lock lock(mutex); - // if (queue.is_shutdown) - // return; // TODO + if (is_shutdown) + return; if (queue.size() == DBMS_SYSTEM_LOG_QUEUE_SIZE / 2) { @@ -191,18 +192,18 @@ uint64_t SystemLogBase::notifyFlushImpl(bool force) uint64_t this_thread_requested_offset; { - std::lock_guard lock(mutex); + std::lock_guard lock(queue->mutex); if (is_shutdown) return uint64_t(-1); - this_thread_requested_offset = queue_front_index + queue.size(); + this_thread_requested_offset = queue->queue_front_index + queue->queue.size(); // Publish our flush request, taking care not to overwrite the requests // made by other threads. is_force_prepare_tables |= force; - requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); + queue->requested_flush_up_to = std::max(queue->requested_flush_up_to, this_thread_requested_offset); - flush_event.notify_all(); + queue->flush_event.notify_all(); } LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index cf6efc68119..109334964d2 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -78,10 +78,8 @@ public: void add(const LogElement & element); size_t size() const { return queue.size(); } - void push_back(const LogElement & element) - { - queue.push_back(element); - } + //void push_back(const LogElement & element) { queue.push_back(element); } + void shutdown() { is_shutdown = true; } // Queue is bounded. But its size is quite large to not block in all normal cases. std::vector queue; @@ -101,7 +99,9 @@ public: // Logged overflow message at this queue front index uint64_t logged_queue_full_at_index = -1; +private: Poco::Logger * log; + bool is_shutdown = false; }; template diff --git a/src/Loggers/OwnSplitChannel.h b/src/Loggers/OwnSplitChannel.h index 155d0872465..a6ee8af5b14 100644 --- a/src/Loggers/OwnSplitChannel.h +++ b/src/Loggers/OwnSplitChannel.h @@ -12,7 +12,7 @@ namespace DB { template class SystemLogQueue; struct TextLogElement; - using FooBar = SystemLogQueue; + using TextLogQueue = SystemLogQueue; } #endif @@ -33,7 +33,7 @@ public: void addChannel(Poco::AutoPtr channel, const std::string & name); #ifndef WITHOUT_TEXT_LOG - void addTextLog(std::shared_ptr log_queue, int max_priority); + void addTextLog(std::shared_ptr log_queue, int max_priority); #endif void setLevel(const std::string & name, int level); @@ -48,7 +48,7 @@ private: std::map channels; #ifndef WITHOUT_TEXT_LOG - std::weak_ptr text_log; + std::weak_ptr text_log; std::atomic text_log_max_priority = -1; #endif }; From 103a689edc1291e6b243c8000c8100b043caf8c6 Mon Sep 17 00:00:00 2001 From: therealnick233 Date: Fri, 14 Jul 2023 18:36:42 +0800 Subject: [PATCH 0563/2047] Fixed typo in combinators.md line 303, 'Jonh' -> 'John' --- docs/en/sql-reference/aggregate-functions/combinators.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index a395b350a55..18ff5073e3f 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -300,7 +300,7 @@ SELECT groupArrayResample(30, 75, 30)(name, age) FROM people Consider the results. -`Jonh` is out of the sample because he’s too young. Other people are distributed according to the specified age intervals. +`John` is out of the sample because he’s too young. Other people are distributed according to the specified age intervals. Now let’s count the total number of people and their average wage in the specified age intervals. From f7705fff58e871913028a1683c6d7b21a82b9c16 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Fri, 14 Jul 2023 12:45:05 +0200 Subject: [PATCH 0564/2047] Update README.md --- README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 1036e1a97e1..932a10be0ab 100644 --- a/README.md +++ b/README.md @@ -23,7 +23,7 @@ curl https://clickhouse.com/ | sh ## Upcoming Events -* [**v23.6 Release Webinar**](https://clickhouse.com/company/events/v23-6-release-call?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-06) - Jun 29 - 23.6 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. +* [**v23.7 Release Webinar**](https://clickhouse.com/company/events/v23-7-community-release-call?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-07) - Jul 27 - 23.7 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. * [**ClickHouse Meetup in Boston**](https://www.meetup.com/clickhouse-boston-user-group/events/293913596) - Jul 18 * [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/293913441) - Jul 19 * [**ClickHouse Meetup in Toronto**](https://www.meetup.com/clickhouse-toronto-user-group/events/294183127) - Jul 20 @@ -34,13 +34,13 @@ Also, keep an eye out for upcoming meetups around the world. Somewhere else you ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" -* **Recording available**: [**v23.4 Release Webinar**](https://www.youtube.com/watch?v=4rrf6bk_mOg) Faster Parquet Reading, Asynchonous Connections to Reoplicas, Trailing Comma before FROM, extractKeyValuePairs, integrations updates, and so much more! Watch it now! +* **Recording available**: [**v23.6 Release Webinar**](https://www.youtube.com/watch?v=cuf_hYn7dqU) All the features of 23.7, one convenient video! Watch it now! * **All release webinar recordings**: [YouTube playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3jAlSy1JxyP8zluvXaN3nxU) - ## Interested in joining ClickHouse and making it your full time job? + ## Interested in joining ClickHouse and making it your full-time job? -We are a globally diverse and distributed team, united behind a common goal of creating industry-leading, real-time analytics. Here, you will have an opportunity to solve some of the most cutting edge technical challenges and have direct ownership of your work and vision. If you are a contributor by nature, a thinker as well as a doer - we’ll definitely click! +We are a globally diverse and distributed team, united behind a common goal of creating industry-leading, real-time analytics. Here, you will have an opportunity to solve some of the most cutting-edge technical challenges and have direct ownership of your work and vision. If you are a contributor by nature, a thinker and a doer - we’ll definitely click! Check out our **current openings** here: https://clickhouse.com/company/careers From c6c6389f3f64024a096b8df06850a1d4273a36ba Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 14 Jul 2023 11:57:18 +0000 Subject: [PATCH 0565/2047] Add queries with explicitly enabled analyzer --- .../Optimizations/optimizePrewhere.cpp | 80 ------------------- ...ct_in_order_optimization_explain.reference | 31 +++++++ ..._distinct_in_order_optimization_explain.sh | 28 +++++-- 3 files changed, 54 insertions(+), 85 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index bcd3244b5a9..d9e98a9f41c 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB { @@ -61,75 +60,6 @@ void matchDAGOutputNodesOrderWithHeader(ActionsDAGPtr & actions_dag, const Block namespace QueryPlanOptimizations { -#ifdef WHATEVERSOMETHING -static void removeAliases(ActionsDAG * dag) -{ - using Node = ActionsDAG::Node; - struct Frame - { - const ActionsDAG::Node * node; - const ActionsDAG::Node * parent; - size_t next_child = 0; - }; - std::vector stack; - std::vector> aliases; - - /// collect aliases - auto output_nodes = dag->getOutputs(); - for (const auto * output_node : output_nodes) - { - stack.push_back({output_node, nullptr}); - while (!stack.empty()) - { - auto & frame = stack.back(); - const auto * parent = frame.parent; - const auto * node = frame.node; - - if (frame.next_child < node->children.size()) - { - auto next_frame = Frame{.node = node->children[frame.next_child], .parent = node}; - ++frame.next_child; - stack.push_back(next_frame); - continue; - } - - if (parent && node->type == ActionsDAG::ActionType::ALIAS) - aliases.emplace_back(const_cast(node), const_cast(parent)); - - stack.pop_back(); - } - } - - /// remove aliases from output nodes if any - for(auto it = output_nodes.begin(); it != output_nodes.end();) - { - if ((*it)->type == ActionsDAG::ActionType::ALIAS) - it = output_nodes.erase(it); - else - ++it; - } - - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "aliases found: {}", aliases.size()); - - /// disconnect aliases - for(auto [alias, parent]: aliases) - { - /// find alias in parent's children and replace it with alias child - for (auto & child : parent->children) - { - if (child == alias) - { - child = alias->children.front(); - break; - } - } - } - - /// remove aliases - dag->removeUnusedActions(); -} -#endif - void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) { if (stack.size() < 3) @@ -231,8 +161,6 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) storage.supportedPrewhereColumns(), &Poco::Logger::get("QueryPlanOptimizePrewhere")}; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "filter expression\n{}", filter_step->getExpression()->dumpDAG()); - auto optimize_result = where_optimizer.optimize(filter_step->getExpression(), filter_step->getFilterColumnName(), read_from_merge_tree->getContext(), @@ -249,10 +177,6 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) prewhere_info->need_filter = true; auto & prewhere_filter_actions = optimize_result->prewhere_filter_actions; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "prewhere_filter_actions\n{}", prewhere_filter_actions->dumpDAG()); - - // removeAliases(prewhere_filter_actions.get()); - // LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "removeAliases\n{}", prewhere_filter_actions->dumpDAG()); ActionsChain actions_chain; @@ -335,9 +259,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) prewhere_info->prewhere_column_name = prewere_filter_node_name; prewhere_info->remove_prewhere_column = !prewhere_actions_chain_node->getChildRequiredOutputColumnsNames().contains(prewere_filter_node_name); - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "header BEFORE prewhere update\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); read_from_merge_tree->updatePrewhereInfo(prewhere_info); - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "header AFTER prewhere update\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); QueryPlan::Node * replace_old_filter_node = nullptr; bool remove_filter_node = false; @@ -398,12 +320,10 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) bool apply_match_step = false; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "read header\n{}", read_from_merge_tree->getOutputStream().header.dumpStructure()); /// If column order does not match old filter step column order, match dag output nodes with header if (!blocksHaveEqualStructure(read_from_merge_tree->getOutputStream().header, filter_step->getOutputStream().header)) { apply_match_step = true; - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "rename_actions_dag\n{}", rename_actions_dag->dumpDAG()); matchDAGOutputNodesOrderWithHeader(rename_actions_dag, filter_step->getOutputStream().header); } diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index 0a123a2a50f..3cee6da3c3d 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -56,6 +56,7 @@ Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC -- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query +-- disable new analyzer -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns Sorting (Stream): a ASC Sorting (Stream): a ASC @@ -80,3 +81,33 @@ Sorting (Stream): a DESC, b DESC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC +-- enable new analyzer +-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns +Sorting (Stream): a_1 ASC +Sorting (Stream): a_1 ASC +Sorting (Stream): a_1 ASC +Sorting (Stream): a ASC +-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a ASC, b ASC +-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a DESC, b DESC +-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause +Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): a ASC, b ASC +-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a_1 DESC, b_0 DESC +Sorting (Stream): a DESC, b DESC +-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization +Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): a_0 ASC, b_1 ASC +Sorting (Stream): a ASC, b ASC diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index 46919ae49b2..4c4e660030e 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -81,18 +81,36 @@ $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain plan sorting=1 select disti echo "-- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query" ENABLE_READ_IN_ORDER="set optimize_read_in_order=1" +echo "-- disable new analyzer" +DISABLE_ANALYZER="set allow_experimental_analyzer=0" echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" -$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" $CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES +echo "-- enable new analyzer" +ENABLE_ANALYZER="set allow_experimental_analyzer=1" +echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES +echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization, but direction used from ORDER BY clause" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC" | eval $FIND_SORTING_PROPERTIES +echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct a from distinct_in_order_explain order by a, b" | eval $FIND_SORTING_PROPERTIES +echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES + +echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES + $CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync" From 57ee7916cc348074f52940b225f703f473d972cb Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 14 Jul 2023 12:01:26 +0000 Subject: [PATCH 0566/2047] Add tests to nullable types --- tests/queries/0_stateless/02809_has_subsequence.reference | 7 +++++++ tests/queries/0_stateless/02809_has_subsequence.sql | 8 ++++++++ 2 files changed, 15 insertions(+) diff --git a/tests/queries/0_stateless/02809_has_subsequence.reference b/tests/queries/0_stateless/02809_has_subsequence.reference index 5f533d7f5bb..66da41ccc87 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.reference +++ b/tests/queries/0_stateless/02809_has_subsequence.reference @@ -55,3 +55,10 @@ hasSubsequenceCaseInsensitiveUTF8 0 1 0 +Nullable +\N +\N +\N +1 +1 +1 diff --git a/tests/queries/0_stateless/02809_has_subsequence.sql b/tests/queries/0_stateless/02809_has_subsequence.sql index bcc491a95fe..13b92164cf5 100644 --- a/tests/queries/0_stateless/02809_has_subsequence.sql +++ b/tests/queries/0_stateless/02809_has_subsequence.sql @@ -58,3 +58,11 @@ select hasSubsequenceCaseInsensitiveUTF8('для онлайн обработки select hasSubsequenceCaseInsensitiveUTF8('для онлайн обработки аналитических запросов (OLAP)', materialize('зло()')); select hasSubsequenceCaseInsensitiveUTF8(materialize('для онлайн обработки аналитических запросов (OLAP)'), materialize('аналитика')); select hasSubsequenceCaseInsensitiveUTF8(materialize('для онлайн обработки аналитических запросов (OLAP)'), materialize('аналитика для аналитиков')); + +select 'Nullable'; +select hasSubsequence(Null, Null); +select hasSubsequence(Null, 'a'); +select hasSubsequence(Null::Nullable(String), 'arg'::Nullable(String)); +select hasSubsequence('garbage'::Nullable(String), 'a'); +select hasSubsequence('garbage'::Nullable(String), 'arg'::Nullable(String)); +select hasSubsequence(materialize('garbage'::Nullable(String)), materialize('arg'::Nullable(String))); \ No newline at end of file From 62e6a53d1caefa6d907f81d9151ffd930e3242d4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 14 Jul 2023 12:16:09 +0000 Subject: [PATCH 0567/2047] Remove debug traces --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index d3d8c0f2bc8..48adf36e678 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -644,7 +644,6 @@ Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const Prewher if (prewhere_info->prewhere_actions) { block = prewhere_info->prewhere_actions->updateHeader(std::move(block)); - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "updateHeader()\n{}", block.dumpStructure()); auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) @@ -655,7 +654,6 @@ Block IMergeTreeSelectAlgorithm::applyPrewhereActions(Block block, const Prewher if (prewhere_info->remove_prewhere_column) { - LOG_DEBUG(&Poco::Logger::get(__PRETTY_FUNCTION__), "remove_column_name\n{}", prewhere_info->prewhere_column_name); block.erase(prewhere_info->prewhere_column_name); } else if (prewhere_info->need_filter) From 9738d248c31feb4944339eb13162d1a24e7db44f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 14 Jul 2023 12:17:48 +0000 Subject: [PATCH 0568/2047] Revert unnecessary changes --- src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index d9e98a9f41c..ca8a412bf2e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -60,6 +60,7 @@ void matchDAGOutputNodesOrderWithHeader(ActionsDAGPtr & actions_dag, const Block namespace QueryPlanOptimizations { + void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) { if (stack.size() < 3) From 4e188a637979516df49212ac2b2043efdf16feaa Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 14 Jul 2023 08:39:37 -0400 Subject: [PATCH 0569/2047] Update s3.md regarding partitioned data --- .../engines/table-engines/integrations/s3.md | 21 +++---------------- 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 0e2b48ef6a6..051945538b2 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -57,7 +57,8 @@ Notice that the S3 endpoint in the `ENGINE` configuration uses the parameter tok :::note As shown in the example, querying from S3 tables that are partitioned is -not directly supported at this time, but can be accomplished by querying the bucket contents with a wildcard. +not directly supported at this time, but can be accomplished by querying the individual partitions +using the S3 table function. The primary use-case for writing partitioned data in S3 is to enable transferring that data into another @@ -127,23 +128,7 @@ FROM s3('http://minio:10000/clickhouse//test_45.csv', 'minioadmin', 'minioadminp └────┴────┴────┘ ``` -#### Select from all partitions - -```sql -SELECT * -FROM s3('http://minio:10000/clickhouse//**', 'minioadmin', 'minioadminpassword', 'CSV') -``` -```response -┌─c1─┬─c2─┬─c3─┐ -│ 3 │ 2 │ 1 │ -└────┴────┴────┘ -┌─c1─┬─c2─┬─c3─┐ -│ 1 │ 2 │ 3 │ -└────┴────┴────┘ -┌─c1─┬─c2─┬─c3─┐ -│ 78 │ 43 │ 45 │ -└────┴────┴────┘ -``` +#### Limitation You may naturally try to `Select * from p`, but as noted above, this query will fail; use the preceding query. From 132052fbf2404d8faa5e14fbc2cbd5f8e16bdc1f Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 29 Jun 2023 12:16:40 -0700 Subject: [PATCH 0570/2047] Implement partial support for endianness-independent serialization --- .../AggregateFunctionGroupArray.h | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.h b/src/AggregateFunctions/AggregateFunctionGroupArray.h index 7a5e6a8cb2d..b5905105457 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -266,19 +266,20 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { const auto & value = this->data(place).value; - size_t size = value.size(); + const size_t size = value.size(); writeVarUInt(size, buf); - buf.write(reinterpret_cast(value.data()), size * sizeof(value[0])); + for (const auto & element : value) + writeBinaryLittleEndian(element, buf); if constexpr (Trait::last) - DB::writeIntBinary(this->data(place).total_values, buf); + writeBinaryLittleEndian(this->data(place).total_values, buf); if constexpr (Trait::sampler == Sampler::RNG) { - DB::writeIntBinary(this->data(place).total_values, buf); + writeBinaryLittleEndian(this->data(place).total_values, buf); WriteBufferFromOwnString rng_buf; rng_buf << this->data(place).rng; - DB::writeStringBinary(rng_buf.str(), buf); + writeStringBinary(rng_buf.str(), buf); } } @@ -297,16 +298,17 @@ public: auto & value = this->data(place).value; value.resize_exact(size, arena); - buf.readStrict(reinterpret_cast(value.data()), size * sizeof(value[0])); + for (auto & element : value) + readBinaryLittleEndian(element, buf); if constexpr (Trait::last) - DB::readIntBinary(this->data(place).total_values, buf); + readBinaryLittleEndian(this->data(place).total_values, buf); if constexpr (Trait::sampler == Sampler::RNG) { - DB::readIntBinary(this->data(place).total_values, buf); + readBinaryLittleEndian(this->data(place).total_values, buf); std::string rng_string; - DB::readStringBinary(rng_string, buf); + readStringBinary(rng_string, buf); ReadBufferFromString rng_buf(rng_string); rng_buf >> this->data(place).rng; } @@ -603,14 +605,14 @@ public: node->write(buf); if constexpr (Trait::last) - DB::writeIntBinary(data(place).total_values, buf); + writeBinaryLittleEndian(data(place).total_values, buf); if constexpr (Trait::sampler == Sampler::RNG) { - DB::writeIntBinary(data(place).total_values, buf); + writeBinaryLittleEndian(data(place).total_values, buf); WriteBufferFromOwnString rng_buf; rng_buf << data(place).rng; - DB::writeStringBinary(rng_buf.str(), buf); + writeStringBinary(rng_buf.str(), buf); } } @@ -636,13 +638,13 @@ public: value[i] = Node::read(buf, arena); if constexpr (Trait::last) - DB::readIntBinary(data(place).total_values, buf); + readBinaryLittleEndian(data(place).total_values, buf); if constexpr (Trait::sampler == Sampler::RNG) { - DB::readIntBinary(data(place).total_values, buf); + readBinaryLittleEndian(data(place).total_values, buf); std::string rng_string; - DB::readStringBinary(rng_string, buf); + readStringBinary(rng_string, buf); ReadBufferFromString rng_buf(rng_string); rng_buf >> data(place).rng; } From 0527a32282f7d9153eecbb9c740be18140f741c4 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 6 Jul 2023 11:27:38 -0700 Subject: [PATCH 0571/2047] Implement platform-independent serialization for further aggregate functions --- .../AggregateFunctionBoundingRatio.h | 51 +++++++++++-------- .../AggregateFunctionDeltaSum.h | 16 +++--- .../AggregateFunctionDeltaSumTimestamp.h | 24 ++++----- src/AggregateFunctions/QuantileApprox.h | 28 +++++----- src/AggregateFunctions/ReservoirSampler.h | 12 ++--- src/Common/TransformEndianness.hpp | 6 +++ 6 files changed, 76 insertions(+), 61 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index 935adbf2b7d..82e4f1122a8 100644 --- a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -67,29 +67,38 @@ struct AggregateFunctionBoundingRatioData } } - void serialize(WriteBuffer & buf) const - { - writeBinary(empty, buf); - - if (!empty) - { - writePODBinary(left, buf); - writePODBinary(right, buf); - } - } - - void deserialize(ReadBuffer & buf) - { - readBinary(empty, buf); - - if (!empty) - { - readPODBinary(left, buf); - readPODBinary(right, buf); - } - } + void serialize(WriteBuffer & buf) const; + void deserialize(ReadBuffer & buf); }; +template +inline void transformEndianness(AggregateFunctionBoundingRatioData::Point & p) +{ + transformEndianness(p.x); + transformEndianness(p.y); +} + +void AggregateFunctionBoundingRatioData::serialize(WriteBuffer & buf) const +{ + writeBinaryLittleEndian(empty, buf); + + if (!empty) + { + writeBinaryLittleEndian(left, buf); + writeBinaryLittleEndian(right, buf); + } +} + +void AggregateFunctionBoundingRatioData::deserialize(ReadBuffer & buf) +{ + readBinaryLittleEndian(empty, buf); + + if (!empty) + { + readBinaryLittleEndian(left, buf); + readBinaryLittleEndian(right, buf); + } +} class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper { diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSum.h b/src/AggregateFunctions/AggregateFunctionDeltaSum.h index 199d2706d3a..d64f949825a 100644 --- a/src/AggregateFunctions/AggregateFunctionDeltaSum.h +++ b/src/AggregateFunctions/AggregateFunctionDeltaSum.h @@ -103,18 +103,18 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { - writeIntBinary(this->data(place).sum, buf); - writeIntBinary(this->data(place).first, buf); - writeIntBinary(this->data(place).last, buf); - writePODBinary(this->data(place).seen, buf); + writeBinaryLittleEndian(this->data(place).sum, buf); + writeBinaryLittleEndian(this->data(place).first, buf); + writeBinaryLittleEndian(this->data(place).last, buf); + writeBinaryLittleEndian(this->data(place).seen, buf); } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { - readIntBinary(this->data(place).sum, buf); - readIntBinary(this->data(place).first, buf); - readIntBinary(this->data(place).last, buf); - readPODBinary(this->data(place).seen, buf); + readBinaryLittleEndian(this->data(place).sum, buf); + readBinaryLittleEndian(this->data(place).first, buf); + readBinaryLittleEndian(this->data(place).last, buf); + readBinaryLittleEndian(this->data(place).seen, buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h index 5ca07bb0bdf..5eeb1425afb 100644 --- a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h +++ b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h @@ -144,22 +144,22 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { - writeIntBinary(this->data(place).sum, buf); - writeIntBinary(this->data(place).first, buf); - writeIntBinary(this->data(place).first_ts, buf); - writeIntBinary(this->data(place).last, buf); - writeIntBinary(this->data(place).last_ts, buf); - writePODBinary(this->data(place).seen, buf); + writeBinaryLittleEndian(this->data(place).sum, buf); + writeBinaryLittleEndian(this->data(place).first, buf); + writeBinaryLittleEndian(this->data(place).first_ts, buf); + writeBinaryLittleEndian(this->data(place).last, buf); + writeBinaryLittleEndian(this->data(place).last_ts, buf); + writeBinaryLittleEndian(this->data(place).seen, buf); } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { - readIntBinary(this->data(place).sum, buf); - readIntBinary(this->data(place).first, buf); - readIntBinary(this->data(place).first_ts, buf); - readIntBinary(this->data(place).last, buf); - readIntBinary(this->data(place).last_ts, buf); - readPODBinary(this->data(place).seen, buf); + readBinaryLittleEndian(this->data(place).sum, buf); + readBinaryLittleEndian(this->data(place).first, buf); + readBinaryLittleEndian(this->data(place).first_ts, buf); + readBinaryLittleEndian(this->data(place).last, buf); + readBinaryLittleEndian(this->data(place).last_ts, buf); + readBinaryLittleEndian(this->data(place).seen, buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override diff --git a/src/AggregateFunctions/QuantileApprox.h b/src/AggregateFunctions/QuantileApprox.h index f58f1396fb4..6b2a6cf4398 100644 --- a/src/AggregateFunctions/QuantileApprox.h +++ b/src/AggregateFunctions/QuantileApprox.h @@ -233,35 +233,35 @@ public: void write(WriteBuffer & buf) const { - writeIntBinary(compress_threshold, buf); - writeFloatBinary(relative_error, buf); - writeIntBinary(count, buf); - writeIntBinary(sampled.size(), buf); + writeBinaryLittleEndian(compress_threshold, buf); + writeBinaryLittleEndian(relative_error, buf); + writeBinaryLittleEndian(count, buf); + writeBinaryLittleEndian(sampled.size(), buf); for (const auto & stats : sampled) { - writeFloatBinary(stats.value, buf); - writeIntBinary(stats.g, buf); - writeIntBinary(stats.delta, buf); + writeBinaryLittleEndian(stats.value, buf); + writeBinaryLittleEndian(stats.g, buf); + writeBinaryLittleEndian(stats.delta, buf); } } void read(ReadBuffer & buf) { - readIntBinary(compress_threshold, buf); - readFloatBinary(relative_error, buf); - readIntBinary(count, buf); + readBinaryLittleEndian(compress_threshold, buf); + readBinaryLittleEndian(relative_error, buf); + readBinaryLittleEndian(count, buf); size_t sampled_len = 0; - readIntBinary(sampled_len, buf); + readBinaryLittleEndian(sampled_len, buf); sampled.resize(sampled_len); for (size_t i = 0; i < sampled_len; ++i) { auto stats = sampled[i]; - readFloatBinary(stats.value, buf); - readIntBinary(stats.g, buf); - readIntBinary(stats.delta, buf); + readBinaryLittleEndian(stats.value, buf); + readBinaryLittleEndian(stats.g, buf); + readBinaryLittleEndian(stats.delta, buf); } } diff --git a/src/AggregateFunctions/ReservoirSampler.h b/src/AggregateFunctions/ReservoirSampler.h index 3d723d5aace..7409a3fa0dd 100644 --- a/src/AggregateFunctions/ReservoirSampler.h +++ b/src/AggregateFunctions/ReservoirSampler.h @@ -207,8 +207,8 @@ public: void read(DB::ReadBuffer & buf) { - DB::readIntBinary(sample_count, buf); - DB::readIntBinary(total_values, buf); + DB::readBinaryLittleEndian(sample_count, buf); + DB::readBinaryLittleEndian(total_values, buf); size_t size = std::min(total_values, sample_count); static constexpr size_t MAX_RESERVOIR_SIZE = 1_GiB; @@ -224,22 +224,22 @@ public: rng_buf >> rng; for (size_t i = 0; i < samples.size(); ++i) - DB::readBinary(samples[i], buf); + DB::readBinaryLittleEndian(samples[i], buf); sorted = false; } void write(DB::WriteBuffer & buf) const { - DB::writeIntBinary(sample_count, buf); - DB::writeIntBinary(total_values, buf); + DB::writeBinaryLittleEndian(sample_count, buf); + DB::writeBinaryLittleEndian(total_values, buf); DB::WriteBufferFromOwnString rng_buf; rng_buf << rng; DB::writeStringBinary(rng_buf.str(), buf); for (size_t i = 0; i < std::min(sample_count, total_values); ++i) - DB::writeBinary(samples[i], buf); + DB::writeBinaryLittleEndian(samples[i], buf); } private: diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 4d690d75d9e..0a9055dde15 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -59,4 +59,10 @@ inline void transformEndianness(std::pair & pair) transformEndianness(pair.first); transformEndianness(pair.second); } + +template +inline void transformEndianness(StrongTypedef & x) +{ + transformEndianness(x.toUnderType()); +} } From dff5d78126318782cc7f3935c38013b010695c07 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Sun, 18 Jun 2023 02:03:14 +0000 Subject: [PATCH 0572/2047] Add jemalloc support --- contrib/jemalloc-cmake/CMakeLists.txt | 4 +- .../internal/jemalloc_internal_defs.h.in | 435 ++++++++++++++++++ 2 files changed, 438 insertions(+), 1 deletion(-) create mode 100644 contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 20025dfc63e..9a4ad372be3 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ if (SANITIZE OR NOT ( - ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64)) OR + ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X )) OR (OS_DARWIN AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) )) if (ENABLE_JEMALLOC) @@ -148,6 +148,8 @@ elseif (ARCH_PPC64LE) set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_ppc64le") elseif (ARCH_RISCV64) set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_riscv64") +elseif (ARCH_S390X) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_s390x") else () message (FATAL_ERROR "internal jemalloc: This arch is not supported") endif () diff --git a/contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in new file mode 100644 index 00000000000..531f2bca0c2 --- /dev/null +++ b/contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in @@ -0,0 +1,435 @@ +/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ +#ifndef JEMALLOC_INTERNAL_DEFS_H_ +#define JEMALLOC_INTERNAL_DEFS_H_ +/* + * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all + * public APIs to be prefixed. This makes it possible, with some care, to use + * multiple allocators simultaneously. + */ +/* #undef JEMALLOC_PREFIX */ +/* #undef JEMALLOC_CPREFIX */ + +/* + * Define overrides for non-standard allocator-related functions if they are + * present on the system. + */ +#define JEMALLOC_OVERRIDE___LIBC_CALLOC +#define JEMALLOC_OVERRIDE___LIBC_FREE +#define JEMALLOC_OVERRIDE___LIBC_MALLOC +#define JEMALLOC_OVERRIDE___LIBC_MEMALIGN +#define JEMALLOC_OVERRIDE___LIBC_REALLOC +#define JEMALLOC_OVERRIDE___LIBC_VALLOC +#define JEMALLOC_OVERRIDE___LIBC_PVALLOC +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ + +/* + * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. + * For shared libraries, symbol visibility mechanisms prevent these symbols + * from being exported, but for static libraries, naming collisions are a real + * possibility. + */ +#define JEMALLOC_PRIVATE_NAMESPACE je_ + +/* + * Hyper-threaded CPUs may need a special instruction inside spin loops in + * order to yield to another virtual CPU. + */ +#define CPU_SPINWAIT +/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ +#define HAVE_CPU_SPINWAIT 0 + +/* + * Number of significant bits in virtual addresses. This may be less than the + * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 + * bits are the same as bit 47. + */ +#define LG_VADDR 64 + +/* Defined if C11 atomics are available. */ +#define JEMALLOC_C11_ATOMICS + +/* Defined if GCC __atomic atomics are available. */ +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS + +/* Defined if GCC __sync atomics are available. */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS + +/* + * Defined if __builtin_clz() and __builtin_clzl() are available. + */ +#define JEMALLOC_HAVE_BUILTIN_CLZ + +/* + * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. + */ +/* #undef JEMALLOC_OS_UNFAIR_LOCK */ + +/* Defined if syscall(2) is usable. */ +#define JEMALLOC_USE_SYSCALL + +/* + * Defined if secure_getenv(3) is available. + */ +#define JEMALLOC_HAVE_SECURE_GETENV + +/* + * Defined if issetugid(2) is available. + */ +/* #undef JEMALLOC_HAVE_ISSETUGID */ + +/* Defined if pthread_atfork(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_ATFORK + +/* Defined if pthread_setname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_SETNAME_NP + +/* Defined if pthread_getname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_GETNAME_NP + +/* Defined if pthread_get_name_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GET_NAME_NP */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC + +/* + * Defined if mach_absolute_time() is available. + */ +/* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ + +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + +/* + * Defined if _malloc_thread_cleanup() exists. At least in the case of + * FreeBSD, pthread_key_create() allocates, which if used during malloc + * bootstrapping will cause recursion into the pthreads library. Therefore, if + * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in + * malloc_tsd. + */ +/* #undef JEMALLOC_MALLOC_THREAD_CLEANUP */ + +/* + * Defined if threaded initialization is known to be safe on this platform. + * Among other things, it must be possible to initialize a mutex without + * triggering allocation in order for threaded allocation to be safe. + */ +#define JEMALLOC_THREADED_INIT + +/* + * Defined if the pthreads implementation defines + * _pthread_mutex_init_calloc_cb(), in which case the function is used in order + * to avoid recursive allocation during mutex initialization. + */ +/* #undef JEMALLOC_MUTEX_INIT_CB */ + +/* Non-empty if the tls_model attribute is supported. */ +#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) + +/* + * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables + * inline functions. + */ +/* #undef JEMALLOC_DEBUG */ + +/* JEMALLOC_STATS enables statistics calculation. */ +#define JEMALLOC_STATS + +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + +/* JEMALLOC_PROF enables allocation profiling. */ +/* #undef JEMALLOC_PROF */ + +/* Use libunwind for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBUNWIND */ + +/* Use libgcc for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBGCC */ + +/* Use gcc intrinsics for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_GCC */ + +/* JEMALLOC_PAGEID enabled page id */ +/* #undef JEMALLOC_PAGEID */ + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +#define JEMALLOC_HAVE_PRCTL + +/* + * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage + * segment (DSS). + */ +#define JEMALLOC_DSS + +/* Support memory filling (junk/zero). */ +#define JEMALLOC_FILL + +/* Support utrace(2)-based tracing. */ +/* #undef JEMALLOC_UTRACE */ + +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + +/* Support optional abort() on OOM. */ +/* #undef JEMALLOC_XMALLOC */ + +/* Support lazy locking (avoid locking unless a second thread is launched). */ +/* #undef JEMALLOC_LAZY_LOCK */ + +/* + * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size + * classes). + */ +/* #undef LG_QUANTUM */ + +/* One page is 2^LG_PAGE bytes. */ +#define LG_PAGE 12 + +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + +/* + * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the + * system does not explicitly support huge pages; system calls that require + * explicit huge page support are separately configured. + */ +#define LG_HUGEPAGE 20 + +/* + * If defined, adjacent virtual memory mappings with identical attributes + * automatically coalesce, and they fragment when changes are made to subranges. + * This is the normal order of things for mmap()/munmap(), but on Windows + * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. + * mappings do *not* coalesce/fragment. + */ +#define JEMALLOC_MAPS_COALESCE + +/* + * If defined, retain memory for later reuse by default rather than using e.g. + * munmap() to unmap freed extents. This is enabled on 64-bit Linux because + * common sequences of mmap()/munmap() calls will cause virtual memory map + * holes. + */ +#define JEMALLOC_RETAIN + +/* TLS is used to map arenas and magazine caches to threads. */ +#define JEMALLOC_TLS + +/* + * Used to mark unreachable code to quiet "end of non-void" compiler warnings. + * Don't use this directly; instead use unreachable() from util.h + */ +#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable + +/* + * ffs*() functions to use for bitmapping. Don't use these directly; instead, + * use ffs_*() from util.h. + */ +#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll +#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl +#define JEMALLOC_INTERNAL_FFS __builtin_ffs + +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + +/* + * If defined, explicitly attempt to more uniformly distribute large allocation + * pointer alignments across all cache indices. + */ +#define JEMALLOC_CACHE_OBLIVIOUS + +/* + * If defined, enable logging facilities. We make this a configure option to + * avoid taking extra branches everywhere. + */ +/* #undef JEMALLOC_LOG */ + +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + +/* + * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. + */ +/* #undef JEMALLOC_ZONE */ + +/* + * Methods for determining whether the OS overcommits. + * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's + * /proc/sys/vm.overcommit_memory file. + * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. + */ +/* #undef JEMALLOC_SYSCTL_VM_OVERCOMMIT */ +#define JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY + +/* Defined if madvise(2) is available. */ +#define JEMALLOC_HAVE_MADVISE + +/* + * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE + * arguments to madvise(2). + */ +#define JEMALLOC_HAVE_MADVISE_HUGE + +/* + * Methods for purging unused pages differ between operating systems. + * + * madvise(..., MADV_FREE) : This marks pages as being unused, such that they + * will be discarded rather than swapped out. + * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is + * defined, this immediately discards pages, + * such that new pages will be demand-zeroed if + * the address region is later touched; + * otherwise this behaves similarly to + * MADV_FREE, though typically with higher + * system overhead. + */ +#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_DONTNEED +#define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS + +/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ +/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ + +/* + * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. + */ +#define JEMALLOC_MADVISE_DONTDUMP + +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_NOCORE */ + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + +/* + * Defined if transparent huge pages (THPs) are supported via the + * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. + */ +/* #undef JEMALLOC_THP */ + +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +/* #undef JEMALLOC_HAVE_MALLOC_SIZE */ + +/* Define if operating system has alloca.h header. */ +#define JEMALLOC_HAS_ALLOCA_H + +/* C99 restrict keyword supported. */ +#define JEMALLOC_HAS_RESTRICT + +/* For use by hash code. */ +#define JEMALLOC_BIG_ENDIAN + +/* sizeof(int) == 2^LG_SIZEOF_INT. */ +#define LG_SIZEOF_INT 2 + +/* sizeof(long) == 2^LG_SIZEOF_LONG. */ +#define LG_SIZEOF_LONG 3 + +/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ +#define LG_SIZEOF_LONG_LONG 3 + +/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ +#define LG_SIZEOF_INTMAX_T 3 + +/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ +/* #undef JEMALLOC_GLIBC_MALLOC_HOOK */ + +/* glibc memalign hook. */ +/* #undef JEMALLOC_GLIBC_MEMALIGN_HOOK */ + +/* pthread support */ +#define JEMALLOC_HAVE_PTHREAD + +/* dlsym() support */ +#define JEMALLOC_HAVE_DLSYM + +/* Adaptive mutex support in pthreads. */ +#define JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP + +/* GNU specific sched_getcpu support */ +#define JEMALLOC_HAVE_SCHED_GETCPU + +/* GNU specific sched_setaffinity support */ +#define JEMALLOC_HAVE_SCHED_SETAFFINITY + +/* + * If defined, all the features necessary for background threads are present. + */ +#define JEMALLOC_BACKGROUND_THREAD + +/* + * If defined, jemalloc symbols are not exported (doesn't work when + * JEMALLOC_PREFIX is not defined). + */ +/* #undef JEMALLOC_EXPORT */ + +/* config.malloc_conf options string. */ +#define JEMALLOC_CONFIG_MALLOC_CONF "" + +/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ +#define JEMALLOC_IS_MALLOC + +/* + * Defined if strerror_r returns char * if _GNU_SOURCE is defined. + */ +#define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE + +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +#define JEMALLOC_ENABLE_CXX + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +/* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ + +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +#define JEMALLOC_ZERO_REALLOC_DEFAULT_FREE + +#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ From efa78a377eaebd5a8c369f56732e63a51eb6fd07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 16:54:16 +0300 Subject: [PATCH 0573/2047] Update CMakeLists.txt --- contrib/jemalloc-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 9a4ad372be3..ab4d9d46c63 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ if (SANITIZE OR NOT ( - ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X )) OR + ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X)) OR (OS_DARWIN AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) )) if (ENABLE_JEMALLOC) From b2546c8bb524df19a376f925ec14d1a730b72768 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 16:54:34 +0300 Subject: [PATCH 0574/2047] Update CMakeLists.txt --- contrib/jemalloc-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index ab4d9d46c63..f2b37d7efae 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ if (SANITIZE OR NOT ( - ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X)) OR + ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X)) OR (OS_DARWIN AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) )) if (ENABLE_JEMALLOC) From f0f69a2e3a7dabb0169c785e34f6c4a6639ccd83 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 16:55:57 +0300 Subject: [PATCH 0575/2047] Update CMakeLists.txt --- contrib/jemalloc-cmake/CMakeLists.txt | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index f2b37d7efae..15e965ed841 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -17,17 +17,17 @@ if (NOT ENABLE_JEMALLOC) endif () if (NOT OS_LINUX) - message (WARNING "jemalloc support on non-linux is EXPERIMENTAL") + message (WARNING "jemalloc support on non-Linux is EXPERIMENTAL") endif() if (OS_LINUX) - # ThreadPool select job randomly, and there can be some threads that had been - # performed some memory heavy task before and will be inactive for some time, - # but until it will became active again, the memory will not be freed since by - # default each thread has it's own arena, but there should be not more then + # ThreadPool select job randomly, and there can be some threads that have been + # performed some memory-heavy tasks before and will be inactive for some time, + # but until it becomes active again, the memory will not be freed since, by + # default, each thread has its arena, but there should be no more than # 4*CPU arenas (see opt.nareans description). # - # By enabling percpu_arena number of arenas limited to number of CPUs and hence + # By enabling percpu_arena number of arenas is limited to the number of CPUs, and hence # this problem should go away. # # muzzy_decay_ms -- use MADV_FREE when available on newer Linuxes, to @@ -38,7 +38,7 @@ if (OS_LINUX) else() set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:5000,dirty_decay_ms:5000") endif() -# CACHE variable is empty, to allow changing defaults without necessity +# CACHE variable is empty to allow changing defaults without the necessity # to purge cache set (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE "" CACHE STRING "Change default configuration string of JEMalloc" ) if (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE) @@ -174,7 +174,7 @@ target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_PROF=1) # jemalloc provides support for two different libunwind flavors: the original HP libunwind and the one coming with gcc / g++ / libstdc++. # The latter is identified by `JEMALLOC_PROF_LIBGCC` and uses `_Unwind_Backtrace` method instead of `unw_backtrace`. -# At the time ClickHouse uses LLVM libunwind which follows libgcc's way of backtracing. +# At the time ClickHouse uses LLVM libunwind which follows libgcc's way of backtracking. # # ClickHouse has to provide `unw_backtrace` method by the means of [commit 8e2b31e](https://github.com/ClickHouse/libunwind/commit/8e2b31e766dd502f6df74909e04a7dbdf5182eb1). target_compile_definitions (_jemalloc PRIVATE -DJEMALLOC_PROF_LIBGCC=1) From feebad3d3bf8bb26e3df77845308c27126184403 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Fri, 14 Jul 2023 14:39:24 +0000 Subject: [PATCH 0576/2047] Try to fix tests by adding mutex like in onProgress() --- src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 544442c4c05..a05efa7ec3f 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -176,6 +176,7 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) void WriteBufferFromHTTPServerResponse::onMemoryUsage(Int64 usage) { + std::lock_guard lock(mutex); peak_memory_usage = usage; } From 4f6d59ce7f2eaf032a8e833153da909b155bf3ae Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 14 Jul 2023 05:57:46 +0200 Subject: [PATCH 0577/2047] tests: drop existing view in test_materialized_mysql_database This should avoid failures of other tests in case of some previous test failed Signed-off-by: Azat Khuzhin --- .../test_materialized_mysql_database/materialized_with_ddl.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 8cf9e67bf63..73f6e11d7f8 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -992,6 +992,7 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): ) check_query(clickhouse_node, "SHOW TABLES FROM db FORMAT TSV", "t\n") clickhouse_node.query("SYSTEM STOP MERGES db.t") + clickhouse_node.query("DROP VIEW v IF EXISTS") clickhouse_node.query("CREATE VIEW v AS SELECT * FROM db.t") mysql_node.query("INSERT INTO db.t VALUES (1, 1), (2, 2)") mysql_node.query("DELETE FROM db.t WHERE a = 2;") From f762a0e61523c1e46eb690a59a2452e4e8d0f3af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 18:56:53 +0300 Subject: [PATCH 0578/2047] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 932a10be0ab..f0a7dbe2408 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ Also, keep an eye out for upcoming meetups around the world. Somewhere else you ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" -* **Recording available**: [**v23.6 Release Webinar**](https://www.youtube.com/watch?v=cuf_hYn7dqU) All the features of 23.7, one convenient video! Watch it now! +* **Recording available**: [**v23.6 Release Webinar**](https://www.youtube.com/watch?v=cuf_hYn7dqU) All the features of 23.6, one convenient video! Watch it now! * **All release webinar recordings**: [YouTube playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3jAlSy1JxyP8zluvXaN3nxU) From a3a576cd865ab8f114e82e2860b89fc86365078f Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 14 Jul 2023 17:35:31 +0000 Subject: [PATCH 0579/2047] Add usage of in s3_storage_class in SETTINGS --- src/Backups/BackupFactory.h | 1 + src/Backups/BackupSettings.cpp | 1 + src/Backups/BackupSettings.h | 3 +++ src/Backups/BackupsWorker.cpp | 1 + src/Backups/registerBackupEngineS3.cpp | 2 +- 5 files changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index 92a5e16533c..f163cd0eb23 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -30,6 +30,7 @@ public: String compression_method; int compression_level = -1; String password; + String s3_storage_class; ContextPtr context; bool is_internal_backup = false; std::shared_ptr backup_coordination; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 882342467fe..baf9064884f 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -21,6 +21,7 @@ namespace ErrorCodes M(String, id) \ M(String, compression_method) \ M(String, password) \ + M(String, s3_storage_class) \ M(Bool, structure_only) \ M(Bool, async) \ M(Bool, decrypt_files_from_encrypted_disks) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 2c899687e6e..d15b83d8f19 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -25,6 +25,9 @@ struct BackupSettings /// Password used to encrypt the backup. String password; + /// S3 storage class. + String s3_storage_class = ""; + /// If this is set to true then only create queries will be written to backup, /// without the data of tables. bool structure_only = false; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 53a076705c4..df60f3de740 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -344,6 +344,7 @@ void BackupsWorker::doBackup( backup_create_params.compression_method = backup_settings.compression_method; backup_create_params.compression_level = backup_settings.compression_level; backup_create_params.password = backup_settings.password; + backup_create_params.s3_storage_class = backup_settings.s3_storage_class; backup_create_params.is_internal_backup = backup_settings.internal; backup_create_params.backup_coordination = backup_coordination; backup_create_params.backup_uuid = backup_settings.backup_uuid; diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 4f5a0d3e865..219973d8fc4 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -52,7 +52,7 @@ void registerBackupEngineS3(BackupFactory & factory) const auto & args = params.backup_info.args; String s3_uri, access_key_id, secret_access_key; - String storage_class; + String storage_class = params.s3_storage_class; if (!id_arg.empty()) { From 254b0a65b3c82124d4053e6d4d6099ec23c8980e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 20:03:48 +0200 Subject: [PATCH 0580/2047] Update CCTZ --- contrib/cctz | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz b/contrib/cctz index 5e05432420f..8529bcef5cd 160000 --- a/contrib/cctz +++ b/contrib/cctz @@ -1 +1 @@ -Subproject commit 5e05432420f9692418e2e12aff09859e420b14a2 +Subproject commit 8529bcef5cd996b7c0f4d7475286b76b5d126c4c From 32bc5866ea1b2622bbd0b18d92d748b4d6a4f86e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 20:04:58 +0200 Subject: [PATCH 0581/2047] Update build.md for clang-16 --- docs/en/development/build.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 83a4550df88..dfec20be58e 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -42,7 +42,7 @@ sudo add-apt-repository -y ppa:ubuntu-toolchain-r/test For other Linux distribution - check the availability of LLVM's [prebuild packages](https://releases.llvm.org/download.html). -As of April 2023, any version of Clang >= 15 will work. +As of April 2023, clang-16 or higher will work. GCC as a compiler is not supported. To build with a specific Clang version: @@ -86,8 +86,8 @@ The build requires the following components: - Git (used to checkout the sources, not needed for the build) - CMake 3.20 or newer -- Compiler: Clang 15 or newer -- Linker: lld 15 or newer +- Compiler: clang-16 or newer +- Linker: lld-16 or newer - Ninja - Yasm - Gawk From 94112895a160bb9eddc226aff55f05f56b55b143 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 21:09:58 +0300 Subject: [PATCH 0582/2047] Add RISC-V 64 to the docs --- docs/en/development/build.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 83a4550df88..d55f7d8cced 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -11,7 +11,8 @@ Supported platforms: - x86_64 - AArch64 -- Power9 (experimental) +- PowerPC 64 LE (experimental) +- RISC-V 64 (experimental) ## Building on Ubuntu From 527d77bc0fb37dd7af6d39e9f490f8621da3032e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 21:12:12 +0200 Subject: [PATCH 0583/2047] system.licenses table will display hard forks --- .../0_stateless/02813_system_licenses_base.reference | 7 +++++++ tests/queries/0_stateless/02813_system_licenses_base.sql | 1 + utils/list-licenses/list-licenses.sh | 4 ++-- 3 files changed, 10 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02813_system_licenses_base.reference create mode 100644 tests/queries/0_stateless/02813_system_licenses_base.sql diff --git a/tests/queries/0_stateless/02813_system_licenses_base.reference b/tests/queries/0_stateless/02813_system_licenses_base.reference new file mode 100644 index 00000000000..89c42cc7a2f --- /dev/null +++ b/tests/queries/0_stateless/02813_system_licenses_base.reference @@ -0,0 +1,7 @@ +Row 1: +────── +library_name: poco +license_type: Boost +license_path: /base/poco/LICENSE +license_text: Boost Software License - Version 1.0 - August 17th, 2003 + diff --git a/tests/queries/0_stateless/02813_system_licenses_base.sql b/tests/queries/0_stateless/02813_system_licenses_base.sql new file mode 100644 index 00000000000..e4b2ca3d36b --- /dev/null +++ b/tests/queries/0_stateless/02813_system_licenses_base.sql @@ -0,0 +1 @@ +SELECT * REPLACE substring(license_text, 1, position(license_text, '\n')) AS license_text FROM system.licenses WHERE library_name = 'poco' FORMAT Vertical; diff --git a/utils/list-licenses/list-licenses.sh b/utils/list-licenses/list-licenses.sh index dd23e6321c8..cee5cf87a08 100755 --- a/utils/list-licenses/list-licenses.sh +++ b/utils/list-licenses/list-licenses.sh @@ -12,7 +12,7 @@ fi ROOT_PATH="$(git rev-parse --show-toplevel)" LIBS_PATH="${ROOT_PATH}/contrib" -ls -1 -d ${LIBS_PATH}/*/ | ${GREP_CMD} -F -v -- '-cmake' | LC_ALL=C sort | while read LIB; do +ls -1 -d ${LIBS_PATH}/*/ "${ROOT_PATH}/base/poco" | ${GREP_CMD} -F -v -- '-cmake' | LC_ALL=C sort | while read LIB; do LIB_NAME=$(basename $LIB) LIB_LICENSE=$( @@ -72,7 +72,7 @@ ls -1 -d ${LIBS_PATH}/*/ | ${GREP_CMD} -F -v -- '-cmake' | LC_ALL=C sort | while echo "HPND") || echo "Unknown") - RELATIVE_PATH=$(echo "$LIB_LICENSE" | sed -r -e 's!^.+/contrib/!/contrib/!') + RELATIVE_PATH=$(echo "$LIB_LICENSE" | sed -r -e 's!^.+/(contrib|base)/!/\1/!') echo -e "$LIB_NAME\t$LICENSE_TYPE\t$RELATIVE_PATH" fi From 6abdde12cafb74b45cf5cb650caac133606dab45 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 14 Jul 2023 18:31:45 +0200 Subject: [PATCH 0584/2047] Print Zxid in keeper stat command in hex (so as ZooKeeper) Signed-off-by: Azat Khuzhin --- src/Coordination/FourLetterCommand.cpp | 18 ++++++++++++++++-- .../test_keeper_four_word_command/test.py | 4 ++-- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 0490213d296..d1ff03482b6 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -18,6 +18,20 @@ #include #include +namespace +{ + +String formatZxid(int64_t zxid) +{ + /// ZooKeeper print zxid in hex and + String hex = getHexUIntLowercase(zxid); + /// without leading zeros + trimLeft(hex, '0'); + return "0x" + hex; +} + +} + namespace DB { @@ -348,7 +362,7 @@ String ServerStatCommand::run() write("Sent", toString(stats.getPacketsSent())); write("Connections", toString(keeper_info.alive_connections_count)); write("Outstanding", toString(keeper_info.outstanding_requests_count)); - write("Zxid", toString(keeper_info.last_zxid)); + write("Zxid", formatZxid(keeper_info.last_zxid)); write("Mode", keeper_info.getRole()); write("Node count", toString(keeper_info.total_nodes_count)); @@ -381,7 +395,7 @@ String StatCommand::run() write("Sent", toString(stats.getPacketsSent())); write("Connections", toString(keeper_info.alive_connections_count)); write("Outstanding", toString(keeper_info.outstanding_requests_count)); - write("Zxid", toString(keeper_info.last_zxid)); + write("Zxid", formatZxid(keeper_info.last_zxid)); write("Mode", keeper_info.getRole()); write("Node count", toString(keeper_info.total_nodes_count)); diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index b546d879c75..1d5bc6a6541 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -329,7 +329,7 @@ def test_cmd_srvr(started_cluster): assert result["Received"] == "10" assert result["Sent"] == "10" assert int(result["Connections"]) == 1 - assert int(result["Zxid"]) > 10 + assert int(result["Zxid"], 16) > 10 assert result["Mode"] == "leader" assert result["Node count"] == "14" @@ -369,7 +369,7 @@ def test_cmd_stat(started_cluster): assert result["Received"] == "10" assert result["Sent"] == "10" assert int(result["Connections"]) == 1 - assert int(result["Zxid"]) >= 10 + assert int(result["Zxid"], 16) >= 10 assert result["Mode"] == "leader" assert result["Node count"] == "14" From 9307e60c20fd634da1e88f2e839a30c963e7f4c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 23:21:26 +0300 Subject: [PATCH 0585/2047] Revert "Millisecond and microsecond support in date_diff / age functions" --- .../functions/date-time-functions.md | 6 +- .../functions/date-time-functions.md | 6 +- .../functions/date-time-functions.md | 2 - src/Core/DecimalFunctions.h | 6 +- src/Functions/DateTimeTransforms.h | 71 ++---------- src/Functions/TransformDateTime64.h | 7 +- src/Functions/dateDiff.cpp | 58 +++------- src/Functions/toStartOfInterval.cpp | 1 + ...0479_date_and_datetime_to_number.reference | 1 - .../00479_date_and_datetime_to_number.sql | 1 - .../02160_special_functions.reference | 10 -- .../0_stateless/02160_special_functions.sql | 12 -- .../02477_age_datetime64.reference | 106 ------------------ .../0_stateless/02477_age_datetime64.sql | 65 ----------- 14 files changed, 29 insertions(+), 323 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 40ede6e0b62..3f61e7a214d 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -718,7 +718,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d ## age -Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 microsecond. +Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 second. E.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for `day` unit, 0 months for `month` unit, 0 years for `year` unit. For an alternative to `age`, see function `date\_diff`. @@ -734,8 +734,6 @@ age('unit', startdate, enddate, [timezone]) - `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). Possible values: - - `microsecond` (possible abbreviations: `us`, `u`) - - `millisecond` (possible abbreviations: `ms`) - `second` (possible abbreviations: `ss`, `s`) - `minute` (possible abbreviations: `mi`, `n`) - `hour` (possible abbreviations: `hh`, `h`) @@ -811,8 +809,6 @@ Aliases: `dateDiff`, `DATE_DIFF`, `timestampDiff`, `timestamp_diff`, `TIMESTAMP_ - `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). Possible values: - - `microsecond` (possible abbreviations: `us`, `u`) - - `millisecond` (possible abbreviations: `ms`) - `second` (possible abbreviations: `ss`, `s`) - `minute` (possible abbreviations: `mi`, `n`) - `hour` (possible abbreviations: `hh`, `h`) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 864ae6e905e..17ab04b7799 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -621,7 +621,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d ## age -Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 микросекунду. +Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 секунду. Например, разница между `2021-12-29` и `2022-01-01` 3 дня для единицы `day`, 0 месяцев для единицы `month`, 0 лет для единицы `year`. **Синтаксис** @@ -635,8 +635,6 @@ age('unit', startdate, enddate, [timezone]) - `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md). Возможные значения: - - `microsecond` (возможные сокращения: `us`, `u`) - - `millisecond` (возможные сокращения: `ms`) - `second` (возможные сокращения: `ss`, `s`) - `minute` (возможные сокращения: `mi`, `n`) - `hour` (возможные сокращения: `hh`, `h`) @@ -710,8 +708,6 @@ date_diff('unit', startdate, enddate, [timezone]) - `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md). Возможные значения: - - `microsecond` (возможные сокращения: `us`, `u`) - - `millisecond` (возможные сокращения: `ms`) - `second` (возможные сокращения: `ss`, `s`) - `minute` (возможные сокращения: `mi`, `n`) - `hour` (возможные сокращения: `hh`, `h`) diff --git a/docs/zh/sql-reference/functions/date-time-functions.md b/docs/zh/sql-reference/functions/date-time-functions.md index e4b70322477..53dadc23c6d 100644 --- a/docs/zh/sql-reference/functions/date-time-functions.md +++ b/docs/zh/sql-reference/functions/date-time-functions.md @@ -643,8 +643,6 @@ date_diff('unit', startdate, enddate, [timezone]) - `unit` — `value`对应的时间单位。类型为[String](../../sql-reference/data-types/string.md)。 可能的值: - - `microsecond` - - `millisecond` - `second` - `minute` - `hour` diff --git a/src/Core/DecimalFunctions.h b/src/Core/DecimalFunctions.h index 17d95650730..357cff2c541 100644 --- a/src/Core/DecimalFunctions.h +++ b/src/Core/DecimalFunctions.h @@ -48,11 +48,7 @@ inline auto scaleMultiplier(UInt32 scale) /** Components of DecimalX value: * whole - represents whole part of decimal, can be negative or positive. - * fractional - for fractional part of decimal. - * - * 0.123 represents 0 / 0.123 - * -0.123 represents 0 / -0.123 - * -1.123 represents -1 / 0.123 + * fractional - for fractional part of decimal, always positive. */ template struct DecimalComponents diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index a1c880f6956..510a88db2b6 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -19,9 +19,6 @@ namespace DB { -static constexpr auto microsecond_multiplier = 1000000; -static constexpr auto millisecond_multiplier = 1000; - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -1380,36 +1377,6 @@ struct ToRelativeSecondNumImpl using FactorTransform = ZeroTransform; }; -template -struct ToRelativeSubsecondNumImpl -{ - static constexpr auto name = "toRelativeSubsecondNumImpl"; - - static inline Int64 execute(const DateTime64 & t, DateTime64::NativeType scale, const DateLUTImpl &) - { - static_assert(scale_multiplier == 1000 || scale_multiplier == 1000000); - if (scale == scale_multiplier) - return t.value; - if (scale > scale_multiplier) - return t.value / (scale / scale_multiplier); - return t.value * (scale_multiplier / scale); - } - static inline Int64 execute(UInt32 t, const DateLUTImpl &) - { - return t * scale_multiplier; - } - static inline Int64 execute(Int32 d, const DateLUTImpl & time_zone) - { - return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))) * scale_multiplier; - } - static inline Int64 execute(UInt16 d, const DateLUTImpl & time_zone) - { - return static_cast(time_zone.fromDayNum(DayNum(d)) * scale_multiplier); - } - - using FactorTransform = ZeroTransform; -}; - struct ToYYYYMMImpl { static constexpr auto name = "toYYYYMM"; @@ -1509,47 +1476,25 @@ struct ToYYYYMMDDhhmmssImpl using FactorTransform = ZeroTransform; }; -struct DateTimeComponentsWithFractionalPart : public DateLUTImpl::DateTimeComponents -{ - UInt16 millisecond; - UInt16 microsecond; -}; - struct ToDateTimeComponentsImpl { static constexpr auto name = "toDateTimeComponents"; - static inline DateTimeComponentsWithFractionalPart execute(const DateTime64 & t, DateTime64::NativeType scale_multiplier, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(Int64 t, const DateLUTImpl & time_zone) { - auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); - - if (t.value < 0 && components.fractional) - { - components.fractional = scale_multiplier + (components.whole ? Int64(-1) : Int64(1)) * components.fractional; - --components.whole; - } - Int64 fractional = components.fractional; - if (scale_multiplier > microsecond_multiplier) - fractional = fractional / (scale_multiplier / microsecond_multiplier); - else if (scale_multiplier < microsecond_multiplier) - fractional = fractional * (microsecond_multiplier / scale_multiplier); - - constexpr Int64 divider = microsecond_multiplier/ millisecond_multiplier; - UInt16 millisecond = static_cast(fractional / divider); - UInt16 microsecond = static_cast(fractional % divider); - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(components.whole), millisecond, microsecond}; + return time_zone.toDateTimeComponents(t); } - static inline DateTimeComponentsWithFractionalPart execute(UInt32 t, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(UInt32 t, const DateLUTImpl & time_zone) { - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(static_cast(t)), 0, 0}; + return time_zone.toDateTimeComponents(static_cast(t)); } - static inline DateTimeComponentsWithFractionalPart execute(Int32 d, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(Int32 d, const DateLUTImpl & time_zone) { - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(ExtendedDayNum(d)), 0, 0}; + return time_zone.toDateTimeComponents(ExtendedDayNum(d)); } - static inline DateTimeComponentsWithFractionalPart execute(UInt16 d, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(UInt16 d, const DateLUTImpl & time_zone) { - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(DayNum(d)), 0, 0}; + return time_zone.toDateTimeComponents(DayNum(d)); } using FactorTransform = ZeroTransform; diff --git a/src/Functions/TransformDateTime64.h b/src/Functions/TransformDateTime64.h index fcee2753066..3dab9efeb6b 100644 --- a/src/Functions/TransformDateTime64.h +++ b/src/Functions/TransformDateTime64.h @@ -5,7 +5,7 @@ namespace DB { -/** Transform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform. +/** Tansform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform. * * Depending on what overloads of Transform::execute() are available, when called with DateTime64 value, * invokes Transform::execute() with either: @@ -80,10 +80,7 @@ public: } else { - auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); - if (t.value < 0 && components.fractional) - --components.whole; - + const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); return wrapped_transform.execute(static_cast(components.whole), std::forward(args)...); } } diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 6bfbbb7c735..8361e9db166 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -174,13 +174,12 @@ public: { auto res = static_cast(transform_y.execute(y, timezone_y)) - static_cast(transform_x.execute(x, timezone_x)); - DateTimeComponentsWithFractionalPart a_comp; - DateTimeComponentsWithFractionalPart b_comp; + DateLUTImpl::DateTimeComponents a_comp; + DateLUTImpl::DateTimeComponents b_comp; Int64 adjust_value; - auto x_microseconds = TransformDateTime64>(transform_x.getScaleMultiplier()).execute(x, timezone_x); - auto y_microseconds = TransformDateTime64>(transform_y.getScaleMultiplier()).execute(y, timezone_y); - - if (x_microseconds <= y_microseconds) + auto x_seconds = TransformDateTime64>(transform_x.getScaleMultiplier()).execute(x, timezone_x); + auto y_seconds = TransformDateTime64>(transform_y.getScaleMultiplier()).execute(y, timezone_y); + if (x_seconds <= y_seconds) { a_comp = TransformDateTime64(transform_x.getScaleMultiplier()).execute(x, timezone_x); b_comp = TransformDateTime64(transform_y.getScaleMultiplier()).execute(y, timezone_y); @@ -193,16 +192,14 @@ public: adjust_value = 1; } - if constexpr (std::is_same_v>>) { if ((a_comp.date.month > b_comp.date.month) || ((a_comp.date.month == b_comp.date.month) && ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) + ))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -213,9 +210,8 @@ public: || ((x_month_in_quarter == y_month_in_quarter) && ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) + ))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -223,9 +219,8 @@ public: if ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) + ))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -235,44 +230,25 @@ public: if ((x_day_of_week > y_day_of_week) || ((x_day_of_week == y_day_of_week) && (a_comp.time.hour > b_comp.time.hour)) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { if ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { if ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))) res += adjust_value; } else if constexpr (std::is_same_v>>) { - if ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))) - res += adjust_value; - } - else if constexpr (std::is_same_v>>) - { - if ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))) - res += adjust_value; - } - else if constexpr (std::is_same_v>>) - { - if (a_comp.microsecond > b_comp.microsecond) + if (a_comp.time.second > b_comp.time.second) res += adjust_value; } return res; @@ -397,10 +373,6 @@ public: impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "second" || unit == "ss" || unit == "s") impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); - else if (unit == "millisecond" || unit == "ms") - impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); - else if (unit == "microsecond" || unit == "us" || unit == "u") - impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} does not support '{}' unit", getName(), unit); diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 48bf88cb14c..649242d0d86 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference index 168b733d702..1375ccb1542 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference @@ -4,4 +4,3 @@ 201707 20170721 20170721112233 -19691231235959 diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql index 1e35e99a802..71151690028 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql @@ -4,4 +4,3 @@ SELECT toYYYYMMDDhhmmss(toDate('2017-07-21')); SELECT toYYYYMM(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDD(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDDhhmmss(toDateTime('2017-07-21T11:22:33')); -SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); diff --git a/tests/queries/0_stateless/02160_special_functions.reference b/tests/queries/0_stateless/02160_special_functions.reference index 5e7e3383d8d..3a1dcd88902 100644 --- a/tests/queries/0_stateless/02160_special_functions.reference +++ b/tests/queries/0_stateless/02160_special_functions.reference @@ -33,14 +33,4 @@ Hello 2021-01-01 1 1 -86400000 -172800000 -86461000 -86401299 -701 -701 -800 -60200201 -60 -10 1 diff --git a/tests/queries/0_stateless/02160_special_functions.sql b/tests/queries/0_stateless/02160_special_functions.sql index 64919536be3..6d18e7d0d25 100644 --- a/tests/queries/0_stateless/02160_special_functions.sql +++ b/tests/queries/0_stateless/02160_special_functions.sql @@ -41,16 +41,4 @@ SELECT TIMESTAMPSUB(DATE '2022-01-01', INTERVAL 1 YEAR); SELECT DATE_DIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); SELECT DATEDIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); -SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); - -SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); -SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); - SELECT EXISTS (SELECT 1); diff --git a/tests/queries/0_stateless/02477_age_datetime64.reference b/tests/queries/0_stateless/02477_age_datetime64.reference index c8c716e1e9a..3b4459dd26d 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.reference +++ b/tests/queries/0_stateless/02477_age_datetime64.reference @@ -111,109 +111,3 @@ SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), ma 1 SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); 1 --- DateTime64 vs DateTime64 with fractional part -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); -5100200 -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); -5100200 -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); -5100 -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); -5099 -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); -4 -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); -4 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); -5 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); -4 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); -4 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -3 -SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); -3 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); -2 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); -2 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); -1 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); -16 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); -15 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); -5 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); -4 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); -8 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); -7 --- DateTime64 vs DateTime64 with negative time -SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); -2349 -SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); -2 -SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -5 -SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -4 diff --git a/tests/queries/0_stateless/02477_age_datetime64.sql b/tests/queries/0_stateless/02477_age_datetime64.sql index 889137395a3..1bed93991ca 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.sql +++ b/tests/queries/0_stateless/02477_age_datetime64.sql @@ -75,68 +75,3 @@ SELECT age('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), SELECT age('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC'))); SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDate('2015-08-19', 'UTC'))); SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); - --- DateTime64 vs DateTime64 with fractional part -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); - -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); - -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); - -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); - -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); - -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); - -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); - -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); - -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); - -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); - --- DateTime64 vs DateTime64 with negative time -SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); \ No newline at end of file From cbe0a0e748cd0301292bb8079bd093e998d3031c Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 15 Jul 2023 03:04:56 +0200 Subject: [PATCH 0586/2047] upd threshold due to dt64 range change --- src/IO/ReadHelpers.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index b8ce162ec91..2636898c1b3 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1005,8 +1005,8 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re } } } - /// 9908870400 is time_t value for 2184-01-01 UTC (a bit over the last year supported by DateTime64) - else if (whole >= 9908870400LL) + /// 10413792000 is time_t value for 2300-01-01 UTC (a bit over the last year supported by DateTime64) + else if (whole >= 10413792000LL) { /// Unix timestamp with subsecond precision, already scaled to integer. /// For disambiguation we support only time since 2001-09-09 01:46:40 UTC and less than 30 000 years in future. From 761cc7afa584438824d76ed2ed9385a0b340fc04 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 15 Jul 2023 03:17:18 +0200 Subject: [PATCH 0587/2047] add tests --- .../0_stateless/01802_toDateTime64_large_values.reference | 4 ++++ .../queries/0_stateless/01802_toDateTime64_large_values.sql | 5 ++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01802_toDateTime64_large_values.reference b/tests/queries/0_stateless/01802_toDateTime64_large_values.reference index e60b1c30314..f3810cc3d4b 100644 --- a/tests/queries/0_stateless/01802_toDateTime64_large_values.reference +++ b/tests/queries/0_stateless/01802_toDateTime64_large_values.reference @@ -8,3 +8,7 @@ SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); 2205-12-12 12:12:12.000000 SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); 2205-12-12 12:12:12.000000 +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); +2299-12-31 23:59:59.000 +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); +2299-12-31 23:59:59.000 diff --git a/tests/queries/0_stateless/01802_toDateTime64_large_values.sql b/tests/queries/0_stateless/01802_toDateTime64_large_values.sql index d82d4433b2d..5c2e65188c3 100644 --- a/tests/queries/0_stateless/01802_toDateTime64_large_values.sql +++ b/tests/queries/0_stateless/01802_toDateTime64_large_values.sql @@ -4,4 +4,7 @@ SELECT toDateTime64('2205-12-12 12:12:12', 0, 'UTC'); SELECT toDateTime64('2205-12-12 12:12:12', 0, 'Asia/Istanbul'); SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); -SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); \ No newline at end of file +SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); + +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); \ No newline at end of file From 801d0955ececaa7141b3ed5a4210afab6eb19d1d Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 14 Jul 2023 12:08:40 +0300 Subject: [PATCH 0588/2047] Add tests --- tests/integration/helpers/cluster.py | 18 +++ tests/integration/parallel_skip.json | 6 +- tests/integration/test_dns_cache/test.py | 32 +++-- .../test_http_failover/__init__.py | 0 .../test_http_failover/configs/listen.xml | 1 + tests/integration/test_http_failover/test.py | 113 ++++++++++++++++++ 6 files changed, 152 insertions(+), 18 deletions(-) create mode 100644 tests/integration/test_http_failover/__init__.py create mode 100644 tests/integration/test_http_failover/configs/listen.xml create mode 100644 tests/integration/test_http_failover/test.py diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0614cbf0e0d..8548bc2b1b2 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3522,6 +3522,24 @@ class ClickHouseInstance: return error + def append_hosts(self, name, ip): + self.exec_in_container( + (["bash", "-c", "echo '{}' {} >> /etc/hosts".format(ip, name)]), + privileged=True, + user="root", + ) + + def set_hosts(self, hosts): + entries = ["127.0.0.1 localhost", "::1 localhost"] + for host in hosts: + entries.append(f"{host[0]} {host[1]}") + + self.exec_in_container( + ["bash", "-c", 'echo -e "{}" > /etc/hosts'.format("\\n".join(entries))], + privileged=True, + user="root", + ) + # Connects to the instance via HTTP interface, sends a query and returns both the answer and the error message # as a tuple (output, error). def http_query_and_get_answer_with_error( diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index d060218456a..407fe7d1b01 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -68,5 +68,9 @@ "test_server_reload/test.py::test_remove_postgresql_port", "test_server_reload/test.py::test_remove_tcp_port", - "test_keeper_map/test.py::test_keeper_map_without_zk" + "test_keeper_map/test.py::test_keeper_map_without_zk", + + "test_http_failover/test.py::test_url_destination_host_with_multiple_addrs", + "test_http_failover/test.py::test_url_invalid_hostname", + "test_http_failover/test.py::test_url_ip_change" ] diff --git a/tests/integration/test_dns_cache/test.py b/tests/integration/test_dns_cache/test.py index 98417930713..9c1c9797383 100644 --- a/tests/integration/test_dns_cache/test.py +++ b/tests/integration/test_dns_cache/test.py @@ -55,6 +55,13 @@ def cluster_without_dns_cache_update(): # node1 is a source, node2 downloads data # node2 has long dns_cache_update_period, so dns cache update wouldn't work def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): + # In this case we should manually set up the static DNS entries on the source host + # to exclude resplving addresses automatically added by docker. + # We use ipv6 for hosts, but resolved DNS entries may contain an unexpected ipv4 address. + node2.set_hosts([("2001:3984:3989::1:1111", "node1")]) + # drop DNS cache + node2.query("SYSTEM DROP DNS CACHE") + # First we check, that normal replication works node1.query( "INSERT INTO test_table_drop VALUES ('2018-10-01', 1), ('2018-10-02', 2), ('2018-10-03', 3)" @@ -64,6 +71,7 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update): # We change source node ip cluster.restart_instance_with_ip_change(node1, "2001:3984:3989::1:7777") + node2.set_hosts([("2001:3984:3989::1:7777", "node1")]) # Put some data to source node1 node1.query( @@ -163,17 +171,8 @@ def test_ip_change_update_dns_cache(cluster_with_dns_cache_update): assert_eq_with_retry(node4, "SELECT count(*) from test_table_update", "7") -def set_hosts(node, hosts): - new_content = "\\n".join(["127.0.0.1 localhost", "::1 localhost"] + hosts) - node.exec_in_container( - ["bash", "-c", 'echo -e "{}" > /etc/hosts'.format(new_content)], - privileged=True, - user="root", - ) - - def test_dns_cache_update(cluster_with_dns_cache_update): - set_hosts(node4, ["127.255.255.255 lost_host"]) + node4.set_hosts([("127.255.255.255", "lost_host")]) with pytest.raises(QueryRuntimeException): node4.query("SELECT * FROM remote('lost_host', 'system', 'one')") @@ -184,7 +183,7 @@ def test_dns_cache_update(cluster_with_dns_cache_update): with pytest.raises(QueryRuntimeException): node4.query("SELECT * FROM distributed_lost_host") - set_hosts(node4, ["127.0.0.1 lost_host"]) + node4.set_hosts([("127.0.0.1", "lost_host")]) # Wait a bit until dns cache will be updated assert_eq_with_retry( @@ -239,11 +238,10 @@ def test_user_access_ip_change(cluster_with_dns_cache_update, node): == "0\n" ) - set_hosts( - node, + node.set_hosts( [ - "127.255.255.255 node3", - "2001:3984:3989::1:88{}4 unknown_host".format(node_num), + ("127.255.255.255", "node3"), + (f"2001:3984:3989::1:88{node_num}4", "unknown_host"), ], ) @@ -260,7 +258,7 @@ def test_user_access_ip_change(cluster_with_dns_cache_update, node): node4.query("SELECT * FROM remote('{}', 'system', 'one')".format(node_name)) # now wrong addresses are cached - set_hosts(node, []) + node.set_hosts([]) retry_count = 60 if node_name == "node5": # client is not allowed to connect, so execute it directly in container to send query from localhost @@ -298,7 +296,7 @@ def test_host_is_drop_from_cache_after_consecutive_failures( # Note that the list of hosts in variable since lost_host will be there too (and it's dropped and added back) # dns_update_short -> dns_max_consecutive_failures set to 6 assert node4.wait_for_log_line( - "Cannot resolve host \\(InvalidHostThatDoesNotExist\\), error 0: Host not found." + "Code: 198. DB::Exception: Not found address of host: InvalidHostThatDoesNotExist." ) assert node4.wait_for_log_line( "Cached hosts not found:.*InvalidHostThatDoesNotExist**", diff --git a/tests/integration/test_http_failover/__init__.py b/tests/integration/test_http_failover/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_http_failover/configs/listen.xml b/tests/integration/test_http_failover/configs/listen.xml new file mode 100644 index 00000000000..3abb37d5da2 --- /dev/null +++ b/tests/integration/test_http_failover/configs/listen.xml @@ -0,0 +1 @@ +:: diff --git a/tests/integration/test_http_failover/test.py b/tests/integration/test_http_failover/test.py new file mode 100644 index 00000000000..41b55ef635c --- /dev/null +++ b/tests/integration/test_http_failover/test.py @@ -0,0 +1,113 @@ +import pytest +from contextlib import nullcontext as does_not_raise +from helpers.cluster import ClickHouseCluster +from helpers.client import QueryRuntimeException +from helpers.test_tools import exec_query_with_retry +from helpers.test_tools import assert_eq_with_retry + + +ACCESSIBLE_IPV4 = "10.5.172.10" +OTHER_ACCESSIBLE_IPV4 = "10.5.172.20" +NOT_ACCESSIBLE_IPV4 = "10.5.172.11" + +ACCESSIBLE_IPV6 = "2001:3984:3989::1:1000" +NOT_ACCESSIBLE_IPV6 = "2001:3984:3989::1:1001" + +DST_NODE_IPV4 = ACCESSIBLE_IPV4 +DST_NODE_IPV6 = ACCESSIBLE_IPV6 +SRC_NODE_IPV6 = "2001:3984:3989::1:2000" + + +cluster = ClickHouseCluster(__file__) + +# Destination node +dst_node = cluster.add_instance( + "dst_node", + with_zookeeper=True, + ipv4_address=DST_NODE_IPV4, + ipv6_address=DST_NODE_IPV6, + main_configs=["configs/listen.xml"], +) +# Source node +src_node = cluster.add_instance( + "src_node", + with_zookeeper=True, + ipv6_address=SRC_NODE_IPV6, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + pass + + +@pytest.fixture +def dst_node_addrs(started_cluster, request): + src_node.set_hosts([(ip, "dst_node") for ip in request.param]) + src_node.query("SYSTEM DROP DNS CACHE") + + yield + + # Clear static DNS entries + src_node.set_hosts([]) + src_node.query("SYSTEM DROP DNS CACHE") + + +@pytest.mark.parametrize( + "dst_node_addrs, expectation", + [ + ((ACCESSIBLE_IPV4, ACCESSIBLE_IPV6), does_not_raise()), + ((NOT_ACCESSIBLE_IPV4, ACCESSIBLE_IPV6), does_not_raise()), + ((ACCESSIBLE_IPV4, NOT_ACCESSIBLE_IPV6), does_not_raise()), + ( + (NOT_ACCESSIBLE_IPV4, NOT_ACCESSIBLE_IPV6), + pytest.raises(QueryRuntimeException), + ), + ], + indirect=["dst_node_addrs"], +) +def test_url_destination_host_with_multiple_addrs(dst_node_addrs, expectation): + with expectation: + result = src_node.query( + "SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')" + ) + assert result == "42\n" + + +def test_url_invalid_hostname(started_cluster): + with pytest.raises(QueryRuntimeException): + src_node.query( + "SELECT count(*) FROM url('http://notvalidhost:8123/?query=SELECT+1', TSV, 'column1 UInt32');" + ) + + +def test_url_ip_change(started_cluster): + assert ( + src_node.query( + "SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')" + ) + == "42\n" + ) + + started_cluster.restart_instance_with_ip_change(dst_node, OTHER_ACCESSIBLE_IPV4) + + # Ensure that only new IPV4 address is accessible + src_node.set_hosts( + [(OTHER_ACCESSIBLE_IPV4, "dst_node"), (NOT_ACCESSIBLE_IPV6, "dst_node")] + ) + src_node.query("SYSTEM DROP DNS CACHE") + + assert ( + src_node.query( + "SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')" + ) + == "42\n" + ) From 08defa36b2f9d5a9b9d70a21132d4d766b92c509 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 14 Jul 2023 12:09:22 +0300 Subject: [PATCH 0589/2047] Add code --- .../Net/include/Poco/Net/HTTPClientSession.h | 2 +- .../S3/ProxyResolverConfiguration.cpp | 2 +- src/IO/HTTPCommon.cpp | 134 +++++++++++------- src/IO/HTTPCommon.h | 4 +- src/IO/S3/PocoHTTPClient.cpp | 8 +- 5 files changed, 89 insertions(+), 61 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPClientSession.h b/base/poco/Net/include/Poco/Net/HTTPClientSession.h index d495d662f75..167a06eb7ff 100644 --- a/base/poco/Net/include/Poco/Net/HTTPClientSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPClientSession.h @@ -306,7 +306,7 @@ namespace Net DEFAULT_KEEP_ALIVE_TIMEOUT = 8 }; - void reconnect(); + virtual void reconnect(); /// Connects the underlying socket to the HTTP server. int write(const char * buffer, std::streamsize length); diff --git a/src/Disks/ObjectStorages/S3/ProxyResolverConfiguration.cpp b/src/Disks/ObjectStorages/S3/ProxyResolverConfiguration.cpp index 14db39b3f3d..18c644f3680 100644 --- a/src/Disks/ObjectStorages/S3/ProxyResolverConfiguration.cpp +++ b/src/Disks/ObjectStorages/S3/ProxyResolverConfiguration.cpp @@ -60,7 +60,7 @@ ClientConfigurationPerRequest ProxyResolverConfiguration::getConfiguration(const { auto resolved_endpoint = endpoint; resolved_endpoint.setHost(resolved_hosts[i].toString()); - session = makeHTTPSession(resolved_endpoint, timeouts, false); + session = makeHTTPSession(resolved_endpoint, timeouts); try { diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 2f5e0a172a0..a5816911c09 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -24,9 +25,9 @@ #include +#include #include #include -#include namespace ProfileEvents @@ -54,6 +55,78 @@ namespace session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); } + template + requires std::derived_from + class HTTPSessionAdapter : public Session + { + static_assert(std::has_virtual_destructor_v, "The base class must have a virtual destructor"); + + public: + HTTPSessionAdapter(const std::string & host, UInt16 port) : Session(host, port), log{&Poco::Logger::get("HTTPSessionAdapter")} { } + ~HTTPSessionAdapter() override = default; + + protected: + void reconnect() override + { + // First of all will try to establish connection with last used addr. + if (!Session::getResolvedHost().empty()) + { + try + { + Session::reconnect(); + return; + } + catch (...) + { + Session::close(); + LOG_TRACE( + log, + "Last ip ({}) is unreachable for {}:{}. Will try another resolved address.", + Session::getResolvedHost(), + Session::getHost(), + Session::getPort()); + } + } + + const auto endpoinds = DNSResolver::instance().resolveHostAll(Session::getHost()); + + for (auto it = endpoinds.begin();;) + { + try + { + Session::setResolvedHost(it->toString()); + Session::reconnect(); + + LOG_TRACE( + log, + "Created HTTP(S) session with {}:{} ({}:{})", + Session::getHost(), + Session::getPort(), + it->toString(), + Session::getPort()); + + break; + } + catch (...) + { + Session::close(); + if (++it == endpoinds.end()) + { + Session::setResolvedHost(""); + throw; + } + LOG_TRACE( + log, + "Failed to create connection with {}:{}, Will try another resolved address. {}", + Session::getResolvedHost(), + Session::getPort(), + getCurrentExceptionMessage(false)); + } + } + } + Poco::Logger * log; + }; + bool isHTTPS(const Poco::URI & uri) { if (uri.getScheme() == "https") @@ -64,28 +137,21 @@ namespace throw Exception(ErrorCodes::UNSUPPORTED_URI_SCHEME, "Unsupported scheme in URI '{}'", uri.toString()); } - HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive, bool resolve_host = true) + HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive) { HTTPSessionPtr session; if (https) { #if USE_SSL - /// Cannot resolve host in advance, otherwise SNI won't work in Poco. - /// For more information about SNI, see the https://en.wikipedia.org/wiki/Server_Name_Indication - auto https_session = std::make_shared(host, port); - if (resolve_host) - https_session->setResolvedHost(DNSResolver::instance().resolveHost(host).toString()); - - session = std::move(https_session); + session = std::make_shared>(host, port); #else throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without HTTPS support"); #endif } else { - String resolved_host = resolve_host ? DNSResolver::instance().resolveHost(host).toString() : host; - session = std::make_shared(resolved_host, port); + session = std::make_shared>(host, port); } ProfileEvents::increment(ProfileEvents::CreatedHTTPConnections); @@ -104,7 +170,6 @@ namespace const String proxy_host; const UInt16 proxy_port; const bool proxy_https; - const bool resolve_host; using Base = PoolBase; @@ -113,7 +178,7 @@ namespace /// Pool is global, we shouldn't attribute this memory to query/user. MemoryTrackerSwitcher switcher{&total_memory_tracker}; - auto session = makeHTTPSessionImpl(host, port, https, true, resolve_host); + auto session = makeHTTPSessionImpl(host, port, https, true); if (!proxy_host.empty()) { const String proxy_scheme = proxy_https ? "https" : "http"; @@ -137,7 +202,6 @@ namespace UInt16 proxy_port_, bool proxy_https_, size_t max_pool_size_, - bool resolve_host_, bool wait_on_pool_size_limit) : Base( static_cast(max_pool_size_), @@ -149,7 +213,6 @@ namespace , proxy_host(proxy_host_) , proxy_port(proxy_port_) , proxy_https(proxy_https_) - , resolve_host(resolve_host_) { } }; @@ -197,24 +260,6 @@ namespace std::mutex mutex; std::unordered_map endpoints_pool; - void updateHostIfIpChanged(Entry & session, const String & new_ip) - { - const auto old_ip = session->getResolvedHost().empty() ? session->getHost() : session->getResolvedHost(); - - if (new_ip != old_ip) - { - session->reset(); - if (session->getResolvedHost().empty()) - { - session->setHost(new_ip); - } - else - { - session->setResolvedHost(new_ip); - } - } - } - protected: HTTPSessionPool() = default; @@ -230,7 +275,6 @@ namespace const Poco::URI & proxy_uri, const ConnectionTimeouts & timeouts, size_t max_connections_per_endpoint, - bool resolve_host, bool wait_on_pool_size_limit) { std::unique_lock lock(mutex); @@ -261,7 +305,6 @@ namespace proxy_port, proxy_https, max_connections_per_endpoint, - resolve_host, wait_on_pool_size_limit)); /// Some routines held session objects until the end of its lifetime. Also this routines may create another sessions in this time frame. @@ -273,17 +316,6 @@ namespace auto retry_timeout = timeouts.connection_timeout.totalMicroseconds(); auto session = pool_ptr->second->get(retry_timeout); - const auto & session_data = session->sessionData(); - if (session_data.empty() || !Poco::AnyCast(&session_data)) - { - session->reset(); - - if (resolve_host) - updateHostIfIpChanged(session, DNSResolver::instance().resolveHost(host).toString()); - } - - session->attachSessionData({}); - setTimeouts(*session, timeouts); return session; @@ -301,13 +333,13 @@ void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_ response.set("Keep-Alive", "timeout=" + std::to_string(timeout.totalSeconds())); } -HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, bool resolve_host) +HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts) { const std::string & host = uri.getHost(); UInt16 port = uri.getPort(); bool https = isHTTPS(uri); - auto session = makeHTTPSessionImpl(host, port, https, false, resolve_host); + auto session = makeHTTPSessionImpl(host, port, https, false); setTimeouts(*session, timeouts); return session; } @@ -317,10 +349,9 @@ PooledHTTPSessionPtr makePooledHTTPSession( const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, - bool resolve_host, bool wait_on_pool_size_limit) { - return makePooledHTTPSession(uri, {}, timeouts, per_endpoint_pool_size, resolve_host, wait_on_pool_size_limit); + return makePooledHTTPSession(uri, {}, timeouts, per_endpoint_pool_size, wait_on_pool_size_limit); } PooledHTTPSessionPtr makePooledHTTPSession( @@ -328,10 +359,9 @@ PooledHTTPSessionPtr makePooledHTTPSession( const Poco::URI & proxy_uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, - bool resolve_host, bool wait_on_pool_size_limit) { - return HTTPSessionPool::instance().getSession(uri, proxy_uri, timeouts, per_endpoint_pool_size, resolve_host, wait_on_pool_size_limit); + return HTTPSessionPool::instance().getSession(uri, proxy_uri, timeouts, per_endpoint_pool_size, wait_on_pool_size_limit); } bool isRedirect(const Poco::Net::HTTPResponse::HTTPStatus status) { return status == Poco::Net::HTTPResponse::HTTP_MOVED_PERMANENTLY || status == Poco::Net::HTTPResponse::HTTP_FOUND || status == Poco::Net::HTTPResponse::HTTP_SEE_OTHER || status == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT; } diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index 4733f366c8a..082491b2851 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -70,14 +70,13 @@ void markSessionForReuse(PooledHTTPSessionPtr session); void setResponseDefaultHeaders(HTTPServerResponse & response, size_t keep_alive_timeout); /// Create session object to perform requests and set required parameters. -HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, bool resolve_host = true); +HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts); /// As previous method creates session, but tooks it from pool, without and with proxy uri. PooledHTTPSessionPtr makePooledHTTPSession( const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, - bool resolve_host = true, bool wait_on_pool_size_limit = true); PooledHTTPSessionPtr makePooledHTTPSession( @@ -85,7 +84,6 @@ PooledHTTPSessionPtr makePooledHTTPSession( const Poco::URI & proxy_uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, - bool resolve_host = true, bool wait_on_pool_size_limit = true); bool isRedirect(Poco::Net::HTTPResponse::HTTPStatus status); diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index d64ddf0ec38..1a367a8199d 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -336,9 +336,9 @@ void PocoHTTPClient::makeRequestInternalImpl( /// This can lead to request signature difference on S3 side. if constexpr (pooled) session = makePooledHTTPSession( - target_uri, timeouts, http_connection_pool_size, /* resolve_host = */ true, wait_on_pool_size_limit); + target_uri, timeouts, http_connection_pool_size, wait_on_pool_size_limit); else - session = makeHTTPSession(target_uri, timeouts, /* resolve_host = */ false); + session = makeHTTPSession(target_uri, timeouts); bool use_tunnel = request_configuration.proxy_scheme == Aws::Http::Scheme::HTTP && target_uri.getScheme() == "https"; session->setProxy( @@ -352,9 +352,9 @@ void PocoHTTPClient::makeRequestInternalImpl( { if constexpr (pooled) session = makePooledHTTPSession( - target_uri, timeouts, http_connection_pool_size, /* resolve_host = */ true, wait_on_pool_size_limit); + target_uri, timeouts, http_connection_pool_size, wait_on_pool_size_limit); else - session = makeHTTPSession(target_uri, timeouts, /* resolve_host = */ false); + session = makeHTTPSession(target_uri, timeouts); } /// In case of error this address will be written to logs From 7d3b28f9c398e4d224243004a6f5c5eddfc4cce2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 14 Jul 2023 21:22:16 +0200 Subject: [PATCH 0590/2047] Bump version of the distributed_ddl_entry_format_version to 5 by default This will enable the following features for distributed DDL queries: - opentelemetry support (#41484) - initial_query_id pass through (#50015) Signed-off-by: Azat Khuzhin --- src/Core/Settings.h | 2 +- ...distributed_ddl_output_mode_long.reference | 32 +++++++++---------- .../02761_ddl_initial_query_id.reference | 2 +- .../0_stateless/02761_ddl_initial_query_id.sh | 5 +-- 4 files changed, 21 insertions(+), 20 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f1a314fed37..d54645bb0de 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -629,7 +629,7 @@ class IColumn; M(Bool, database_replicated_allow_only_replicated_engine, false, "Allow to create only Replicated tables in database with engine Replicated", 0) \ M(Bool, database_replicated_allow_replicated_engine_arguments, true, "Allow to create only Replicated tables in database with engine Replicated with explicit arguments", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \ - M(UInt64, distributed_ddl_entry_format_version, 3, "Compatibility version of distributed DDL (ON CLUSTER) queries", 0) \ + M(UInt64, distributed_ddl_entry_format_version, 5, "Compatibility version of distributed DDL (ON CLUSTER) queries", 0) \ \ M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ M(UInt64, external_storage_max_read_bytes, 0, "Limit maximum number of bytes when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ diff --git a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference index 4397810b68d..bc33c2fa913 100644 --- a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference +++ b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference @@ -27,19 +27,19 @@ localhost 9000 57 Code: 57. Error: Table default.never_throw already exists. (TA localhost 9000 0 1 0 localhost 1 \N \N 1 0 distributed_ddl_queue -2 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.none already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.throw already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.null_status already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.never_throw already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.none already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.throw already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.null_status already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.never_throw already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 diff --git a/tests/queries/0_stateless/02761_ddl_initial_query_id.reference b/tests/queries/0_stateless/02761_ddl_initial_query_id.reference index 5c6f448eed5..aac7a0892e1 100644 --- a/tests/queries/0_stateless/02761_ddl_initial_query_id.reference +++ b/tests/queries/0_stateless/02761_ddl_initial_query_id.reference @@ -1,4 +1,4 @@ -default distributed_ddl_entry_format_version +distributed_ddl_entry_format_version=OPENTELEMETRY_ENABLED_VERSION (older then PRESERVE_INITIAL_QUERY_ID_VERSION) DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost distributed_ddl_entry_format_version=PRESERVE_INITIAL_QUERY_ID_VERSION DROP TABLE IF EXISTS default.foo diff --git a/tests/queries/0_stateless/02761_ddl_initial_query_id.sh b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh index b8b35ef01f7..83890c3f335 100755 --- a/tests/queries/0_stateless/02761_ddl_initial_query_id.sh +++ b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh @@ -4,9 +4,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -echo "default distributed_ddl_entry_format_version" +echo "distributed_ddl_entry_format_version=OPENTELEMETRY_ENABLED_VERSION (older then PRESERVE_INITIAL_QUERY_ID_VERSION)" +OPENTELEMETRY_ENABLED_VERSION=4 query_id="$(random_str 10)" -$CLICKHOUSE_CLIENT --query_id "$query_id" --distributed_ddl_output_mode=none -q "DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost" +$CLICKHOUSE_CLIENT --distributed_ddl_entry_format_version=$OPENTELEMETRY_ENABLED_VERSION --query_id "$query_id" --distributed_ddl_output_mode=none -q "DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost" $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" $CLICKHOUSE_CLIENT -q "SELECT query FROM system.query_log WHERE initial_query_id = '$query_id' AND type != 'QueryStart'" From 7761e5ef9d75bd006e5b0e655682eb0aad9236d8 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 15 Jul 2023 14:42:08 +0200 Subject: [PATCH 0591/2047] support async_deduplicate_token for async insert --- src/Interpreters/AsynchronousInsertQueue.cpp | 14 ++++-- src/Interpreters/AsynchronousInsertQueue.h | 3 +- src/Processors/Chunk.h | 10 ++-- .../MergeTree/MergeTreeDataWriter.cpp | 40 +++++++++------ src/Storages/MergeTree/MergeTreeDataWriter.h | 7 +-- .../MergeTree/ReplicatedMergeTreeSink.cpp | 50 +++++++++++-------- .../MergeTree/tests/gtest_async_inserts.cpp | 31 +++++++----- .../02481_async_insert_dedup.python | 19 +++++-- .../02481_async_insert_dedup_token.reference | 3 ++ .../02481_async_insert_dedup_token.sh | 9 ++++ 10 files changed, 120 insertions(+), 66 deletions(-) create mode 100644 tests/queries/0_stateless/02481_async_insert_dedup_token.reference create mode 100755 tests/queries/0_stateless/02481_async_insert_dedup_token.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index dc2310cfebf..4370c87afba 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -93,6 +93,10 @@ UInt128 AsynchronousInsertQueue::InsertQuery::calculateHash() const for (const auto & setting : settings.allChanged()) { + /// We don't consider this setting because it is only for deduplication, + /// which means we can put two inserts with different tokens in the same block safely. + if (setting.getName() == "insert_deduplication_token") + continue; siphash.update(setting.getName()); applyVisitor(FieldVisitorHash(siphash), setting.getValue()); } @@ -107,9 +111,10 @@ bool AsynchronousInsertQueue::InsertQuery::operator==(const InsertQuery & other) return query_str == other.query_str && settings == other.settings; } -AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_, MemoryTracker * user_memory_tracker_) +AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_, const String & async_dedup_token_, MemoryTracker * user_memory_tracker_) : bytes(std::move(bytes_)) , query_id(std::move(query_id_)) + , async_dedup_token(async_dedup_token_) , user_memory_tracker(user_memory_tracker_) , create_time(std::chrono::system_clock::now()) { @@ -222,7 +227,7 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) /// to avoid buffering of huge amount of data in memory. auto read_buf = getReadBufferFromASTInsertQuery(query); - LimitReadBuffer limit_buf(*read_buf, settings.async_insert_max_data_size, /* trow_exception */ false, /* exact_limit */ {}); + LimitReadBuffer limit_buf(*read_buf, settings.async_insert_max_data_size, /* throw_exception */ false, /* exact_limit */ {}); WriteBufferFromString write_buf(bytes); copyData(limit_buf, write_buf); @@ -248,7 +253,7 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context) if (auto quota = query_context->getQuota()) quota->used(QuotaType::WRITTEN_BYTES, bytes.size()); - auto entry = std::make_shared(std::move(bytes), query_context->getCurrentQueryId(), CurrentThread::getUserMemoryTracker()); + auto entry = std::make_shared(std::move(bytes), query_context->getCurrentQueryId(), settings.insert_deduplication_token, CurrentThread::getUserMemoryTracker()); InsertQuery key{query, settings}; InsertDataPtr data_to_process; @@ -451,7 +456,7 @@ try StreamingFormatExecutor executor(header, format, std::move(on_error), std::move(adding_defaults_transform)); std::unique_ptr last_buffer; - auto chunk_info = std::make_shared(); + auto chunk_info = std::make_shared(); for (const auto & entry : data->entries) { auto buffer = std::make_unique(entry->bytes); @@ -460,6 +465,7 @@ try size_t num_rows = executor.execute(*buffer); total_rows += num_rows; chunk_info->offsets.push_back(total_rows); + chunk_info->tokens.push_back(entry->async_dedup_token); /// Keep buffer, because it still can be used /// in destructor, while resetting buffer at next iteration. diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index bc60c86d067..a75119988d6 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -92,10 +92,11 @@ private: public: String bytes; const String query_id; + const String async_dedup_token; MemoryTracker * const user_memory_tracker; const std::chrono::time_point create_time; - Entry(String && bytes_, String && query_id_, MemoryTracker * user_memory_tracker_); + Entry(String && bytes_, String && query_id_, const String & async_dedup_token, MemoryTracker * user_memory_tracker_); void finish(std::exception_ptr exception_ = nullptr); std::future getFuture() { return promise.get_future(); } diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 413872d512d..7969cc7cceb 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -115,15 +115,17 @@ private: using Chunks = std::vector; /// ChunkOffsets marks offsets of different sub-chunks, which will be used by async inserts. -class ChunkOffsets : public ChunkInfo +class AsyncInsertInfo : public ChunkInfo { public: - ChunkOffsets() = default; - explicit ChunkOffsets(const std::vector & offsets_) : offsets(offsets_) {} + AsyncInsertInfo() = default; + explicit AsyncInsertInfo(const std::vector & offsets_, const std::vector & tokens_) : offsets(offsets_), tokens(tokens_) {} + std::vector offsets; + std::vector tokens; }; -using ChunkOffsetsPtr = std::shared_ptr; +using AsyncInsertInfoPtr = std::shared_ptr; /// Extension to support delayed defaults. AddingDefaultsProcessor uses it to replace missing values with column defaults. class ChunkMissingValues : public ChunkInfo diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 7e306880e9c..ea5d64212f5 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -171,23 +171,23 @@ void MergeTreeDataWriter::TemporaryPart::finalize() projection->getDataPartStorage().precommitTransaction(); } -std::vector scatterOffsetsBySelector(ChunkOffsetsPtr chunk_offsets, const IColumn::Selector & selector, size_t partition_num) +std::vector scatterAsyncInsertInfoBySelector(AsyncInsertInfoPtr async_insert_info, const IColumn::Selector & selector, size_t partition_num) { - if (nullptr == chunk_offsets) + if (nullptr == async_insert_info) { return {}; } if (selector.empty()) { - return {chunk_offsets}; + return {async_insert_info}; } - std::vector result(partition_num); + std::vector result(partition_num); std::vector last_row_for_partition(partition_num, -1); size_t offset_idx = 0; for (size_t i = 0; i < selector.size(); ++i) { ++last_row_for_partition[selector[i]]; - if (i + 1 == chunk_offsets->offsets[offset_idx]) + if (i + 1 == async_insert_info->offsets[offset_idx]) { for (size_t part_id = 0; part_id < last_row_for_partition.size(); ++part_id) { @@ -196,9 +196,12 @@ std::vector scatterOffsetsBySelector(ChunkOffsetsPtr chunk_offs continue; size_t offset = static_cast(last_row + 1); if (result[part_id] == nullptr) - result[part_id] = std::make_shared(); + result[part_id] = std::make_shared(); if (result[part_id]->offsets.empty() || offset > *result[part_id]->offsets.rbegin()) + { result[part_id]->offsets.push_back(offset); + result[part_id]->tokens.push_back(async_insert_info->tokens[offset_idx]); + } } ++offset_idx; } @@ -207,7 +210,7 @@ std::vector scatterOffsetsBySelector(ChunkOffsetsPtr chunk_offs } BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( - const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, ChunkOffsetsPtr chunk_offsets) + const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, AsyncInsertInfoPtr async_insert_info) { BlocksWithPartition result; if (!block || !block.rows()) @@ -218,8 +221,11 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( if (!metadata_snapshot->hasPartitionKey()) /// Table is not partitioned. { result.emplace_back(Block(block), Row{}); - if (chunk_offsets != nullptr) - result[0].offsets = std::move(chunk_offsets->offsets); + if (async_insert_info != nullptr) + { + result[0].offsets = std::move(async_insert_info->offsets); + result[0].tokens = std::move(async_insert_info->tokens); + } return result; } @@ -236,7 +242,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( IColumn::Selector selector; buildScatterSelector(partition_columns, partition_num_to_first_row, selector, max_parts); - auto chunk_offsets_with_partition = scatterOffsetsBySelector(chunk_offsets, selector, partition_num_to_first_row.size()); + auto async_insert_info_with_partition = scatterAsyncInsertInfoBySelector(async_insert_info, selector, partition_num_to_first_row.size()); size_t partitions_count = partition_num_to_first_row.size(); result.reserve(partitions_count); @@ -255,8 +261,11 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( /// NOTE: returning a copy of the original block so that calculated partition key columns /// do not interfere with possible calculated primary key columns of the same name. result.emplace_back(Block(block), get_partition(0)); - if (!chunk_offsets_with_partition.empty()) - result[0].offsets = std::move(chunk_offsets_with_partition[0]->offsets); + if (!async_insert_info_with_partition.empty()) + { + result[0].offsets = std::move(async_insert_info_with_partition[0]->offsets); + result[0].tokens = std::move(async_insert_info_with_partition[0]->tokens); + } return result; } @@ -270,8 +279,11 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( result[i].block.getByPosition(col).column = std::move(scattered[i]); } - for (size_t i = 0; i < chunk_offsets_with_partition.size(); ++i) - result[i].offsets = std::move(chunk_offsets_with_partition[i]->offsets); + for (size_t i = 0; i < async_insert_info_with_partition.size(); ++i) + { + result[i].offsets = std::move(async_insert_info_with_partition[i]->offsets); + result[i].tokens = std::move(async_insert_info_with_partition[i]->tokens); + } return result; } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index 795453b2afa..d6c980d83d8 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -23,14 +23,15 @@ struct BlockWithPartition Block block; Row partition; std::vector offsets; + std::vector tokens; BlockWithPartition(Block && block_, Row && partition_) : block(block_), partition(std::move(partition_)) { } - BlockWithPartition(Block && block_, Row && partition_, std::vector && offsets_) - : block(block_), partition(std::move(partition_)), offsets(std::move(offsets_)) + BlockWithPartition(Block && block_, Row && partition_, std::vector && offsets_, std::vector && tokens_) + : block(block_), partition(std::move(partition_)), offsets(std::move(offsets_)), tokens(std::move(tokens_)) { } }; @@ -51,7 +52,7 @@ public: * (split rows by partition) * Works deterministically: if same block was passed, function will return same result in same order. */ - static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, ChunkOffsetsPtr chunk_offsets = nullptr); + static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, AsyncInsertInfoPtr chunk_offsets = nullptr); /// This structure contains not completely written temporary part. /// Some writes may happen asynchronously, e.g. for blob storages. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 4128654a632..72d19dee038 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -78,7 +78,7 @@ struct ReplicatedMergeTreeSinkImpl::DelayedChunk unmerged_block_with_partition(std::move(unmerged_block_with_partition_)), part_counters(std::move(part_counters_)) { - initBlockIDMap(); + initBlockIDMap(); } void initBlockIDMap() @@ -209,8 +209,8 @@ std::vector testSelfDeduplicate(std::vector data, std::vectorinsert(datum); } Block block({ColumnWithTypeAndName(std::move(column), DataTypePtr(new DataTypeInt64()), "a")}); - - BlockWithPartition block1(std::move(block), Row(), std::move(offsets)); + std::vector tokens(offsets.size()); + BlockWithPartition block1(std::move(block), Row(), std::move(offsets), std::move(tokens)); ProfileEvents::Counters profile_counters; ReplicatedMergeTreeSinkImpl::DelayedChunk::Partition part( &Poco::Logger::get("testSelfDeduplicate"), MergeTreeDataWriter::TemporaryPart(), 0, std::move(hashes), std::move(block1), std::nullopt, std::move(profile_counters)); @@ -242,22 +242,29 @@ namespace size_t start = 0; auto cols = block.block.getColumns(); std::vector block_id_vec; - for (auto offset : block.offsets) + for (size_t i = 0; i < block.offsets.size(); ++i) { - SipHash hash; - for (size_t i = start; i < offset; ++i) + size_t offset = block.offsets[i]; + std::string_view token = block.tokens[i]; + if (token.empty()) { - for (const auto & col : cols) - col->updateHashWithValue(i, hash); - } - union - { - char bytes[16]; - UInt64 words[2]; - } hash_value; - hash.get128(hash_value.bytes); + SipHash hash; + for (size_t j = start; j < offset; ++j) + { + for (const auto & col : cols) + col->updateHashWithValue(j, hash); + } + union + { + char bytes[16]; + UInt64 words[2]; + } hash_value; + hash.get128(hash_value.bytes); - block_id_vec.push_back(partition_id + "_" + DB::toString(hash_value.words[0]) + "_" + DB::toString(hash_value.words[1])); + block_id_vec.push_back(partition_id + "_" + DB::toString(hash_value.words[0]) + "_" + DB::toString(hash_value.words[1])); + } + else + block_id_vec.push_back(partition_id + "_" + std::string(token)); start = offset; } @@ -418,18 +425,18 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) convertDynamicColumnsToTuples(block, storage_snapshot); - ChunkOffsetsPtr chunk_offsets; + AsyncInsertInfoPtr async_insert_info; if constexpr (async_insert) { const auto & chunk_info = chunk.getChunkInfo(); - if (const auto * chunk_offsets_ptr = typeid_cast(chunk_info.get())) - chunk_offsets = std::make_shared(chunk_offsets_ptr->offsets); + if (const auto * async_insert_info_ptr = typeid_cast(chunk_info.get())) + async_insert_info = std::make_shared(async_insert_info_ptr->offsets, async_insert_info_ptr->tokens); else throw Exception(ErrorCodes::LOGICAL_ERROR, "No chunk info for async inserts"); } - auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context, chunk_offsets); + auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context, async_insert_info); using DelayedPartition = typename ReplicatedMergeTreeSinkImpl::DelayedChunk::Partition; using DelayedPartitions = std::vector; @@ -453,7 +460,7 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) { /// we copy everything but offsets which we move because they are only used by async insert if (settings.optimize_on_insert && storage.writer.getMergingMode() != MergeTreeData::MergingParams::Mode::Ordinary) - unmerged_block.emplace(Block(current_block.block), Row(current_block.partition), std::move(current_block.offsets)); + unmerged_block.emplace(Block(current_block.block), Row(current_block.partition), std::move(current_block.offsets), std::move(current_block.tokens)); } /// Write part to the filesystem under temporary name. Calculate a checksum. @@ -468,7 +475,6 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) if constexpr (async_insert) { - /// TODO consider insert_deduplication_token block_id = getHashesForBlocks(unmerged_block.has_value() ? *unmerged_block : current_block, temp_part.part->info.partition_id); LOG_TRACE(log, "async insert part, part id {}, block id {}, offsets {}, size {}", temp_part.part->info.partition_id, toString(block_id), toString(current_block.offsets), current_block.offsets.size()); } diff --git a/src/Storages/MergeTree/tests/gtest_async_inserts.cpp b/src/Storages/MergeTree/tests/gtest_async_inserts.cpp index f67c2f7fb0f..2d8cd0acc3e 100644 --- a/src/Storages/MergeTree/tests/gtest_async_inserts.cpp +++ b/src/Storages/MergeTree/tests/gtest_async_inserts.cpp @@ -8,7 +8,7 @@ namespace DB { -std::vector scatterOffsetsBySelector(ChunkOffsetsPtr chunk_offsets, const IColumn::Selector & selector, size_t partition_num); +std::vector scatterAsyncInsertInfoBySelector(AsyncInsertInfoPtr chunk_offsets, const IColumn::Selector & selector, size_t partition_num); class AsyncInsertsTest : public ::testing::TestPartResult {}; @@ -16,31 +16,36 @@ class AsyncInsertsTest : public ::testing::TestPartResult TEST(AsyncInsertsTest, testScatterOffsetsBySelector) { - auto test_impl = [](std::vector offsets, std::vector selector_data, size_t part_num, std::vector> expected) + auto test_impl = [](std::vector offsets, std::vector selector_data, std::vector tokens, size_t part_num, std::vector>> expected) { - auto offset_ptr = std::make_shared(offsets); + auto offset_ptr = std::make_shared(offsets, tokens); IColumn::Selector selector(selector_data.size()); size_t num_rows = selector_data.size(); for (size_t i = 0; i < num_rows; i++) selector[i] = selector_data[i]; - auto results = scatterOffsetsBySelector(offset_ptr, selector, part_num); + auto results = scatterAsyncInsertInfoBySelector(offset_ptr, selector, part_num); ASSERT_EQ(results.size(), expected.size()); for (size_t i = 0; i < results.size(); i++) { - auto result = results[i]->offsets; + auto result = results[i]; auto expect = expected[i]; - ASSERT_EQ(result.size(), expect.size()); - for (size_t j = 0; j < result.size(); j++) - ASSERT_EQ(result[j], expect[j]); + ASSERT_EQ(result->offsets.size(), expect.size()); + ASSERT_EQ(result->tokens.size(), expect.size()); + for (size_t j = 0; j < expect.size(); j++) + { + ASSERT_EQ(result->offsets[j], std::get<0>(expect[j])); + ASSERT_EQ(result->tokens[j], std::get<1>(expect[j])); + } } }; - test_impl({5}, {0,1,0,1,0}, 2, {{3},{2}}); - test_impl({5,10}, {0,1,0,1,0,1,0,1,0,1}, 2, {{3,5},{2,5}}); - test_impl({4,8,12}, {0,1,0,1,0,2,0,2,1,2,1,2}, 3, {{2,4},{2,4},{2,4}}); - test_impl({1,2,3,4,5}, {0,1,2,3,4}, 5, {{1},{1},{1},{1},{1}}); - test_impl({3,6,10}, {1,1,1,2,2,2,0,0,0,0}, 3, {{4},{3},{3}}); + test_impl({1}, {0}, {"a"}, 1, {{{1,"a"}}}); + test_impl({5}, {0,1,0,1,0}, {"a"}, 2, {{{3,"a"}},{{2,"a"}}}); + test_impl({5,10}, {0,1,0,1,0,1,0,1,0,1}, {"a", "b"}, 2, {{{3,"a"},{5,"b"}},{{2,"a"},{5,"b"}}}); + test_impl({4,8,12}, {0,1,0,1,0,2,0,2,1,2,1,2}, {"a", "b", "c"}, 3, {{{2, "a"},{4, "b"}},{{2,"a"},{4,"c"}},{{2,"b"},{4,"c"}}}); + test_impl({1,2,3,4,5}, {0,1,2,3,4}, {"a", "b", "c", "d", "e"}, 5, {{{1,"a"}},{{1,"b"}},{{1, "c"}},{{1, "d"}},{{1, "e"}}}); + test_impl({3,6,10}, {1,1,1,2,2,2,0,0,0,0}, {"a", "b", "c"}, 3, {{{4, "c"}},{{3, "a"}},{{3, "b"}}}); } std::vector testSelfDeduplicate(std::vector data, std::vector offsets, std::vector hashes); diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index 9fd82da1038..ca83253eaf8 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -39,7 +39,7 @@ client.query("DROP TABLE IF EXISTS t_async_insert_dedup_no_part SYNC") # generate data and push to queue -def generate_data(q, total_number): +def generate_data(q, total_number, use_token): old_data = [] max_chunk_size = 30 partitions = ["2022-11-11 10:10:10", "2022-12-12 10:10:10"] @@ -63,12 +63,16 @@ def generate_data(q, total_number): end = start + chunk_size if end > total_number: end = total_number + + token = "" for i in range(start, end + 1): partition = partitions[random.randint(0, 1)] insert_stmt += "('{}', {}),".format(partition, i) + if use_token: + token = str(i) insert_stmt = insert_stmt[:-1] - q.put(insert_stmt) - old_data.append(insert_stmt) + q.put((insert_stmt, token)) + old_data.append((insert_stmt, token)) last_number = end if end >= total_number: break @@ -80,13 +84,14 @@ def fetch_and_insert_data(q, client): while True: insert = q.get() client.query( - insert, + insert[0], settings={ "async_insert": 1, "async_insert_deduplicate": 1, "wait_for_async_insert": 0, "async_insert_busy_timeout_ms": 1500, "insert_keeper_fault_injection_probability": 0, + "insert_deduplication_token": insert[1], }, ) q.task_done() @@ -110,7 +115,11 @@ ORDER BY (KeyID, EventDate) SETTINGS use_async_block_ids_cache = 1 q = queue.Queue(100) total_number = 10000 -gen = Thread(target=generate_data, args=[q, total_number]) +use_token = False +if sys.argv[-1] == "token": + use_token = True + +gen = Thread(target=generate_data, args=[q, total_number, use_token]) gen.start() for i in range(3): diff --git a/tests/queries/0_stateless/02481_async_insert_dedup_token.reference b/tests/queries/0_stateless/02481_async_insert_dedup_token.reference new file mode 100644 index 00000000000..a91c59a7fc5 --- /dev/null +++ b/tests/queries/0_stateless/02481_async_insert_dedup_token.reference @@ -0,0 +1,3 @@ +5 + +10000 diff --git a/tests/queries/0_stateless/02481_async_insert_dedup_token.sh b/tests/queries/0_stateless/02481_async_insert_dedup_token.sh new file mode 100755 index 00000000000..8ef6eecda24 --- /dev/null +++ b/tests/queries/0_stateless/02481_async_insert_dedup_token.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: long, zookeeper, no-parallel, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# We should have correct env vars from shell_config.sh to run this test +python3 "$CURDIR"/02481_async_insert_dedup.python token From 290c145acfc5accb2344b63356904af5f0a784ba Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 15 Jul 2023 14:53:46 +0000 Subject: [PATCH 0592/2047] Add array_agg as alias of groupArray for PostgreSQL compatibility --- .../aggregate-functions/reference/grouparray.md | 2 ++ src/AggregateFunctions/AggregateFunctionGroupArray.cpp | 1 + tests/queries/0_stateless/02813_array_agg.reference | 6 ++++++ tests/queries/0_stateless/02813_array_agg.sql | 10 ++++++++++ 4 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02813_array_agg.reference create mode 100644 tests/queries/0_stateless/02813_array_agg.sql diff --git a/docs/en/sql-reference/aggregate-functions/reference/grouparray.md b/docs/en/sql-reference/aggregate-functions/reference/grouparray.md index 18048fa4f71..ad678443df6 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/grouparray.md +++ b/docs/en/sql-reference/aggregate-functions/reference/grouparray.md @@ -44,3 +44,5 @@ Result: ``` The groupArray function will remove ᴺᵁᴸᴸ value based on the above results. + +- Alias: `array_agg`. diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index bb1368b9ff8..1c54038929b 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -125,6 +125,7 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory) AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("groupArray", { createAggregateFunctionGroupArray, properties }); + factory.registerAlias("array_agg", "groupArray", AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("groupArraySample", { createAggregateFunctionGroupArraySample, properties }); factory.registerFunction("groupArrayLast", { createAggregateFunctionGroupArray, properties }); } diff --git a/tests/queries/0_stateless/02813_array_agg.reference b/tests/queries/0_stateless/02813_array_agg.reference new file mode 100644 index 00000000000..202f6aa4bac --- /dev/null +++ b/tests/queries/0_stateless/02813_array_agg.reference @@ -0,0 +1,6 @@ +['hello, world!','hello, world!','hello, world!','hello, world!','hello, world!'] +['hello, world!'] +['hello, world!'] +['hello, world!'] +['hello, world!'] +['hello, world!'] diff --git a/tests/queries/0_stateless/02813_array_agg.sql b/tests/queries/0_stateless/02813_array_agg.sql new file mode 100644 index 00000000000..91d8d0774d3 --- /dev/null +++ b/tests/queries/0_stateless/02813_array_agg.sql @@ -0,0 +1,10 @@ +drop table if exists t; +create table t (n Int32, s String) engine=MergeTree order by n; + +insert into t select number, 'hello, world!' from numbers (5); + +select array_agg(s) from t; + +select aRray_Agg(s) from t group by n; + +drop table t; From 386adfad3365d3026ec8a3fe11536eead780262d Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 15 Jul 2023 16:21:58 +0000 Subject: [PATCH 0593/2047] Avro input format support Union with single type --- .../Formats/Impl/AvroRowInputFormat.cpp | 28 ++++++++++++++---- .../02813_avro_union_with_one_type.reference | 5 ++++ .../02813_avro_union_with_one_type.sh | 13 ++++++++ .../0_stateless/data_avro/union_one_type.avro | Bin 0 -> 304 bytes 4 files changed, 41 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02813_avro_union_with_one_type.reference create mode 100755 tests/queries/0_stateless/02813_avro_union_with_one_type.sh create mode 100644 tests/queries/0_stateless/data_avro/union_one_type.avro diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 4cd73cb23b5..771247a983b 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -367,14 +367,25 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro break; case avro::AVRO_UNION: { - if (root_node->leaves() == 2 + if (root_node->leaves() == 1) + { + auto nested_deserialize = createDeserializeFn(root_node->leafAt(0), target_type); + return [nested_deserialize](IColumn & column, avro::Decoder & decoder) + { + decoder.decodeUnionIndex(); + nested_deserialize(column, decoder); + return true; + }; + } + /// FIXME Support UNION has more than two datatypes. + else if ( + root_node->leaves() == 2 && (root_node->leafAt(0)->type() == avro::AVRO_NULL || root_node->leafAt(1)->type() == avro::AVRO_NULL)) { int non_null_union_index = root_node->leafAt(0)->type() == avro::AVRO_NULL ? 1 : 0; if (target.isNullable()) { - auto nested_deserialize = this->createDeserializeFn( - root_node->leafAt(non_null_union_index), removeNullable(target_type)); + auto nested_deserialize = createDeserializeFn(root_node->leafAt(non_null_union_index), removeNullable(target_type)); return [non_null_union_index, nested_deserialize](IColumn & column, avro::Decoder & decoder) { ColumnNullable & col = assert_cast(column); @@ -393,7 +404,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(const avro } else if (null_as_default) { - auto nested_deserialize = this->createDeserializeFn(root_node->leafAt(non_null_union_index), target_type); + auto nested_deserialize = createDeserializeFn(root_node->leafAt(non_null_union_index), target_type); return [non_null_union_index, nested_deserialize](IColumn & column, avro::Decoder & decoder) { int union_index = static_cast(decoder.decodeUnionIndex()); @@ -1169,12 +1180,19 @@ DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node) case avro::Type::AVRO_NULL: return std::make_shared(); case avro::Type::AVRO_UNION: - if (node->leaves() == 2 && (node->leafAt(0)->type() == avro::Type::AVRO_NULL || node->leafAt(1)->type() == avro::Type::AVRO_NULL)) + if (node->leaves() == 1) + { + return avroNodeToDataType(node->leafAt(0)); + } + else if ( + node->leaves() == 2 + && (node->leafAt(0)->type() == avro::Type::AVRO_NULL || node->leafAt(1)->type() == avro::Type::AVRO_NULL)) { int nested_leaf_index = node->leafAt(0)->type() == avro::Type::AVRO_NULL ? 1 : 0; auto nested_type = avroNodeToDataType(node->leafAt(nested_leaf_index)); return nested_type->canBeInsideNullable() ? makeNullable(nested_type) : nested_type; } + /// FIXME Support UNION has more than two datatypes. throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Avro type UNION is not supported for inserting."); case avro::Type::AVRO_SYMBOLIC: return avroNodeToDataType(avro::resolveSymbol(node)); diff --git a/tests/queries/0_stateless/02813_avro_union_with_one_type.reference b/tests/queries/0_stateless/02813_avro_union_with_one_type.reference new file mode 100644 index 00000000000..c65bed48055 --- /dev/null +++ b/tests/queries/0_stateless/02813_avro_union_with_one_type.reference @@ -0,0 +1,5 @@ +name String +favorite_number Int32 +favorite_color String +Alyssa 256 yellow +Ben 7 red diff --git a/tests/queries/0_stateless/02813_avro_union_with_one_type.sh b/tests/queries/0_stateless/02813_avro_union_with_one_type.sh new file mode 100755 index 00000000000..b58dc9126da --- /dev/null +++ b/tests/queries/0_stateless/02813_avro_union_with_one_type.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +DATA_DIR=$CUR_DIR/data_avro + +$CLICKHOUSE_LOCAL -q "desc file('$DATA_DIR/union_one_type.avro')" +$CLICKHOUSE_LOCAL -q "select * from file('$DATA_DIR/union_one_type.avro')" diff --git a/tests/queries/0_stateless/data_avro/union_one_type.avro b/tests/queries/0_stateless/data_avro/union_one_type.avro new file mode 100644 index 0000000000000000000000000000000000000000..07e6140e5e2f020fd44f59a674598d7c4c7e7214 GIT binary patch literal 304 zcmeZI%3@>@Nh~YM*GtY%NloU+E6vFf1M`cMGg5OCXE9eRl~fj_Dp@Hg6{RNU7o{la zC@AG6=7L2+i&KkW{NjSdWUydrMPhD2PO2Wr6p&aG4L_46{V)2D+d7n$7llp literal 0 HcmV?d00001 From 0d48dca171c146871996046855dfa9790aa421e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 15 Jul 2023 20:36:16 +0300 Subject: [PATCH 0594/2047] Update materialized_with_ddl.py --- .../test_materialized_mysql_database/materialized_with_ddl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 73f6e11d7f8..8926c0e5c81 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -992,7 +992,7 @@ def select_without_columns(clickhouse_node, mysql_node, service_name): ) check_query(clickhouse_node, "SHOW TABLES FROM db FORMAT TSV", "t\n") clickhouse_node.query("SYSTEM STOP MERGES db.t") - clickhouse_node.query("DROP VIEW v IF EXISTS") + clickhouse_node.query("DROP VIEW IF EXISTS v") clickhouse_node.query("CREATE VIEW v AS SELECT * FROM db.t") mysql_node.query("INSERT INTO db.t VALUES (1, 1), (2, 2)") mysql_node.query("DELETE FROM db.t WHERE a = 2;") From aef9575c3ee8f0f30070a44b5ff1867a014f1b03 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 15 Jul 2023 20:18:39 +0000 Subject: [PATCH 0595/2047] Fix test --- ...stinct_in_order_optimization_explain.reference | 9 +++++++-- ...2317_distinct_in_order_optimization_explain.sh | 15 ++++++++++----- 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index 3cee6da3c3d..21ce47cc685 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -51,12 +51,12 @@ MergeTreeInOrder MergeTreeInOrder -- enabled, only part of distinct columns form prefix of sorting key MergeTreeThread +=== disable new analyzer === -- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC -- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query --- disable new analyzer -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns Sorting (Stream): a ASC Sorting (Stream): a ASC @@ -81,7 +81,12 @@ Sorting (Stream): a DESC, b DESC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC --- enable new analyzer +=== enable new analyzer === +-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a_1 ASC, b_0 ASC +Sorting (Stream): a_1 ASC, b ASC -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns Sorting (Stream): a_1 ASC Sorting (Stream): a_1 ASC diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh index 4c4e660030e..db8fa7c1600 100755 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh @@ -76,13 +76,14 @@ $CLICKHOUSE_CLIENT --read_in_order_two_level_merge_threshold=2 -nq "$ENABLE_OPTI echo "-- enabled, only part of distinct columns form prefix of sorting key" $CLICKHOUSE_CLIENT --max_threads=0 -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_READING_DEFAULT +echo "=== disable new analyzer ===" +DISABLE_ANALYZER="set allow_experimental_analyzer=0" + echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0" | eval $FIND_SORTING_PROPERTIES echo "-- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query" ENABLE_READ_IN_ORDER="set optimize_read_in_order=1" -echo "-- disable new analyzer" -DISABLE_ANALYZER="set allow_experimental_analyzer=0" echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" $CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" @@ -95,10 +96,14 @@ echo "-- enabled, check that ReadFromMergeTree sorting description is NOT overwr $CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a DESC, b DESC" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization" -$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES +$CLICKHOUSE_CLIENT -nq "$DISABLE_ANALYZER;$ENABLE_OPTIMIZATION;set optimize_read_in_order=0;set optimize_aggregation_in_order=0;set optimize_read_in_window_order=0;explain plan sorting=1 select distinct a,b from distinct_in_order_explain" | eval $FIND_SORTING_PROPERTIES -echo "-- enable new analyzer" +echo "=== enable new analyzer ===" ENABLE_ANALYZER="set allow_experimental_analyzer=1" + +echo "-- enabled, check that sorting properties are propagated from ReadFromMergeTree till preliminary distinct" +$CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$ENABLE_OPTIMIZATION;explain plan sorting=1 select distinct b, a from distinct_in_order_explain where a > 0 settings optimize_move_to_prewhere=1" | eval $FIND_SORTING_PROPERTIES + echo "-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns" $CLICKHOUSE_CLIENT -nq "$ENABLE_ANALYZER;$DISABLE_OPTIMIZATION;$ENABLE_READ_IN_ORDER;explain plan sorting=1 select distinct b, a from distinct_in_order_explain order by a" | eval $FIND_SORTING_PROPERTIES echo "-- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause" From 8f3469e7ad623b34c6272ba7c0cd132107034481 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 15 Jul 2023 20:18:57 +0000 Subject: [PATCH 0596/2047] Remove test from broken tests --- tests/analyzer_tech_debt.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e0f259306aa..b420149418f 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -82,7 +82,6 @@ 02242_join_rocksdb 02267_join_dup_columns_issue36199 02302_s3_file_pruning -02317_distinct_in_order_optimization_explain 02341_global_join_cte 02345_implicit_transaction 02352_grouby_shadows_arg From 20b77e946a03605907320d13dbfacfb0f413f828 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 15 Jul 2023 23:00:23 +0200 Subject: [PATCH 0597/2047] There is no point in detecting flaky tests --- tests/ci/clickhouse_helper.py | 24 ------------------- tests/ci/compatibility_check.py | 2 -- tests/ci/fast_test_check.py | 2 -- tests/ci/functional_test_check.py | 2 -- tests/ci/install_check.py | 2 -- tests/ci/integration_test_check.py | 2 -- tests/ci/report.py | 2 +- tests/ci/stress_check.py | 2 -- tests/ci/style_check.py | 2 -- tests/ci/unit_tests_check.py | 2 -- tests/integration/ci-runner.py | 38 ++++++++---------------------- 11 files changed, 11 insertions(+), 69 deletions(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 64b64896f66..9410b37d69f 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -190,27 +190,3 @@ def prepare_tests_results_for_clickhouse( result.append(current_row) return result - - -def mark_flaky_tests( - clickhouse_helper: ClickHouseHelper, check_name: str, test_results: TestResults -) -> None: - try: - query = f"""SELECT DISTINCT test_name -FROM checks -WHERE - check_start_time BETWEEN now() - INTERVAL 3 DAY AND now() - AND check_name = '{check_name}' - AND (test_status = 'FAIL' OR test_status = 'FLAKY') - AND pull_request_number = 0 -""" - - tests_data = clickhouse_helper.select_json_each_row("default", query) - master_failed_tests = {row["test_name"] for row in tests_data} - logging.info("Found flaky tests: %s", ", ".join(master_failed_tests)) - - for test_result in test_results: - if test_result.status == "FAIL" and test_result.name in master_failed_tests: - test_result.status = "FLAKY" - except Exception as ex: - logging.error("Exception happened during flaky tests fetch %s", ex) diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 04203617dca..97de7fed2d5 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -13,7 +13,6 @@ from github import Github from build_download_helper import download_builds_filter from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import RerunHelper, get_commit, post_commit_status @@ -231,7 +230,6 @@ def main(): ) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, args.check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index d5198e5c3d7..460e17acd37 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -14,7 +14,6 @@ from github import Github from build_check import get_release_or_pr from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -190,7 +189,6 @@ def main(): state, description, test_results, additional_logs = process_results(output_path) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, NAME, test_results) s3_path_prefix = os.path.join( get_release_or_pr(pr_info, get_version_from_repo())[0], pr_info.sha, diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 9279b19b187..b773d1eddd9 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -16,7 +16,6 @@ from github import Github from build_download_helper import download_all_deb_packages from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -368,7 +367,6 @@ def main(): state = override_status(state, check_name, invert=validate_bugfix_check) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index d619ce96cee..73e1a6ef739 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -15,7 +15,6 @@ from github import Github from build_download_helper import download_builds_filter from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -345,7 +344,6 @@ def main(): return ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, args.check_name, test_results) description = format_description(description) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index e6b2203fb65..222b2197117 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -15,7 +15,6 @@ from github import Github from build_download_helper import download_all_deb_packages from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -276,7 +275,6 @@ def main(): state = override_status(state, check_name, invert=validate_bugfix_check) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) s3_helper = S3Helper() report_url = upload_results( diff --git a/tests/ci/report.py b/tests/ci/report.py index a9014acec12..8b301d08d56 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -349,7 +349,7 @@ def create_test_html_report( has_log_urls = True row = "" - has_error = test_result.status in ("FAIL", "FLAKY", "NOT_FAILED") + has_error = test_result.status in ("FAIL", "NOT_FAILED") if has_error and test_result.raw_logs is not None: row = '' row += "" + test_result.name + "" diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index ac280916a2f..895eb318bc4 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -13,7 +13,6 @@ from github import Github from build_download_helper import download_all_deb_packages from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import RerunHelper, get_commit, post_commit_status @@ -168,7 +167,6 @@ def run_stress_test(docker_image_name): result_path, server_log_path, run_log_path ) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 33a5cd21f39..0871dd7ec6a 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -12,7 +12,6 @@ from typing import List, Tuple from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -189,7 +188,6 @@ def main(): state, description, test_results, additional_files = process_result(temp_path) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, NAME, test_results) report_url = upload_results( s3_helper, pr_info.number, pr_info.sha, test_results, additional_files, NAME diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index 5279ccde492..1c3ee303b27 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -12,7 +12,6 @@ from github import Github from build_download_helper import download_unit_tests from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -159,7 +158,6 @@ def main(): state, description, test_results, additional_logs = process_results(test_output) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 00bca60e7db..190c79a5263 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -487,36 +487,23 @@ class ClickhouseIntegrationTestsRunner: def _update_counters(self, main_counters, current_counters, broken_tests): for test in current_counters["PASSED"]: - if ( - test not in main_counters["PASSED"] - and test not in main_counters["FLAKY"] - ): - is_flaky = False + if test not in main_counters["PASSED"]: if test in main_counters["FAILED"]: main_counters["FAILED"].remove(test) - is_flaky = True if test in main_counters["ERROR"]: main_counters["ERROR"].remove(test) - is_flaky = True if test in main_counters["BROKEN"]: main_counters["BROKEN"].remove(test) - is_flaky = True - if is_flaky: - main_counters["FLAKY"].append(test) + if test not in broken_tests: + main_counters["PASSED"].append(test) else: - if test not in broken_tests: - main_counters["PASSED"].append(test) - else: - main_counters["NOT_FAILED"].append(test) + main_counters["NOT_FAILED"].append(test) for state in ("ERROR", "FAILED"): for test in current_counters[state]: - if test in main_counters["FLAKY"]: - continue if test in main_counters["PASSED"]: main_counters["PASSED"].remove(test) - main_counters["FLAKY"].append(test) continue if test not in broken_tests: if test not in main_counters[state]: @@ -605,7 +592,6 @@ class ClickhouseIntegrationTestsRunner: "PASSED": [], "FAILED": [], "SKIPPED": [], - "FLAKY": [], } tests_times = defaultdict(float) for test in tests_in_group: @@ -627,7 +613,6 @@ class ClickhouseIntegrationTestsRunner: "PASSED": [], "FAILED": [], "SKIPPED": [], - "FLAKY": [], "BROKEN": [], "NOT_FAILED": [], } @@ -757,11 +742,11 @@ class ClickhouseIntegrationTestsRunner: ) log_paths.append(extras_result_path) - if len(counters["PASSED"]) + len(counters["FLAKY"]) == len(tests_in_group): + if len(counters["PASSED"]) == len(tests_in_group): logging.info("All tests from group %s passed", test_group) break if ( - len(counters["PASSED"]) + len(counters["FLAKY"]) >= 0 + len(counters["PASSED"]) >= 0 and len(counters["FAILED"]) == 0 and len(counters["ERROR"]) == 0 ): @@ -825,7 +810,7 @@ class ClickhouseIntegrationTestsRunner: result_state = "failure" if not should_fail: break - assert len(counters["FLAKY"]) == 0 or should_fail + assert should_fail logging.info("Try is OK, all tests passed, going to clear env") clear_ip_tables_and_restart_daemons() logging.info("And going to sleep for some time") @@ -835,7 +820,7 @@ class ClickhouseIntegrationTestsRunner: time.sleep(5) test_result = [] - for state in ("ERROR", "FAILED", "PASSED", "SKIPPED", "FLAKY"): + for state in ("ERROR", "FAILED", "PASSED", "SKIPPED"): if state == "PASSED": text_state = "OK" elif state == "FAILED": @@ -928,7 +913,6 @@ class ClickhouseIntegrationTestsRunner: "PASSED": [], "FAILED": [], "SKIPPED": [], - "FLAKY": [], "BROKEN": [], "NOT_FAILED": [], } @@ -988,7 +972,6 @@ class ClickhouseIntegrationTestsRunner: "FAILED", "PASSED", "SKIPPED", - "FLAKY", "BROKEN", "NOT_FAILED", ): @@ -1004,15 +987,14 @@ class ClickhouseIntegrationTestsRunner: ] failed_sum = len(counters["FAILED"]) + len(counters["ERROR"]) - status_text = "fail: {}, passed: {}, flaky: {}".format( - failed_sum, len(counters["PASSED"]), len(counters["FLAKY"]) + status_text = "fail: {}, passed: {}".format( + failed_sum, len(counters["PASSED"]) ) if self.soft_deadline_time < time.time(): status_text = "Timeout, " + status_text result_state = "failure" - counters["FLAKY"] = [] if not counters or sum(len(counter) for counter in counters.values()) == 0: status_text = "No tests found for some reason! It's a bug" result_state = "failure" From 44fc93ba638e14054e2d17d8ba6e4e8fa578f268 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 15 Jul 2023 21:10:21 +0000 Subject: [PATCH 0598/2047] Automatic style fix --- tests/integration/ci-runner.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 190c79a5263..9629a5821b5 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -987,9 +987,7 @@ class ClickhouseIntegrationTestsRunner: ] failed_sum = len(counters["FAILED"]) + len(counters["ERROR"]) - status_text = "fail: {}, passed: {}".format( - failed_sum, len(counters["PASSED"]) - ) + status_text = "fail: {}, passed: {}".format(failed_sum, len(counters["PASSED"])) if self.soft_deadline_time < time.time(): status_text = "Timeout, " + status_text From 4f6a0c2faccdb0e6652a21467da164b12520846c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 15 Jul 2023 23:11:22 +0200 Subject: [PATCH 0599/2047] Loosen the check --- tests/clickhouse-test | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 46ec19b041d..36e793f03e0 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1213,17 +1213,20 @@ class TestCase: args.timeout - (datetime.now() - start_time).total_seconds(), 20 ) - leftover_tables = clickhouse_execute( - args, - f"SHOW TABLES FROM {database}", - timeout=seconds_left, - settings={ - "log_comment": args.testcase_basename, - }, - ).decode().replace("\n", ", "); + # Check if the test does not cleanup its tables. + # Only for newly added tests. Please extend this check to the old tests as well. + if self.case_file >= '02800': + leftover_tables = clickhouse_execute( + args, + f"SHOW TABLES FROM {database}", + timeout=seconds_left, + settings={ + "log_comment": args.testcase_basename, + }, + ).decode().replace("\n", ", "); - if 0 != len(leftover_tables): - raise Exception(f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally.") + if 0 != len(leftover_tables): + raise Exception(f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally.") drop_database_query = f"DROP DATABASE IF EXISTS {database}" if args.replicated_database: From 095624fbd17ee7c4b1a68525c59a3d5c14ba4baf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 15 Jul 2023 23:33:11 +0200 Subject: [PATCH 0600/2047] Remove default argument value --- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageMergeTree.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 29880f10f28..074f01e7d03 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -373,7 +373,7 @@ void StorageMergeTree::alter( /// Always execute required mutations synchronously, because alters /// should be executed in sequential order. if (!maybe_mutation_commands.empty()) - waitForMutation(mutation_version); + waitForMutation(mutation_version, false); } { @@ -601,7 +601,7 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer Int64 version = startMutation(commands, query_context); if (query_context->getSettingsRef().mutations_sync > 0 || query_context->getCurrentTransaction()) - waitForMutation(version); + waitForMutation(version, false); } bool StorageMergeTree::hasLightweightDeletedMask() const diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 8099f9c16aa..6aecde15117 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -191,7 +191,7 @@ private: /// and into in-memory structures. Wake up merge-mutation task. Int64 startMutation(const MutationCommands & commands, ContextPtr query_context); /// Wait until mutation with version will finish mutation for all parts - void waitForMutation(Int64 version, bool wait_for_another_mutation = false); + void waitForMutation(Int64 version, bool wait_for_another_mutation); void waitForMutation(const String & mutation_id, bool wait_for_another_mutation) override; void waitForMutation(Int64 version, const String & mutation_id, bool wait_for_another_mutation = false); void setMutationCSN(const String & mutation_id, CSN csn) override; From e76c9612fbaa7ae17dbdf35ca45dea15e7a89bce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 00:26:42 +0200 Subject: [PATCH 0601/2047] Fix the "kill_mutation" test --- tests/queries/0_stateless/00834_kill_mutation.sh | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00834_kill_mutation.sh b/tests/queries/0_stateless/00834_kill_mutation.sh index 46b10ccbff4..4c6455d2f53 100755 --- a/tests/queries/0_stateless/00834_kill_mutation.sh +++ b/tests/queries/0_stateless/00834_kill_mutation.sh @@ -27,8 +27,22 @@ ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id FROM system.mutations WHERE dat ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill invalid mutation that blocks another mutation ***'" +# Note: there is a benign race condition. +# The mutation can fail with the message +# "Cannot parse string 'a' as UInt32" +# or +# "Cannot parse string 'b' as UInt32" +# depending on which parts are processed first. +# The mutations are also coalesced together, and the subsequent mutation inherits the failure status of the original mutation. +# When we are waiting for mutations, we are listing all the mutations with identical error messages. +# But due to a race condition and to repeated runs, the original and subsequent mutations can have different error messages, +# therefore the original mutation will not be included in the list. + +# Originally, there was grep "happened during execution of mutations 'mutation_4.txt, mutation_5.txt'", +# but due to this race condition, I've replaced it to grep "happened during execution of mutation" + ${CLICKHOUSE_CLIENT} --query="ALTER TABLE kill_mutation DELETE WHERE toUInt32(s) = 1" -${CLICKHOUSE_CLIENT} --query="ALTER TABLE kill_mutation DELETE WHERE x = 1 SETTINGS mutations_sync = 1" 2>&1 | grep -o "happened during execution of mutations 'mutation_4.txt, mutation_5.txt'" | head -n 1 +${CLICKHOUSE_CLIENT} --query="ALTER TABLE kill_mutation DELETE WHERE x = 1 SETTINGS mutations_sync = 1" 2>&1 | grep -o "happened during execution of mutation" | head -n 1 # but exception doesn't stop mutations, and we will still see them in system.mutations ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' AND table = 'kill_mutation' AND mutation_id = 'mutation_4.txt'" # 1 From 66b66db39d8d511fe3df137a14586072bf538481 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 02:52:06 +0200 Subject: [PATCH 0602/2047] Fix ORDER BY tuple of WINDOW functions --- src/Interpreters/ExpressionAnalyzer.cpp | 11 +++++----- src/Interpreters/GetAggregatesVisitor.cpp | 20 +++++++++++++------ src/Interpreters/GetAggregatesVisitor.h | 2 +- src/Interpreters/TreeRewriter.cpp | 1 + src/Interpreters/TreeRewriter.h | 5 ++--- ...4_order_by_tuple_window_function.reference | 1 + .../02814_order_by_tuple_window_function.sql | 1 + 7 files changed, 25 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/02814_order_by_tuple_window_function.reference create mode 100644 tests/queries/0_stateless/02814_order_by_tuple_window_function.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index f27d23e8e94..9a450fabd5b 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1378,10 +1378,9 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments( void SelectQueryExpressionAnalyzer::appendExpressionsAfterWindowFunctions(ExpressionActionsChain & chain, bool /* only_types */) { ExpressionActionsChain::Step & step = chain.lastStep(columns_after_window); + for (const auto & expression : syntax->expressions_with_window_function) - { getRootActionsForWindowFunctions(expression->clone(), true, step.actions()); - } } void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAGPtr & before_aggregation, ExpressionActionsChain & chain, bool /* only_types */) @@ -1760,9 +1759,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( /// second_stage: Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing. /** First we compose a chain of actions and remember the necessary steps from it. - * Regardless of from_stage and to_stage, we will compose a complete sequence of actions to perform optimization and - * throw out unnecessary columns based on the entire query. In unnecessary parts of the query, we will not execute subqueries. - */ + * Regardless of from_stage and to_stage, we will compose a complete sequence of actions to perform optimization and + * throw out unnecessary columns based on the entire query. In unnecessary parts of the query, we will not execute subqueries. + */ const ASTSelectQuery & query = *query_analyzer.getSelectQuery(); auto context = query_analyzer.getContext(); @@ -1805,7 +1804,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { - // we evaluate sampling for Merge lazily so we need to get all the columns + // we evaluate sampling for Merge lazily, so we need to get all the columns if (storage->getName() == "Merge") { const auto columns = metadata_snapshot->getColumns().getAll(); diff --git a/src/Interpreters/GetAggregatesVisitor.cpp b/src/Interpreters/GetAggregatesVisitor.cpp index dd958693d89..718721308b1 100644 --- a/src/Interpreters/GetAggregatesVisitor.cpp +++ b/src/Interpreters/GetAggregatesVisitor.cpp @@ -1,4 +1,6 @@ #include +#include + namespace DB { @@ -13,7 +15,7 @@ struct WindowExpressionsCollectorChildInfo bool window_function_in_subtree = false; }; -// This visitor travers AST and collects the list of expressions which depend on +// This visitor traverses the AST and collects the list of expressions which depend on // evaluation of window functions. Expression is collected only if // it's not a part of another expression. // @@ -26,15 +28,18 @@ struct WindowExpressionsCollectorMatcher { if (child->as() || child->as()) return false; + if (auto * select = node->as()) { - // We don't analysis WITH statement because it might contain useless aggregates + // We don't analyse the WITH statement because it might contain useless aggregates if (child == select->with()) return false; } - // We procces every expression manually + + // We process every expression manually if (auto * func = node->as()) return false; + return true; } @@ -50,6 +55,8 @@ struct WindowExpressionsCollectorMatcher ASTPtr & ast, const ASTPtr & parent) { + checkStackSize(); + if (auto * func = ast->as()) { if (func->is_window_function) @@ -67,7 +74,7 @@ struct WindowExpressionsCollectorMatcher { func->compute_after_window_functions = true; if ((!parent || !parent->as())) - expressions_with_window_functions.push_back(func); + expressions_with_window_functions.push_back(ast); } return result; @@ -75,15 +82,16 @@ struct WindowExpressionsCollectorMatcher return {}; } - std::vector expressions_with_window_functions {}; + ASTs expressions_with_window_functions; }; using WindowExpressionsCollectorVisitor = InDepthNodeVisitorWithChildInfo; -std::vector getExpressionsWithWindowFunctions(ASTPtr & ast) +ASTs getExpressionsWithWindowFunctions(ASTPtr & ast) { WindowExpressionsCollectorVisitor visitor; visitor.visit(ast); + return std::move(visitor.expressions_with_window_functions); } diff --git a/src/Interpreters/GetAggregatesVisitor.h b/src/Interpreters/GetAggregatesVisitor.h index 3f5804c39a0..fdf54de3e57 100644 --- a/src/Interpreters/GetAggregatesVisitor.h +++ b/src/Interpreters/GetAggregatesVisitor.h @@ -114,6 +114,6 @@ inline void assertNoAggregates(const ASTPtr & ast, const char * description) GetAggregatesVisitor(data).visit(ast); } -std::vector getExpressionsWithWindowFunctions(ASTPtr & ast); +ASTs getExpressionsWithWindowFunctions(ASTPtr & ast); } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index cda5ceeb164..65b5d950975 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1287,6 +1287,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( bool is_changed = replaceAliasColumnsInQuery(query, result.storage_snapshot->metadata->getColumns(), result.array_join_result_to_source, getContext(), excluded_nodes); + /// If query is changed, we need to redo some work to correct name resolution. if (is_changed) { diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index b94043b8983..ea16c432d0f 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -40,11 +40,10 @@ struct TreeRewriterResult NameSet expanded_aliases; Aliases aliases; + std::vector aggregates; - std::vector window_function_asts; - - std::vector expressions_with_window_function; + ASTs expressions_with_window_function; /// Which column is needed to be ARRAY-JOIN'ed to get the specified. /// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v". diff --git a/tests/queries/0_stateless/02814_order_by_tuple_window_function.reference b/tests/queries/0_stateless/02814_order_by_tuple_window_function.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02814_order_by_tuple_window_function.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02814_order_by_tuple_window_function.sql b/tests/queries/0_stateless/02814_order_by_tuple_window_function.sql new file mode 100644 index 00000000000..8ba54fc11d0 --- /dev/null +++ b/tests/queries/0_stateless/02814_order_by_tuple_window_function.sql @@ -0,0 +1 @@ +SELECT 1 ORDER BY tuple(count() OVER ()); From c62089134049efb42fbf76a90c0199a7bcc8c491 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 03:14:44 +0200 Subject: [PATCH 0603/2047] Slightly more changes --- src/Interpreters/ExpressionAnalyzer.cpp | 46 ++++++++++++++----------- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/GetAggregatesVisitor.h | 10 +++--- src/Interpreters/TreeRewriter.cpp | 35 +++++++++---------- src/Interpreters/TreeRewriter.h | 4 +-- 5 files changed, 50 insertions(+), 47 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 9a450fabd5b..9aee61eb8f0 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -548,15 +548,17 @@ void ExpressionAnalyzer::getRootActionsForWindowFunctions(const ASTPtr & ast, bo void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, AggregateDescriptions & descriptions) { - for (const ASTFunction * node : aggregates()) + for (const ASTPtr & ast : aggregates()) { + const ASTFunction & node = typeid_cast(*ast); + AggregateDescription aggregate; - if (node->arguments) - getRootActionsNoMakeSet(node->arguments, actions); + if (node.arguments) + getRootActionsNoMakeSet(node.arguments, actions); - aggregate.column_name = node->getColumnName(); + aggregate.column_name = node.getColumnName(); - const ASTs & arguments = node->arguments ? node->arguments->children : ASTs(); + const ASTs & arguments = node.arguments ? node.arguments->children : ASTs(); aggregate.argument_names.resize(arguments.size()); DataTypes types(arguments.size()); @@ -568,7 +570,7 @@ void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, Aggr { throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier '{}' in aggregate function '{}'", - name, node->formatForErrorMessage()); + name, node.formatForErrorMessage()); } types[i] = dag_node->result_type; @@ -576,8 +578,8 @@ void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, Aggr } AggregateFunctionProperties properties; - aggregate.parameters = (node->parameters) ? getAggregateFunctionParametersArray(node->parameters, "", getContext()) : Array(); - aggregate.function = AggregateFunctionFactory::instance().get(node->name, types, aggregate.parameters, properties); + aggregate.parameters = (node.parameters) ? getAggregateFunctionParametersArray(node.parameters, "", getContext()) : Array(); + aggregate.function = AggregateFunctionFactory::instance().get(node.name, types, aggregate.parameters, properties); descriptions.push_back(aggregate); } @@ -744,12 +746,13 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) } // Window functions - for (const ASTFunction * function_node : syntax->window_function_asts) + for (const ASTPtr & ast : syntax->window_function_asts) { - assert(function_node->is_window_function); + const ASTFunction & function_node = typeid_cast(*ast); + assert(function_node.is_window_function); WindowFunctionDescription window_function; - window_function.function_node = function_node; + window_function.function_node = &function_node; window_function.column_name = window_function.function_node->getColumnName(); window_function.function_parameters @@ -760,7 +763,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) // Requiring a constant reference to a shared pointer to non-const AST // doesn't really look sane, but the visitor does indeed require it. - // Hence we clone the node (not very sane either, I know). + // Hence, we clone the node (not very sane either, I know). getRootActionsNoMakeSet(window_function.function_node->clone(), actions); const ASTs & arguments @@ -793,22 +796,22 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) // Find the window corresponding to this function. It may be either // referenced by name and previously defined in WINDOW clause, or it // may be defined inline. - if (!function_node->window_name.empty()) + if (!function_node.window_name.empty()) { - auto it = window_descriptions.find(function_node->window_name); + auto it = window_descriptions.find(function_node.window_name); if (it == std::end(window_descriptions)) { throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Window '{}' is not defined (referenced by '{}')", - function_node->window_name, - function_node->formatForErrorMessage()); + function_node.window_name, + function_node.formatForErrorMessage()); } it->second.window_functions.push_back(window_function); } else { - const auto & definition = function_node->window_definition->as< + const auto & definition = function_node.window_definition->as< const ASTWindowDefinition &>(); WindowDescription desc; desc.window_name = definition.getDefaultWindowName(); @@ -1323,10 +1326,13 @@ void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(Expression GetAggregatesVisitor(data).visit(select_query->orderBy()); /// TODO: data.aggregates -> aggregates() - for (const ASTFunction * node : data.aggregates) - if (node->arguments) - for (auto & argument : node->arguments->children) + for (const ASTPtr & ast : data.aggregates) + { + const ASTFunction & node = typeid_cast(*ast); + if (node.arguments) + for (auto & argument : node.arguments->children) getRootActions(argument, only_types, step.actions()); + } } void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments( diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 271c3943afc..941194e69ff 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -168,7 +168,7 @@ protected: const ConstStoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists. const TableJoin & analyzedJoin() const { return *syntax->analyzed_join; } const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; } - const std::vector & aggregates() const { return syntax->aggregates; } + const ASTs & aggregates() const { return syntax->aggregates; } /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. void initGlobalSubqueriesAndExternalTables(bool do_global, bool is_explain); diff --git a/src/Interpreters/GetAggregatesVisitor.h b/src/Interpreters/GetAggregatesVisitor.h index fdf54de3e57..7bf6591af69 100644 --- a/src/Interpreters/GetAggregatesVisitor.h +++ b/src/Interpreters/GetAggregatesVisitor.h @@ -26,8 +26,8 @@ public: // Explicit empty initializers are needed to make designated initializers // work on GCC 10. std::unordered_set uniq_names {}; - std::vector aggregates {}; - std::vector window_functions {}; + ASTs aggregates; + ASTs window_functions; }; static bool needChildVisit(const ASTPtr & node, const ASTPtr & child) @@ -61,7 +61,7 @@ public: } private: - static void visit(const ASTFunction & node, const ASTPtr &, Data & data) + static void visit(const ASTFunction & node, const ASTPtr & ast, Data & data) { if (isAggregateFunction(node)) { @@ -74,7 +74,7 @@ private: return; data.uniq_names.insert(column_name); - data.aggregates.push_back(&node); + data.aggregates.push_back(ast); } else if (node.is_window_function) { @@ -87,7 +87,7 @@ private: return; data.uniq_names.insert(column_name); - data.window_functions.push_back(&node); + data.window_functions.push_back(ast); } } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 65b5d950975..aa493a1b55d 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -731,7 +731,7 @@ void expandGroupByAll(ASTSelectQuery * select_query) select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, group_expression_list); } -std::vector getAggregates(ASTPtr & query, const ASTSelectQuery & select_query) +ASTs getAggregates(ASTPtr & query, const ASTSelectQuery & select_query) { /// There can not be aggregate functions inside the WHERE and PREWHERE. if (select_query.where()) @@ -743,11 +743,12 @@ std::vector getAggregates(ASTPtr & query, const ASTSelectQu GetAggregatesVisitor(data).visit(query); /// There can not be other aggregate functions within the aggregate functions. - for (const ASTFunction * node : data.aggregates) + for (const ASTPtr & ast : data.aggregates) { - if (node->arguments) + const ASTFunction & node = typeid_cast(*ast); + if (node.arguments) { - for (auto & arg : node->arguments->children) + for (auto & arg : node.arguments->children) { assertNoAggregates(arg, "inside another aggregate function"); // We also can't have window functions inside aggregate functions, @@ -759,7 +760,7 @@ std::vector getAggregates(ASTPtr & query, const ASTSelectQu return data.aggregates; } -std::vector getWindowFunctions(ASTPtr & query, const ASTSelectQuery & select_query) +ASTs getWindowFunctions(ASTPtr & query, const ASTSelectQuery & select_query) { /// There can not be window functions inside the WHERE, PREWHERE and HAVING if (select_query.having()) @@ -777,20 +778,16 @@ std::vector getWindowFunctions(ASTPtr & query, const ASTSel /// Window functions cannot be inside aggregates or other window functions. /// Aggregate functions can be inside window functions because they are /// calculated earlier. - for (const ASTFunction * node : data.window_functions) + for (const ASTPtr & ast : data.window_functions) { - if (node->arguments) - { - for (auto & arg : node->arguments->children) - { - assertNoWindows(arg, "inside another window function"); - } - } + const ASTFunction & node = typeid_cast(*ast); - if (node->window_definition) - { - assertNoWindows(node->window_definition, "inside window definition"); - } + if (node.arguments) + for (auto & arg : node.arguments->children) + assertNoWindows(arg, "inside another window function"); + + if (node.window_definition) + assertNoWindows(node.window_definition, "inside window definition"); } return data.window_functions; @@ -1357,8 +1354,8 @@ TreeRewriterResultPtr TreeRewriter::analyze( GetAggregatesVisitor(data).visit(query); /// There can not be other aggregate functions within the aggregate functions. - for (const ASTFunction * node : data.aggregates) - for (auto & arg : node->arguments->children) + for (const ASTPtr & node : data.aggregates) + for (auto & arg : typeid_cast(*node).arguments->children) assertNoAggregates(arg, "inside another aggregate function"); result.aggregates = data.aggregates; } diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index ea16c432d0f..206a63541a6 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -41,8 +41,8 @@ struct TreeRewriterResult Aliases aliases; - std::vector aggregates; - std::vector window_function_asts; + ASTs aggregates; + ASTs window_function_asts; ASTs expressions_with_window_function; /// Which column is needed to be ARRAY-JOIN'ed to get the specified. From 5d0c1e83a355757da87ad99ef46a97c4be92ef29 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 03:14:58 +0200 Subject: [PATCH 0604/2047] Remove a whitespace --- tests/queries/0_stateless/02364_window_view_segfault.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02364_window_view_segfault.sh b/tests/queries/0_stateless/02364_window_view_segfault.sh index 3def22f4a9e..9accf7f30c3 100755 --- a/tests/queries/0_stateless/02364_window_view_segfault.sh +++ b/tests/queries/0_stateless/02364_window_view_segfault.sh @@ -12,6 +12,6 @@ opts=( ${CLICKHOUSE_CLIENT} "${opts[@]}" --multiquery --multiline --query """ DROP TABLE IF EXISTS mt ON CLUSTER test_shard_localhost; DROP TABLE IF EXISTS wv ON CLUSTER test_shard_localhost; -CREATE TABLE mt ON CLUSTER test_shard_localhost (a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE mt ON CLUSTER test_shard_localhost (a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); CREATE WINDOW VIEW wv ON CLUSTER test_shard_localhost TO input_deduplicated INNER ENGINE Memory WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), hopStart(wid) AS w_start, hopEnd(wid) AS w_end FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; """ 2>&1 | grep -q -e "Code: 344" -e "Code: 60" && echo 'ok' || echo 'fail' ||: From a0fef7496926e0a64797294d97c488b6eb0cf59e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 04:08:00 +0200 Subject: [PATCH 0605/2047] Update test reference --- tests/queries/0_stateless/00834_kill_mutation.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00834_kill_mutation.reference b/tests/queries/0_stateless/00834_kill_mutation.reference index 1685343c2b1..49fabab2f8a 100644 --- a/tests/queries/0_stateless/00834_kill_mutation.reference +++ b/tests/queries/0_stateless/00834_kill_mutation.reference @@ -2,7 +2,7 @@ 1 waiting default kill_mutation mutation_3.txt DELETE WHERE toUInt32(s) = 1 *** Create and kill invalid mutation that blocks another mutation *** -happened during execution of mutations 'mutation_4.txt, mutation_5.txt' +happened during execution of mutation 1 waiting default kill_mutation mutation_4.txt DELETE WHERE toUInt32(s) = 1 2001-01-01 2 b From 315f4afe11d2a297f7b444f29366ba006c0127aa Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 05:20:09 +0000 Subject: [PATCH 0606/2047] Add any_value as a compatibility alias for any --- src/AggregateFunctions/AggregateFunctionAny.cpp | 1 + tests/queries/0_stateless/02813_any_value.reference | 2 ++ tests/queries/0_stateless/02813_any_value.sql | 2 ++ 3 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02813_any_value.reference create mode 100644 tests/queries/0_stateless/02813_any_value.sql diff --git a/src/AggregateFunctions/AggregateFunctionAny.cpp b/src/AggregateFunctions/AggregateFunctionAny.cpp index 7f57062126b..fc8f50efabe 100644 --- a/src/AggregateFunctions/AggregateFunctionAny.cpp +++ b/src/AggregateFunctions/AggregateFunctionAny.cpp @@ -49,6 +49,7 @@ void registerAggregateFunctionsAny(AggregateFunctionFactory & factory) AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("any", { createAggregateFunctionAny, properties }); + factory.registerAlias("any_value", "any", AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("anyLast", { createAggregateFunctionAnyLast, properties }); factory.registerFunction("anyHeavy", { createAggregateFunctionAnyHeavy, properties }); diff --git a/tests/queries/0_stateless/02813_any_value.reference b/tests/queries/0_stateless/02813_any_value.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/02813_any_value.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/02813_any_value.sql b/tests/queries/0_stateless/02813_any_value.sql new file mode 100644 index 00000000000..d2c2324349c --- /dev/null +++ b/tests/queries/0_stateless/02813_any_value.sql @@ -0,0 +1,2 @@ +select any_value(number) from numbers(10); +select aNy_VaLue(number) from numbers(10); From 1eaa387da7bc0cfa17dc9b666f38dff6250595c2 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 05:23:15 +0000 Subject: [PATCH 0607/2047] update doc --- docs/en/sql-reference/aggregate-functions/reference/any.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/any.md b/docs/en/sql-reference/aggregate-functions/reference/any.md index db19f524b31..f79fe66c05d 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/any.md +++ b/docs/en/sql-reference/aggregate-functions/reference/any.md @@ -12,3 +12,5 @@ To get a determinate result, you can use the ‘min’ or ‘max’ function ins In some cases, you can rely on the order of execution. This applies to cases when SELECT comes from a subquery that uses ORDER BY. When a `SELECT` query has the `GROUP BY` clause or at least one aggregate function, ClickHouse (in contrast to MySQL) requires that all expressions in the `SELECT`, `HAVING`, and `ORDER BY` clauses be calculated from keys or from aggregate functions. In other words, each column selected from the table must be used either in keys or inside aggregate functions. To get behavior like in MySQL, you can put the other columns in the `any` aggregate function. + +- Alias: `any_value` From 4669951db550f4d70039c678ffacafa15b966f6c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Jun 2023 09:06:29 +0200 Subject: [PATCH 0608/2047] Fix timeout for hedged requests Reset the timeout for hedged requests after each packet (such as Progress packets) to avoid triggering send_timeout/receive_timeout as a total timeout for the query in some cases. Here is a simple query that demonstrates the problem: select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1 Signed-off-by: Azat Khuzhin --- src/Client/HedgedConnections.cpp | 2 ++ .../0_stateless/02805_distributed_queries_timeouts.reference | 0 .../queries/0_stateless/02805_distributed_queries_timeouts.sql | 3 +++ 3 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02805_distributed_queries_timeouts.reference create mode 100644 tests/queries/0_stateless/02805_distributed_queries_timeouts.sql diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index fb6fe258d48..0efad1188fa 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -353,6 +353,8 @@ bool HedgedConnections::resumePacketReceiver(const HedgedConnections::ReplicaLoc if (replica_state.packet_receiver->isPacketReady()) { + /// Reset the socket timeout after some packet received + replica_state.packet_receiver->setTimeout(hedged_connections_factory.getConnectionTimeouts().receive_timeout); last_received_packet = replica_state.packet_receiver->getPacket(); return true; } diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.reference b/tests/queries/0_stateless/02805_distributed_queries_timeouts.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql new file mode 100644 index 00000000000..0b7337d1255 --- /dev/null +++ b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql @@ -0,0 +1,3 @@ +select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=1 format Null; +select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=1, use_hedged_requests=0 format Null; +select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=0 format Null; From 20a671b8cf2830733eea2ed7e7c6cd80d70ed81a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 15 Jul 2023 19:44:29 +0200 Subject: [PATCH 0609/2047] Skip protection from double decompression if inode from maps cannot be obtained Under some circumstances, like using qemu-$ARCH-static, /proc/self/maps will not contain information about /proc/self/exe. Well, strictly speaking it does contains, however qemu will not pass it to the user program:
strace $ sudo strace -s10000 -f arch-chroot . /qemu-riscv64-static /clickhouse ... execve("/qemu-riscv64-static", ["/qemu-riscv64-static", "/clickhouse"], 0x7fffffffe458 /* 20 vars */) = 0 readlinkat(AT_FDCWD, "/proc/self/exe", "/qemu-riscv64-static", 4096) = 20 openat(AT_FDCWD, "/proc/self/maps", O_RDONLY|O_CLOEXEC) = 4 [pid 3126] read(4, "00010000-00111000 r--p 00000000 fe:01 30312571 /clickhouse\n00111000-00119000 r--p 00100000 fe:01 30312571 /clickhouse\n00119000-0011a000 rw-p 00108000 fe:01 30312571 /clickhouse\n0011a000-0013d000 rw-p 00000000 00:00 0 \n4000000000-4000001000 ---p 00000000 00:00 0 \n4000001000-4000801000 rw-p 00000000 00:00 0 \n4000801000-400081a000 r--p 00000000 fe:01 30316932 /lib/riscv64-linux-gnu/ld-2.32.so\n400081a000-400081b000 ---p 00000000 00:00 0 \n400081b000-400081c000 r--p 00019000 fe:01 30316932 /lib/riscv64-linux-gnu/ld-2.32.so\n400081c000-400081e000 rw-p 0001a000 fe:01 30316932 /lib/riscv64-linux-gnu/ld-2.32.so\n400081e000-400081f000 r--p 00000000 00:00 0 \n400081f000-4000922000 r--p 00000000 fe:01 30316935 /lib/riscv64-linux-gnu/libc-2.32.so\n4000922000-4000926000 r--p 00102000 fe:01 30316935 /lib/riscv64-linux-gnu/libc-2.32.so\n4000926000-4000928000 rw-p 00106000 fe:01 30316935 /lib/riscv64-linux-gnu/libc-2.32.so\n4000928000-400092d000 rw-p 00000000 00:00 0 \n400092d000-40009af000 r--p 00000000 fe:01 30316943 /lib/riscv64-linux-gnu/libm-2.32.so\n40009af000-40009b0000 r--p 00081000 fe:01 30316943 /lib/riscv64-linux-gnu/libm-2.32.so\n40009b0000-40009b1000 rw-p 00082000 fe:01 30316943 /lib/riscv64-linux-gnu/libm-2.32.so\n40009b1000-40009c5000 r--p 00000000 fe:01 30316946 /lib/riscv64-linux-gnu/libpthread-2.32.so\n40009c5000-40009c6000 r--p 00013000 fe:01 30316946 /lib/riscv64-linux-gnu/libpthread-2.32.so\n40009c6000-40009c7000 rw-p 00014000 fe:01 30316946 /lib/riscv64-linux-gnu/libpthread-2.32.so\n40009c7000-40009cb000 rw-p 00000000 00:00 0 \n40009cb000-40009cd000 r--p 00000000 fe:01 30316939 /lib/riscv64-linux-gnu/libdl-2.32.so\n40009cd000-40009ce000 r--p 00001000 fe:01 30316939 /lib/riscv64-linux-gnu/libdl-2.32.so\n40009ce000-40009cf000 rw-p 00002000 fe:01 30316939 /lib/riscv64-linux-gnu/libdl-2.32.so\n40009cf000-40009d1000 rw-p 00000000 00:00 0 \n7fffe8000000-7fffeffff000 rwxp 00000000 00:00 0 \n7fffeffff000-7ffff0000000 ---p 00000000 00:00 0 \n7ffff0000000-7ffff0021000 rw-p 00000000 00:00 0 \n7ffff0021000-7ffff4000000 ---p 00000000 00:00 0 \n7ffff6b4b000-7ffff6b5b000 rw-p 00000000 00:00 0 \n7ffff71ff000-7ffff7200000 ---p 00000000 00:00 0 \n7ffff7200000-7ffff7a00000 rw-p 00000000 00:00 0\n7ffff7a00000-7ffff7a3c000 r--p 00000000 fe:01 30316953 /qemu-riscv64-static\n7ffff7a3c000-7ffff7c74000 r-xp 0003c000 fe:01 30316953 /qemu-riscv64-static\n7ffff7c74000-7ffff7d77000 r--p 00274000 fe:01 30316953 /qemu-riscv64-static\n7ffff7d77000-7ffff7dce000 r--p 00377000 fe:01 30316953 /qemu-riscv64-static\n7ffff7dce000-7ffff7df7000 rw-p 003ce000 fe:01 30316953 /qemu-riscv64-static\n7ffff7df7000-7ffff7e0c000 rw-p 00000000 00:00 0 [heap]\n7ffff7e0c000-7ffff7e70000 rw-p 00000000 00:00 0 [heap]\n7ffff7f42000-7ffff7ff9000 rw-p 00000000 00:00 0 \n7ffff7ff9000-7ffff7ffd000 r--p 00000000 00:00 0 [vvar]\n7ffff7ffd000-7ffff7fff000 r-xp 00000000 00:00 0 [vdso]\n7ffffffde000-7ffffffff000 rw-p 00000000 00:00 0 [stack]\nffffffffff600000-ffffffffff601000 --xp 00000000 00:00 0 [vsyscall]\n", 4096) = 3608 [pid 3126] read(4, "", 1024) = 0 [pid 3126] close(4) = 0 [pid 3126] write(3, "10000-111000 r-xp 00000000 fe:01 30312571", 41) = 41 [pid 3126] write(3, " /clickhouse\n", 44) = 44 [pid 3126] write(3, "111000-119000 r--p 00100000 fe:01 30312571", 42) = 42 [pid 3126] write(3, " /clickhouse\n", 43) = 43 [pid 3126] write(3, "119000-11a000 rw-p 00108000 fe:01 30312571", 42) = 42 [pid 3126] write(3, " /clickhouse\n", 43) = 43 [pid 3126] write(3, "11a000-13d000 rw-p 00000000 00:00 0", 35) = 35 [pid 3126] write(3, " \n", 39) = 39 [pid 3126] write(3, "4000000000-4000001000 ---p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "4000001000-4000801000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " [stack]\n", 38) = 38 [pid 3126] write(3, "4000801000-400081a000 r-xp 00000000 fe:01 30316932", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/ld-2.32.so\n", 57 [pid 3127] <... clock_nanosleep resumed>0x7ffff79ff060) = 0 [pid 3126] <... write resumed>) = 57 [pid 3127] clock_nanosleep(CLOCK_REALTIME, 0, {tv_sec=0, tv_nsec=10000000}, [pid 3126] write(3, "400081a000-400081b000 ---p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "400081b000-400081c000 r--p 00019000 fe:01 30316932", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/ld-2.32.so\n", 57) = 57 [pid 3126] write(3, "400081c000-400081e000 rw-p 0001a000 fe:01 30316932", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/ld-2.32.so\n", 57) = 57 [pid 3126] write(3, "400081e000-400081f000 r-xp 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "400081f000-4000922000 r-xp 00000000 fe:01 30316935", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libc-2.32.so\n", 59) = 59 [pid 3126] write(3, "4000922000-4000926000 r--p 00102000 fe:01 30316935", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libc-2.32.so\n", 59) = 59 [pid 3126] write(3, "4000926000-4000928000 rw-p 00106000 fe:01 30316935", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libc-2.32.so\n", 59) = 59 [pid 3126] write(3, "4000928000-400092d000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "400092d000-40009af000 r-xp 00000000 fe:01 30316943", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libm-2.32.so\n", 59) = 59 [pid 3126] write(3, "40009af000-40009b0000 r--p 00081000 fe:01 30316943", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libm-2.32.so\n", 59) = 59 [pid 3126] write(3, "40009b0000-40009b1000 rw-p 00082000 fe:01 30316943", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libm-2.32.so\n", 59) = 59 [pid 3126] write(3, "40009b1000-40009c5000 r-xp 00000000 fe:01 30316946", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libpthread-2.32.so\n", 65) = 65 [pid 3126] write(3, "40009c5000-40009c6000 r--p 00013000 fe:01 30316946", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libpthread-2.32.so\n", 65) = 65 [pid 3126] write(3, "40009c6000-40009c7000 rw-p 00014000 fe:01 30316946", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libpthread-2.32.so\n", 65) = 65 [pid 3126] write(3, "40009c7000-40009cb000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "40009cb000-40009cd000 r-xp 00000000 fe:01 30316939", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libdl-2.32.so\n", 60) = 60 [pid 3126] write(3, "40009cd000-40009ce000 r--p 00001000 fe:01 30316939", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libdl-2.32.so\n", 60) = 60 [pid 3126] write(3, "40009ce000-40009cf000 rw-p 00002000 fe:01 30316939", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libdl-2.32.so\n", 60) = 60 [pid 3126] write(3, "40009cf000-40009d1000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31
Signed-off-by: Azat Khuzhin --- .../decompressor.cpp | 93 ++++++++++--------- 1 file changed, 47 insertions(+), 46 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index d41b9b1ebe1..4a4985120fd 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -430,55 +430,58 @@ int main(int/* argc*/, char* argv[]) return 1; } + int lock = -1; + /// Protection from double decompression #if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// get inode of this executable uint64_t inode = getInode(self); - if (inode == 0) + /// In some cases /proc/self/maps may not contain the inode for the + /// /proc/self/exe, one of such examples are using qemu-*-static, in this + /// case maps will be proxied through the qemu, and it will remove + /// information about itself from it. + if (inode != 0) { - std::cerr << "Unable to obtain inode." << std::endl; - return 1; - } - - std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; - int lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); - if (lock < 0) - { - perror("lock open"); - return 1; - } - - /// lock file should be closed on exec call - fcntl(lock, F_SETFD, FD_CLOEXEC); - - if (lockf(lock, F_LOCK, 0)) - { - perror("lockf"); - return 1; - } - - /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to - /// 32bit conversion of input_info.st_ino - if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) - input_info.st_ino &= 0x00000000FFFFFFFF; - - /// if decompression was performed by another process since this copy was started - /// then file referred by path "self" is already pointing to different inode - if (input_info.st_ino != inode) - { - struct stat lock_info; - if (0 != fstat(lock, &lock_info)) + std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; + lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); + if (lock < 0) { - perror("fstat lock"); + perror("lock open"); return 1; } - /// size 1 of lock file indicates that another decompressor has found active executable - if (lock_info.st_size == 1) - execv(self, argv); + /// lock file should be closed on exec call + fcntl(lock, F_SETFD, FD_CLOEXEC); - printf("No target executable - decompression only was performed.\n"); - return 0; + if (lockf(lock, F_LOCK, 0)) + { + perror("lockf"); + return 1; + } + + /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to + /// 32bit conversion of input_info.st_ino + if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) + input_info.st_ino &= 0x00000000FFFFFFFF; + + /// if decompression was performed by another process since this copy was started + /// then file referred by path "self" is already pointing to different inode + if (input_info.st_ino != inode) + { + struct stat lock_info; + if (0 != fstat(lock, &lock_info)) + { + perror("fstat lock"); + return 1; + } + + /// size 1 of lock file indicates that another decompressor has found active executable + if (lock_info.st_size == 1) + execv(self, argv); + + printf("No target executable - decompression only was performed.\n"); + return 0; + } } #endif @@ -546,21 +549,19 @@ int main(int/* argc*/, char* argv[]) if (has_exec) { -#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// write one byte to the lock in case other copies of compressed are running to indicate that /// execution should be performed - write(lock, "1", 1); -#endif + if (lock >= 0) + write(lock, "1", 1); execv(self, argv); /// This part of code will be reached only if error happened perror("execv"); return 1; } -#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// since inodes can be reused - it's a precaution if lock file already exists and have size of 1 - ftruncate(lock, 0); -#endif + if (lock >= 0) + ftruncate(lock, 0); printf("No target executable - decompression only was performed.\n"); } From 6c5fa1d4a971aa6f099bdde2f907e89239fa4563 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 13 Feb 2023 17:46:28 +0100 Subject: [PATCH 0610/2047] Enable PREWHERE splitting into steps --- src/Core/Settings.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8f304f0aab6..fc49923b221 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -127,8 +127,8 @@ class IColumn; \ M(Bool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.", 0) \ M(Bool, optimize_move_to_prewhere_if_final, false, "If query has `FINAL`, the optimization `move_to_prewhere` is not always correct and it is enabled only if both settings `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are turned on", 0) \ - M(Bool, move_all_conditions_to_prewhere, false, "Move all viable conditions from WHERE to PREWHERE", 0) \ - M(Bool, enable_multiple_prewhere_read_steps, false, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ + M(Bool, move_all_conditions_to_prewhere, true, "Move all viable conditions from WHERE to PREWHERE", 0) \ + M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ From 420446e1a41bbc3006b866000c8e35a0dcdfed13 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 24 Feb 2023 19:13:12 +0100 Subject: [PATCH 0611/2047] Update tests --- .../01582_move_to_prewhere_compact_parts.reference | 3 +-- .../01824_move_to_prewhere_many_columns.reference | 6 ++---- .../0_stateless/01917_prewhere_column_type.reference | 1 + tests/queries/0_stateless/01917_prewhere_column_type.sql | 2 +- .../0_stateless/02156_storage_merge_prewhere.reference | 3 +-- 5 files changed, 6 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.reference b/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.reference index 6b762abd192..30b5ae9c648 100644 --- a/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.reference +++ b/tests/queries/0_stateless/01582_move_to_prewhere_compact_parts.reference @@ -9,5 +9,4 @@ SELECT x3, x4 FROM prewhere_move -PREWHERE x1 > 100 -WHERE (x1 > 100) AND ((x2 > 100) AND (x3 > 100) AND (x4 > 100)) +PREWHERE (x1 > 100) AND (x2 > 100) AND (x3 > 100) AND (x4 > 100) diff --git a/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.reference b/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.reference index adce19321d5..686a864f222 100644 --- a/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.reference +++ b/tests/queries/0_stateless/01824_move_to_prewhere_many_columns.reference @@ -3,12 +3,10 @@ 35 SELECT count() FROM t_move_to_prewhere -PREWHERE a AND b AND c -WHERE (a AND b AND c) AND (NOT ignore(fat_string)) +PREWHERE a AND b AND c AND (NOT ignore(fat_string)) 1 Compact 2 Compact 35 SELECT count() FROM t_move_to_prewhere -PREWHERE a -WHERE a AND (b AND c AND (NOT ignore(fat_string))) +PREWHERE a AND b AND c AND (NOT ignore(fat_string)) diff --git a/tests/queries/0_stateless/01917_prewhere_column_type.reference b/tests/queries/0_stateless/01917_prewhere_column_type.reference index 58c9bdf9d01..2bbe845f4ef 100644 --- a/tests/queries/0_stateless/01917_prewhere_column_type.reference +++ b/tests/queries/0_stateless/01917_prewhere_column_type.reference @@ -1 +1,2 @@ 111 +111 diff --git a/tests/queries/0_stateless/01917_prewhere_column_type.sql b/tests/queries/0_stateless/01917_prewhere_column_type.sql index c0bc0c3e36b..7ddcb17fbb9 100644 --- a/tests/queries/0_stateless/01917_prewhere_column_type.sql +++ b/tests/queries/0_stateless/01917_prewhere_column_type.sql @@ -6,7 +6,7 @@ CREATE TABLE t1 ( s String, f Float32, e UInt16 ) ENGINE = MergeTree ORDER BY tu INSERT INTO t1 VALUES ('111', 1, 1); -SELECT s FROM t1 WHERE f AND (e = 1); -- { serverError 59 } +SELECT s FROM t1 WHERE f AND (e = 1); SELECT s FROM t1 PREWHERE f; -- { serverError 59 } SELECT s FROM t1 PREWHERE f WHERE (e = 1); -- { serverError 59 } SELECT s FROM t1 PREWHERE f WHERE f AND (e = 1); -- { serverError 59 } diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 2dc83f1eaa5..30f9b1ab175 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -1,7 +1,6 @@ SELECT count() FROM t_02156_merge1 -PREWHERE k = 3 -WHERE (k = 3) AND notEmpty(v) +PREWHERE (k = 3) AND notEmpty(v) 2 SELECT count() FROM t_02156_merge2 From d23f67bb04ad677f2c03187201fa941956df8be8 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 28 Feb 2023 13:06:11 +0100 Subject: [PATCH 0612/2047] Couple more cases with float --- tests/queries/0_stateless/01917_prewhere_column_type.reference | 3 +++ tests/queries/0_stateless/01917_prewhere_column_type.sql | 3 +++ 2 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/01917_prewhere_column_type.reference b/tests/queries/0_stateless/01917_prewhere_column_type.reference index 2bbe845f4ef..99c26d55eda 100644 --- a/tests/queries/0_stateless/01917_prewhere_column_type.reference +++ b/tests/queries/0_stateless/01917_prewhere_column_type.reference @@ -1,2 +1,5 @@ 111 111 +111 +111 +111 diff --git a/tests/queries/0_stateless/01917_prewhere_column_type.sql b/tests/queries/0_stateless/01917_prewhere_column_type.sql index 7ddcb17fbb9..9ce87ab548c 100644 --- a/tests/queries/0_stateless/01917_prewhere_column_type.sql +++ b/tests/queries/0_stateless/01917_prewhere_column_type.sql @@ -7,6 +7,9 @@ CREATE TABLE t1 ( s String, f Float32, e UInt16 ) ENGINE = MergeTree ORDER BY tu INSERT INTO t1 VALUES ('111', 1, 1); SELECT s FROM t1 WHERE f AND (e = 1); +SELECT s FROM t1 WHERE f AND (e = 1) SETTINGS optimize_move_to_prewhere=true; +SELECT s FROM t1 WHERE f AND (e = 1) SETTINGS optimize_move_to_prewhere=false; +SELECT s FROM t1 PREWHERE f AND (e = 1); SELECT s FROM t1 PREWHERE f; -- { serverError 59 } SELECT s FROM t1 PREWHERE f WHERE (e = 1); -- { serverError 59 } SELECT s FROM t1 PREWHERE f WHERE f AND (e = 1); -- { serverError 59 } From 2f9043f16cfd24bfb6f9a8160dfd6dfa5242f1b1 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 5 Jun 2023 20:11:36 +0200 Subject: [PATCH 0613/2047] Properly deal with columns computed at previous steps --- .../MergeTree/MergeTreeBlockReadUtils.cpp | 17 ++++++++++++++++- ...teps_in_prewhere_reuse_computation.reference | 14 ++++++++++++++ ...read_steps_in_prewhere_reuse_computation.sql | 17 +++++++++++++++++ 3 files changed, 47 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.reference create mode 100644 tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.sql diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index c7434eab05d..d830ba37e71 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -328,11 +328,22 @@ MergeTreeReadTaskColumns getReadTaskColumns( NameSet columns_from_previous_steps; auto add_step = [&](const PrewhereExprStep & step) { - Names step_column_names = step.actions->getActionsDAG().getRequiredColumnsNames(); + Names step_column_names; + + /// Computation results from previous steps might be used in the current step as well. In such a case these + /// computed columns will be present in the current step inputs. They don't need to be read from the disk so + /// exclude them from the list of columns to read. This filtering must be done before injecting required + /// columns to avoid adding unnecessary columns or failing to find required columns that are computation + /// results from previous steps. + /// Example: step1: sin(a)>b, step2: sin(a)>c + for (const auto & name : step.actions->getActionsDAG().getRequiredColumnsNames()) + if (!columns_from_previous_steps.contains(name)) + step_column_names.push_back(name); injectRequiredColumns( data_part_info_for_reader, storage_snapshot, with_subcolumns, step_column_names); + /// More columns could have been added, filter them as well by the list of columns from previous steps. Names columns_to_read_in_step; for (const auto & name : step_column_names) { @@ -343,6 +354,10 @@ MergeTreeReadTaskColumns getReadTaskColumns( columns_from_previous_steps.insert(name); } + /// Add results of the step to the list of already "known" columns so that we don't read or compute them again. + for (const auto & name : step.actions->getActionsDAG().getNames()) + columns_from_previous_steps.insert(name); + result.pre_columns.push_back(storage_snapshot->getColumnsByNames(options, columns_to_read_in_step)); }; diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.reference b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.reference new file mode 100644 index 00000000000..904d46b184a --- /dev/null +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.reference @@ -0,0 +1,14 @@ +-- { echoOn } + +SELECT a FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c; +1 +2 +SELECT sin(a) > 2 FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c; +0 +0 +SELECT sin(a) < a FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c AND sin(a) > -a; +1 +1 +SELECT sin(a) < a FROM t_02559 PREWHERE sin(a) < b AND a <= c AND sin(a) > -a; +1 +1 diff --git a/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.sql b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.sql new file mode 100644 index 00000000000..544f5f03cad --- /dev/null +++ b/tests/queries/0_stateless/02559_multiple_read_steps_in_prewhere_reuse_computation.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS t_02559; +CREATE TABLE t_02559 (a Int64, b Int64, c Int64) ENGINE = MergeTree ORDER BY a; + +INSERT INTO t_02559 SELECT number, number, number FROM numbers(3); + +SET enable_multiple_prewhere_read_steps = 1; + +-- { echoOn } + +SELECT a FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c; +SELECT sin(a) > 2 FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c; +SELECT sin(a) < a FROM t_02559 PREWHERE sin(a) < b AND sin(a) < c AND sin(a) > -a; +SELECT sin(a) < a FROM t_02559 PREWHERE sin(a) < b AND a <= c AND sin(a) > -a; + +-- {echoOff} + +DROP TABLE t_02559; From 4ee0bacd945415e9d252d89697541c101e8efaf4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 7 Jun 2023 16:38:54 +0200 Subject: [PATCH 0614/2047] Fix indentation --- ...771_ignore_data_skipping_indices.reference | 74 +++++++++---------- 1 file changed, 37 insertions(+), 37 deletions(-) diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference index 786360783fd..fcede2caf2a 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference @@ -1,40 +1,40 @@ 1 2 3 1 2 3 1 2 3 - ReadFromMergeTree (default.data_02771) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 1/1 - Skip - Name: x_idx - Description: minmax GRANULARITY 1 - Parts: 0/1 - Granules: 0/1 - Skip - Name: y_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 - Skip - Name: xy_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 - ReadFromMergeTree (default.data_02771) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 1/1 - Skip - Name: x_idx - Description: minmax GRANULARITY 1 - Parts: 0/1 - Granules: 0/1 - Skip - Name: y_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + Skip + Name: xy_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 From 3e4182fc4fa7f0852aa5e28acd09eb9156f67caf Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 7 Jun 2023 23:19:43 +0200 Subject: [PATCH 0615/2047] Include source columns for sampling into required columns list --- .../QueryPlan/Optimizations/optimizePrewhere.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index ca8a412bf2e..3352567943a 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -138,8 +138,11 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) if (table_expression_modifiers && table_expression_modifiers->hasSampleSizeRatio()) { const auto & sampling_key = storage_snapshot->getMetadataForQuery()->getSamplingKey(); - const auto & sampling_columns = sampling_key.sample_block.getColumnsWithTypeAndName(); - required_columns_after_filter.insert(required_columns_after_filter.end(), sampling_columns.begin(), sampling_columns.end()); + const auto & sampling_source_columns = sampling_key.expression->getRequiredColumnsWithTypes(); + for (const auto & column : sampling_source_columns) + required_columns_after_filter.push_back(ColumnWithTypeAndName(column.type, column.name)); + const auto & sampling_result_columns = sampling_key.sample_block.getColumnsWithTypeAndName(); + required_columns_after_filter.insert(required_columns_after_filter.end(), sampling_result_columns.begin(), sampling_result_columns.end()); } const auto & storage = storage_snapshot->storage; From 68a2c6301f982086bf5bebf47b827212d653bedd Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 6 Jul 2023 14:21:42 +0200 Subject: [PATCH 0616/2047] Fix 01786_explain_merge_tree --- .../01786_explain_merge_tree.reference | 160 +++++++++--------- 1 file changed, 79 insertions(+), 81 deletions(-) diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.reference b/tests/queries/0_stateless/01786_explain_merge_tree.reference index 8d3954484dd..794acc310ce 100644 --- a/tests/queries/0_stateless/01786_explain_merge_tree.reference +++ b/tests/queries/0_stateless/01786_explain_merge_tree.reference @@ -1,85 +1,83 @@ - ReadFromMergeTree (default.test_index) - Indexes: - MinMax - Keys: - y - Condition: (y in [1, +Inf)) - Parts: 4/5 - Granules: 11/12 - Partition - Keys: - y - bitAnd(z, 3) - Condition: and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])) - Parts: 3/4 - Granules: 10/11 - PrimaryKey - Keys: - x - y - Condition: and((x in [11, +Inf)), (y in [1, +Inf))) - Parts: 2/3 - Granules: 6/10 - Skip - Name: t_minmax - Description: minmax GRANULARITY 2 - Parts: 1/2 - Granules: 4/6 - Skip - Name: t_set - Description: set GRANULARITY 2 - Parts: 1/1 - Granules: 2/4 + ReadFromMergeTree (default.test_index) + Indexes: + MinMax + Keys: + y + Condition: (y in [1, +Inf)) + Parts: 4/5 + Granules: 11/12 + Partition + Keys: + y + bitAnd(z, 3) + Condition: and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])) + Parts: 3/4 + Granules: 10/11 + PrimaryKey + Keys: + x + y + Condition: and((x in [11, +Inf)), (y in [1, +Inf))) + Parts: 2/3 + Granules: 6/10 + Skip + Name: t_minmax + Description: minmax GRANULARITY 2 + Parts: 1/2 + Granules: 4/6 + Skip + Name: t_set + Description: set GRANULARITY 2 + Parts: 1/1 + Granules: 2/4 ----------------- - "Node Type": "ReadFromMergeTree", - "Description": "default.test_index", - "Indexes": [ - { - "Type": "MinMax", - "Keys": ["y"], - "Condition": "(y in [1, +Inf))", - "Initial Parts": 5, - "Selected Parts": 4, - "Initial Granules": 12, - "Selected Granules": 11 - }, - { - "Type": "Partition", - "Keys": ["y", "bitAnd(z, 3)"], - "Condition": "and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))", - "Initial Parts": 4, - "Selected Parts": 3, - "Initial Granules": 11, - "Selected Granules": 10 - }, - { - "Type": "PrimaryKey", - "Keys": ["x", "y"], - "Condition": "and((x in [11, +Inf)), (y in [1, +Inf)))", - "Initial Parts": 3, - "Selected Parts": 2, - "Initial Granules": 10, - "Selected Granules": 6 - }, - { - "Type": "Skip", - "Name": "t_minmax", - "Description": "minmax GRANULARITY 2", - "Initial Parts": 2, - "Selected Parts": 1, - "Initial Granules": 6, - "Selected Granules": 4 - }, - { - "Type": "Skip", - "Name": "t_set", - "Description": "set GRANULARITY 2", - "Initial Parts": 1, - "Selected Parts": 1, - "Initial Granules": 4, - "Selected Granules": 2 - } - ] + "Node Type": "ReadFromMergeTree", + "Description": "default.test_index", + "Indexes": [ + { + "Type": "MinMax", + "Keys": ["y"], + "Condition": "(y in [1, +Inf))", + "Initial Parts": 5, + "Selected Parts": 4, + "Initial Granules": 12, + "Selected Granules": 11 + }, + { + "Type": "Partition", + "Keys": ["y", "bitAnd(z, 3)"], + "Condition": "and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))", + "Initial Parts": 4, + "Selected Parts": 3, + "Initial Granules": 11, + "Selected Granules": 10 + }, + { + "Type": "PrimaryKey", + "Keys": ["x", "y"], + "Condition": "and((x in [11, +Inf)), (y in [1, +Inf)))", + "Initial Parts": 3, + "Selected Parts": 2, + "Initial Granules": 10, + "Selected Granules": 6 + }, + { + "Type": "Skip", + "Name": "t_minmax", + "Description": "minmax GRANULARITY 2", + "Initial Parts": 2, + "Selected Parts": 1, + "Initial Granules": 6, + "Selected Granules": 4 + }, + { + "Type": "Skip", + "Name": "t_set", + "Description": "set GRANULARITY 2", + "Initial Parts": 1, + "Selected Parts": 1, + "Initial Granules": 4, + "Selected Granules": 2 } ] } From 6886e843deff60f03b93f28802816261ed330e27 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 14 Jul 2023 17:23:46 +0200 Subject: [PATCH 0617/2047] Fix flakiness due to randomized enable_multiple_prewhere_read_steps setting --- tests/queries/1_stateful/00091_prewhere_two_conditions.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/1_stateful/00091_prewhere_two_conditions.sql b/tests/queries/1_stateful/00091_prewhere_two_conditions.sql index 745bb125c2b..cbfbbaa2662 100644 --- a/tests/queries/1_stateful/00091_prewhere_two_conditions.sql +++ b/tests/queries/1_stateful/00091_prewhere_two_conditions.sql @@ -4,6 +4,7 @@ SET max_bytes_to_read = 600000000; SET optimize_move_to_prewhere = 1; +SET enable_multiple_prewhere_read_steps = 1; SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND URL != '' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; @@ -11,6 +12,8 @@ SELECT uniq(*) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014 WITH toTimeZone(EventTime, 'Asia/Dubai') AS xyz SELECT uniq(*) FROM test.hits WHERE xyz >= '2014-03-20 00:00:00' AND xyz < '2014-03-21 00:00:00' AND EventDate = '2014-03-21'; SET optimize_move_to_prewhere = 0; +SET enable_multiple_prewhere_read_steps = 0; SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; -- { serverError 307 } SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND URL != '' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; -- { serverError 307 } +SELECT uniq(URL) FROM test.hits PREWHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND URL != '' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; -- { serverError 307 } From a4f7b7717768893983be2973cef9b1a5684b1949 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:21:00 +0000 Subject: [PATCH 0618/2047] Add array_concat_agg for compatibility with BigQuery --- .../AggregateFunctionFactory.cpp | 29 ++++++++++++++++++- .../AggregateFunctionFactory.h | 3 ++ .../registerAggregateFunctions.cpp | 8 +++++ src/Common/IFactoryWithAliases.h | 20 ++++++------- .../02813_array_concat_agg.reference | 5 ++++ .../0_stateless/02813_array_concat_agg.sql | 9 ++++++ 6 files changed, 63 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02813_array_concat_agg.reference create mode 100644 tests/queries/0_stateless/02813_array_concat_agg.sql diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index 6cacf66500f..f2dbb931b1f 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -59,6 +59,34 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat } } +void AggregateFunctionFactory::registerAliasForAggregateFunctionWithCombinator( + const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness) +{ + if (!isAggregateFunctionName(real_name)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "{}: can't create alias '{}', the real name '{}' is not registered", + getFactoryName(), + alias_name, + real_name); + + auto alias_name_lowercase = Poco::toLower(alias_name); + + if (aggregate_functions.contains(alias_name) || case_insensitive_aggregate_functions.contains(alias_name_lowercase)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", getFactoryName(), alias_name); + + if (case_sensitiveness == CaseInsensitive) + { + if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", getFactoryName(), alias_name); + case_insensitive_name_mapping[alias_name_lowercase] = real_name; + } + + if (!aliases.emplace(alias_name, real_name).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: alias name '{}' is not unique", getFactoryName(), alias_name); +} + static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types) { DataTypes res_types; @@ -222,7 +250,6 @@ AggregateFunctionPtr AggregateFunctionFactory::tryGet( : nullptr; } - std::optional AggregateFunctionFactory::tryGetProperties(String name) const { if (name.size() > MAX_AGGREGATE_FUNCTION_NAME_LENGTH) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.h b/src/AggregateFunctions/AggregateFunctionFactory.h index dab0d28e851..6c2b539bd8c 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.h +++ b/src/AggregateFunctions/AggregateFunctionFactory.h @@ -62,6 +62,9 @@ public: Value creator, CaseSensitiveness case_sensitiveness = CaseSensitive); + void registerAliasForAggregateFunctionWithCombinator( + const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive); + /// Throws an exception if not found. AggregateFunctionPtr get(const String & name, diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 91248a52ae9..87d96f0d1ce 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -193,6 +193,14 @@ void registerAggregateFunctions() registerAggregateFunctionCombinatorDistinct(factory); registerAggregateFunctionCombinatorMap(factory); } + + { + auto & factory = AggregateFunctionFactory::instance(); + + /// Must register after registerCombinator + factory.registerAliasForAggregateFunctionWithCombinator( + "array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive); + } } } diff --git a/src/Common/IFactoryWithAliases.h b/src/Common/IFactoryWithAliases.h index b2ac4ab289e..af5656ffb75 100644 --- a/src/Common/IFactoryWithAliases.h +++ b/src/Common/IFactoryWithAliases.h @@ -10,6 +10,8 @@ namespace DB { +class AggregateFunctionFactory; + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -22,6 +24,7 @@ namespace ErrorCodes template class IFactoryWithAliases : public IHints<2, IFactoryWithAliases> { + friend AggregateFunctionFactory; protected: using Value = ValueType; @@ -55,9 +58,9 @@ public: const String factory_name = getFactoryName(); String real_dict_name; - if (creator_map.count(real_name)) + if (creator_map.contains(real_name)) real_dict_name = real_name; - else if (auto real_name_lowercase = Poco::toLower(real_name); case_insensitive_creator_map.count(real_name_lowercase)) + else if (auto real_name_lowercase = Poco::toLower(real_name); case_insensitive_creator_map.contains(real_name_lowercase)) real_dict_name = real_name_lowercase; else throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: can't create alias '{}', the real name '{}' is not registered", @@ -65,7 +68,7 @@ public: String alias_name_lowercase = Poco::toLower(alias_name); - if (creator_map.count(alias_name) || case_insensitive_creator_map.count(alias_name_lowercase)) + if (creator_map.contains(alias_name) || case_insensitive_creator_map.contains(alias_name_lowercase)) throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", factory_name, alias_name); @@ -93,7 +96,7 @@ public: bool isCaseInsensitive(const String & name) const { String name_lowercase = Poco::toLower(name); - return getCaseInsensitiveMap().count(name_lowercase) || case_insensitive_aliases.count(name_lowercase); + return getCaseInsensitiveMap().contains(name_lowercase) || case_insensitive_aliases.contains(name_lowercase); } const String & aliasTo(const String & name) const @@ -106,14 +109,11 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: name '{}' is not alias", getFactoryName(), name); } - bool isAlias(const String & name) const - { - return aliases.count(name) || case_insensitive_aliases.contains(name); - } + bool isAlias(const String & name) const { return aliases.contains(name) || case_insensitive_aliases.contains(name); } bool hasNameOrAlias(const String & name) const { - return getMap().count(name) || getCaseInsensitiveMap().count(name) || isAlias(name); + return getMap().contains(name) || getCaseInsensitiveMap().contains(name) || isAlias(name); } /// Return the canonical name (the name used in registration) if it's different from `name`. @@ -129,7 +129,7 @@ public: private: using InnerMap = std::unordered_map; // name -> creator - using AliasMap = std::unordered_map; // alias -> original type + using AliasMap = std::unordered_map; // alias -> original name virtual const InnerMap & getMap() const = 0; virtual const InnerMap & getCaseInsensitiveMap() const = 0; diff --git a/tests/queries/0_stateless/02813_array_concat_agg.reference b/tests/queries/0_stateless/02813_array_concat_agg.reference new file mode 100644 index 00000000000..7144a499922 --- /dev/null +++ b/tests/queries/0_stateless/02813_array_concat_agg.reference @@ -0,0 +1,5 @@ +[1,2,3,4,5,6] +[1,2,3,4,5,6] +1 [1,2,3] +2 [4,5] +3 [6] diff --git a/tests/queries/0_stateless/02813_array_concat_agg.sql b/tests/queries/0_stateless/02813_array_concat_agg.sql new file mode 100644 index 00000000000..94fe133db7d --- /dev/null +++ b/tests/queries/0_stateless/02813_array_concat_agg.sql @@ -0,0 +1,9 @@ +drop table if exists t; + +create table t (n UInt32, a Array(Int32)) engine=Memory; +insert into t values (1, [1,2,3]), (2, [4,5]), (3, [6]); + +select array_concat_agg(a) from t; +select ArrAy_cOncAt_aGg(a) from t; +select n, array_concat_agg(a) from t group by n order by n; +drop table t; From 4955e07326c4220af989a8938dda9fc3c783c9de Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:27:34 +0000 Subject: [PATCH 0619/2047] add docs --- .../reference/arrayconcatagg.md | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md new file mode 100644 index 00000000000..50e4ed63787 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md @@ -0,0 +1,32 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/arrayconcatagg +sidebar_position: 110 +--- + +# array_concat_agg +- Alias of `groupArrayArray`. The function is case insensitive. + +**Exampla** + +```text +SELECT * +FROM t + +┌─a───────┐ +│ [1,2,3] │ +│ [4,5] │ +│ [6] │ +└─────────┘ + +``` + +Query: + +```sql +SELECT array_concat_agg(a) AS a +FROM t + +┌─a─────────────┐ +│ [1,2,3,4,5,6] │ +└───────────────┘ +``` From f59370e5f98f72fa136529ed4a4f545faf18b510 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:50:29 +0000 Subject: [PATCH 0620/2047] Fix style --- .../reference/{arrayconcatagg.md => array_concat_agg.md} | 2 +- src/AggregateFunctions/registerAggregateFunctions.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename docs/en/sql-reference/aggregate-functions/reference/{arrayconcatagg.md => array_concat_agg.md} (97%) diff --git a/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md b/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md similarity index 97% rename from docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md rename to docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md index 50e4ed63787..db1f1a10859 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md +++ b/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md @@ -6,7 +6,7 @@ sidebar_position: 110 # array_concat_agg - Alias of `groupArrayArray`. The function is case insensitive. -**Exampla** +**Example** ```text SELECT * diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 87d96f0d1ce..92484ad93ff 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -194,7 +194,7 @@ void registerAggregateFunctions() registerAggregateFunctionCombinatorMap(factory); } - { + { auto & factory = AggregateFunctionFactory::instance(); /// Must register after registerCombinator From 6899070f95ea81b6a77090f06fb45332f475d349 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:54:13 +0000 Subject: [PATCH 0621/2047] fix --- .../reference/{array_concat_agg.md => arrayconcatagg.md} | 2 +- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) rename docs/en/sql-reference/aggregate-functions/reference/{array_concat_agg.md => arrayconcatagg.md} (86%) diff --git a/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md similarity index 86% rename from docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md rename to docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md index db1f1a10859..3c71129bdb5 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md +++ b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md @@ -1,5 +1,5 @@ --- -slug: /en/sql-reference/aggregate-functions/reference/arrayconcatagg +slug: /en/sql-reference/aggregate-functions/reference/array_concat_agg sidebar_position: 110 --- diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2802e52c288..fc2cd5640b1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -990,6 +990,7 @@ addressToLine addressToLineWithInlines addressToSymbol adviced +agg aggregatefunction aggregatingmergetree aggregatio From 43cd600e21b4e3ac0d38e6be1daa153f3e9765ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 17:35:51 +0300 Subject: [PATCH 0622/2047] Update 02813_any_value.sql --- tests/queries/0_stateless/02813_any_value.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02813_any_value.sql b/tests/queries/0_stateless/02813_any_value.sql index d2c2324349c..6bd2b66fde1 100644 --- a/tests/queries/0_stateless/02813_any_value.sql +++ b/tests/queries/0_stateless/02813_any_value.sql @@ -1,2 +1,3 @@ +SET max_block_size = 10, max_threads = 1; select any_value(number) from numbers(10); select aNy_VaLue(number) from numbers(10); From 71d6206fc45e8622230ed2396f2662d76851eb37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 17:42:22 +0300 Subject: [PATCH 0623/2047] Fix Python --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 790adf9df62..abd109d00b2 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1229,7 +1229,7 @@ class TestCase: .replace("\n", ", ") ) - if 0 != len(leftover_tables): + if len(leftover_tables) != 0: raise Exception( f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally." ) From bbade814f3e59bf31fb86cadb17b0ffed1661257 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 17:10:28 +0000 Subject: [PATCH 0624/2047] fix --- .../AggregateFunctionFactory.cpp | 28 ------------- .../AggregateFunctionFactory.h | 3 -- .../AggregateFunctionGroupArray.cpp | 1 + .../registerAggregateFunctions.cpp | 8 ---- src/Common/IFactoryWithAliases.h | 40 +++++++++---------- 5 files changed, 21 insertions(+), 59 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index f2dbb931b1f..f52c9ac8510 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -59,34 +59,6 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat } } -void AggregateFunctionFactory::registerAliasForAggregateFunctionWithCombinator( - const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness) -{ - if (!isAggregateFunctionName(real_name)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "{}: can't create alias '{}', the real name '{}' is not registered", - getFactoryName(), - alias_name, - real_name); - - auto alias_name_lowercase = Poco::toLower(alias_name); - - if (aggregate_functions.contains(alias_name) || case_insensitive_aggregate_functions.contains(alias_name_lowercase)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", getFactoryName(), alias_name); - - if (case_sensitiveness == CaseInsensitive) - { - if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", getFactoryName(), alias_name); - case_insensitive_name_mapping[alias_name_lowercase] = real_name; - } - - if (!aliases.emplace(alias_name, real_name).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: alias name '{}' is not unique", getFactoryName(), alias_name); -} - static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types) { DataTypes res_types; diff --git a/src/AggregateFunctions/AggregateFunctionFactory.h b/src/AggregateFunctions/AggregateFunctionFactory.h index 6c2b539bd8c..dab0d28e851 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.h +++ b/src/AggregateFunctions/AggregateFunctionFactory.h @@ -62,9 +62,6 @@ public: Value creator, CaseSensitiveness case_sensitiveness = CaseSensitive); - void registerAliasForAggregateFunctionWithCombinator( - const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive); - /// Throws an exception if not found. AggregateFunctionPtr get(const String & name, diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index bb1368b9ff8..a09e9ddfbf6 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -125,6 +125,7 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory) AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("groupArray", { createAggregateFunctionGroupArray, properties }); + factory.registerAliasUnchecked("array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("groupArraySample", { createAggregateFunctionGroupArraySample, properties }); factory.registerFunction("groupArrayLast", { createAggregateFunctionGroupArray, properties }); } diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 92484ad93ff..91248a52ae9 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -193,14 +193,6 @@ void registerAggregateFunctions() registerAggregateFunctionCombinatorDistinct(factory); registerAggregateFunctionCombinatorMap(factory); } - - { - auto & factory = AggregateFunctionFactory::instance(); - - /// Must register after registerCombinator - factory.registerAliasForAggregateFunctionWithCombinator( - "array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive); - } } } diff --git a/src/Common/IFactoryWithAliases.h b/src/Common/IFactoryWithAliases.h index af5656ffb75..07440dd2463 100644 --- a/src/Common/IFactoryWithAliases.h +++ b/src/Common/IFactoryWithAliases.h @@ -10,8 +10,6 @@ namespace DB { -class AggregateFunctionFactory; - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -24,7 +22,6 @@ namespace ErrorCodes template class IFactoryWithAliases : public IHints<2, IFactoryWithAliases> { - friend AggregateFunctionFactory; protected: using Value = ValueType; @@ -55,35 +52,38 @@ public: { const auto & creator_map = getMap(); const auto & case_insensitive_creator_map = getCaseInsensitiveMap(); - const String factory_name = getFactoryName(); - String real_dict_name; - if (creator_map.contains(real_name)) - real_dict_name = real_name; - else if (auto real_name_lowercase = Poco::toLower(real_name); case_insensitive_creator_map.contains(real_name_lowercase)) - real_dict_name = real_name_lowercase; - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: can't create alias '{}', the real name '{}' is not registered", - factory_name, alias_name, real_name); + auto real_name_lowercase = Poco::toLower(real_name); + if (!creator_map.contains(real_name) && !case_insensitive_creator_map.contains(real_name_lowercase)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "{}: can't create alias '{}', the real name '{}' is not registered", + getFactoryName(), + alias_name, + real_name); + registerAliasUnchecked(alias_name, real_name, case_sensitiveness); + } + + /// We need sure the real_name exactly exists when call the function directly. + void registerAliasUnchecked(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive) + { String alias_name_lowercase = Poco::toLower(alias_name); - - if (creator_map.contains(alias_name) || case_insensitive_creator_map.contains(alias_name_lowercase)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", - factory_name, alias_name); + String real_name_lowercase = Poco::toLower(real_name); + const String factory_name = getFactoryName(); if (case_sensitiveness == CaseInsensitive) { - if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_dict_name).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", - factory_name, alias_name); + if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", factory_name, alias_name); case_insensitive_name_mapping[alias_name_lowercase] = real_name; } - if (!aliases.emplace(alias_name, real_dict_name).second) + if (!aliases.emplace(alias_name, real_name).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: alias name '{}' is not unique", factory_name, alias_name); } + std::vector getAllRegisteredNames() const override { std::vector result; From a644317f577f6c5b573ddc99589ac8745d8e2e13 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Sun, 16 Jul 2023 18:07:22 +0000 Subject: [PATCH 0625/2047] Revert "Merge pull request #52129 from ClickHouse/revert-51291-ADQM-868" This reverts commit 1b9bcae68ee971a241b2a70b20408de095f1529f, reversing changes made to 7284749d05f31a687cd8091e1ffc249afd7c16b5. --- .../functions/date-time-functions.md | 6 +- .../functions/date-time-functions.md | 6 +- .../functions/date-time-functions.md | 2 + src/Core/DecimalFunctions.h | 6 +- src/Functions/DateTimeTransforms.h | 71 ++++++++++-- src/Functions/TransformDateTime64.h | 7 +- src/Functions/dateDiff.cpp | 58 +++++++--- src/Functions/toStartOfInterval.cpp | 1 - ...0479_date_and_datetime_to_number.reference | 1 + .../00479_date_and_datetime_to_number.sql | 1 + .../02160_special_functions.reference | 10 ++ .../0_stateless/02160_special_functions.sql | 12 ++ .../02477_age_datetime64.reference | 106 ++++++++++++++++++ .../0_stateless/02477_age_datetime64.sql | 65 +++++++++++ 14 files changed, 323 insertions(+), 29 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 19eeda967fe..ce1a4f4d283 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -722,7 +722,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d ## age -Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 second. +Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 microsecond. E.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for `day` unit, 0 months for `month` unit, 0 years for `year` unit. For an alternative to `age`, see function `date\_diff`. @@ -738,6 +738,8 @@ age('unit', startdate, enddate, [timezone]) - `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). Possible values: + - `microsecond` (possible abbreviations: `us`, `u`) + - `millisecond` (possible abbreviations: `ms`) - `second` (possible abbreviations: `ss`, `s`) - `minute` (possible abbreviations: `mi`, `n`) - `hour` (possible abbreviations: `hh`, `h`) @@ -813,6 +815,8 @@ Aliases: `dateDiff`, `DATE_DIFF`, `timestampDiff`, `timestamp_diff`, `TIMESTAMP_ - `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). Possible values: + - `microsecond` (possible abbreviations: `us`, `u`) + - `millisecond` (possible abbreviations: `ms`) - `second` (possible abbreviations: `ss`, `s`) - `minute` (possible abbreviations: `mi`, `n`) - `hour` (possible abbreviations: `hh`, `h`) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 779728ca0fe..4db8a1ec6f8 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -625,7 +625,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d ## age -Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 секунду. +Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 микросекунду. Например, разница между `2021-12-29` и `2022-01-01` 3 дня для единицы `day`, 0 месяцев для единицы `month`, 0 лет для единицы `year`. **Синтаксис** @@ -639,6 +639,8 @@ age('unit', startdate, enddate, [timezone]) - `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md). Возможные значения: + - `microsecond` (возможные сокращения: `us`, `u`) + - `millisecond` (возможные сокращения: `ms`) - `second` (возможные сокращения: `ss`, `s`) - `minute` (возможные сокращения: `mi`, `n`) - `hour` (возможные сокращения: `hh`, `h`) @@ -712,6 +714,8 @@ date_diff('unit', startdate, enddate, [timezone]) - `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md). Возможные значения: + - `microsecond` (возможные сокращения: `us`, `u`) + - `millisecond` (возможные сокращения: `ms`) - `second` (возможные сокращения: `ss`, `s`) - `minute` (возможные сокращения: `mi`, `n`) - `hour` (возможные сокращения: `hh`, `h`) diff --git a/docs/zh/sql-reference/functions/date-time-functions.md b/docs/zh/sql-reference/functions/date-time-functions.md index 53dadc23c6d..e4b70322477 100644 --- a/docs/zh/sql-reference/functions/date-time-functions.md +++ b/docs/zh/sql-reference/functions/date-time-functions.md @@ -643,6 +643,8 @@ date_diff('unit', startdate, enddate, [timezone]) - `unit` — `value`对应的时间单位。类型为[String](../../sql-reference/data-types/string.md)。 可能的值: + - `microsecond` + - `millisecond` - `second` - `minute` - `hour` diff --git a/src/Core/DecimalFunctions.h b/src/Core/DecimalFunctions.h index 357cff2c541..17d95650730 100644 --- a/src/Core/DecimalFunctions.h +++ b/src/Core/DecimalFunctions.h @@ -48,7 +48,11 @@ inline auto scaleMultiplier(UInt32 scale) /** Components of DecimalX value: * whole - represents whole part of decimal, can be negative or positive. - * fractional - for fractional part of decimal, always positive. + * fractional - for fractional part of decimal. + * + * 0.123 represents 0 / 0.123 + * -0.123 represents 0 / -0.123 + * -1.123 represents -1 / 0.123 */ template struct DecimalComponents diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 510a88db2b6..a1c880f6956 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -19,6 +19,9 @@ namespace DB { +static constexpr auto microsecond_multiplier = 1000000; +static constexpr auto millisecond_multiplier = 1000; + namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -1377,6 +1380,36 @@ struct ToRelativeSecondNumImpl using FactorTransform = ZeroTransform; }; +template +struct ToRelativeSubsecondNumImpl +{ + static constexpr auto name = "toRelativeSubsecondNumImpl"; + + static inline Int64 execute(const DateTime64 & t, DateTime64::NativeType scale, const DateLUTImpl &) + { + static_assert(scale_multiplier == 1000 || scale_multiplier == 1000000); + if (scale == scale_multiplier) + return t.value; + if (scale > scale_multiplier) + return t.value / (scale / scale_multiplier); + return t.value * (scale_multiplier / scale); + } + static inline Int64 execute(UInt32 t, const DateLUTImpl &) + { + return t * scale_multiplier; + } + static inline Int64 execute(Int32 d, const DateLUTImpl & time_zone) + { + return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))) * scale_multiplier; + } + static inline Int64 execute(UInt16 d, const DateLUTImpl & time_zone) + { + return static_cast(time_zone.fromDayNum(DayNum(d)) * scale_multiplier); + } + + using FactorTransform = ZeroTransform; +}; + struct ToYYYYMMImpl { static constexpr auto name = "toYYYYMM"; @@ -1476,25 +1509,47 @@ struct ToYYYYMMDDhhmmssImpl using FactorTransform = ZeroTransform; }; +struct DateTimeComponentsWithFractionalPart : public DateLUTImpl::DateTimeComponents +{ + UInt16 millisecond; + UInt16 microsecond; +}; + struct ToDateTimeComponentsImpl { static constexpr auto name = "toDateTimeComponents"; - static inline DateLUTImpl::DateTimeComponents execute(Int64 t, const DateLUTImpl & time_zone) + static inline DateTimeComponentsWithFractionalPart execute(const DateTime64 & t, DateTime64::NativeType scale_multiplier, const DateLUTImpl & time_zone) { - return time_zone.toDateTimeComponents(t); + auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); + + if (t.value < 0 && components.fractional) + { + components.fractional = scale_multiplier + (components.whole ? Int64(-1) : Int64(1)) * components.fractional; + --components.whole; + } + Int64 fractional = components.fractional; + if (scale_multiplier > microsecond_multiplier) + fractional = fractional / (scale_multiplier / microsecond_multiplier); + else if (scale_multiplier < microsecond_multiplier) + fractional = fractional * (microsecond_multiplier / scale_multiplier); + + constexpr Int64 divider = microsecond_multiplier/ millisecond_multiplier; + UInt16 millisecond = static_cast(fractional / divider); + UInt16 microsecond = static_cast(fractional % divider); + return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(components.whole), millisecond, microsecond}; } - static inline DateLUTImpl::DateTimeComponents execute(UInt32 t, const DateLUTImpl & time_zone) + static inline DateTimeComponentsWithFractionalPart execute(UInt32 t, const DateLUTImpl & time_zone) { - return time_zone.toDateTimeComponents(static_cast(t)); + return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(static_cast(t)), 0, 0}; } - static inline DateLUTImpl::DateTimeComponents execute(Int32 d, const DateLUTImpl & time_zone) + static inline DateTimeComponentsWithFractionalPart execute(Int32 d, const DateLUTImpl & time_zone) { - return time_zone.toDateTimeComponents(ExtendedDayNum(d)); + return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(ExtendedDayNum(d)), 0, 0}; } - static inline DateLUTImpl::DateTimeComponents execute(UInt16 d, const DateLUTImpl & time_zone) + static inline DateTimeComponentsWithFractionalPart execute(UInt16 d, const DateLUTImpl & time_zone) { - return time_zone.toDateTimeComponents(DayNum(d)); + return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(DayNum(d)), 0, 0}; } using FactorTransform = ZeroTransform; diff --git a/src/Functions/TransformDateTime64.h b/src/Functions/TransformDateTime64.h index 3dab9efeb6b..fcee2753066 100644 --- a/src/Functions/TransformDateTime64.h +++ b/src/Functions/TransformDateTime64.h @@ -5,7 +5,7 @@ namespace DB { -/** Tansform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform. +/** Transform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform. * * Depending on what overloads of Transform::execute() are available, when called with DateTime64 value, * invokes Transform::execute() with either: @@ -80,7 +80,10 @@ public: } else { - const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); + auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); + if (t.value < 0 && components.fractional) + --components.whole; + return wrapped_transform.execute(static_cast(components.whole), std::forward(args)...); } } diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 8361e9db166..6bfbbb7c735 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -174,12 +174,13 @@ public: { auto res = static_cast(transform_y.execute(y, timezone_y)) - static_cast(transform_x.execute(x, timezone_x)); - DateLUTImpl::DateTimeComponents a_comp; - DateLUTImpl::DateTimeComponents b_comp; + DateTimeComponentsWithFractionalPart a_comp; + DateTimeComponentsWithFractionalPart b_comp; Int64 adjust_value; - auto x_seconds = TransformDateTime64>(transform_x.getScaleMultiplier()).execute(x, timezone_x); - auto y_seconds = TransformDateTime64>(transform_y.getScaleMultiplier()).execute(y, timezone_y); - if (x_seconds <= y_seconds) + auto x_microseconds = TransformDateTime64>(transform_x.getScaleMultiplier()).execute(x, timezone_x); + auto y_microseconds = TransformDateTime64>(transform_y.getScaleMultiplier()).execute(y, timezone_y); + + if (x_microseconds <= y_microseconds) { a_comp = TransformDateTime64(transform_x.getScaleMultiplier()).execute(x, timezone_x); b_comp = TransformDateTime64(transform_y.getScaleMultiplier()).execute(y, timezone_y); @@ -192,14 +193,16 @@ public: adjust_value = 1; } + if constexpr (std::is_same_v>>) { if ((a_comp.date.month > b_comp.date.month) || ((a_comp.date.month == b_comp.date.month) && ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) - ))))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -210,8 +213,9 @@ public: || ((x_month_in_quarter == y_month_in_quarter) && ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) - ))))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -219,8 +223,9 @@ public: if ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) - ))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -230,25 +235,44 @@ public: if ((x_day_of_week > y_day_of_week) || ((x_day_of_week == y_day_of_week) && (a_comp.time.hour > b_comp.time.hour)) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { if ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { if ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))) + || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { - if (a_comp.time.second > b_comp.time.second) + if ((a_comp.time.second > b_comp.time.second) + || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))) + res += adjust_value; + } + else if constexpr (std::is_same_v>>) + { + if ((a_comp.millisecond > b_comp.millisecond) + || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))) + res += adjust_value; + } + else if constexpr (std::is_same_v>>) + { + if (a_comp.microsecond > b_comp.microsecond) res += adjust_value; } return res; @@ -373,6 +397,10 @@ public: impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "second" || unit == "ss" || unit == "s") impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); + else if (unit == "millisecond" || unit == "ms") + impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); + else if (unit == "microsecond" || unit == "us" || unit == "u") + impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} does not support '{}' unit", getName(), unit); diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 649242d0d86..48bf88cb14c 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference index 1375ccb1542..168b733d702 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference @@ -4,3 +4,4 @@ 201707 20170721 20170721112233 +19691231235959 diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql index 71151690028..1e35e99a802 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql @@ -4,3 +4,4 @@ SELECT toYYYYMMDDhhmmss(toDate('2017-07-21')); SELECT toYYYYMM(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDD(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDDhhmmss(toDateTime('2017-07-21T11:22:33')); +SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); diff --git a/tests/queries/0_stateless/02160_special_functions.reference b/tests/queries/0_stateless/02160_special_functions.reference index 3a1dcd88902..5e7e3383d8d 100644 --- a/tests/queries/0_stateless/02160_special_functions.reference +++ b/tests/queries/0_stateless/02160_special_functions.reference @@ -33,4 +33,14 @@ Hello 2021-01-01 1 1 +86400000 +172800000 +86461000 +86401299 +701 +701 +800 +60200201 +60 +10 1 diff --git a/tests/queries/0_stateless/02160_special_functions.sql b/tests/queries/0_stateless/02160_special_functions.sql index 6d18e7d0d25..64919536be3 100644 --- a/tests/queries/0_stateless/02160_special_functions.sql +++ b/tests/queries/0_stateless/02160_special_functions.sql @@ -41,4 +41,16 @@ SELECT TIMESTAMPSUB(DATE '2022-01-01', INTERVAL 1 YEAR); SELECT DATE_DIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); SELECT DATEDIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); +SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); + +SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); +SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); + SELECT EXISTS (SELECT 1); diff --git a/tests/queries/0_stateless/02477_age_datetime64.reference b/tests/queries/0_stateless/02477_age_datetime64.reference index 3b4459dd26d..c8c716e1e9a 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.reference +++ b/tests/queries/0_stateless/02477_age_datetime64.reference @@ -111,3 +111,109 @@ SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), ma 1 SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); 1 +-- DateTime64 vs DateTime64 with fractional part +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); +5100200 +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); +5100200 +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); +5100 +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); +5099 +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); +4 +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); +4 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); +5 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); +4 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); +4 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +3 +SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); +3 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); +2 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); +2 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); +1 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); +16 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); +15 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); +5 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); +4 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); +8 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); +7 +-- DateTime64 vs DateTime64 with negative time +SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); +2349 +SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); +2 +SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +5 +SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +4 diff --git a/tests/queries/0_stateless/02477_age_datetime64.sql b/tests/queries/0_stateless/02477_age_datetime64.sql index 1bed93991ca..889137395a3 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.sql +++ b/tests/queries/0_stateless/02477_age_datetime64.sql @@ -75,3 +75,68 @@ SELECT age('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), SELECT age('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC'))); SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDate('2015-08-19', 'UTC'))); SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); + +-- DateTime64 vs DateTime64 with fractional part +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); + +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); + +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); + +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); + +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); + +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); + +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); + +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); + +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); + +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); + +-- DateTime64 vs DateTime64 with negative time +SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); \ No newline at end of file From abac46817ab9683f18ccabedeeb627d4f9652e20 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Sun, 16 Jul 2023 18:15:03 +0000 Subject: [PATCH 0626/2047] Move all tests to to one file --- ...0479_date_and_datetime_to_number.reference | 1 - .../00479_date_and_datetime_to_number.sql | 1 - .../02160_special_functions.reference | 10 -- .../0_stateless/02160_special_functions.sql | 12 -- .../02477_age_datetime64.reference | 106 --------------- .../0_stateless/02477_age_datetime64.sql | 65 --------- .../0_stateless/02814_age_datediff.reference | 128 ++++++++++++++++++ .../0_stateless/02814_age_datediff.sql | 79 +++++++++++ 8 files changed, 207 insertions(+), 195 deletions(-) create mode 100644 tests/queries/0_stateless/02814_age_datediff.reference create mode 100644 tests/queries/0_stateless/02814_age_datediff.sql diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference index 168b733d702..1375ccb1542 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference @@ -4,4 +4,3 @@ 201707 20170721 20170721112233 -19691231235959 diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql index 1e35e99a802..71151690028 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql @@ -4,4 +4,3 @@ SELECT toYYYYMMDDhhmmss(toDate('2017-07-21')); SELECT toYYYYMM(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDD(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDDhhmmss(toDateTime('2017-07-21T11:22:33')); -SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); diff --git a/tests/queries/0_stateless/02160_special_functions.reference b/tests/queries/0_stateless/02160_special_functions.reference index 5e7e3383d8d..3a1dcd88902 100644 --- a/tests/queries/0_stateless/02160_special_functions.reference +++ b/tests/queries/0_stateless/02160_special_functions.reference @@ -33,14 +33,4 @@ Hello 2021-01-01 1 1 -86400000 -172800000 -86461000 -86401299 -701 -701 -800 -60200201 -60 -10 1 diff --git a/tests/queries/0_stateless/02160_special_functions.sql b/tests/queries/0_stateless/02160_special_functions.sql index 64919536be3..6d18e7d0d25 100644 --- a/tests/queries/0_stateless/02160_special_functions.sql +++ b/tests/queries/0_stateless/02160_special_functions.sql @@ -41,16 +41,4 @@ SELECT TIMESTAMPSUB(DATE '2022-01-01', INTERVAL 1 YEAR); SELECT DATE_DIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); SELECT DATEDIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); -SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); - -SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); -SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); - SELECT EXISTS (SELECT 1); diff --git a/tests/queries/0_stateless/02477_age_datetime64.reference b/tests/queries/0_stateless/02477_age_datetime64.reference index c8c716e1e9a..3b4459dd26d 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.reference +++ b/tests/queries/0_stateless/02477_age_datetime64.reference @@ -111,109 +111,3 @@ SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), ma 1 SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); 1 --- DateTime64 vs DateTime64 with fractional part -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); -5100200 -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); -5100200 -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); -5100 -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); -5099 -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); -4 -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); -4 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); -5 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); -4 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); -4 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -3 -SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); -3 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); -2 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); -2 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); -1 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); -16 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); -15 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); -5 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); -4 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); -8 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); -7 --- DateTime64 vs DateTime64 with negative time -SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); -2349 -SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); -2 -SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -5 -SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -4 diff --git a/tests/queries/0_stateless/02477_age_datetime64.sql b/tests/queries/0_stateless/02477_age_datetime64.sql index 889137395a3..1bed93991ca 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.sql +++ b/tests/queries/0_stateless/02477_age_datetime64.sql @@ -75,68 +75,3 @@ SELECT age('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), SELECT age('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC'))); SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDate('2015-08-19', 'UTC'))); SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); - --- DateTime64 vs DateTime64 with fractional part -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); - -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); - -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); - -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); - -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); - -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); - -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); - -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); - -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); - -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); - --- DateTime64 vs DateTime64 with negative time -SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); \ No newline at end of file diff --git a/tests/queries/0_stateless/02814_age_datediff.reference b/tests/queries/0_stateless/02814_age_datediff.reference new file mode 100644 index 00000000000..2b065211213 --- /dev/null +++ b/tests/queries/0_stateless/02814_age_datediff.reference @@ -0,0 +1,128 @@ +-- DateTime64 vs DateTime64 with fractional part +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); +5100200 +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); +5100200 +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); +5100 +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); +5099 +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); +4 +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); +4 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); +5 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); +4 +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); +4 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +3 +SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +2 +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); +3 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); +2 +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); +2 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); +2 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); +1 +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); +1 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); +16 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); +15 +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); +15 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); +5 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); +4 +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); +4 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); +8 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); +7 +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); +7 +-- DateTime64 vs DateTime64 with negative time +SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); +2349 +SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); +2 +SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +5 +SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +4 +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date); +86400000 +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32); +172800000 +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime); +86461000 +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64); +86401299 +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date); +701 +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date); +701 +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); +800 +SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); +60200201 +SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); +60 +SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); +10 +SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); +19691231235959 \ No newline at end of file diff --git a/tests/queries/0_stateless/02814_age_datediff.sql b/tests/queries/0_stateless/02814_age_datediff.sql new file mode 100644 index 00000000000..bc77f6de5d1 --- /dev/null +++ b/tests/queries/0_stateless/02814_age_datediff.sql @@ -0,0 +1,79 @@ + +-- DateTime64 vs DateTime64 with fractional part +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); +SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); + +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); +SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); + +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); +SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); + +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); +SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); + +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); +SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); + +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); +SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); + +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); +SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); + +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); +SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); + +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); +SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); + +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); +SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); + +-- DateTime64 vs DateTime64 with negative time +SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); +SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); + +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime); +SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date); +SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); +SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); + +SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); +SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); + +SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); \ No newline at end of file From 1c34d750143b3e4a659231116094a50e0e461327 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Sun, 16 Jul 2023 18:21:10 +0000 Subject: [PATCH 0627/2047] Add new lines --- tests/queries/0_stateless/02814_age_datediff.reference | 2 +- tests/queries/0_stateless/02814_age_datediff.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02814_age_datediff.reference b/tests/queries/0_stateless/02814_age_datediff.reference index 2b065211213..49165643657 100644 --- a/tests/queries/0_stateless/02814_age_datediff.reference +++ b/tests/queries/0_stateless/02814_age_datediff.reference @@ -125,4 +125,4 @@ SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970- SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); 10 SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); -19691231235959 \ No newline at end of file +19691231235959 diff --git a/tests/queries/0_stateless/02814_age_datediff.sql b/tests/queries/0_stateless/02814_age_datediff.sql index bc77f6de5d1..4d32af5d29d 100644 --- a/tests/queries/0_stateless/02814_age_datediff.sql +++ b/tests/queries/0_stateless/02814_age_datediff.sql @@ -76,4 +76,4 @@ SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021- SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); -SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); \ No newline at end of file +SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); From 59ecdbd53ec9f8abaf841d4cb0410e93d03bb32e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 22:30:04 +0200 Subject: [PATCH 0628/2047] Fix test 02497_storage_file_reader_selection --- .../0_stateless/02497_storage_file_reader_selection.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh index 8ea3adcbd2f..aa43e81f131 100755 --- a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh +++ b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh @@ -7,11 +7,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_FILE=test_02497_$CLICKHOUSE_TEST_UNIQUE_NAME.tsv echo -e 'key\nfoo\nbar' > $DATA_FILE -$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap:" -$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferOrdinary" +$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferMMap:" && echo 1 || echo 'Fail' +$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferOrdinary" && echo 'Fail' || echo 0 -$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap" -$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferOrdinary" +$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferMMap" && echo 'Fail' || echo 0 +$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferOrdinary" && echo 1 || echo 'Fail' $CLICKHOUSE_CLIENT --storage_file_read_method=mmap -nq "SELECT * FROM file('/dev/null', 'LineAsString') FORMAT Null -- { serverError BAD_ARGUMENTS }" From 72e8303ee7dd73ff0038cb5cbf28b63c98d1a183 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 22:49:06 +0200 Subject: [PATCH 0629/2047] Remove try/catch from DatabaseFilesystem --- .../Serializations/SerializationWrapper.h | 1 - src/Databases/DatabaseFilesystem.cpp | 23 ++++--------------- src/Databases/DatabaseFilesystem.h | 2 +- 3 files changed, 6 insertions(+), 20 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index bf922888af9..31900f93148 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -77,7 +77,6 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; - void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; }; diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 7eaf474eea0..60ae55fff11 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -128,17 +128,17 @@ bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr context_) if (tryGetTableFromCache(name)) return true; - return checkTableFilePath(getTablePath(name), context_, /* throw_on_error */false); + return checkTableFilePath(getTablePath(name), context_, /* throw_on_error */ false); } -StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_) const +StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_, bool throw_on_error) const { /// Check if table exists in loaded tables map. if (auto table = tryGetTableFromCache(name)) return table; auto table_path = getTablePath(name); - checkTableFilePath(table_path, context_, /* throw_on_error */true); + checkTableFilePath(table_path, context_, throw_on_error); /// If the file exists, create a new table using TableFunctionFile and return it. auto args = makeASTFunction("file", std::make_shared(table_path)); @@ -158,7 +158,7 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_) const { /// getTableImpl can throw exceptions, do not catch them to show correct error to user. - if (auto storage = getTableImpl(name, context_)) + if (auto storage = getTableImpl(name, context_, true)) return storage; throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", @@ -167,20 +167,7 @@ StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_ StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr context_) const { - try - { - return getTableImpl(name, context_); - } - catch (const Exception & e) - { - /// Ignore exceptions thrown by TableFunctionFile, which indicate that there is no table - /// see tests/02722_database_filesystem.sh for more details. - if (e.code() == ErrorCodes::FILE_DOESNT_EXIST) - { - return nullptr; - } - throw; - } + return getTableImpl(name, context_, false); } bool DatabaseFilesystem::empty() const diff --git a/src/Databases/DatabaseFilesystem.h b/src/Databases/DatabaseFilesystem.h index 7fe620401dc..b72891b9a5c 100644 --- a/src/Databases/DatabaseFilesystem.h +++ b/src/Databases/DatabaseFilesystem.h @@ -48,7 +48,7 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; protected: - StoragePtr getTableImpl(const String & name, ContextPtr context) const; + StoragePtr getTableImpl(const String & name, ContextPtr context, bool throw_on_error) const; StoragePtr tryGetTableFromCache(const std::string & name) const; From 1050fab8decfc01533527280b6f78f757c053bc8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 23:12:08 +0200 Subject: [PATCH 0630/2047] Remove another exception --- src/Databases/DatabaseFilesystem.cpp | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 60ae55fff11..a6365b0348c 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -11,9 +11,11 @@ #include #include #include +#include #include + namespace fs = std::filesystem; namespace DB @@ -138,10 +140,15 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont return table; auto table_path = getTablePath(name); - checkTableFilePath(table_path, context_, throw_on_error); + if (!checkTableFilePath(table_path, context_, throw_on_error)) + return {}; + + String format = FormatFactory::instance().getFormatFromFileName(table_path, throw_on_error); + if (format.empty()) + return {}; /// If the file exists, create a new table using TableFunctionFile and return it. - auto args = makeASTFunction("file", std::make_shared(table_path)); + auto args = makeASTFunction("file", std::make_shared(table_path), std::make_shared(format)); auto table_function = TableFunctionFactory::instance().get(args, context_); if (!table_function) From 422881ac2c27564fa0e7036c9a595b1f35237d45 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 23:21:30 +0200 Subject: [PATCH 0631/2047] Fix logical error: cannot get name of not a column: Set --- src/Interpreters/evaluateConstantExpression.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 5a333172b14..f43b8bd168f 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -61,6 +62,11 @@ std::pair> evaluateConstantExpression(co ast->setAlias("constant_expression"); } + if (ast->as() != nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Element of set in IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument " + "is not a constant expression (unexpected AST node type): {}", ast->getID()); + ReplaceQueryParameterVisitor param_visitor(context->getQueryParameters()); param_visitor.visit(ast); @@ -94,18 +100,18 @@ std::pair> evaluateConstantExpression(co if (!result_column) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Element of set in IN, VALUES or LIMIT or aggregate function parameter " + "Element of set in IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument " "is not a constant expression (result column not found): {}", result_name); if (result_column->empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty result column after evaluation " - "of constant expression for IN, VALUES or LIMIT or aggregate function parameter"); + "of constant expression for IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument"); /// Expressions like rand() or now() are not constant if (!isColumnConst(*result_column)) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Element of set in IN, VALUES or LIMIT or aggregate function parameter " + "Element of set in IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument " "is not a constant expression (result column is not const): {}", result_name); return std::make_pair((*result_column)[0], result_type); From c443c798b120b0cc6ab2a33c2c4cdb78c056508b Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Sun, 16 Jul 2023 17:22:01 -0400 Subject: [PATCH 0632/2047] upgrade qpl to v1.2.0 and libaccel to v4.0 --- contrib/idxd-config | 2 +- contrib/qpl | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/idxd-config b/contrib/idxd-config index f6605c41a73..a836ce0e420 160000 --- a/contrib/idxd-config +++ b/contrib/idxd-config @@ -1 +1 @@ -Subproject commit f6605c41a735e3fdfef2d2d18655a33af6490b99 +Subproject commit a836ce0e42052a69bffbbc14239ab4097f3b77f1 diff --git a/contrib/qpl b/contrib/qpl index 3f8f5cea277..faaf1935045 160000 --- a/contrib/qpl +++ b/contrib/qpl @@ -1 +1 @@ -Subproject commit 3f8f5cea27739f5261e8fd577dc233ffe88bf679 +Subproject commit faaf19350459c076e66bb5df11743c3fade59b73 From de879a6004a9304596a274d6175af8a56a8b14fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 23:22:48 +0200 Subject: [PATCH 0633/2047] Add a test --- .../02815_logical_error_cannot_get_column_name_of_set.reference | 0 .../02815_logical_error_cannot_get_column_name_of_set.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.reference create mode 100644 tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql diff --git a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.reference b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql new file mode 100644 index 00000000000..9d59f89a30a --- /dev/null +++ b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql @@ -0,0 +1 @@ +SELECT * FROM numbers(SETTINGS x = 1); -- { serverError BAD_ARGUMENTS } From 18bcbe400bdbc642995062ed5691d6ffe778127b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 23:26:21 +0200 Subject: [PATCH 0634/2047] Remove unused headers --- src/Interpreters/evaluateConstantExpression.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index f43b8bd168f..a2e17f5b834 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -15,14 +15,13 @@ #include #include #include -#include #include #include #include #include -#include #include + namespace DB { From e4384e104b75909f978a61cb5ccc413e220c5811 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 00:06:58 +0200 Subject: [PATCH 0635/2047] Add a test that clickhouse-client or local do not throw/catch on startup --- programs/main.cpp | 5 ++ src/Common/Exception.cpp | 10 ++++ src/Common/Exception.h | 10 ++++ ...02815_no_throw_in_simple_queries.reference | 5 ++ .../02815_no_throw_in_simple_queries.sh | 56 +++++++++++++++++++ 5 files changed, 86 insertions(+) create mode 100644 tests/queries/0_stateless/02815_no_throw_in_simple_queries.reference create mode 100755 tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh diff --git a/programs/main.cpp b/programs/main.cpp index 9a3ad47a86e..4af9e3a3067 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -466,6 +466,11 @@ int main(int argc_, char ** argv_) checkHarmfulEnvironmentVariables(argv_); #endif + /// This is used for testing. For example, + /// clickhouse-local should be able to run a simple query without throw/catch. + if (getenv("CLICKHOUSE_TERMINATE_ON_ANY_EXCEPTION")) // NOLINT(concurrency-mt-unsafe) + DB::terminate_on_any_exception = true; + /// Reset new handler to default (that throws std::bad_alloc) /// It is needed because LLVM library clobbers it. std::set_new_handler(nullptr); diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 054a60cb91d..ff62def67bd 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -50,6 +50,8 @@ void abortOnFailedAssertion(const String & description) abort(); } +bool terminate_on_any_exception = false; + /// - Aborts the process if error code is LOGICAL_ERROR. /// - Increments error codes statistics. void handle_error_code([[maybe_unused]] const std::string & msg, int code, bool remote, const Exception::FramePointers & trace) @@ -84,6 +86,8 @@ Exception::Exception(const MessageMasked & msg_masked, int code, bool remote_) : Poco::Exception(msg_masked.msg, code) , remote(remote_) { + if (terminate_on_any_exception) + std::terminate(); capture_thread_frame_pointers = thread_frame_pointers; handle_error_code(msg_masked.msg, code, remote, getStackFramePointers()); } @@ -92,6 +96,8 @@ Exception::Exception(MessageMasked && msg_masked, int code, bool remote_) : Poco::Exception(msg_masked.msg, code) , remote(remote_) { + if (terminate_on_any_exception) + std::terminate(); capture_thread_frame_pointers = thread_frame_pointers; handle_error_code(message(), code, remote, getStackFramePointers()); } @@ -99,6 +105,8 @@ Exception::Exception(MessageMasked && msg_masked, int code, bool remote_) Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc) : Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION) { + if (terminate_on_any_exception) + std::terminate(); capture_thread_frame_pointers = thread_frame_pointers; #ifdef STD_EXCEPTION_HAS_STACK_TRACE auto * stack_trace_frames = exc.get_stack_trace_frames(); @@ -111,6 +119,8 @@ Exception::Exception(CreateFromPocoTag, const Poco::Exception & exc) Exception::Exception(CreateFromSTDTag, const std::exception & exc) : Poco::Exception(demangle(typeid(exc).name()) + ": " + String(exc.what()), ErrorCodes::STD_EXCEPTION) { + if (terminate_on_any_exception) + std::terminate(); capture_thread_frame_pointers = thread_frame_pointers; #ifdef STD_EXCEPTION_HAS_STACK_TRACE auto * stack_trace_frames = exc.get_stack_trace_frames(); diff --git a/src/Common/Exception.h b/src/Common/Exception.h index f80dfe7f0a2..c09c80472da 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -20,6 +20,10 @@ namespace DB void abortOnFailedAssertion(const String & description); +/// This flag can be set for testing purposes - to check that no exceptions are thrown. +extern bool terminate_on_any_exception; + + class Exception : public Poco::Exception { public: @@ -27,17 +31,23 @@ public: Exception() { + if (terminate_on_any_exception) + std::terminate(); capture_thread_frame_pointers = thread_frame_pointers; } Exception(const PreformattedMessage & msg, int code): Exception(msg.text, code) { + if (terminate_on_any_exception) + std::terminate(); capture_thread_frame_pointers = thread_frame_pointers; message_format_string = msg.format_string; } Exception(PreformattedMessage && msg, int code): Exception(std::move(msg.text), code) { + if (terminate_on_any_exception) + std::terminate(); capture_thread_frame_pointers = thread_frame_pointers; message_format_string = msg.format_string; } diff --git a/tests/queries/0_stateless/02815_no_throw_in_simple_queries.reference b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.reference new file mode 100644 index 00000000000..1e7d6b54cce --- /dev/null +++ b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.reference @@ -0,0 +1,5 @@ +Aborted +1 +1 +1 +2 diff --git a/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh new file mode 100755 index 00000000000..cbebe33f295 --- /dev/null +++ b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +export CLICKHOUSE_TERMINATE_ON_ANY_EXCEPTION=1 + +# The environment variable works as expected: +bash -c " + abort_handler() + { + exit 0 + } + trap 'abort_handler' ABRT + $CLICKHOUSE_LOCAL --query 'this is wrong' +" 2>&1 | grep -o 'Aborted' + +# No exceptions are thrown in simple cases: +$CLICKHOUSE_LOCAL --query "SELECT 1" +$CLICKHOUSE_LOCAL --query "SHOW TABLES" +$CLICKHOUSE_LOCAL --query "SELECT * FROM system.tables FORMAT Null" + +# The same for the client app: +$CLICKHOUSE_CLIENT --query "SELECT 1" +$CLICKHOUSE_CLIENT --query "SHOW TABLES" +$CLICKHOUSE_CLIENT --query "SELECT * FROM system.tables FORMAT Null" + +# Multi queries are ok: +$CLICKHOUSE_LOCAL --multiquery "SELECT 1; SELECT 2;" + +# It can run in interactive mode: +function run() +{ + command=$1 + expect << EOF + +log_user 0 +set timeout 60 +match_max 100000 + +spawn bash -c "$command" + +expect ":) " + +send -- "SELECT 1\r" +expect "1" +expect ":) " + +send -- "exit\r" +expect eof + +EOF +} + +run "$CLICKHOUSE_LOCAL" From ac408d0b63b8ec13c8b80ca65f887b21cf29795f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 00:10:42 +0200 Subject: [PATCH 0636/2047] Add a check for hasToken --- src/Functions/HasTokenImpl.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Functions/HasTokenImpl.h b/src/Functions/HasTokenImpl.h index fdec5fcb0b7..ab6b6399486 100644 --- a/src/Functions/HasTokenImpl.h +++ b/src/Functions/HasTokenImpl.h @@ -39,6 +39,9 @@ struct HasTokenImpl if (start_pos != nullptr) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' does not support start_pos argument", name); + if (pattern.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle cannot be empty, because empty string isn't a token"); + if (haystack_offsets.empty()) return; From ace98464aef63545a23265fb71adc5147cfc0207 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 00:12:37 +0200 Subject: [PATCH 0637/2047] Add a test --- tests/queries/0_stateless/02816_has_token_empty.reference | 2 ++ tests/queries/0_stateless/02816_has_token_empty.sql | 7 +++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02816_has_token_empty.reference create mode 100644 tests/queries/0_stateless/02816_has_token_empty.sql diff --git a/tests/queries/0_stateless/02816_has_token_empty.reference b/tests/queries/0_stateless/02816_has_token_empty.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/02816_has_token_empty.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/02816_has_token_empty.sql b/tests/queries/0_stateless/02816_has_token_empty.sql new file mode 100644 index 00000000000..e5d6156debd --- /dev/null +++ b/tests/queries/0_stateless/02816_has_token_empty.sql @@ -0,0 +1,7 @@ +SELECT hasTokenCaseInsensitive('K(G', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasTokenCaseInsensitive('Hello', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasTokenCaseInsensitive('', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasTokenCaseInsensitive('', 'Hello'); +SELECT hasToken('Hello', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasToken('', 'Hello'); +SELECT hasToken('', ''); -- { serverError BAD_ARGUMENTS } From e6f71e3ad2bd0b6c33772606783a9e225402f986 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 00:30:53 +0200 Subject: [PATCH 0638/2047] Fix error in groupArrayMovingSum --- .../AggregateFunctionGroupArrayMoving.cpp | 196 ++++++++++++++++- .../AggregateFunctionGroupArrayMoving.h | 207 ------------------ ...up_array_moving_zero_window_size.reference | 0 ...17_group_array_moving_zero_window_size.sql | 2 + 4 files changed, 195 insertions(+), 210 deletions(-) delete mode 100644 src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h create mode 100644 tests/queries/0_stateless/02817_group_array_moving_zero_window_size.reference create mode 100644 tests/queries/0_stateless/02817_group_array_moving_zero_window_size.sql diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp index e2acccce516..026b8d1956f 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp @@ -1,10 +1,25 @@ +#include #include -#include #include #include #include -#include #include +#include +#include +#include + +#include +#include + +#include +#include + +#include +#include + +#include + +#define AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE 0xFFFFFF namespace DB @@ -13,11 +28,186 @@ struct Settings; namespace ErrorCodes { + extern const int TOO_LARGE_ARRAY_SIZE; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; } +template +struct MovingData +{ + /// For easy serialization. + static_assert(std::has_unique_object_representations_v || std::is_floating_point_v); + + using Accumulator = T; + + /// Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena + using Allocator = MixedAlignedArenaAllocator; + using Array = PODArray; + + Array value; /// Prefix sums. + T sum{}; + + void NO_SANITIZE_UNDEFINED add(T val, Arena * arena) + { + sum += val; + value.push_back(sum, arena); + } +}; + +template +struct MovingSumData : public MovingData +{ + static constexpr auto name = "groupArrayMovingSum"; + + T NO_SANITIZE_UNDEFINED get(size_t idx, UInt64 window_size) const + { + if (idx < window_size) + return this->value[idx]; + else + return this->value[idx] - this->value[idx - window_size]; + } +}; + +template +struct MovingAvgData : public MovingData +{ + static constexpr auto name = "groupArrayMovingAvg"; + + T NO_SANITIZE_UNDEFINED get(size_t idx, UInt64 window_size) const + { + if (idx < window_size) + return this->value[idx] / T(window_size); + else + return (this->value[idx] - this->value[idx - window_size]) / T(window_size); + } +}; + + +template +class MovingImpl final + : public IAggregateFunctionDataHelper> +{ + static constexpr bool limit_num_elems = LimitNumElements::value; + UInt64 window_size; + +public: + using ResultT = typename Data::Accumulator; + + using ColumnSource = ColumnVectorOrDecimal; + + /// Probably for overflow function in the future. + using ColumnResult = ColumnVectorOrDecimal; + + explicit MovingImpl(const DataTypePtr & data_type_, UInt64 window_size_ = std::numeric_limits::max()) + : IAggregateFunctionDataHelper>({data_type_}, {}, createResultType(data_type_)) + , window_size(window_size_) {} + + String getName() const override { return Data::name; } + + static DataTypePtr createResultType(const DataTypePtr & argument) + { + return std::make_shared(getReturnTypeElement(argument)); + } + + void NO_SANITIZE_UNDEFINED add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + auto value = static_cast(*columns[0]).getData()[row_num]; + this->data(place).add(static_cast(value), arena); + } + + void NO_SANITIZE_UNDEFINED merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override + { + auto & cur_elems = this->data(place); + auto & rhs_elems = this->data(rhs); + + size_t cur_size = cur_elems.value.size(); + + if (rhs_elems.value.size()) + cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.end(), arena); + + for (size_t i = cur_size; i < cur_elems.value.size(); ++i) + { + cur_elems.value[i] += cur_elems.sum; + } + + cur_elems.sum += rhs_elems.sum; + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + const auto & value = this->data(place).value; + size_t size = value.size(); + writeVarUInt(size, buf); + buf.write(reinterpret_cast(value.data()), size * sizeof(value[0])); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override + { + size_t size = 0; + readVarUInt(size, buf); + + if (unlikely(size > AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE)) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size (maximum: {})", AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE); + + if (size > 0) + { + auto & value = this->data(place).value; + value.resize(size, arena); + buf.readStrict(reinterpret_cast(value.data()), size * sizeof(value[0])); + this->data(place).sum = value.back(); + } + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + const auto & data = this->data(place); + size_t size = data.value.size(); + + ColumnArray & arr_to = assert_cast(to); + ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); + + offsets_to.push_back(offsets_to.back() + size); + + if (size) + { + typename ColumnResult::Container & data_to = assert_cast(arr_to.getData()).getData(); + + for (size_t i = 0; i < size; ++i) + { + if (!limit_num_elems) + { + data_to.push_back(data.get(i, size)); + } + else + { + data_to.push_back(data.get(i, window_size)); + } + } + } + } + + bool allocatesMemoryInArena() const override + { + return true; + } + +private: + static auto getReturnTypeElement(const DataTypePtr & argument) + { + if constexpr (!is_decimal) + return std::make_shared>(); + else + { + using Res = DataTypeDecimal; + return std::make_shared(Res::maxPrecision(), getDecimalScale(*argument)); + } + } +}; + + namespace { @@ -79,7 +269,7 @@ AggregateFunctionPtr createAggregateFunctionMoving( if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name); - if ((type == Field::Types::Int64 && parameters[0].get() < 0) || + if ((type == Field::Types::Int64 && parameters[0].get() <= 0) || (type == Field::Types::UInt64 && parameters[0].get() == 0)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter for aggregate function {} should be positive integer", name); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h deleted file mode 100644 index e6f79d7bca1..00000000000 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h +++ /dev/null @@ -1,207 +0,0 @@ -#pragma once - -#include -#include - -#include -#include -#include - -#include -#include - -#include -#include - -#include - -#include - -#define AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE 0xFFFFFF - - -namespace DB -{ -struct Settings; - -namespace ErrorCodes -{ - extern const int TOO_LARGE_ARRAY_SIZE; -} - -template -struct MovingData -{ - /// For easy serialization. - static_assert(std::has_unique_object_representations_v || std::is_floating_point_v); - - using Accumulator = T; - - /// Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena - using Allocator = MixedAlignedArenaAllocator; - using Array = PODArray; - - Array value; /// Prefix sums. - T sum{}; - - void NO_SANITIZE_UNDEFINED add(T val, Arena * arena) - { - sum += val; - value.push_back(sum, arena); - } -}; - -template -struct MovingSumData : public MovingData -{ - static constexpr auto name = "groupArrayMovingSum"; - - T NO_SANITIZE_UNDEFINED get(size_t idx, UInt64 window_size) const - { - if (idx < window_size) - return this->value[idx]; - else - return this->value[idx] - this->value[idx - window_size]; - } -}; - -template -struct MovingAvgData : public MovingData -{ - static constexpr auto name = "groupArrayMovingAvg"; - - T NO_SANITIZE_UNDEFINED get(size_t idx, UInt64 window_size) const - { - if (idx < window_size) - return this->value[idx] / T(window_size); - else - return (this->value[idx] - this->value[idx - window_size]) / T(window_size); - } -}; - - -template -class MovingImpl final - : public IAggregateFunctionDataHelper> -{ - static constexpr bool limit_num_elems = LimitNumElements::value; - UInt64 window_size; - -public: - using ResultT = typename Data::Accumulator; - - using ColumnSource = ColumnVectorOrDecimal; - - /// Probably for overflow function in the future. - using ColumnResult = ColumnVectorOrDecimal; - - explicit MovingImpl(const DataTypePtr & data_type_, UInt64 window_size_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper>({data_type_}, {}, createResultType(data_type_)) - , window_size(window_size_) {} - - String getName() const override { return Data::name; } - - static DataTypePtr createResultType(const DataTypePtr & argument) - { - return std::make_shared(getReturnTypeElement(argument)); - } - - void NO_SANITIZE_UNDEFINED add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override - { - auto value = static_cast(*columns[0]).getData()[row_num]; - this->data(place).add(static_cast(value), arena); - } - - void NO_SANITIZE_UNDEFINED merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override - { - auto & cur_elems = this->data(place); - auto & rhs_elems = this->data(rhs); - - size_t cur_size = cur_elems.value.size(); - - if (rhs_elems.value.size()) - cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.end(), arena); - - for (size_t i = cur_size; i < cur_elems.value.size(); ++i) - { - cur_elems.value[i] += cur_elems.sum; - } - - cur_elems.sum += rhs_elems.sum; - } - - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override - { - const auto & value = this->data(place).value; - size_t size = value.size(); - writeVarUInt(size, buf); - buf.write(reinterpret_cast(value.data()), size * sizeof(value[0])); - } - - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override - { - size_t size = 0; - readVarUInt(size, buf); - - if (unlikely(size > AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE)) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, - "Too large array size (maximum: {})", AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE); - - if (size > 0) - { - auto & value = this->data(place).value; - value.resize(size, arena); - buf.readStrict(reinterpret_cast(value.data()), size * sizeof(value[0])); - this->data(place).sum = value.back(); - } - } - - void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override - { - const auto & data = this->data(place); - size_t size = data.value.size(); - - ColumnArray & arr_to = assert_cast(to); - ColumnArray::Offsets & offsets_to = arr_to.getOffsets(); - - offsets_to.push_back(offsets_to.back() + size); - - if (size) - { - typename ColumnResult::Container & data_to = assert_cast(arr_to.getData()).getData(); - - for (size_t i = 0; i < size; ++i) - { - if (!limit_num_elems) - { - data_to.push_back(data.get(i, size)); - } - else - { - data_to.push_back(data.get(i, window_size)); - } - } - } - } - - bool allocatesMemoryInArena() const override - { - return true; - } - -private: - static auto getReturnTypeElement(const DataTypePtr & argument) - { - if constexpr (!is_decimal) - return std::make_shared>(); - else - { - using Res = DataTypeDecimal; - return std::make_shared(Res::maxPrecision(), getDecimalScale(*argument)); - } - } -}; - -#undef AGGREGATE_FUNCTION_MOVING_MAX_ARRAY_SIZE - -} diff --git a/tests/queries/0_stateless/02817_group_array_moving_zero_window_size.reference b/tests/queries/0_stateless/02817_group_array_moving_zero_window_size.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02817_group_array_moving_zero_window_size.sql b/tests/queries/0_stateless/02817_group_array_moving_zero_window_size.sql new file mode 100644 index 00000000000..fcbcaf1245b --- /dev/null +++ b/tests/queries/0_stateless/02817_group_array_moving_zero_window_size.sql @@ -0,0 +1,2 @@ +SELECT groupArrayMovingAvg ( toInt64 ( 0 ) ) ( toDecimal32 ( 1 , 1 ) ); -- { serverError BAD_ARGUMENTS } + From 34ec39234f875d8309055b196fd083fc0fe74586 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 01:40:02 +0200 Subject: [PATCH 0639/2047] Fix crash in comparison functions due to incorrect query analysis --- src/Functions/FunctionsComparison.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 66269f72866..cd40880ba54 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -1112,6 +1112,11 @@ private: bool c0_const = isColumnConst(*c0); bool c1_const = isColumnConst(*c1); + /// This is a paranoid check to protect from a broken query analysis. + if (c0->isNullable() != c1->isNullable()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Logical error: columns are assumed to be of identical types, but they are different in Nullable"); + if (c0_const && c1_const) { UInt8 res = 0; From 299149b68207294b5347ed1c2bcf82b68bfda9c8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 01:49:43 +0200 Subject: [PATCH 0640/2047] Fix slow test 02317_distinct_in_order_optimization --- .../02317_distinct_in_order_optimization.sql | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql index a794709caba..d05a25882a7 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization.sql @@ -18,7 +18,7 @@ select distinct a from distinct_in_order settings max_block_size=10, max_threads select '-- create table with not only primary key columns'; drop table if exists distinct_in_order sync; -create table distinct_in_order (a int, b int, c int) engine=MergeTree() order by (a, b); +create table distinct_in_order (a int, b int, c int) engine=MergeTree() order by (a, b) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into distinct_in_order select number % number, number % 5, number % 10 from numbers(1,1000000); select '-- distinct with primary key prefix only'; @@ -59,16 +59,16 @@ drop table if exists distinct_in_order sync; select '-- check that distinct in order returns the same result as ordinary distinct'; drop table if exists distinct_cardinality_low sync; -CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium); +CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; INSERT INTO distinct_cardinality_low SELECT number % 1e1, number % 1e2, number % 1e3 FROM numbers_mt(1e4); drop table if exists distinct_in_order sync; drop table if exists ordinary_distinct sync; select '-- check that distinct in order WITH order by returns the same result as ordinary distinct'; -create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into distinct_in_order select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=1; -create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into ordinary_distinct select distinct * from distinct_cardinality_low order by high settings optimize_distinct_in_order=0; select count() as diff from (select distinct * from distinct_in_order except select * from ordinary_distinct); @@ -76,9 +76,9 @@ drop table if exists distinct_in_order sync; drop table if exists ordinary_distinct sync; select '-- check that distinct in order WITHOUT order by returns the same result as ordinary distinct'; -create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into distinct_in_order select distinct * from distinct_cardinality_low settings optimize_distinct_in_order=1; -create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into ordinary_distinct select distinct * from distinct_cardinality_low settings optimize_distinct_in_order=0; select count() as diff from (select distinct * from distinct_in_order except select * from ordinary_distinct); @@ -86,9 +86,9 @@ drop table if exists distinct_in_order; drop table if exists ordinary_distinct; select '-- check that distinct in order WITHOUT order by and WITH filter returns the same result as ordinary distinct'; -create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table distinct_in_order (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into distinct_in_order select distinct * from distinct_cardinality_low where low > 0 settings optimize_distinct_in_order=1; -create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium); +create table ordinary_distinct (low UInt64, medium UInt64, high UInt64) engine=MergeTree() order by (low, medium) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into ordinary_distinct select distinct * from distinct_cardinality_low where low > 0 settings optimize_distinct_in_order=0; select count() as diff from (select distinct * from distinct_in_order except select * from ordinary_distinct); @@ -102,12 +102,12 @@ drop table if exists sorting_key_contain_function; select '-- bug 42185, distinct in order and empty sort description'; select '-- distinct in order, sorting key tuple()'; -create table sorting_key_empty_tuple (a int, b int) engine=MergeTree() order by tuple(); +create table sorting_key_empty_tuple (a int, b int) engine=MergeTree() order by tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into sorting_key_empty_tuple select number % 2, number % 5 from numbers(1,10); select distinct a from sorting_key_empty_tuple; select '-- distinct in order, sorting key contains function'; -create table sorting_key_contain_function (datetime DateTime, a int) engine=MergeTree() order by (toDate(datetime)); +create table sorting_key_contain_function (datetime DateTime, a int) engine=MergeTree() order by (toDate(datetime)) SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into sorting_key_contain_function values ('2000-01-01', 1); insert into sorting_key_contain_function values ('2000-01-01', 2); select distinct datetime from sorting_key_contain_function; From 4e727cdb2a870c6d0876ba0f251ccc161e0c9777 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 03:21:24 +0300 Subject: [PATCH 0641/2047] Update liftUpFunctions.cpp --- src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp index 47b4e31ed32..34a1fc2bb88 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp @@ -31,7 +31,7 @@ namespace DB::QueryPlanOptimizations /// This is a check that output columns does not have the same name /// This is ok for DAG, but may introduce a bug in a SotringStep cause columns are selected by name. -static bool areOutputsAreConvertableToBlock(const ActionsDAG::NodeRawConstPtrs & outputs) +static bool areOutputsConvertableToBlock(const ActionsDAG::NodeRawConstPtrs & outputs) { std::unordered_set names; for (const auto & output : outputs) @@ -72,7 +72,7 @@ size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan: if (unneeded_for_sorting->trivial()) return 0; - if (!areOutputsAreConvertableToBlock(needed_for_sorting->getOutputs())) + if (!areOutputsConvertableToBlock(needed_for_sorting->getOutputs())) return 0; // Sorting (parent_node) -> Expression (child_node) From d4459a5904aab19586b17f6d4675ed66dc78a614 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 04:16:02 +0200 Subject: [PATCH 0642/2047] Fix style --- tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh index cbebe33f295..a5c6de3ce58 100755 --- a/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh +++ b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh @@ -19,12 +19,12 @@ bash -c " # No exceptions are thrown in simple cases: $CLICKHOUSE_LOCAL --query "SELECT 1" $CLICKHOUSE_LOCAL --query "SHOW TABLES" -$CLICKHOUSE_LOCAL --query "SELECT * FROM system.tables FORMAT Null" +$CLICKHOUSE_LOCAL --query "SELECT * FROM system.tables WHERE database = currentDatabase() FORMAT Null" # The same for the client app: $CLICKHOUSE_CLIENT --query "SELECT 1" $CLICKHOUSE_CLIENT --query "SHOW TABLES" -$CLICKHOUSE_CLIENT --query "SELECT * FROM system.tables FORMAT Null" +$CLICKHOUSE_CLIENT --query "SELECT * FROM system.tables WHERE database = currentDatabase() FORMAT Null" # Multi queries are ok: $CLICKHOUSE_LOCAL --multiquery "SELECT 1; SELECT 2;" From e38a9de7c997f0f23456c978cdd991b9e579d6b3 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Mon, 17 Jul 2023 10:17:20 +0800 Subject: [PATCH 0643/2047] add aspell Signed-off-by: Lloyd-Pottiger --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2802e52c288..e6e8c273d25 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2539,3 +2539,4 @@ znode znodes zookeeperSessionUptime zstd +curdate From 58686d7850e32f7dc78a06297c4882685825f0ef Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Mon, 17 Jul 2023 10:23:39 +0800 Subject: [PATCH 0644/2047] add tests Signed-off-by: Lloyd-Pottiger --- .../queries/0_stateless/02813_func_now_and_alias.reference | 3 +++ tests/queries/0_stateless/02813_func_now_and_alias.sql | 3 +++ .../0_stateless/02813_func_today_and_alias.reference | 6 ++++++ tests/queries/0_stateless/02813_func_today_and_alias.sql | 6 ++++++ 4 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02813_func_now_and_alias.reference create mode 100644 tests/queries/0_stateless/02813_func_now_and_alias.sql create mode 100644 tests/queries/0_stateless/02813_func_today_and_alias.reference create mode 100644 tests/queries/0_stateless/02813_func_today_and_alias.sql diff --git a/tests/queries/0_stateless/02813_func_now_and_alias.reference b/tests/queries/0_stateless/02813_func_now_and_alias.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/02813_func_now_and_alias.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/02813_func_now_and_alias.sql b/tests/queries/0_stateless/02813_func_now_and_alias.sql new file mode 100644 index 00000000000..bcda1f26181 --- /dev/null +++ b/tests/queries/0_stateless/02813_func_now_and_alias.sql @@ -0,0 +1,3 @@ +SELECT now() = current_timestamp(); +SELECT now() = CURRENT_TIMESTAMP(); +SELECT now() = current_TIMESTAMP(); diff --git a/tests/queries/0_stateless/02813_func_today_and_alias.reference b/tests/queries/0_stateless/02813_func_today_and_alias.reference new file mode 100644 index 00000000000..a9e2f17562a --- /dev/null +++ b/tests/queries/0_stateless/02813_func_today_and_alias.reference @@ -0,0 +1,6 @@ +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02813_func_today_and_alias.sql b/tests/queries/0_stateless/02813_func_today_and_alias.sql new file mode 100644 index 00000000000..d379aa74f18 --- /dev/null +++ b/tests/queries/0_stateless/02813_func_today_and_alias.sql @@ -0,0 +1,6 @@ +SELECT today() = current_date(); +SELECT today() = CURRENT_DATE(); +SELECT today() = current_DATE(); +SELECT today() = curdate(); +SELECT today() = CURDATE(); +SELECT today() = curDATE(); From 7716479a37559afefc285191f780f1f424460826 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 17 Jul 2023 10:33:38 +0800 Subject: [PATCH 0645/2047] add comments for https://github.com/ClickHouse/ClickHouse/pull/52112 --- src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 899b84cc132..ad199abaf8f 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -233,6 +233,8 @@ namespace DB checkStatus(components_status, nested_column->getName(), format_name); /// Pass null null_map, because fillArrowArray will decide whether nested_type is nullable, if nullable, it will create a new null_map from nested_column + /// Note that it is only needed by gluten(https://github.com/oap-project/gluten), because array type in gluten is by default nullable. + /// And it does not influence the original ClickHouse logic, because null_map passed to fillArrowArrayWithArrayColumnData is always nullptr for ClickHouse doesn't allow nullable complex types including array type. fillArrowArray(column_name, nested_column, nested_type, nullptr, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } } From 1cebf03cabd591adfd5cd9df600fbaae42551b21 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 17 Jul 2023 10:41:18 +0800 Subject: [PATCH 0646/2047] add length alias --- src/Functions/array/length.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/array/length.cpp b/src/Functions/array/length.cpp index f09ad17892b..91a5e5fdec2 100644 --- a/src/Functions/array/length.cpp +++ b/src/Functions/array/length.cpp @@ -101,6 +101,7 @@ It is ok to have ASCII NUL bytes in strings, and they will be counted as well. .categories{"String", "Array"} }, FunctionFactory::CaseInsensitive); + factory.registerAlias("OCTET_LENGTH", "length", FunctionFactory::CaseInsensitive); } } From 8ea335aca7a102d5d06114799c2ada616b894ea8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 17 Jul 2023 10:43:13 +0800 Subject: [PATCH 0647/2047] update style --- src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index ad199abaf8f..f688efa3290 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -233,8 +233,8 @@ namespace DB checkStatus(components_status, nested_column->getName(), format_name); /// Pass null null_map, because fillArrowArray will decide whether nested_type is nullable, if nullable, it will create a new null_map from nested_column - /// Note that it is only needed by gluten(https://github.com/oap-project/gluten), because array type in gluten is by default nullable. - /// And it does not influence the original ClickHouse logic, because null_map passed to fillArrowArrayWithArrayColumnData is always nullptr for ClickHouse doesn't allow nullable complex types including array type. + /// Note that it is only needed by gluten(https://github.com/oap-project/gluten), because array type in gluten is by default nullable. + /// And it does not influence the original ClickHouse logic, because null_map passed to fillArrowArrayWithArrayColumnData is always nullptr for ClickHouse doesn't allow nullable complex types including array type. fillArrowArray(column_name, nested_column, nested_type, nullptr, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } } From 7915ba253bbc786d1752db4f1f3206cf08631501 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 04:49:04 +0200 Subject: [PATCH 0648/2047] Better version --- src/Interpreters/ActionsVisitor.cpp | 8 ++------ src/Interpreters/evaluateConstantExpression.cpp | 8 -------- src/Parsers/ASTSetQuery.cpp | 10 ++++++++++ src/Parsers/ASTSetQuery.h | 3 +++ ...815_logical_error_cannot_get_column_name_of_set.sql | 2 ++ 5 files changed, 17 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 01f2d4cf22e..efab11003f5 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -16,19 +16,15 @@ #include #include -#include #include #include #include #include -#include #include #include #include -#include #include -#include #include #include @@ -47,7 +43,6 @@ #include #include #include -#include #include #include #include @@ -61,6 +56,7 @@ #include #include + namespace DB { @@ -715,7 +711,7 @@ bool ActionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr & child) node->as()) return false; - /// Do not go to FROM, JOIN, UNION. + /// Do not go to FROM, JOIN, UNION if (child->as() || child->as()) return false; diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index a2e17f5b834..921cd5ff553 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -1,20 +1,17 @@ #include #include -#include #include #include #include #include #include -#include #include #include #include #include #include #include -#include #include #include #include @@ -61,11 +58,6 @@ std::pair> evaluateConstantExpression(co ast->setAlias("constant_expression"); } - if (ast->as() != nullptr) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Element of set in IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument " - "is not a constant expression (unexpected AST node type): {}", ast->getID()); - ReplaceQueryParameterVisitor param_visitor(context->getQueryParameters()); param_visitor.visit(ast); diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index 0b8d76dbb89..76ad812e713 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -64,4 +64,14 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, Forma } } +void ASTSetQuery::appendColumnName(WriteBuffer & ostr) const +{ + Hash hash = getTreeHash(); + + writeCString("__settings_", ostr); + writeText(hash.first, ostr); + ostr.write('_'); + writeText(hash.second, ostr); +} + } diff --git a/src/Parsers/ASTSetQuery.h b/src/Parsers/ASTSetQuery.h index 40abe2de31d..beed052c79a 100644 --- a/src/Parsers/ASTSetQuery.h +++ b/src/Parsers/ASTSetQuery.h @@ -37,6 +37,9 @@ public: void updateTreeHashImpl(SipHash & hash_state) const override; QueryKind getQueryKind() const override { return QueryKind::Set; } + + void appendColumnName(WriteBuffer & ostr) const override; + void appendColumnNameWithoutAlias(WriteBuffer & ostr) const override { return appendColumnName(ostr); } }; } diff --git a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql index 9d59f89a30a..98a8a8992b8 100644 --- a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql +++ b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql @@ -1 +1,3 @@ SELECT * FROM numbers(SETTINGS x = 1); -- { serverError BAD_ARGUMENTS } +SELECT * FROM numbers(numbers(SETTINGS x = 1)); -- { serverError UNKNOWN_FUNCTION } +SELECT * FROM numbers(numbers(SETTINGS x = 1), SETTINGS x = 1); -- { serverError UNKNOWN_FUNCTION } From e1abdd041924b6e69da3ad8bcbbb20857e1f68cd Mon Sep 17 00:00:00 2001 From: frinkr Date: Mon, 17 Jul 2023 10:51:24 +0800 Subject: [PATCH 0649/2047] atomic exchange simplified --- src/Coordination/KeeperDispatcher.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index e04060cf5cc..c50f72c12a7 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -400,8 +400,7 @@ void KeeperDispatcher::shutdown() try { { - bool expected = false; - if (!shutdown_called.compare_exchange_strong(expected, true)) + if (shutdown_called.exchange(true)) return; LOG_DEBUG(log, "Shutting down storage dispatcher"); From 0cdb1d1466367c355660ff4b217e41d605b39dcf Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 17 Jul 2023 10:52:05 +0800 Subject: [PATCH 0650/2047] add docs --- docs/en/sql-reference/functions/array-functions.md | 2 ++ docs/en/sql-reference/functions/string-functions.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 7f2b8f3c605..7aeb1af8f6b 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -102,6 +102,8 @@ The function also works for strings. Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operations/settings/settings.md#optimize-functions-to-subcolumns) setting. With `optimize_functions_to_subcolumns = 1` the function reads only [size0](../../sql-reference/data-types/array.md#array-size) subcolumn instead of reading and processing the whole array column. The query `SELECT length(arr) FROM table` transforms to `SELECT arr.size0 FROM TABLE`. +Alias: `OCTET_LENGTH` + ## emptyArrayUInt8, emptyArrayUInt16, emptyArrayUInt32, emptyArrayUInt64 ## emptyArrayInt8, emptyArrayInt16, emptyArrayInt32, emptyArrayInt64 diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 5175bbf0615..28dcf1c6b77 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -90,6 +90,8 @@ Returns the length of a string in bytes (not: in characters or Unicode code poin The function also works for arrays. +Alias: `OCTET_LENGTH` + ## lengthUTF8 Returns the length of a string in Unicode code points (not: in bytes or characters). It assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. From ebfc9c25f7e22bce3dc78ee5195799ea2c0ca3d8 Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 17 Jul 2023 10:59:40 +0800 Subject: [PATCH 0651/2047] add test --- tests/queries/0_stateless/02815_alias_to_length.reference | 2 ++ tests/queries/0_stateless/02815_alias_to_length.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02815_alias_to_length.reference create mode 100644 tests/queries/0_stateless/02815_alias_to_length.sql diff --git a/tests/queries/0_stateless/02815_alias_to_length.reference b/tests/queries/0_stateless/02815_alias_to_length.reference new file mode 100644 index 00000000000..7290ba859f4 --- /dev/null +++ b/tests/queries/0_stateless/02815_alias_to_length.reference @@ -0,0 +1,2 @@ +4 +4 diff --git a/tests/queries/0_stateless/02815_alias_to_length.sql b/tests/queries/0_stateless/02815_alias_to_length.sql new file mode 100644 index 00000000000..531bb92f8f0 --- /dev/null +++ b/tests/queries/0_stateless/02815_alias_to_length.sql @@ -0,0 +1,2 @@ +SELECT OCTET_LENGTH([1,2,3,4]); +SELECT OCTET_LENGTH('1234'); From 3df812f4f03fa8149a009cad3f6735d8d702627e Mon Sep 17 00:00:00 2001 From: FFFFFFFHHHHHHH <916677625@qq.com> Date: Mon, 17 Jul 2023 11:18:30 +0800 Subject: [PATCH 0652/2047] fix --- tests/queries/0_stateless/02815_alias_to_length.reference | 2 ++ tests/queries/0_stateless/02815_alias_to_length.sql | 6 +++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02815_alias_to_length.reference b/tests/queries/0_stateless/02815_alias_to_length.reference index 7290ba859f4..de958a364ef 100644 --- a/tests/queries/0_stateless/02815_alias_to_length.reference +++ b/tests/queries/0_stateless/02815_alias_to_length.reference @@ -1,2 +1,4 @@ 4 4 +15 +4 diff --git a/tests/queries/0_stateless/02815_alias_to_length.sql b/tests/queries/0_stateless/02815_alias_to_length.sql index 531bb92f8f0..780ac7dac6d 100644 --- a/tests/queries/0_stateless/02815_alias_to_length.sql +++ b/tests/queries/0_stateless/02815_alias_to_length.sql @@ -1,2 +1,6 @@ -SELECT OCTET_LENGTH([1,2,3,4]); SELECT OCTET_LENGTH('1234'); +SELECT OcTet_lenGtH('1234'); +SELECT OCTET_LENGTH('你好,世界'); + +-- This is a implementation-specific behavior of getting the length of an array. +SELECT OCTET_LENGTH([1,2,3,4]); From 0353eb1f4578c88bbea0b6051a9c0db86cd19cc4 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 05:05:33 +0000 Subject: [PATCH 0653/2047] add timezone to datediff() to fix other bug --- tests/queries/0_stateless/02814_age_datediff.reference | 6 ++++-- tests/queries/0_stateless/02814_age_datediff.sql | 5 +++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02814_age_datediff.reference b/tests/queries/0_stateless/02814_age_datediff.reference index 49165643657..cbcb8c8a7b6 100644 --- a/tests/queries/0_stateless/02814_age_datediff.reference +++ b/tests/queries/0_stateless/02814_age_datediff.reference @@ -1,3 +1,5 @@ +-- { echo } + -- DateTime64 vs DateTime64 with fractional part SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); 5100200 @@ -120,9 +122,9 @@ SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02 800 SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); 60200201 -SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); +SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6, 'UTC'), '1970-01-01 00:00:00.000010'::DateTime64(6, 'UTC')); 60 -SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); +SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6, 'UTC'), '1970-01-01 00:00:09.123'::DateTime64(6, 'UTC')); 10 SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); 19691231235959 diff --git a/tests/queries/0_stateless/02814_age_datediff.sql b/tests/queries/0_stateless/02814_age_datediff.sql index 4d32af5d29d..934a95c035f 100644 --- a/tests/queries/0_stateless/02814_age_datediff.sql +++ b/tests/queries/0_stateless/02814_age_datediff.sql @@ -1,3 +1,4 @@ +-- { echo } -- DateTime64 vs DateTime64 with fractional part SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); @@ -73,7 +74,7 @@ SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021- SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); -SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); -SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); +SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6, 'UTC'), '1970-01-01 00:00:00.000010'::DateTime64(6, 'UTC')); +SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6, 'UTC'), '1970-01-01 00:00:09.123'::DateTime64(6, 'UTC')); SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); From 708b980d693c13362b9222a0bfa74d1ab691a9f3 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 05:36:24 +0000 Subject: [PATCH 0654/2047] Fix arg name to fix build --- src/Interpreters/SystemLog.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index bf41ff12d2a..3f5ad8a5b6c 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -109,7 +109,7 @@ public: const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_, - std::shared_ptr> ex_queue = nullptr); + std::shared_ptr> queue_ = nullptr); void shutdown() override; From 0917cf05881f65faddb950daee60f67f53448129 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 17 Jul 2023 08:16:58 +0000 Subject: [PATCH 0655/2047] disable direct join for range dictionary --- src/Interpreters/JoinedTables.cpp | 5 +++ .../02815_range_dict_no_direct_join.reference | 6 ++++ .../02815_range_dict_no_direct_join.sql | 33 +++++++++++++++++++ 3 files changed, 44 insertions(+) create mode 100644 tests/queries/0_stateless/02815_range_dict_no_direct_join.reference create mode 100644 tests/queries/0_stateless/02815_range_dict_no_direct_join.sql diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index ee5c288afbb..29add31fd5d 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -337,6 +337,11 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se LOG_TRACE(&Poco::Logger::get("JoinedTables"), "Can't use dictionary join: dictionary '{}' was not found", dictionary_name); return nullptr; } + if (dictionary->getSpecialKeyType() == DictionarySpecialKeyType::Range) + { + LOG_TRACE(&Poco::Logger::get("JoinedTables"), "Can't use dictionary join: dictionary '{}' is a range dictionary", dictionary_name); + return nullptr; + } auto dictionary_kv = std::dynamic_pointer_cast(dictionary); table_join->setStorageJoin(dictionary_kv); diff --git a/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference b/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference new file mode 100644 index 00000000000..3576991f7a4 --- /dev/null +++ b/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference @@ -0,0 +1,6 @@ +1 0.1 +1 0.2 +2 0.3 +2 0.4 +3 0.5 +3 0.6 diff --git a/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql new file mode 100644 index 00000000000..249ffdf8089 --- /dev/null +++ b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql @@ -0,0 +1,33 @@ +CREATE TABLE discounts +( + advertiser_id UInt64, + discount_start_date Date, + discount_end_date Nullable(Date), + amount Float64 +) +ENGINE = Memory; + +INSERT INTO discounts VALUES (1, '2015-01-01', Null, 0.1); +INSERT INTO discounts VALUES (1, '2015-01-15', Null, 0.2); +INSERT INTO discounts VALUES (2, '2015-01-01', '2015-01-15', 0.3); +INSERT INTO discounts VALUES (2, '2015-01-04', '2015-01-10', 0.4); +INSERT INTO discounts VALUES (3, '1970-01-01', '2015-01-15', 0.5); +INSERT INTO discounts VALUES (3, '1970-01-01', '2015-01-10', 0.6); + +CREATE DICTIONARY discounts_dict +( + advertiser_id UInt64, + discount_start_date Date, + discount_end_date Nullable(Date), + amount Float64 +) +PRIMARY KEY advertiser_id +SOURCE(CLICKHOUSE(TABLE discounts)) +LIFETIME(MIN 600 MAX 900) +LAYOUT(RANGE_HASHED(RANGE_LOOKUP_STRATEGY 'max')) +RANGE(MIN discount_start_date MAX discount_end_date); + +CREATE TABLE ids (id UInt64) ENGINE = Memory; +INSERT INTO ids SELECT * FROM numbers(10); + +SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id SETTINGS join_algorithm = 'direct'; \ No newline at end of file From dfc1cc13c8bb58fbb224e5ed0651fcdb3c10c36d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 17 Jul 2023 08:54:55 +0000 Subject: [PATCH 0656/2047] Better oldest part selection --- tests/integration/test_multiple_disks/test.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 5561d63840b..bc7ac6683af 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -818,9 +818,10 @@ def test_start_stop_moves(start_cluster, name, engine): node1.query(f"SYSTEM STOP MOVES {name}") node1.query(f"SYSTEM STOP MERGES {name}") + first_part = None for i in range(5): data = [] # 5MB in total - for i in range(5): + for _ in range(5): data.append(get_random_string(1024 * 1024)) # 1MB row # jbod size is 40MB, so lets insert 5MB batch 7 times node1.query_with_retry( @@ -829,7 +830,13 @@ def test_start_stop_moves(start_cluster, name, engine): ) ) - first_part = get_oldest_part(node1, name) + # we cannot rely simply on modification time of part because it can be changed + # by different background operations so we explicitly check after the first + # part is inserted + if i == 0: + first_part = get_oldest_part(node1, name) + + assert first_part is not None used_disks = get_used_disks_for_table(node1, name) From 1e984da9e6bbebcb4a345247f006ae28d4ec605d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 11:26:58 +0200 Subject: [PATCH 0657/2047] Fix --- .../02240_filesystem_cache_bypass_cache_threshold.reference | 1 - .../02240_filesystem_cache_bypass_cache_threshold.sql | 3 --- .../queries/0_stateless/02240_filesystem_query_cache.reference | 1 - tests/queries/0_stateless/02240_filesystem_query_cache.sql | 3 +-- .../queries/0_stateless/02286_drop_filesystem_cache.reference | 2 ++ .../02503_cache_on_write_with_small_segment_size.sh | 1 - 6 files changed, 3 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference index 4828d9771b3..c29d736fa84 100644 --- a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference +++ b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference @@ -17,7 +17,6 @@ SETTINGS min_bytes_for_wide_part = 10485760, bypass_cache_threashold = 100, cache_on_write_operations = 1, enable_filesystem_query_cache_limit = 1, - do_not_evict_index_and_mark_files = 0, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); INSERT INTO test SELECT number, toString(number) FROM numbers(100); diff --git a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql index ae2cd1b8cd1..8871f8655dd 100644 --- a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql +++ b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.sql @@ -18,9 +18,6 @@ SETTINGS min_bytes_for_wide_part = 10485760, path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', enable_bypass_cache_with_threashold = 1, bypass_cache_threashold = 100, - cache_on_write_operations = 1, - enable_filesystem_query_cache_limit = 1, - do_not_evict_index_and_mark_files = 0, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); diff --git a/tests/queries/0_stateless/02240_filesystem_query_cache.reference b/tests/queries/0_stateless/02240_filesystem_query_cache.reference index 2e59aea73b9..26340c271e1 100644 --- a/tests/queries/0_stateless/02240_filesystem_query_cache.reference +++ b/tests/queries/0_stateless/02240_filesystem_query_cache.reference @@ -17,7 +17,6 @@ SETTINGS min_bytes_for_wide_part = 10485760, path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', cache_on_write_operations= 1, enable_filesystem_query_cache_limit = 1, - do_not_evict_index_and_mark_files = 0, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); SYSTEM DROP FILESYSTEM CACHE; diff --git a/tests/queries/0_stateless/02240_filesystem_query_cache.sql b/tests/queries/0_stateless/02240_filesystem_query_cache.sql index 02cf54b0caa..d85b3f543e1 100644 --- a/tests/queries/0_stateless/02240_filesystem_query_cache.sql +++ b/tests/queries/0_stateless/02240_filesystem_query_cache.sql @@ -8,7 +8,6 @@ SET skip_download_if_exceeds_query_cache=1; SET filesystem_cache_max_download_size=128; DROP TABLE IF EXISTS test; -SYSTEM DROP FILESYSTEM CACHE; CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key @@ -21,9 +20,9 @@ SETTINGS min_bytes_for_wide_part = 10485760, path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', cache_on_write_operations= 1, enable_filesystem_query_cache_limit = 1, - do_not_evict_index_and_mark_files = 0, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); +SYSTEM DROP FILESYSTEM CACHE; INSERT INTO test SELECT number, toString(number) FROM numbers(100); SELECT * FROM test FORMAT Null; SELECT file_segment_range_begin, file_segment_range_end, size FROM system.filesystem_cache ORDER BY file_segment_range_end, size; diff --git a/tests/queries/0_stateless/02286_drop_filesystem_cache.reference b/tests/queries/0_stateless/02286_drop_filesystem_cache.reference index 279e77e795a..b4e5b6715de 100644 --- a/tests/queries/0_stateless/02286_drop_filesystem_cache.reference +++ b/tests/queries/0_stateless/02286_drop_filesystem_cache.reference @@ -6,6 +6,7 @@ Using storage policy: s3_cache 1 1 1 +0 Using storage policy: local_cache 0 2 @@ -14,3 +15,4 @@ Using storage policy: local_cache 1 1 1 +0 diff --git a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh index 2c526d10cc9..229c68bf8ec 100755 --- a/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh +++ b/tests/queries/0_stateless/02503_cache_on_write_with_small_segment_size.sh @@ -25,7 +25,6 @@ SETTINGS min_bytes_for_wide_part = 0, path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', cache_on_write_operations = 1, enable_filesystem_query_cache_limit = 1, - do_not_evict_index_and_mark_files = 0, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); From 09d8ca0a40464d328f0708c915af09ffe8b1914e Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 11:30:21 +0200 Subject: [PATCH 0658/2047] Update --- tests/config/config.d/storage_conf.xml | 10 +--------- tests/queries/0_stateless/02801_backup_native_copy.sh | 2 +- 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index bd99ff06fb1..8533fef9fc9 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -1,18 +1,10 @@ - - s3 - s3_common_disk/ - http://localhost:11111/test/common/ - clickhouse - clickhouse - 20000 - s3 s3_disk/ - http://localhost:11111/test/00170_test/ + http://localhost:11111/test/common/ clickhouse clickhouse 20000 diff --git a/tests/queries/0_stateless/02801_backup_native_copy.sh b/tests/queries/0_stateless/02801_backup_native_copy.sh index 015dcb19b82..31a7cc3b410 100755 --- a/tests/queries/0_stateless/02801_backup_native_copy.sh +++ b/tests/queries/0_stateless/02801_backup_native_copy.sh @@ -10,7 +10,7 @@ set -e $CLICKHOUSE_CLIENT -nm -q " drop table if exists data; - create table data (key Int) engine=MergeTree() order by tuple() settings disk='s3_common_disk'; + create table data (key Int) engine=MergeTree() order by tuple() settings disk='s3_disk'; insert into data select * from numbers(10); " From 37f8bd070fb8a18c83f225b6bee3fc2d87398804 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 17 Jul 2023 09:55:12 +0000 Subject: [PATCH 0659/2047] Retry blob listing in test_alter_moving_garbage --- .../test_alter_moving_garbage/test.py | 38 ++++++++++++------- 1 file changed, 24 insertions(+), 14 deletions(-) diff --git a/tests/integration/test_alter_moving_garbage/test.py b/tests/integration/test_alter_moving_garbage/test.py index af9fffbb74d..21be46a7e1b 100644 --- a/tests/integration/test_alter_moving_garbage/test.py +++ b/tests/integration/test_alter_moving_garbage/test.py @@ -218,22 +218,32 @@ def test_delete_race_leftovers(cluster): time.sleep(5) # Check that we correctly deleted all outdated parts and no leftovers on s3 - known_remote_paths = set( - node.query( - f"SELECT remote_path FROM system.remote_data_paths WHERE disk_name = 's32'" - ).splitlines() - ) - - all_remote_paths = set( - obj.object_name - for obj in cluster.minio_client.list_objects( - cluster.minio_bucket, "data2/", recursive=True + # Do it with retries because we delete blobs in the background + # and it can be race condition between removing from remote_data_paths and deleting blobs + all_remote_paths = set() + known_remote_paths = set() + for i in range(3): + known_remote_paths = set( + node.query( + f"SELECT remote_path FROM system.remote_data_paths WHERE disk_name = 's32'" + ).splitlines() ) - ) - # Some blobs can be deleted after we listed remote_data_paths - # It's alright, thus we check only that all remote paths are known - # (in other words, all remote paths is subset of known paths) + all_remote_paths = set( + obj.object_name + for obj in cluster.minio_client.list_objects( + cluster.minio_bucket, "data2/", recursive=True + ) + ) + + # Some blobs can be deleted after we listed remote_data_paths + # It's alright, thus we check only that all remote paths are known + # (in other words, all remote paths is subset of known paths) + if all_remote_paths == {p for p in known_remote_paths if p in all_remote_paths}: + break + + time.sleep(1) + assert all_remote_paths == {p for p in known_remote_paths if p in all_remote_paths} # Check that we have all data From 41bdcdabfab16e6f93b4e0371563b149a1f0209b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 17 Jul 2023 12:02:32 +0200 Subject: [PATCH 0660/2047] Create ZK ancestors optimistically --- src/Common/ZooKeeper/ZooKeeper.cpp | 32 +++++++++++++++++---- src/Storages/StorageReplicatedMergeTree.cpp | 3 +- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 5dd7948276d..6f8c2105cad 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -3,8 +3,10 @@ #include "KeeperException.h" #include "TestKeeper.h" -#include #include +#include +#include +#include #include #include @@ -350,15 +352,33 @@ void ZooKeeper::createIfNotExists(const std::string & path, const std::string & void ZooKeeper::createAncestors(const std::string & path) { - size_t pos = 1; + std::string data = ""; + std::string path_created; // Ignored + std::vector pending_nodes; + + size_t last_pos = path.rfind('/'); + std::string current_node = path.substr(0, last_pos); + while (true) { - pos = path.find('/', pos); - if (pos == std::string::npos) + Coordination::Error code = createImpl(current_node, data, CreateMode::Persistent, path_created); + if (code == Coordination::Error::ZNONODE) + { + /// The parent node doesn't exist. Save the current node and try with the parent + last_pos = current_node.rfind('/'); + if (last_pos == std::string::npos || last_pos == 0) + throw KeeperException(code, path); + pending_nodes.emplace_back(std::move(current_node)); + current_node = path.substr(0, last_pos); + } + else if (code == Coordination::Error::ZOK || code == Coordination::Error::ZNODEEXISTS) break; - createIfNotExists(path.substr(0, pos), ""); - ++pos; + else + throw KeeperException(code, path); } + + for (const std::string & pending : pending_nodes | std::views::reverse) + createIfNotExists(pending, data); } void ZooKeeper::checkExistsAndGetCreateAncestorsOps(const std::string & path, Coordination::Requests & requests) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6894368841f..ed21e89a163 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9173,8 +9173,7 @@ std::optional StorageReplicatedMergeTree::tryCreateZeroCopyExclusi String zc_zookeeper_path = *getZeroCopyPartPath(part_name, disk); /// Just recursively create ancestors for lock - zookeeper->createAncestors(zc_zookeeper_path); - zookeeper->createIfNotExists(zc_zookeeper_path, ""); + zookeeper->createAncestors(zc_zookeeper_path + "/"); /// Create actual lock ZeroCopyLock lock(zookeeper, zc_zookeeper_path, replica_name); From 2bc7bb062cb5f7d99844f40f0735ff65575ab6d5 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 10:20:58 +0000 Subject: [PATCH 0661/2047] Add peak to http progress; not display peak if old server --- src/Common/ProgressIndication.cpp | 5 +++- src/Common/ProgressIndication.h | 6 ++-- .../WriteBufferFromHTTPServerResponse.cpp | 28 +++++++------------ .../HTTP/WriteBufferFromHTTPServerResponse.h | 7 ++--- src/Server/HTTPHandler.cpp | 6 ++-- 5 files changed, 23 insertions(+), 29 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index c9068720f8f..960d864660c 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -99,8 +99,11 @@ void ProgressIndication::writeFinalProgress() if (elapsed_ns) std::cout << " (" << formatReadableQuantity(progress.read_rows * 1000000000.0 / elapsed_ns) << " rows/s., " << formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)"; + else + std::cout << ". "; auto peak_memory_usage = getMemoryUsage().peak; - std::cout << ".\nPeak memory usage (for query) " << formatReadableSizeWithBinarySuffix(peak_memory_usage) << "."; + if (peak_memory_usage >= 0) + std::cout << "\nPeak memory usage (for query) " << formatReadableSizeWithBinarySuffix(peak_memory_usage) << "."; } void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index e3c73e8e7e5..a12c025614e 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -22,7 +22,9 @@ struct ThreadEventData UInt64 user_ms = 0; UInt64 system_ms = 0; UInt64 memory_usage = 0; - UInt64 peak_memory_usage = 0; + + // -1 used as flag 'is not show for old servers' + Int64 peak_memory_usage = -1; }; using HostToTimesMap = std::unordered_map; @@ -65,7 +67,7 @@ private: { UInt64 total = 0; UInt64 max = 0; - UInt64 peak = 0; + Int64 peak = -1; }; MemoryUsage getMemoryUsage() const; diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index a05efa7ec3f..9da24cfd9cb 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -29,7 +29,7 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders() } } -void WriteBufferFromHTTPServerResponse::writeHeaderSummary() +void WriteBufferFromHTTPServerResponse::writeHeaderProgressImpl(const char * header_name) { if (headers_finished_sending) return; @@ -43,19 +43,17 @@ void WriteBufferFromHTTPServerResponse::writeHeaderSummary() writeCString("\"}", progress_string_writer); if (response_header_ostr) - *response_header_ostr << "X-ClickHouse-Summary: " << progress_string_writer.str() << "\r\n" << std::flush; + *response_header_ostr << header_name << progress_string_writer.str() << "\r\n" << std::flush; +} + +void WriteBufferFromHTTPServerResponse::writeHeaderSummary() +{ + writeHeaderProgressImpl("X-ClickHouse-Summary: "); } void WriteBufferFromHTTPServerResponse::writeHeaderProgress() { - if (headers_finished_sending) - return; - - WriteBufferFromOwnString progress_string_writer; - accumulated_progress.writeJSON(progress_string_writer); - - if (response_header_ostr) - *response_header_ostr << "X-ClickHouse-Progress: " << progress_string_writer.str() << "\r\n" << std::flush; + writeHeaderProgressImpl("X-ClickHouse-Progress: "); } void WriteBufferFromHTTPServerResponse::writeExceptionCode() @@ -154,7 +152,7 @@ WriteBufferFromHTTPServerResponse::WriteBufferFromHTTPServerResponse( } -void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) +void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress, Int64 peak_memory_usage_) { std::lock_guard lock(mutex); @@ -163,7 +161,7 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) return; accumulated_progress.incrementPiecewiseAtomically(progress); - + peak_memory_usage = peak_memory_usage_; if (send_progress && progress_watch.elapsed() >= send_progress_interval_ms * 1000000) { progress_watch.restart(); @@ -174,12 +172,6 @@ void WriteBufferFromHTTPServerResponse::onProgress(const Progress & progress) } } -void WriteBufferFromHTTPServerResponse::onMemoryUsage(Int64 usage) -{ - std::lock_guard lock(mutex); - peak_memory_usage = usage; -} - WriteBufferFromHTTPServerResponse::~WriteBufferFromHTTPServerResponse() { finalize(); diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h index f849a685aec..b4c66357d3b 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h @@ -43,10 +43,7 @@ public: ~WriteBufferFromHTTPServerResponse() override; /// Writes progress in repeating HTTP headers. - void onProgress(const Progress & progress); - - void onMemoryUsage(Int64 peak_memory_usage); - + void onProgress(const Progress & progress, Int64 peak_memory_usage_); /// Turn compression on or off. /// The setting has any effect only if HTTP headers haven't been sent yet. @@ -92,6 +89,8 @@ private: /// but not finish them with \r\n, allowing to send more headers subsequently. void startSendHeaders(); + // Used for write the header X-ClickHouse-Progress / X-ClickHouse-Summary + void writeHeaderProgressImpl(const char * header_name); // Used for write the header X-ClickHouse-Progress void writeHeaderProgress(); // Used for write the header X-ClickHouse-Summary diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 31cbbf9ee00..c3c7f8fef3f 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -820,10 +820,8 @@ void HTTPHandler::processQuery( /// Note that we add it unconditionally so the progress is available for `X-ClickHouse-Summary` append_callback([&used_output](const Progress & progress) { - used_output.out->onProgress(progress); - auto thread_group = CurrentThread::getGroup(); - auto peak_memory_usage = thread_group->memory_tracker.getPeak(); - used_output.out->onMemoryUsage(peak_memory_usage); + const auto& thread_group = CurrentThread::getGroup(); + used_output.out->onProgress(progress, thread_group->memory_tracker.getPeak()); }); if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) From 1d4d829d6569dcae5d3dbfe840e18e0e9cb413e9 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 10:36:23 +0000 Subject: [PATCH 0662/2047] Remove spaces --- src/Common/ProgressIndication.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index a12c025614e..3776b1d8ae1 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -22,7 +22,7 @@ struct ThreadEventData UInt64 user_ms = 0; UInt64 system_ms = 0; UInt64 memory_usage = 0; - + // -1 used as flag 'is not show for old servers' Int64 peak_memory_usage = -1; }; From 9677cc5e2bee02991cfc0fb9bb5945b69fc7abd5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 17 Jul 2023 12:50:02 +0200 Subject: [PATCH 0663/2047] Apply suggestions from code review --- src/Coordination/KeeperDispatcher.h | 2 +- src/Coordination/KeeperReconfiguration.cpp | 4 ++-- src/Coordination/KeeperServer.cpp | 2 +- src/Coordination/RaftServerConfig.cpp | 4 +++- 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 40f1dac1570..331e3d2e73e 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -135,7 +135,7 @@ public: bool isServerActive() const; void updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion::Version & macros); - void pushClusterUpdates(ClusterUpdateActions&& actions); + void pushClusterUpdates(ClusterUpdateActions && actions); bool reconfigEnabled() const; /// Shutdown internal keeper parts (server, state machine, log storage, etc) diff --git a/src/Coordination/KeeperReconfiguration.cpp b/src/Coordination/KeeperReconfiguration.cpp index f262a07209d..e3642913a7a 100644 --- a/src/Coordination/KeeperReconfiguration.cpp +++ b/src/Coordination/KeeperReconfiguration.cpp @@ -39,7 +39,7 @@ ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std:: if (leaving_arr.size() >= cfg->get_servers().size()) return {}; - std::unordered_set remove_ids; + std::unordered_set remove_ids; ClusterUpdateActions out; for (std::string_view leaving_server : leaving_arr) @@ -65,7 +65,7 @@ ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std:: String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateActions & updates) { RaftServers new_config; - std::unordered_set remove_update_ids; + std::unordered_set remove_update_ids; for (const auto & update : updates) { diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index fd82f220f9b..a4c3d91e1c9 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -789,7 +789,7 @@ std::vector KeeperServer::getDeadSessions() return state_machine->getDeadSessions(); } -bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction& action) +bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction & action) { std::lock_guard _{server_write_mutex}; diff --git a/src/Coordination/RaftServerConfig.cpp b/src/Coordination/RaftServerConfig.cpp index 9090ed68fb6..929eeeb640e 100644 --- a/src/Coordination/RaftServerConfig.cpp +++ b/src/Coordination/RaftServerConfig.cpp @@ -69,11 +69,12 @@ std::optional RaftServerConfig::parse(std::string_view server) RaftServers parseRaftServers(std::string_view servers) { std::vector server_arr; - std::unordered_set ids; + std::unordered_set ids; std::unordered_set endpoints; RaftServers out; for (auto & server : splitInto<','>(server_arr, servers)) + { if (auto maybe_server = RaftServerConfig::parse(server)) { String endpoint = maybe_server->endpoint; @@ -89,6 +90,7 @@ RaftServers parseRaftServers(std::string_view servers) } else return {}; + } return out; } From 477b8f04b61372e5f79d60f1efb08eb42d6508a4 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 13:09:34 +0200 Subject: [PATCH 0664/2047] Fix sticky mutations test (and extremely rare race condition) --- src/Storages/StorageMergeTree.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 074f01e7d03..8cc2fbc2911 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -599,7 +599,14 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer /// Validate partition IDs (if any) before starting mutation getPartitionIdsAffectedByCommands(commands, query_context); - Int64 version = startMutation(commands, query_context); + Int64 version; + { + /// It's important to serialize order of mutations with alter queries because + /// they can depend on each other. + auto alter_lock = lockForAlter(query_context->getSettings().lock_acquire_timeout); + version = startMutation(commands, query_context); + } + if (query_context->getSettingsRef().mutations_sync > 0 || query_context->getCurrentTransaction()) waitForMutation(version, false); } From c374653c7bfc7450590d6a787bd4f413d15a76af Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 12:01:47 +0200 Subject: [PATCH 0665/2047] Revert "Merge pull request #52138 from azat/decompressor-inode" This reverts commit 6524031348c3a1148e27ef926baabb35e833a09c, reversing changes made to 9bf114f9a36556d6f227dea0fa91be131ee99710. This was not a good idea, since the underlying problem was that `/proc/self/exe` was pointing to `qemu-$ARCH-static` (because the code uses realpath() over normal interface readlink(), which is not caught by the qemu linux-user). And this means that later, it will try to overwrite incorrect binary and then execute some garbage. Signed-off-by: Azat Khuzhin --- .../decompressor.cpp | 93 +++++++++---------- 1 file changed, 46 insertions(+), 47 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 4a4985120fd..d41b9b1ebe1 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -430,58 +430,55 @@ int main(int/* argc*/, char* argv[]) return 1; } - int lock = -1; - /// Protection from double decompression #if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// get inode of this executable uint64_t inode = getInode(self); - /// In some cases /proc/self/maps may not contain the inode for the - /// /proc/self/exe, one of such examples are using qemu-*-static, in this - /// case maps will be proxied through the qemu, and it will remove - /// information about itself from it. - if (inode != 0) + if (inode == 0) { - std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; - lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); - if (lock < 0) + std::cerr << "Unable to obtain inode." << std::endl; + return 1; + } + + std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; + int lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); + if (lock < 0) + { + perror("lock open"); + return 1; + } + + /// lock file should be closed on exec call + fcntl(lock, F_SETFD, FD_CLOEXEC); + + if (lockf(lock, F_LOCK, 0)) + { + perror("lockf"); + return 1; + } + + /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to + /// 32bit conversion of input_info.st_ino + if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) + input_info.st_ino &= 0x00000000FFFFFFFF; + + /// if decompression was performed by another process since this copy was started + /// then file referred by path "self" is already pointing to different inode + if (input_info.st_ino != inode) + { + struct stat lock_info; + if (0 != fstat(lock, &lock_info)) { - perror("lock open"); + perror("fstat lock"); return 1; } - /// lock file should be closed on exec call - fcntl(lock, F_SETFD, FD_CLOEXEC); + /// size 1 of lock file indicates that another decompressor has found active executable + if (lock_info.st_size == 1) + execv(self, argv); - if (lockf(lock, F_LOCK, 0)) - { - perror("lockf"); - return 1; - } - - /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to - /// 32bit conversion of input_info.st_ino - if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) - input_info.st_ino &= 0x00000000FFFFFFFF; - - /// if decompression was performed by another process since this copy was started - /// then file referred by path "self" is already pointing to different inode - if (input_info.st_ino != inode) - { - struct stat lock_info; - if (0 != fstat(lock, &lock_info)) - { - perror("fstat lock"); - return 1; - } - - /// size 1 of lock file indicates that another decompressor has found active executable - if (lock_info.st_size == 1) - execv(self, argv); - - printf("No target executable - decompression only was performed.\n"); - return 0; - } + printf("No target executable - decompression only was performed.\n"); + return 0; } #endif @@ -549,19 +546,21 @@ int main(int/* argc*/, char* argv[]) if (has_exec) { +#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// write one byte to the lock in case other copies of compressed are running to indicate that /// execution should be performed - if (lock >= 0) - write(lock, "1", 1); + write(lock, "1", 1); +#endif execv(self, argv); /// This part of code will be reached only if error happened perror("execv"); return 1; } +#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// since inodes can be reused - it's a precaution if lock file already exists and have size of 1 - if (lock >= 0) - ftruncate(lock, 0); + ftruncate(lock, 0); +#endif printf("No target executable - decompression only was performed.\n"); } From 16165d9498cbebd3ecd02df87480230fd0ed880e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 12:02:54 +0200 Subject: [PATCH 0666/2047] Improve error messages for decompressor Signed-off-by: Azat Khuzhin --- utils/self-extracting-executable/decompressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index d41b9b1ebe1..567d9088f13 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -435,7 +435,7 @@ int main(int/* argc*/, char* argv[]) uint64_t inode = getInode(self); if (inode == 0) { - std::cerr << "Unable to obtain inode." << std::endl; + std::cerr << "Unable to obtain inode for exe '" << self << "'." << std::endl; return 1; } From 1fb7605fb4225225d492cf63f7f048d594e89fc3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 12:09:11 +0200 Subject: [PATCH 0667/2047] Fix self extracting binaries under qemu linux-user (qemu-$ARCH-static) The problem was that the decompressor uses realpath(/proc/self/exe) instead of readlink(/proc/self/exe), while realpath() does lots of trickerly [1] which leads to bypassing qemu linux-user override [2] of /proc/self/exe to the executable with with it had been called -- and the reason for this is that the getpid() after unshare returns 1, while reading /proc/self returns the pid that was before unshare (from the chroot) [3]. [1]: https://github.com/bminor/glibc/blob/4290aed05135ae4c0272006442d147f2155e70d7/stdlib/canonicalize.c#L223 [2]: https://github.com/qemu/qemu/blob/ed8ad9728a9c0eec34db9dff61dfa2f1dd625637/linux-user/syscall.c#L8634 [3]: https://gist.github.com/azat/fcbd8b6c26afd505ae5f3387fc15f0e2 But note, that even after this patch qemu without binfmt will not work, due to internally the code calls execv() while qemu does not handle it (see [4]). [4]: https://patchwork.kernel.org/project/qemu-devel/patch/1453091602-21843-1-git-send-email-petrosagg@gmail.com/ Signed-off-by: Azat Khuzhin --- utils/self-extracting-executable/decompressor.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 567d9088f13..91f4bea5a5b 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -362,11 +362,12 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress #else - int read_exe_path(char *exe, size_t/* buf_sz*/) + int read_exe_path(char *exe, size_t buf_sz) { - if (realpath("/proc/self/exe", exe) == nullptr) - return 1; - return 0; + ssize_t n = readlink("/proc/self/exe", exe, buf_sz - 1); + if (n > 0) + exe[n] = '\0'; + return n > 0 && n < static_cast(buf_sz); } #endif From 5b853c19580def53df4add0ca6242d5912a7bfe3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 17 Jul 2023 11:32:22 +0000 Subject: [PATCH 0668/2047] Make ASTFuzzer happy --- src/Parsers/ASTCreateIndexQuery.cpp | 3 +-- src/Parsers/ASTIndexDeclaration.cpp | 9 +++++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ASTCreateIndexQuery.cpp b/src/Parsers/ASTCreateIndexQuery.cpp index 50470fbc1e4..0d580d5bb21 100644 --- a/src/Parsers/ASTCreateIndexQuery.cpp +++ b/src/Parsers/ASTCreateIndexQuery.cpp @@ -56,8 +56,7 @@ void ASTCreateIndexQuery::formatQueryImpl(const FormatSettings & settings, Forma formatOnCluster(settings); - if (!cluster.empty()) - settings.ostr << " "; + settings.ostr << " "; index_decl->formatImpl(settings, state, frame); } diff --git a/src/Parsers/ASTIndexDeclaration.cpp b/src/Parsers/ASTIndexDeclaration.cpp index e647f02f553..12d59681cc3 100644 --- a/src/Parsers/ASTIndexDeclaration.cpp +++ b/src/Parsers/ASTIndexDeclaration.cpp @@ -29,9 +29,14 @@ void ASTIndexDeclaration::formatImpl(const FormatSettings & s, FormatState & sta { if (part_of_create_index_query) { - s.ostr << "("; + if (expr->as()) + { + s.ostr << "("; + expr->formatImpl(s, state, frame); + s.ostr << ")"; + } + else expr->formatImpl(s, state, frame); - s.ostr << ")"; } else { From 10c15a6548a27bbb51eb04082a11fd110a4642a5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 10:33:09 +0000 Subject: [PATCH 0669/2047] Refactor limit_jobs.cmake --- cmake/limit_jobs.cmake | 53 ++++++++++++++++++++---------------------- 1 file changed, 25 insertions(+), 28 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 3a33b3b9989..b2a54fb8d1a 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -1,16 +1,14 @@ -# Usage: -# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # In megabytes -# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") +# Usage from CMake: +# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # megabyte +# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte # include (cmake/limit_jobs.cmake) cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) # Not available under freebsd cmake_host_system_information(RESULT NUMBER_OF_LOGICAL_CORES QUERY NUMBER_OF_LOGICAL_CORES) -# 1 if not set -option(PARALLEL_COMPILE_JOBS "Maximum number of concurrent compilation jobs" "") - -# 1 if not set -option(PARALLEL_LINK_JOBS "Maximum number of concurrent link jobs" "") +# Set to disable the automatic job-limiting +option(PARALLEL_COMPILE_JOBS "Maximum number of concurrent compilation jobs" OFF) +option(PARALLEL_LINK_JOBS "Maximum number of concurrent link jobs" OFF) if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) math(EXPR PARALLEL_COMPILE_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_COMPILER_MEMORY}) @@ -19,17 +17,10 @@ if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_COMPILE_JOBS_LESS TRUE) + set (PARALLEL_COMPILE_JOBS_UNDERUTILIZING TRUE) endif() endif () -if (PARALLEL_COMPILE_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) - set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) - string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) - set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) -endif () - - if (NOT PARALLEL_LINK_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY) math(EXPR PARALLEL_LINK_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_LINKER_MEMORY}) @@ -37,7 +28,7 @@ if (NOT PARALLEL_LINK_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_LINK_JOBS_LESS TRUE) + set (PARALLEL_LINK_JOBS_UNDERUTILIZING TRUE) endif() endif () @@ -52,20 +43,26 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLE set (PARALLEL_LINK_JOBS 2) endif() +message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") + +if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) + if (PARALLEL_COMPILE_JOBS_UNDERUTILIZING) + message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") + endif() + if (PARALLEL_LINK_JOBS_UNDERUTILIZING) + message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") + endif() +endif () + +if (PARALLEL_COMPILE_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) + set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) + string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) + set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) +endif () + if (PARALLEL_LINK_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS}) endif () -if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) - message(STATUS - "${CMAKE_CURRENT_SOURCE_DIR}: Have ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. - Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS} (system has ${NUMBER_OF_LOGICAL_CORES} logical cores)") - if (PARALLEL_COMPILE_JOBS_LESS) - message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") - endif() - if (PARALLEL_LINK_JOBS_LESS) - message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") - endif() -endif () From a110c879852792739182ea9bb1fe55ff193d457f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 17 Jul 2023 13:46:04 +0200 Subject: [PATCH 0670/2047] Update 02240_filesystem_cache_bypass_cache_threshold.reference --- .../02240_filesystem_cache_bypass_cache_threshold.reference | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference index c29d736fa84..eb6c9305ebd 100644 --- a/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference +++ b/tests/queries/0_stateless/02240_filesystem_cache_bypass_cache_threshold.reference @@ -15,8 +15,6 @@ SETTINGS min_bytes_for_wide_part = 10485760, path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache', enable_bypass_cache_with_threashold = 1, bypass_cache_threashold = 100, - cache_on_write_operations = 1, - enable_filesystem_query_cache_limit = 1, delayed_cleanup_interval_ms = 100, disk = 's3_disk'); INSERT INTO test SELECT number, toString(number) FROM numbers(100); From 7204f7054788302e37be3e79ddd3a0ff87b93f01 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 11:52:00 +0000 Subject: [PATCH 0671/2047] cmake_host_system_information provides TOTAL_PHYSICAL_MEMORY on all platforms, including FreeBSD --- cmake/limit_jobs.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index b2a54fb8d1a..2ca9e5519d7 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -3,14 +3,14 @@ # set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte # include (cmake/limit_jobs.cmake) -cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) # Not available under freebsd +cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) cmake_host_system_information(RESULT NUMBER_OF_LOGICAL_CORES QUERY NUMBER_OF_LOGICAL_CORES) # Set to disable the automatic job-limiting option(PARALLEL_COMPILE_JOBS "Maximum number of concurrent compilation jobs" OFF) option(PARALLEL_LINK_JOBS "Maximum number of concurrent link jobs" OFF) -if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) +if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) math(EXPR PARALLEL_COMPILE_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_COMPILER_MEMORY}) if (NOT PARALLEL_COMPILE_JOBS) @@ -21,7 +21,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) endif() endif () -if (NOT PARALLEL_LINK_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY) +if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) math(EXPR PARALLEL_LINK_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_LINKER_MEMORY}) if (NOT PARALLEL_LINK_JOBS) From 8f1e56167c430783841abca00dd16c217cfcec13 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 11:53:12 +0000 Subject: [PATCH 0672/2047] NUMBER_OF_TOTAL_CORES is on all platforms != 0 --- cmake/limit_jobs.cmake | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 2ca9e5519d7..b145c069e37 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -16,7 +16,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) if (NOT PARALLEL_COMPILE_JOBS) set (PARALLEL_COMPILE_JOBS 1) endif () - if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) + if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set (PARALLEL_COMPILE_JOBS_UNDERUTILIZING TRUE) endif() endif () @@ -27,7 +27,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) if (NOT PARALLEL_LINK_JOBS) set (PARALLEL_LINK_JOBS 1) endif () - if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) + if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) set (PARALLEL_LINK_JOBS_UNDERUTILIZING TRUE) endif() endif () @@ -54,13 +54,13 @@ if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) endif() endif () -if (PARALLEL_COMPILE_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_COMPILE_JOBS AND (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) endif () -if (PARALLEL_LINK_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_LINK_JOBS AND (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS}) From 496aaf58c8380efcb8ad7f46f470831269b6a95f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 17 Jul 2023 13:55:14 +0200 Subject: [PATCH 0673/2047] Make clang-tidy happy --- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 6f8c2105cad..e3e89969c17 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -352,7 +352,7 @@ void ZooKeeper::createIfNotExists(const std::string & path, const std::string & void ZooKeeper::createAncestors(const std::string & path) { - std::string data = ""; + std::string data; std::string path_created; // Ignored std::vector pending_nodes; From 96c399ff7b623cbebcf7a8e1ae036df5b0f57767 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 11:58:18 +0000 Subject: [PATCH 0674/2047] Simplify a bit further --- cmake/limit_jobs.cmake | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index b145c069e37..b7cc5cf0967 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -17,7 +17,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_COMPILE_JOBS_UNDERUTILIZING TRUE) + message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") endif() endif () @@ -28,7 +28,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_LINK_JOBS_UNDERUTILIZING TRUE) + message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") endif() endif () @@ -45,15 +45,6 @@ endif() message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") -if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) - if (PARALLEL_COMPILE_JOBS_UNDERUTILIZING) - message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") - endif() - if (PARALLEL_LINK_JOBS_UNDERUTILIZING) - message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") - endif() -endif () - if (PARALLEL_COMPILE_JOBS AND (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) From 152f5527a906d048df2e727e3e592735f8844508 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 17 Jul 2023 11:58:22 +0000 Subject: [PATCH 0675/2047] Remove wrong assert --- tests/integration/ci-runner.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 9629a5821b5..31f4a7666c8 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -810,7 +810,6 @@ class ClickhouseIntegrationTestsRunner: result_state = "failure" if not should_fail: break - assert should_fail logging.info("Try is OK, all tests passed, going to clear env") clear_ip_tables_and_restart_daemons() logging.info("And going to sleep for some time") From 2993cf3cba6d9d4653cf4981e7fb4235a3a8acfa Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 14:01:31 +0200 Subject: [PATCH 0676/2047] Fix flaky test test_lost_part --- tests/integration/test_lost_part/test.py | 410 ++++++++++++----------- 1 file changed, 207 insertions(+), 203 deletions(-) diff --git a/tests/integration/test_lost_part/test.py b/tests/integration/test_lost_part/test.py index 0bc24268040..b83707145f4 100644 --- a/tests/integration/test_lost_part/test.py +++ b/tests/integration/test_lost_part/test.py @@ -39,257 +39,261 @@ def test_lost_part_same_replica(start_cluster): node1.query("DROP TABLE IF EXISTS mt0 SYNC") node2.query("DROP TABLE IF EXISTS mt0 SYNC") - for node in [node1, node2]: - node.query( - f"CREATE TABLE mt0 (id UInt64, date Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/t', '{node.name}') ORDER BY tuple() PARTITION BY date " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," - "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" - ) + try: + for node in [node1, node2]: + node.query( + f"CREATE TABLE mt0 (id UInt64, date Date) ENGINE ReplicatedMergeTree('/clickhouse/tables/t', '{node.name}') ORDER BY tuple() PARTITION BY date " + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + ) - node1.query("SYSTEM STOP MERGES mt0") - node2.query("SYSTEM STOP REPLICATION QUEUES") + node1.query("SYSTEM STOP MERGES mt0") + node2.query("SYSTEM STOP REPLICATION QUEUES") - for i in range(5): - node1.query(f"INSERT INTO mt0 VALUES ({i}, toDate('2020-10-01'))") + for i in range(5): + node1.query(f"INSERT INTO mt0 VALUES ({i}, toDate('2020-10-01'))") - for i in range(20): - parts_to_merge = node1.query( - "SELECT parts_to_merge FROM system.replication_queue WHERE table='mt0' AND length(parts_to_merge) > 0" - ) - if parts_to_merge: - parts_list = list(sorted(ast.literal_eval(parts_to_merge))) - print("Got parts list", parts_list) - if len(parts_list) < 3: - raise Exception(f"Got too small parts list {parts_list}") - break - time.sleep(1) + for i in range(20): + parts_to_merge = node1.query( + "SELECT parts_to_merge FROM system.replication_queue WHERE table='mt0' AND length(parts_to_merge) > 0" + ) + if parts_to_merge: + parts_list = list(sorted(ast.literal_eval(parts_to_merge))) + print("Got parts list", parts_list) + if len(parts_list) < 3: + raise Exception(f"Got too small parts list {parts_list}") + break + time.sleep(1) - victim_part_from_the_middle = random.choice(parts_list[1:-1]) - print("Will corrupt part", victim_part_from_the_middle) + victim_part_from_the_middle = random.choice(parts_list[1:-1]) + print("Will corrupt part", victim_part_from_the_middle) - remove_part_from_disk(node1, "mt0", victim_part_from_the_middle) + remove_part_from_disk(node1, "mt0", victim_part_from_the_middle) - node1.query("DETACH TABLE mt0") + node1.query("DETACH TABLE mt0") - node1.query("ATTACH TABLE mt0") + node1.query("ATTACH TABLE mt0") - node1.query("SYSTEM START MERGES mt0") - res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt0") - print("result: ", res) - print("error: ", res) + node1.query("SYSTEM START MERGES mt0") + res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt0") + print("result: ", res) + print("error: ", res) - for i in range(10): - result = node1.query("SELECT count() FROM system.replication_queue") - if int(result) == 0: - break - time.sleep(1) - else: - assert False, "Still have something in replication queue:\n" + node1.query( - "SELECT count() FROM system.replication_queue FORMAT Vertical" - ) + for i in range(10): + result = node1.query("SELECT count() FROM system.replication_queue") + if int(result) == 0: + break + time.sleep(1) + else: + assert False, "Still have something in replication queue:\n" + node1.query( + "SELECT count() FROM system.replication_queue FORMAT Vertical" + ) - assert node1.contains_in_log( - "Created empty part" - ), f"Seems like empty part {victim_part_from_the_middle} is not created or log message changed" + assert node1.contains_in_log( + "Created empty part" + ), f"Seems like empty part {victim_part_from_the_middle} is not created or log message changed" - assert node1.query("SELECT COUNT() FROM mt0") == "4\n" + assert node1.query("SELECT COUNT() FROM mt0") == "4\n" - node2.query("SYSTEM START REPLICATION QUEUES") + node2.query("SYSTEM START REPLICATION QUEUES") - assert_eq_with_retry(node2, "SELECT COUNT() FROM mt0", "4") - assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") - - node1.query("DROP TABLE IF EXISTS mt0 SYNC") - node2.query("DROP TABLE IF EXISTS mt0 SYNC") + assert_eq_with_retry(node2, "SELECT COUNT() FROM mt0", "4") + assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") + finally: + node1.query("DROP TABLE IF EXISTS mt0 SYNC") + node2.query("DROP TABLE IF EXISTS mt0 SYNC") def test_lost_part_other_replica(start_cluster): node1.query("DROP TABLE IF EXISTS mt1 SYNC") node2.query("DROP TABLE IF EXISTS mt1 SYNC") - for node in [node1, node2]: - node.query( - f"CREATE TABLE mt1 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t1', '{node.name}') ORDER BY tuple() " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," - "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + try: + for node in [node1, node2]: + node.query( + f"CREATE TABLE mt1 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t1', '{node.name}') ORDER BY tuple() " + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + ) + + node1.query("SYSTEM STOP MERGES mt1") + node2.query("SYSTEM STOP REPLICATION QUEUES") + + for i in range(5): + node1.query(f"INSERT INTO mt1 VALUES ({i})") + + for i in range(20): + parts_to_merge = node1.query( + "SELECT parts_to_merge FROM system.replication_queue WHERE table='mt1' AND length(parts_to_merge) > 0" + ) + if parts_to_merge: + parts_list = list(sorted(ast.literal_eval(parts_to_merge))) + print("Got parts list", parts_list) + if len(parts_list) < 3: + raise Exception("Got too small parts list {}".format(parts_list)) + break + time.sleep(1) + + victim_part_from_the_middle = random.choice(parts_list[1:-1]) + print("Will corrupt part", victim_part_from_the_middle) + + remove_part_from_disk(node1, "mt1", victim_part_from_the_middle) + + # other way to detect broken parts + node1.query("CHECK TABLE mt1") + + node2.query("SYSTEM START REPLICATION QUEUES") + res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt1") + print("result: ", res) + print("error: ", res) + + for i in range(10): + result = node2.query("SELECT count() FROM system.replication_queue") + if int(result) == 0: + break + time.sleep(1) + else: + assert False, "Still have something in replication queue:\n" + node2.query( + "SELECT * FROM system.replication_queue FORMAT Vertical" + ) + + assert node1.contains_in_log( + "Created empty part" + ), "Seems like empty part {} is not created or log message changed".format( + victim_part_from_the_middle ) - node1.query("SYSTEM STOP MERGES mt1") - node2.query("SYSTEM STOP REPLICATION QUEUES") + assert_eq_with_retry(node2, "SELECT COUNT() FROM mt1", "4") + assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") - for i in range(5): - node1.query(f"INSERT INTO mt1 VALUES ({i})") + node1.query("SYSTEM START MERGES mt1") - for i in range(20): - parts_to_merge = node1.query( - "SELECT parts_to_merge FROM system.replication_queue WHERE table='mt1' AND length(parts_to_merge) > 0" - ) - if parts_to_merge: - parts_list = list(sorted(ast.literal_eval(parts_to_merge))) - print("Got parts list", parts_list) - if len(parts_list) < 3: - raise Exception("Got too small parts list {}".format(parts_list)) - break - time.sleep(1) - - victim_part_from_the_middle = random.choice(parts_list[1:-1]) - print("Will corrupt part", victim_part_from_the_middle) - - remove_part_from_disk(node1, "mt1", victim_part_from_the_middle) - - # other way to detect broken parts - node1.query("CHECK TABLE mt1") - - node2.query("SYSTEM START REPLICATION QUEUES") - res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt1") - print("result: ", res) - print("error: ", res) - - for i in range(10): - result = node2.query("SELECT count() FROM system.replication_queue") - if int(result) == 0: - break - time.sleep(1) - else: - assert False, "Still have something in replication queue:\n" + node2.query( - "SELECT * FROM system.replication_queue FORMAT Vertical" - ) - - assert node1.contains_in_log( - "Created empty part" - ), "Seems like empty part {} is not created or log message changed".format( - victim_part_from_the_middle - ) - - assert_eq_with_retry(node2, "SELECT COUNT() FROM mt1", "4") - assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") - - node1.query("SYSTEM START MERGES mt1") - - assert_eq_with_retry(node1, "SELECT COUNT() FROM mt1", "4") - assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") - - node1.query("DROP TABLE IF EXISTS mt1 SYNC") - node2.query("DROP TABLE IF EXISTS mt1 SYNC") + assert_eq_with_retry(node1, "SELECT COUNT() FROM mt1", "4") + assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") + finally: + node1.query("DROP TABLE IF EXISTS mt1 SYNC") + node2.query("DROP TABLE IF EXISTS mt1 SYNC") def test_lost_part_mutation(start_cluster): node1.query("DROP TABLE IF EXISTS mt2 SYNC") node2.query("DROP TABLE IF EXISTS mt2 SYNC") - for node in [node1, node2]: - node.query( - f"CREATE TABLE mt2 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t2', '{node.name}') ORDER BY tuple() " - "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," - "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + try: + for node in [node1, node2]: + node.query( + f"CREATE TABLE mt2 (id UInt64) ENGINE ReplicatedMergeTree('/clickhouse/tables/t2', '{node.name}') ORDER BY tuple() " + "SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + ) + + node1.query("SYSTEM STOP MERGES mt2") + node2.query("SYSTEM STOP REPLICATION QUEUES") + + for i in range(2): + node1.query(f"INSERT INTO mt2 VALUES ({i})") + + node1.query( + "ALTER TABLE mt2 UPDATE id = 777 WHERE 1", settings={"mutations_sync": "0"} ) - node1.query("SYSTEM STOP MERGES mt2") - node2.query("SYSTEM STOP REPLICATION QUEUES") + for i in range(20): + parts_to_mutate = node1.query( + "SELECT count() FROM system.replication_queue WHERE table='mt2'" + ) + # two mutations for both replicas + if int(parts_to_mutate) == 4: + break + time.sleep(1) - for i in range(2): - node1.query(f"INSERT INTO mt2 VALUES ({i})") + remove_part_from_disk(node1, "mt2", "all_1_1_0") - node1.query( - "ALTER TABLE mt2 UPDATE id = 777 WHERE 1", settings={"mutations_sync": "0"} - ) + # other way to detect broken parts + node1.query("CHECK TABLE mt2") - for i in range(20): - parts_to_mutate = node1.query( - "SELECT count() FROM system.replication_queue WHERE table='mt2'" - ) - # two mutations for both replicas - if int(parts_to_mutate) == 4: - break - time.sleep(1) + node1.query("SYSTEM START MERGES mt2") + res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt2") + print("result: ", res) + print("error: ", res) - remove_part_from_disk(node1, "mt2", "all_1_1_0") + for i in range(10): + result = node1.query("SELECT count() FROM system.replication_queue") + if int(result) == 0: + break + time.sleep(1) + else: + assert False, "Still have something in replication queue:\n" + node1.query( + "SELECT * FROM system.replication_queue FORMAT Vertical" + ) - # other way to detect broken parts - node1.query("CHECK TABLE mt2") + assert_eq_with_retry(node1, "SELECT COUNT() FROM mt2", "1") + assert_eq_with_retry(node1, "SELECT SUM(id) FROM mt2", "777") + assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") - node1.query("SYSTEM START MERGES mt2") - res, err = node1.query_and_get_answer_with_error("SYSTEM SYNC REPLICA mt2") - print("result: ", res) - print("error: ", res) + node2.query("SYSTEM START REPLICATION QUEUES") - for i in range(10): - result = node1.query("SELECT count() FROM system.replication_queue") - if int(result) == 0: - break - time.sleep(1) - else: - assert False, "Still have something in replication queue:\n" + node1.query( - "SELECT * FROM system.replication_queue FORMAT Vertical" - ) - - assert_eq_with_retry(node1, "SELECT COUNT() FROM mt2", "1") - assert_eq_with_retry(node1, "SELECT SUM(id) FROM mt2", "777") - assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") - - node2.query("SYSTEM START REPLICATION QUEUES") - - assert_eq_with_retry(node2, "SELECT COUNT() FROM mt2", "1") - assert_eq_with_retry(node2, "SELECT SUM(id) FROM mt2", "777") - assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") - - node1.query("DROP TABLE IF EXISTS mt2 SYNC") - node2.query("DROP TABLE IF EXISTS mt2 SYNC") + assert_eq_with_retry(node2, "SELECT COUNT() FROM mt2", "1") + assert_eq_with_retry(node2, "SELECT SUM(id) FROM mt2", "777") + assert_eq_with_retry(node2, "SELECT COUNT() FROM system.replication_queue", "0") + finally: + node1.query("DROP TABLE IF EXISTS mt2 SYNC") + node2.query("DROP TABLE IF EXISTS mt2 SYNC") def test_lost_last_part(start_cluster): node1.query("DROP TABLE IF EXISTS mt3 SYNC") node2.query("DROP TABLE IF EXISTS mt3 SYNC") - for node in [node1, node2]: - node.query( - f"CREATE TABLE mt3 (id UInt64, p String) ENGINE ReplicatedMergeTree('/clickhouse/tables/t3', '{node.name}') " - "ORDER BY tuple() PARTITION BY p SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," - "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + try: + for node in [node1, node2]: + node.query( + f"CREATE TABLE mt3 (id UInt64, p String) ENGINE ReplicatedMergeTree('/clickhouse/tables/t3', '{node.name}') " + "ORDER BY tuple() PARTITION BY p SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, cleanup_thread_preferred_points_per_iteration=0," + "merge_selecting_sleep_ms=100, max_merge_selecting_sleep_ms=1000" + ) + + node1.query("SYSTEM STOP MERGES mt3") + node2.query("SYSTEM STOP REPLICATION QUEUES") + + for i in range(1): + node1.query(f"INSERT INTO mt3 VALUES ({i}, 'x')") + + # actually not important + node1.query( + "ALTER TABLE mt3 UPDATE id = 777 WHERE 1", settings={"mutations_sync": "0"} ) - node1.query("SYSTEM STOP MERGES mt3") - node2.query("SYSTEM STOP REPLICATION QUEUES") + partition_id = node1.query("select partitionId('x')").strip() + remove_part_from_disk(node1, "mt3", f"{partition_id}_0_0_0") - for i in range(1): - node1.query(f"INSERT INTO mt3 VALUES ({i}, 'x')") + # other way to detect broken parts + node1.query("CHECK TABLE mt3") - # actually not important - node1.query( - "ALTER TABLE mt3 UPDATE id = 777 WHERE 1", settings={"mutations_sync": "0"} - ) + node1.query("SYSTEM START MERGES mt3") - partition_id = node1.query("select partitionId('x')").strip() - remove_part_from_disk(node1, "mt3", f"{partition_id}_0_0_0") + for i in range(100): + result = node1.query( + "SELECT count() FROM system.replication_queue WHERE table='mt3'" + ) + assert int(result) <= 2, "Have a lot of entries in queue {}".format( + node1.query("SELECT * FROM system.replication_queue FORMAT Vertical") + ) + if node1.contains_in_log("Cannot create empty part") and node1.contains_in_log( + "DROP/DETACH PARTITION" + ): + break + if node1.contains_in_log( + "Created empty part 8b8f0fede53df97513a9fb4cb19dc1e4_0_0_0 " + ): + break + time.sleep(0.5) + else: + assert False, "Don't have required messages in node1 log" - # other way to detect broken parts - node1.query("CHECK TABLE mt3") + node1.query(f"ALTER TABLE mt3 DROP PARTITION ID '{partition_id}'") - node1.query("SYSTEM START MERGES mt3") - - for i in range(10): - result = node1.query( - "SELECT count() FROM system.replication_queue WHERE table='mt3'" - ) - assert int(result) <= 2, "Have a lot of entries in queue {}".format( - node1.query("SELECT * FROM system.replication_queue FORMAT Vertical") - ) - if node1.contains_in_log("Cannot create empty part") and node1.contains_in_log( - "DROP/DETACH PARTITION" - ): - break - if node1.contains_in_log( - "Created empty part 8b8f0fede53df97513a9fb4cb19dc1e4_0_0_0 " - ): - break - time.sleep(1) - else: - assert False, "Don't have required messages in node1 log" - - node1.query(f"ALTER TABLE mt3 DROP PARTITION ID '{partition_id}'") - - assert_eq_with_retry(node1, "SELECT COUNT() FROM mt3", "0") - assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") - - node1.query("DROP TABLE IF EXISTS mt3 SYNC") - node2.query("DROP TABLE IF EXISTS mt3 SYNC") + assert_eq_with_retry(node1, "SELECT COUNT() FROM mt3", "0") + assert_eq_with_retry(node1, "SELECT COUNT() FROM system.replication_queue", "0") + finally: + node1.query("DROP TABLE IF EXISTS mt3 SYNC") + node2.query("DROP TABLE IF EXISTS mt3 SYNC") From a2e63404b3ffc9a1a9a5e6380918ab4c033e55a2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 12:01:17 +0000 Subject: [PATCH 0677/2047] Further simplify logic --- cmake/limit_jobs.cmake | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index b7cc5cf0967..4d81619aa13 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -45,15 +45,14 @@ endif() message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") -if (PARALLEL_COMPILE_JOBS AND (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) endif () -if (PARALLEL_LINK_JOBS AND (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS}) endif () - From 355faa4251f7870492e0d019e3ca982bd93f1e90 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 17 Jul 2023 20:08:32 +0800 Subject: [PATCH 0678/2047] ci fix --- docs/en/interfaces/formats.md | 4 ++-- docs/en/operations/settings/settings-formats.md | 8 ++++---- src/Core/Settings.h | 4 ++-- src/Formats/FormatFactory.cpp | 4 ++-- src/Formats/FormatSettings.h | 4 ++-- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 4 ++-- 6 files changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 2207a6f0442..f45c55a9734 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,8 +472,8 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_allow_check_deserialize](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_check_deserialize) - Allow to check whether the csv input field can be deserialized. Default value - `false`. -- [input_format_csv_set_default_if_deserialize_failed](/docs/en/operations/settings/settings-formats.md/#input_format_csv_set_default_if_deserialize_failed) - Set default value to column if the csv input field deserialize failed. Default value - `false`. +- [input_format_csv_allow_check_field_deserialization](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_check_field_deserialization) - Allow to check whether the csv input field can be successful deserialized. Default value - `false`. +- [input_format_csv_set_default_if_deserialization_failed](/docs/en/operations/settings/settings-formats.md/#input_format_csv_set_default_if_deserialization_failed) - Set default value to column if the csv input field deserialization failed. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index b74504ac5fd..409ac4bd58a 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -969,15 +969,15 @@ Result a b ``` -### input_format_csv_allow_check_deserialize {#input_format_csv_allow_check_deserialize} +### input_format_csv_allow_check_field_deserialization {#input_format_csv_allow_check_field_deserialization} Allow to use whitespace or tab as field delimiter in CSV strings. Default value: `false`. -### input_format_csv_set_default_if_deserialize_failed {#input_format_csv_set_default_if_deserialize_failed} +### input_format_csv_set_default_if_deserialization_failed {#input_format_csv_set_default_if_deserialization_failed} -Allow to set default value to column if the csv input field deserialize failed +Allow to set default value to column if the csv input field's deserialization failed Default value: `false`. @@ -988,7 +988,7 @@ Query ```bash echo 'a,b,c' > 1.txt ./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize=true, input_format_csv_set_default_if_deserialize_failed=true FORMAT CSV" +cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_field_deserialization=true, input_format_csv_set_default_if_deserialization_failed=true FORMAT CSV" ./clickhouse local -q "select * from test_tbl" ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5ea355c61c2..9d93ba9ad2c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,8 +872,8 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_allow_check_deserialize, false, "Allow to check the csv input field deserialize whether success or not.", 0) \ - M(Bool, input_format_csv_set_default_if_deserialize_failed, false, "All to set column default value if the input field deserialize failed.", 0) \ + M(Bool, input_format_csv_allow_check_field_deserialization, false, "Allow to check the csv input field deserialization whether success or not.", 0) \ + M(Bool, input_format_csv_set_default_if_deserialization_failed, false, "All to set column default value if the input field's deserialization failed.", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index ac7ceb7e791..b3b9609f9fe 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,8 +73,8 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; - format_settings.csv.allow_check_deserialize = settings.input_format_csv_allow_check_deserialize; - format_settings.csv.set_default_if_deserialize_failed = settings.input_format_csv_set_default_if_deserialize_failed; + format_settings.csv.allow_check_field_deserialization = settings.input_format_csv_allow_check_field_deserialization; + format_settings.csv.set_default_if_deserialization_failed = settings.input_format_csv_set_default_if_deserialization_failed; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index e9de7a51108..4d4eb926992 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -141,8 +141,8 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; - bool allow_check_deserialize=false; - bool set_default_if_deserialize_failed=false; + bool allow_check_field_deserialization=false; + bool set_default_if_deserialization_failed=false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index ada9d78711a..e1be6b21610 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -320,7 +320,7 @@ bool CSVFormatReader::readField( size_t col_size = column.size(); try { - if (format_settings.csv.allow_check_deserialize) + if (format_settings.csv.allow_check_field_deserialization) { std::string field; readCSVField(field, *buf, format_settings.csv); @@ -346,7 +346,7 @@ bool CSVFormatReader::readField( catch (Exception & e) { LOG_DEBUG(&Poco::Logger::get("CSVRowInputFormat"), "Failed to deserialize CSV column, exception message:{}", e.what()); - if (format_settings.csv.set_default_if_deserialize_failed) + if (format_settings.csv.set_default_if_deserialization_failed) { // Reset the column and buffer position, then skip the field and set column default value. if (column.size() == col_size + 1) From 71e84656636b0b7a74974d209f3913e0ee1fb5bf Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 17 Jul 2023 12:09:36 +0000 Subject: [PATCH 0679/2047] Automatic style fix --- tests/integration/test_lost_part/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_lost_part/test.py b/tests/integration/test_lost_part/test.py index b83707145f4..382539df7de 100644 --- a/tests/integration/test_lost_part/test.py +++ b/tests/integration/test_lost_part/test.py @@ -278,9 +278,9 @@ def test_lost_last_part(start_cluster): assert int(result) <= 2, "Have a lot of entries in queue {}".format( node1.query("SELECT * FROM system.replication_queue FORMAT Vertical") ) - if node1.contains_in_log("Cannot create empty part") and node1.contains_in_log( - "DROP/DETACH PARTITION" - ): + if node1.contains_in_log( + "Cannot create empty part" + ) and node1.contains_in_log("DROP/DETACH PARTITION"): break if node1.contains_in_log( "Created empty part 8b8f0fede53df97513a9fb4cb19dc1e4_0_0_0 " From db4735a10508e02b50565bba8a1e71161df90f82 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 17 Jul 2023 12:12:25 +0000 Subject: [PATCH 0680/2047] Implemented requested changes --- src/Functions/array/arrayIntersect.cpp | 47 +++++++++++--------------- 1 file changed, 20 insertions(+), 27 deletions(-) diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 7a8bde7ab5f..83b26b56071 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -510,7 +510,7 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable map.clear(); bool all_has_nullable = all_nullable; - bool current_has_nullable = false; + bool current_has_nullable; for (size_t arg_num = 0; arg_num < args; ++arg_num) { @@ -549,7 +549,8 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable ++(*value); } } - + // We update offsets for all the arrays except the first one. Offsets for the first array would be updated later. + // It is needed to iterate the first array again so that the elements in the result would have fixed order. if (arg_num) { prev_off[arg_num] = off; @@ -570,15 +571,21 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable else off = (*arg.offsets)[row]; + bool is_map_serialized = false; for (auto i : collections::range(prev_off[0], off)) { - all_has_nullable = all_nullable; - current_has_nullable = false; typename Map::LookupResult pair = nullptr; if (arg.null_map && (*arg.null_map)[i]) { current_has_nullable = true; + if (all_has_nullable && !null_added) + { + ++result_offset; + result_data.insertDefault(); + null_map.push_back(1); + null_added = true; + } if (null_added) continue; } @@ -591,51 +598,37 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable else { const char * data = nullptr; - pair = map.find(columns[0]->serializeValueIntoArena(i, arena, data)); + if (!is_map_serialized) + { + pair = map.find(columns[0]->serializeValueIntoArena(i, arena, data)); + is_map_serialized = true; + } } prev_off[0] = off; if (arg.is_const) prev_off[0] = 0; - if (!current_has_nullable) - all_has_nullable = false; - if (pair && pair->getMapped() == args) { + // We increase pair->getMapped() here to not skip duplicate values from the first array. ++pair->getMapped(); ++result_offset; if constexpr (is_numeric_column) { - if (pair->getKey() == columns[0]->getElement(i)) - { - result_data.insertValue(pair->getKey()); - } + result_data.insertValue(pair->getKey()); } else if constexpr (std::is_same_v || std::is_same_v) { - if (pair->getKey() == columns[0]->getDataAt(i)) - { - result_data.insertData(pair->getKey().data, pair->getKey().size); - } + result_data.insertData(pair->getKey().data, pair->getKey().size); } else { const char * data = nullptr; - if (pair->getKey() == columns[0]->serializeValueIntoArena(i, arena, data)) - { - result_data.deserializeAndInsertFromArena(pair->getKey().data); - } + result_data.deserializeAndInsertFromArena(pair->getKey().data); } if (all_nullable) null_map.push_back(0); } - if (all_has_nullable && !null_added) - { - ++result_offset; - result_data.insertDefault(); - null_map.push_back(1); - null_added = true; - } } result_offsets.getElement(row) = result_offset; From a347408e203eee4f0b03341eacc6780003027c27 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 14:13:50 +0200 Subject: [PATCH 0681/2047] Fix style check --- docs/en/engines/table-engines/integrations/s3queue.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 9fd6c88a966..f857df8b1c8 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -69,7 +69,7 @@ SETTINGS Possible values: -- unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer. +- unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKeeper. - ordered — With ordered mode, only the max name of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. Default value: `unordered`. @@ -222,4 +222,3 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table :::note If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: - From e24883d89b680676e047a8eb2d38f746bfd62f54 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 12:24:03 +0000 Subject: [PATCH 0682/2047] Cut peak memory to fix tests --- .../0_stateless/00416_pocopatch_progress_in_http_headers.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh index 3127118c3ba..e77aa1bf250 100755 --- a/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh +++ b/tests/queries/0_stateless/00416_pocopatch_progress_in_http_headers.sh @@ -10,7 +10,7 @@ result="" lines_expected=4 counter=0 while [ $counter -lt $RETRIES ] && [ "$(echo "$result" | wc -l)" != "$lines_expected" ]; do - result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=5&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT max(number) FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]') + result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=5&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d 'SELECT max(number) FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]' | sed 's/,\"peak_mem[^}]*//') let counter=counter+1 done echo "$result" @@ -19,7 +19,7 @@ result="" lines_expected=12 counter=0 while [ $counter -lt $RETRIES ] && [ "$(echo "$result" | wc -l)" != "$lines_expected" ]; do - result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" -d 'SELECT number FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]') + result=$(${CLICKHOUSE_CURL} -vsS "${CLICKHOUSE_URL}&max_block_size=1&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0&output_format_parallel_formatting=0" -d 'SELECT number FROM numbers(10)' 2>&1 | grep -E 'Content-Encoding|X-ClickHouse-Progress|^[0-9]'| sed 's/,\"peak_mem[^}]*//') let counter=counter+1 done echo "$result" From 4a18ec62ea819b6049457ac8c8fd0391b57ef61f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 17 Jul 2023 14:52:11 +0200 Subject: [PATCH 0683/2047] Addressed review comments on docs and function parameters --- .../TableFunctionAzureBlobStorage.cpp | 53 +++++++------------ .../TableFunctionAzureBlobStorage.h | 14 ++++- 2 files changed, 32 insertions(+), 35 deletions(-) diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index 4a9e13a9abf..03cba67069a 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -121,48 +121,35 @@ void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const { configuration.account_name = fourth_arg; configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); - if (!is_format_arg(sixth_arg)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - configuration.format = sixth_arg; + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name/structure"); + if (is_format_arg(sixth_arg)) + configuration.format = sixth_arg; + else + configuration.structure = sixth_arg; } } else if (engine_args.size() == 7) { auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format, compression and structure must be last arguments"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); - if (!is_format_arg(sixth_arg)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - configuration.format = sixth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); - } + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); } else if (engine_args.size() == 8) { auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); - if (!is_format_arg(sixth_arg)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - configuration.format = sixth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); - configuration.structure = checkAndGetLiteralArgument(engine_args[7], "structure"); - } + configuration.account_name = fourth_arg; + configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (!is_format_arg(sixth_arg)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + configuration.format = sixth_arg; + configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); + configuration.structure = checkAndGetLiteralArgument(engine_args[7], "structure"); } configuration.blobs_paths = {configuration.blob_path}; diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.h b/src/TableFunctions/TableFunctionAzureBlobStorage.h index f0efd15dace..efbb6a5b1e0 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.h +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.h @@ -13,13 +13,23 @@ namespace DB class Context; -/* AzureBlob(source, [access_key_id, secret_access_key,] [format, structure, compression]) - creates a temporary storage for a file in AzureBlob. +/* AzureBlob(source, [access_key_id, secret_access_key,] [format, compression, structure]) - creates a temporary storage for a file in AzureBlob. */ class TableFunctionAzureBlobStorage : public ITableFunction { public: static constexpr auto name = "azureBlobStorage"; - static constexpr auto signature = "- connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]\n"; + + static constexpr auto signature = " - connection_string, container_name, blobpath\n" + " - connection_string, container_name, blobpath, structure \n" + " - connection_string, container_name, blobpath, format \n" + " - connection_string, container_name, blobpath, format, compression \n" + " - connection_string, container_name, blobpath, format, compression, structure \n" + " - storage_account_url, container_name, blobpath, account_name, account_key\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, structure\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n"; static size_t getMaxNumberOfArguments() { return 8; } From 195b29b4a02237a8aa337dfc6baff9dc0d0e85f8 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 17 Jul 2023 14:53:37 +0200 Subject: [PATCH 0684/2047] Addressed comments on docs --- .../en/sql-reference/table-functions/azureBlobStorageCluster.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md b/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md index 7718be74428..77acdf51549 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md @@ -5,7 +5,7 @@ sidebar_label: azureBlobStorageCluster title: "azureBlobStorageCluster Table Function" --- -Allows processing files from [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs). in parallel from many nodes in a specified cluster. On initiator it creates a connection to all nodes in the cluster, discloses asterisks in S3 file path, and dispatches each file dynamically. On the worker node it asks the initiator about the next task to process and processes it. This is repeated until all tasks are finished. +Allows processing files from [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs) in parallel from many nodes in a specified cluster. On initiator it creates a connection to all nodes in the cluster, discloses asterisks in S3 file path, and dispatches each file dynamically. On the worker node it asks the initiator about the next task to process and processes it. This is repeated until all tasks are finished. This table function is similar to the [s3Cluster function](../../sql-reference/table-functions/s3Cluster.md). **Syntax** From 018546a57d4553c44613c11aa3b0eb616461e60c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 1 Jul 2023 17:39:50 +0200 Subject: [PATCH 0685/2047] Avoid keeping lock Context::getLock() while recalculating access rights of a connected user. --- src/Access/AccessControl.cpp | 53 +++------ src/Access/AccessControl.h | 10 +- src/Access/ContextAccess.cpp | 74 +++++++------ src/Access/ContextAccess.h | 51 +-------- src/Access/ContextAccessParams.cpp | 172 +++++++++++++++++++++++++++++ src/Access/ContextAccessParams.h | 64 +++++++++++ src/Interpreters/Context.cpp | 102 +++++++++-------- src/Interpreters/Context.h | 7 +- 8 files changed, 362 insertions(+), 171 deletions(-) create mode 100644 src/Access/ContextAccessParams.cpp create mode 100644 src/Access/ContextAccessParams.h diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 6179c823b56..41ac3f42ee2 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -72,18 +72,26 @@ public: std::shared_ptr getContextAccess(const ContextAccessParams & params) { - std::lock_guard lock{mutex}; - auto x = cache.get(params); - if (x) { - if ((*x)->tryGetUser()) - return *x; - /// No user, probably the user has been dropped while it was in the cache. - cache.remove(params); + std::lock_guard lock{mutex}; + auto x = cache.get(params); + if (x) + { + if ((*x)->getUserID() && !(*x)->tryGetUser()) + cache.remove(params); /// The user has been dropped while it was in the cache. + else + return *x; + } } + auto res = std::make_shared(access_control, params); res->initialize(); - cache.add(params, res); + + { + std::lock_guard lock{mutex}; + cache.add(params, res); + } + return res; } @@ -713,35 +721,6 @@ int AccessControl::getBcryptWorkfactor() const } -std::shared_ptr AccessControl::getContextAccess( - const UUID & user_id, - const std::vector & current_roles, - bool use_default_roles, - const Settings & settings, - const String & current_database, - const ClientInfo & client_info) const -{ - ContextAccessParams params; - params.user_id = user_id; - params.current_roles.insert(current_roles.begin(), current_roles.end()); - params.use_default_roles = use_default_roles; - params.current_database = current_database; - params.readonly = settings.readonly; - params.allow_ddl = settings.allow_ddl; - params.allow_introspection = settings.allow_introspection_functions; - params.interface = client_info.interface; - params.http_method = client_info.http_method; - params.address = client_info.current_address.host(); - params.quota_key = client_info.quota_key; - - /// Extract the last entry from comma separated list of X-Forwarded-For addresses. - /// Only the last proxy can be trusted (if any). - params.forwarded_address = client_info.getLastForwardedFor(); - - return getContextAccess(params); -} - - std::shared_ptr AccessControl::getContextAccess(const ContextAccessParams & params) const { return context_access_cache->getContextAccess(params); diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 2a8293a49e7..74816090f88 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -25,7 +25,7 @@ namespace Poco namespace DB { class ContextAccess; -struct ContextAccessParams; +class ContextAccessParams; struct User; using UserPtr = std::shared_ptr; class EnabledRoles; @@ -181,14 +181,6 @@ public: void setSettingsConstraintsReplacePrevious(bool enable) { settings_constraints_replace_previous = enable; } bool doesSettingsConstraintsReplacePrevious() const { return settings_constraints_replace_previous; } - std::shared_ptr getContextAccess( - const UUID & user_id, - const std::vector & current_roles, - bool use_default_roles, - const Settings & settings, - const String & current_database, - const ClientInfo & client_info) const; - std::shared_ptr getContextAccess(const ContextAccessParams & params) const; std::shared_ptr getEnabledRoles( diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 9c57853679f..cb8f1a5a48e 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -221,15 +222,15 @@ namespace } -ContextAccess::ContextAccess(const AccessControl & access_control_, const Params & params_) - : access_control(&access_control_) - , params(params_) +std::shared_ptr ContextAccess::fromContext(const ContextPtr & context) { + return context->getAccess(); } -ContextAccess::ContextAccess(FullAccess) - : is_full_access(true), access(std::make_shared(AccessRights::getFullAccess())), access_with_implicit(access) +ContextAccess::ContextAccess(const AccessControl & access_control_, const Params & params_) + : access_control(&access_control_) + , params(params_) { } @@ -251,18 +252,31 @@ ContextAccess::~ContextAccess() void ContextAccess::initialize() { - std::lock_guard lock{mutex}; - subscription_for_user_change = access_control->subscribeForChanges( - *params.user_id, [weak_ptr = weak_from_this()](const UUID &, const AccessEntityPtr & entity) - { - auto ptr = weak_ptr.lock(); - if (!ptr) - return; - UserPtr changed_user = entity ? typeid_cast(entity) : nullptr; - std::lock_guard lock2{ptr->mutex}; - ptr->setUser(changed_user); - }); - setUser(access_control->read(*params.user_id)); + std::lock_guard lock{mutex}; + + if (params.full_access) + { + access = std::make_shared(AccessRights::getFullAccess()); + access_with_implicit = access; + return; + } + + if (!params.user_id) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No user in current context, it's a bug"); + + subscription_for_user_change = access_control->subscribeForChanges( + *params.user_id, + [weak_ptr = weak_from_this()](const UUID &, const AccessEntityPtr & entity) + { + auto ptr = weak_ptr.lock(); + if (!ptr) + return; + UserPtr changed_user = entity ? typeid_cast(entity) : nullptr; + std::lock_guard lock2{ptr->mutex}; + ptr->setUser(changed_user); + }); + + setUser(access_control->read(*params.user_id)); } @@ -294,10 +308,10 @@ void ContextAccess::setUser(const UserPtr & user_) const current_roles = user->granted_roles.findGranted(user->default_roles); current_roles_with_admin_option = user->granted_roles.findGrantedWithAdminOption(user->default_roles); } - else + else if (params.current_roles) { - current_roles = user->granted_roles.findGranted(params.current_roles); - current_roles_with_admin_option = user->granted_roles.findGrantedWithAdminOption(params.current_roles); + current_roles = user->granted_roles.findGranted(*params.current_roles); + current_roles_with_admin_option = user->granted_roles.findGrantedWithAdminOption(*params.current_roles); } subscription_for_roles_changes.reset(); @@ -316,12 +330,16 @@ void ContextAccess::setRolesInfo(const std::shared_ptr & { assert(roles_info_); roles_info = roles_info_; + enabled_row_policies = access_control->getEnabledRowPolicies( *params.user_id, roles_info->enabled_roles); + enabled_quota = access_control->getEnabledQuota( *params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key); + enabled_settings = access_control->getEnabledSettings( *params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles); + calculateAccessRights(); } @@ -417,14 +435,6 @@ std::optional ContextAccess::getQuotaUsage() const } -std::shared_ptr ContextAccess::getFullAccess() -{ - static const std::shared_ptr res = - [] { return std::shared_ptr(new ContextAccess{kFullAccess}); }(); - return res; -} - - SettingsChanges ContextAccess::getDefaultSettings() const { std::lock_guard lock{mutex}; @@ -478,7 +488,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg throw Exception(ErrorCodes::UNKNOWN_USER, "{}: User has been dropped", getUserName()); } - if (is_full_access) + if (params.full_access) return true; auto access_granted = [&] @@ -706,7 +716,7 @@ bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const return false; }; - if (is_full_access) + if (params.full_access) return true; if (user_was_dropped) @@ -806,7 +816,7 @@ void ContextAccess::checkAdminOption(const std::vector & role_ids, const s void ContextAccess::checkGranteeIsAllowed(const UUID & grantee_id, const IAccessEntity & grantee) const { - if (is_full_access) + if (params.full_access) return; auto current_user = getUser(); @@ -816,7 +826,7 @@ void ContextAccess::checkGranteeIsAllowed(const UUID & grantee_id, const IAccess void ContextAccess::checkGranteesAreAllowed(const std::vector & grantee_ids) const { - if (is_full_access) + if (params.full_access) return; auto current_user = getUser(); diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 60bad0118fc..4c96ef5c11f 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -30,47 +31,18 @@ class AccessControl; class IAST; struct IAccessEntity; using ASTPtr = std::shared_ptr; - - -struct ContextAccessParams -{ - std::optional user_id; - boost::container::flat_set current_roles; - bool use_default_roles = false; - UInt64 readonly = 0; - bool allow_ddl = false; - bool allow_introspection = false; - String current_database; - ClientInfo::Interface interface = ClientInfo::Interface::TCP; - ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; - Poco::Net::IPAddress address; - String forwarded_address; - String quota_key; - - auto toTuple() const - { - return std::tie( - user_id, current_roles, use_default_roles, readonly, allow_ddl, allow_introspection, - current_database, interface, http_method, address, forwarded_address, quota_key); - } - - friend bool operator ==(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return lhs.toTuple() == rhs.toTuple(); } - friend bool operator !=(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return !(lhs == rhs); } - friend bool operator <(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return lhs.toTuple() < rhs.toTuple(); } - friend bool operator >(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return rhs < lhs; } - friend bool operator <=(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return !(rhs < lhs); } - friend bool operator >=(const ContextAccessParams & lhs, const ContextAccessParams & rhs) { return !(lhs < rhs); } -}; +class Context; +using ContextPtr = std::shared_ptr; class ContextAccess : public std::enable_shared_from_this { public: + static std::shared_ptr fromContext(const ContextPtr & context); + using Params = ContextAccessParams; const Params & getParams() const { return params; } - ContextAccess(const AccessControl & access_control_, const Params & params_); - /// Returns the current user. Throws if user is nullptr. UserPtr getUser() const; /// Same as above, but can return nullptr. @@ -161,22 +133,12 @@ public: /// Checks if grantees are allowed for the current user, throws an exception if not. void checkGranteesAreAllowed(const std::vector & grantee_ids) const; - /// Makes an instance of ContextAccess which provides full access to everything - /// without any limitations. This is used for the global context. - static std::shared_ptr getFullAccess(); - + ContextAccess(const AccessControl & access_control_, const Params & params_); ~ContextAccess(); private: friend class AccessControl; - struct FullAccess {}; - static const FullAccess kFullAccess; - - /// Makes an instance of ContextAccess which provides full access to everything - /// without any limitations. This is used for the global context. - explicit ContextAccess(FullAccess); - void initialize(); void setUser(const UserPtr & user_) const TSA_REQUIRES(mutex); void setRolesInfo(const std::shared_ptr & roles_info_) const TSA_REQUIRES(mutex); @@ -223,7 +185,6 @@ private: const AccessControl * access_control = nullptr; const Params params; - const bool is_full_access = false; mutable std::atomic user_was_dropped = false; mutable std::atomic trace_log = nullptr; diff --git a/src/Access/ContextAccessParams.cpp b/src/Access/ContextAccessParams.cpp new file mode 100644 index 00000000000..7963e83dddf --- /dev/null +++ b/src/Access/ContextAccessParams.cpp @@ -0,0 +1,172 @@ +#include +#include +#include + + +namespace DB +{ + +ContextAccessParams::ContextAccessParams( + const std::optional user_id_, + bool full_access_, + bool use_default_roles_, + const std::shared_ptr> & current_roles_, + const Settings & settings_, + const String & current_database_, + const ClientInfo & client_info_) + : user_id(user_id_) + , full_access(full_access_) + , use_default_roles(use_default_roles_) + , current_roles(current_roles_) + , readonly(settings_.readonly) + , allow_ddl(settings_.allow_ddl) + , allow_introspection(settings_.allow_introspection_functions) + , current_database(current_database_) + , interface(client_info_.interface) + , http_method(client_info_.http_method) + , address(client_info_.current_address.host()) + , forwarded_address(client_info_.getLastForwardedFor()) + , quota_key(client_info_.quota_key) +{ +} + +String ContextAccessParams::toString() const +{ + WriteBufferFromOwnString out; + auto separator = [&] { return out.stringView().empty() ? "" : ", "; }; + if (user_id) + out << separator() << "user_id = " << *user_id; + if (full_access) + out << separator() << "full_access = " << full_access; + if (use_default_roles) + out << separator() << "use_default_roles = " << use_default_roles; + if (current_roles && !current_roles->empty()) + { + out << separator() << "current_roles = ["; + for (size_t i = 0; i != current_roles->size(); ++i) + { + if (i) + out << ", "; + out << (*current_roles)[i]; + } + out << "]"; + } + if (readonly) + out << separator() << "readonly = " << readonly; + if (allow_ddl) + out << separator() << "allow_ddl = " << allow_ddl; + if (allow_introspection) + out << separator() << "allow_introspection = " << allow_introspection; + if (!current_database.empty()) + out << separator() << "current_database = " << current_database; + out << separator() << "interface = " << magic_enum::enum_name(interface); + if (http_method != ClientInfo::HTTPMethod::UNKNOWN) + out << separator() << "http_method = " << magic_enum::enum_name(http_method); + if (!address.isWildcard()) + out << separator() << "address = " << address.toString(); + if (!forwarded_address.empty()) + out << separator() << "forwarded_address = " << forwarded_address; + if (!quota_key.empty()) + out << separator() << "quota_key = " << quota_key; + return out.str(); +} + +bool operator ==(const ContextAccessParams & left, const ContextAccessParams & right) +{ + auto check_equals = [](const auto & x, const auto & y) + { + if constexpr (::detail::is_shared_ptr_v>) + { + if (!x) + return !y; + else if (!y) + return false; + else + return *x == *y; + } + else + { + return x == y; + } + }; + + #define CONTEXT_ACCESS_PARAMS_EQUALS(name) \ + if (!check_equals(left.name, right.name)) \ + return false; + + CONTEXT_ACCESS_PARAMS_EQUALS(user_id) + CONTEXT_ACCESS_PARAMS_EQUALS(full_access) + CONTEXT_ACCESS_PARAMS_EQUALS(use_default_roles) + CONTEXT_ACCESS_PARAMS_EQUALS(current_roles) + CONTEXT_ACCESS_PARAMS_EQUALS(readonly) + CONTEXT_ACCESS_PARAMS_EQUALS(allow_ddl) + CONTEXT_ACCESS_PARAMS_EQUALS(allow_introspection) + CONTEXT_ACCESS_PARAMS_EQUALS(current_database) + CONTEXT_ACCESS_PARAMS_EQUALS(interface) + CONTEXT_ACCESS_PARAMS_EQUALS(http_method) + CONTEXT_ACCESS_PARAMS_EQUALS(address) + CONTEXT_ACCESS_PARAMS_EQUALS(forwarded_address) + CONTEXT_ACCESS_PARAMS_EQUALS(quota_key) + + #undef CONTEXT_ACCESS_PARAMS_EQUALS + + return true; /// All fields are equal, operator == must return true. +} + +bool operator <(const ContextAccessParams & left, const ContextAccessParams & right) +{ + auto check_less = [](const auto & x, const auto & y) + { + if constexpr (::detail::is_shared_ptr_v>) + { + if (!x) + return y ? -1 : 0; + else if (!y) + return 1; + else if (*x == *y) + return 0; + else if (*x < *y) + return -1; + else + return 1; + } + else + { + if (x == y) + return 0; + else if (x < y) + return -1; + else + return 1; + } + }; + + #define CONTEXT_ACCESS_PARAMS_LESS(name) \ + if (auto cmp = check_less(left.name, right.name); cmp != 0) \ + return cmp < 0; + + CONTEXT_ACCESS_PARAMS_LESS(user_id) + CONTEXT_ACCESS_PARAMS_LESS(full_access) + CONTEXT_ACCESS_PARAMS_LESS(use_default_roles) + CONTEXT_ACCESS_PARAMS_LESS(current_roles) + CONTEXT_ACCESS_PARAMS_LESS(readonly) + CONTEXT_ACCESS_PARAMS_LESS(allow_ddl) + CONTEXT_ACCESS_PARAMS_LESS(allow_introspection) + CONTEXT_ACCESS_PARAMS_LESS(current_database) + CONTEXT_ACCESS_PARAMS_LESS(interface) + CONTEXT_ACCESS_PARAMS_LESS(http_method) + CONTEXT_ACCESS_PARAMS_LESS(address) + CONTEXT_ACCESS_PARAMS_LESS(forwarded_address) + CONTEXT_ACCESS_PARAMS_LESS(quota_key) + + #undef CONTEXT_ACCESS_PARAMS_LESS + + return false; /// All fields are equal, operator < must return false. +} + +bool ContextAccessParams::dependsOnSettingName(std::string_view setting_name) +{ + return (setting_name == "readonly") || (setting_name == "allow_ddl") || (setting_name == "allow_introspection_functions"); +} + +} diff --git a/src/Access/ContextAccessParams.h b/src/Access/ContextAccessParams.h new file mode 100644 index 00000000000..740ec997964 --- /dev/null +++ b/src/Access/ContextAccessParams.h @@ -0,0 +1,64 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +struct Settings; + +/// Parameters which are used to calculate access rights and some related stuff like roles or constraints. +class ContextAccessParams +{ +public: + ContextAccessParams( + const std::optional user_id_, + bool full_access_, + bool use_default_roles_, + const std::shared_ptr> & current_roles_, + const Settings & settings_, + const String & current_database_, + const ClientInfo & client_info_); + + const std::optional user_id; + + /// Full access to everything without any limitations. + /// This is used for the global context. + const bool full_access; + + const bool use_default_roles; + const std::shared_ptr> current_roles; + + const UInt64 readonly; + const bool allow_ddl; + const bool allow_introspection; + + const String current_database; + + const ClientInfo::Interface interface; + const ClientInfo::HTTPMethod http_method; + const Poco::Net::IPAddress address; + + /// The last entry from comma separated list of X-Forwarded-For addresses. + /// Only the last proxy can be trusted (if any). + const String forwarded_address; + + const String quota_key; + + /// Outputs `ContextAccessParams` to string for logging. + String toString() const; + + friend bool operator <(const ContextAccessParams & left, const ContextAccessParams & right); + friend bool operator ==(const ContextAccessParams & left, const ContextAccessParams & right); + friend bool operator !=(const ContextAccessParams & left, const ContextAccessParams & right) { return !(left == right); } + friend bool operator >(const ContextAccessParams & left, const ContextAccessParams & right) { return right < left; } + friend bool operator <=(const ContextAccessParams & left, const ContextAccessParams & right) { return !(right < left); } + friend bool operator >=(const ContextAccessParams & left, const ContextAccessParams & right) { return !(left < right); } + + static bool dependsOnSettingName(std::string_view setting_name); +}; + +} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5019933c2af..abc33c1b8d4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1063,8 +1063,16 @@ void Context::setUser(const UUID & user_id_) user_id = user_id_; - access = getAccessControl().getContextAccess( - user_id_, /* current_roles = */ {}, /* use_default_roles = */ true, settings, current_database, client_info); + ContextAccessParams params{ + user_id, + /* full_access= */ false, + /* use_default_roles = */ true, + /* current_roles = */ nullptr, + settings, + current_database, + client_info}; + + access = getAccessControl().getContextAccess(params); auto user = access->getUser(); @@ -1108,7 +1116,7 @@ void Context::setCurrentRoles(const std::vector & current_roles_) if (current_roles ? (*current_roles == current_roles_) : current_roles_.empty()) return; current_roles = std::make_shared>(current_roles_); - calculateAccessRights(); + need_recalculate_access = true; } void Context::setCurrentRolesDefault() @@ -1133,20 +1141,6 @@ std::shared_ptr Context::getRolesInfo() const } -void Context::calculateAccessRights() -{ - auto lock = getLock(); - if (user_id) - access = getAccessControl().getContextAccess( - *user_id, - current_roles ? *current_roles : std::vector{}, - /* use_default_roles = */ false, - settings, - current_database, - client_info); -} - - template void Context::checkAccessImpl(const Args &... args) const { @@ -1166,11 +1160,50 @@ void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, void Context::checkAccess(const AccessRightsElement & element) const { return checkAccessImpl(element); } void Context::checkAccess(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } - std::shared_ptr Context::getAccess() const { - auto lock = getLock(); - return access ? access : ContextAccess::getFullAccess(); + /// A helper function to collect parameters for calculating access rights, called with Context::getLock() acquired. + auto get_params = [this]() + { + /// If setUserID() was never called then this must be the global context with the full access. + bool full_access = !user_id; + + return ContextAccessParams{user_id, full_access, /* use_default_roles= */ false, current_roles, settings, current_database, client_info}; + }; + + /// Check if the current access rights are still valid, otherwise get parameters for recalculating access rights. + std::optional params; + + { + auto lock = getLock(); + if (access && !need_recalculate_access) + return access; /// No need to recalculate access rights. + + params.emplace(get_params()); + + if (access && (access->getParams() == *params)) + { + need_recalculate_access = false; + return access; /// No need to recalculate access rights. + } + } + + /// Calculate new access rights according to the collected parameters. + /// NOTE: AccessControl::getContextAccess() may require some IO work, so Context::getLock() must be unlocked while we're doing this. + auto res = getAccessControl().getContextAccess(*params); + + { + /// If the parameters of access rights were not changed while we were calculated them + /// then we store the new access rights in the Context to allow reusing it later. + auto lock = getLock(); + if (get_params() == *params) + { + access = res; + need_recalculate_access = false; + } + } + + return res; } RowPolicyFilterPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const @@ -1700,27 +1733,8 @@ Settings Context::getSettings() const void Context::setSettings(const Settings & settings_) { auto lock = getLock(); - const auto old_readonly = settings.readonly; - const auto old_allow_ddl = settings.allow_ddl; - const auto old_allow_introspection_functions = settings.allow_introspection_functions; - const auto old_display_secrets = settings.format_display_secrets_in_show_and_select; - settings = settings_; - - if ((settings.readonly != old_readonly) - || (settings.allow_ddl != old_allow_ddl) - || (settings.allow_introspection_functions != old_allow_introspection_functions) - || (settings.format_display_secrets_in_show_and_select != old_display_secrets)) - calculateAccessRights(); -} - -void Context::recalculateAccessRightsIfNeeded(std::string_view name) -{ - if (name == "readonly" - || name == "allow_ddl" - || name == "allow_introspection_functions" - || name == "format_display_secrets_in_show_and_select") - calculateAccessRights(); + need_recalculate_access = true; } void Context::setSetting(std::string_view name, const String & value) @@ -1732,7 +1746,8 @@ void Context::setSetting(std::string_view name, const String & value) return; } settings.set(name, value); - recalculateAccessRightsIfNeeded(name); + if (ContextAccessParams::dependsOnSettingName(name)) + need_recalculate_access = true; } void Context::setSetting(std::string_view name, const Field & value) @@ -1744,7 +1759,8 @@ void Context::setSetting(std::string_view name, const Field & value) return; } settings.set(name, value); - recalculateAccessRightsIfNeeded(name); + if (ContextAccessParams::dependsOnSettingName(name)) + need_recalculate_access = true; } void Context::applySettingChange(const SettingChange & change) @@ -1853,7 +1869,7 @@ void Context::setCurrentDatabase(const String & name) DatabaseCatalog::instance().assertDatabaseExists(name); auto lock = getLock(); current_database = name; - calculateAccessRights(); + need_recalculate_access = true; } void Context::setCurrentQueryId(const String & query_id) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 6cbb0e58911..2c32ad28d01 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -248,7 +248,8 @@ private: std::optional user_id; std::shared_ptr> current_roles; std::shared_ptr settings_constraints_and_current_profiles; - std::shared_ptr access; + mutable std::shared_ptr access; + mutable bool need_recalculate_access = true; std::shared_ptr row_policies_of_initial_user; String current_database; Settings settings; /// Setting for query execution. @@ -1149,10 +1150,6 @@ private: void initGlobal(); - /// Compute and set actual user settings, client_info.current_user should be set - void calculateAccessRights(); - void recalculateAccessRightsIfNeeded(std::string_view setting_name); - template void checkAccessImpl(const Args &... args) const; From 0e4b75a282f14d812763a43cb3519c94beeb138c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 1 Jul 2023 19:58:38 +0200 Subject: [PATCH 0686/2047] Avoid keeping lock Context::getLock() while calculating access rights when a user logs in. --- src/Access/ContextAccess.cpp | 35 +++++---- src/Access/ContextAccess.h | 6 +- src/Access/ContextAccessParams.cpp | 5 ++ src/Access/ContextAccessParams.h | 3 + src/Interpreters/Context.cpp | 115 +++++++++++++++++------------ src/Interpreters/Context.h | 23 +++--- src/Interpreters/Session.cpp | 3 - 7 files changed, 107 insertions(+), 83 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index cb8f1a5a48e..51bb7794735 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -240,6 +240,7 @@ ContextAccess::~ContextAccess() enabled_settings.reset(); enabled_quota.reset(); enabled_row_policies.reset(); + row_policies_of_initial_user.reset(); access_with_implicit.reset(); access.reset(); roles_info.reset(); @@ -264,6 +265,12 @@ void ContextAccess::initialize() if (!params.user_id) throw Exception(ErrorCodes::LOGICAL_ERROR, "No user in current context, it's a bug"); + if (!params.initial_user.empty()) + { + if (auto initial_user_id = access_control->find(params.initial_user)) + row_policies_of_initial_user = access_control->tryGetDefaultRowPolicies(*initial_user_id); + } + subscription_for_user_change = access_control->subscribeForChanges( *params.user_id, [weak_ptr = weak_from_this()](const UUID &, const AccessEntityPtr & entity) @@ -331,8 +338,7 @@ void ContextAccess::setRolesInfo(const std::shared_ptr & assert(roles_info_); roles_info = roles_info_; - enabled_row_policies = access_control->getEnabledRowPolicies( - *params.user_id, roles_info->enabled_roles); + enabled_row_policies = access_control->getEnabledRowPolicies(*params.user_id, roles_info->enabled_roles); enabled_quota = access_control->getEnabledQuota( *params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key); @@ -399,21 +405,24 @@ std::shared_ptr ContextAccess::getRolesInfo() const return no_roles; } -std::shared_ptr ContextAccess::getEnabledRowPolicies() const +RowPolicyFilterPtr ContextAccess::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const { std::lock_guard lock{mutex}; - if (enabled_row_policies) - return enabled_row_policies; - static const auto no_row_policies = std::make_shared(); - return no_row_policies; -} -RowPolicyFilterPtr ContextAccess::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, RowPolicyFilterPtr combine_with_filter) const -{ - std::lock_guard lock{mutex}; + RowPolicyFilterPtr filter; if (enabled_row_policies) - return enabled_row_policies->getFilter(database, table_name, filter_type, combine_with_filter); - return combine_with_filter; + filter = enabled_row_policies->getFilter(database, table_name, filter_type); + + if (row_policies_of_initial_user) + { + /// Find and set extra row policies to be used based on `client_info.initial_user`, if the initial user exists. + /// TODO: we need a better solution here. It seems we should pass the initial row policy + /// because a shard is allowed to not have the initial user or it might be another user + /// with the same name. + filter = row_policies_of_initial_user->getFilter(database, table_name, filter_type, filter); + } + + return filter; } std::shared_ptr ContextAccess::getQuota() const diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 4c96ef5c11f..4bd67f8881b 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -53,12 +53,9 @@ public: /// Returns information about current and enabled roles. std::shared_ptr getRolesInfo() const; - /// Returns information about enabled row policies. - std::shared_ptr getEnabledRowPolicies() const; - /// Returns the row policy filter for a specified table. /// The function returns nullptr if there is no filter to apply. - RowPolicyFilterPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, RowPolicyFilterPtr combine_with_filter = {}) const; + RowPolicyFilterPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; /// Returns the quota to track resource consumption. std::shared_ptr getQuota() const; @@ -198,6 +195,7 @@ private: mutable std::shared_ptr access TSA_GUARDED_BY(mutex); mutable std::shared_ptr access_with_implicit TSA_GUARDED_BY(mutex); mutable std::shared_ptr enabled_row_policies TSA_GUARDED_BY(mutex); + mutable std::shared_ptr row_policies_of_initial_user TSA_GUARDED_BY(mutex); mutable std::shared_ptr enabled_quota TSA_GUARDED_BY(mutex); mutable std::shared_ptr enabled_settings TSA_GUARDED_BY(mutex); diff --git a/src/Access/ContextAccessParams.cpp b/src/Access/ContextAccessParams.cpp index 7963e83dddf..ec839a37b1a 100644 --- a/src/Access/ContextAccessParams.cpp +++ b/src/Access/ContextAccessParams.cpp @@ -27,6 +27,7 @@ ContextAccessParams::ContextAccessParams( , address(client_info_.current_address.host()) , forwarded_address(client_info_.getLastForwardedFor()) , quota_key(client_info_.quota_key) + , initial_user((client_info_.initial_user != client_info_.current_user) ? client_info_.initial_user : "") { } @@ -68,6 +69,8 @@ String ContextAccessParams::toString() const out << separator() << "forwarded_address = " << forwarded_address; if (!quota_key.empty()) out << separator() << "quota_key = " << quota_key; + if (!initial_user.empty()) + out << separator() << "initial_user = " << initial_user; return out.str(); } @@ -107,6 +110,7 @@ bool operator ==(const ContextAccessParams & left, const ContextAccessParams & r CONTEXT_ACCESS_PARAMS_EQUALS(address) CONTEXT_ACCESS_PARAMS_EQUALS(forwarded_address) CONTEXT_ACCESS_PARAMS_EQUALS(quota_key) + CONTEXT_ACCESS_PARAMS_EQUALS(initial_user) #undef CONTEXT_ACCESS_PARAMS_EQUALS @@ -158,6 +162,7 @@ bool operator <(const ContextAccessParams & left, const ContextAccessParams & ri CONTEXT_ACCESS_PARAMS_LESS(address) CONTEXT_ACCESS_PARAMS_LESS(forwarded_address) CONTEXT_ACCESS_PARAMS_LESS(quota_key) + CONTEXT_ACCESS_PARAMS_LESS(initial_user) #undef CONTEXT_ACCESS_PARAMS_LESS diff --git a/src/Access/ContextAccessParams.h b/src/Access/ContextAccessParams.h index 740ec997964..8b68fa44ed4 100644 --- a/src/Access/ContextAccessParams.h +++ b/src/Access/ContextAccessParams.h @@ -48,6 +48,9 @@ public: const String quota_key; + /// Initial user is used to combine row policies with. + const String initial_user; + /// Outputs `ContextAccessParams` to string for logging. String toString() const; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index abc33c1b8d4..5fae9374705 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1057,33 +1057,54 @@ ConfigurationPtr Context::getUsersConfig() return shared->users_config; } -void Context::setUser(const UUID & user_id_) +void Context::setUser(const UUID & user_id_, bool set_current_profiles_, bool set_current_roles_, bool set_current_database_) { + /// Prepare lists of user's profiles, constraints, settings, roles. + + std::shared_ptr user; + std::shared_ptr temp_access; + if (set_current_profiles_ || set_current_roles_ || set_current_database_) + { + std::optional params; + { + auto lock = getLock(); + params.emplace(ContextAccessParams{user_id_, /* full_access= */ false, /* use_default_roles = */ true, {}, settings, current_database, client_info}); + } + /// `temp_access` is used here only to extract information about the user, not to actually check access. + /// NOTE: AccessControl::getContextAccess() may require some IO work, so Context::getLock() must be unlocked while we're doing this. + temp_access = getAccessControl().getContextAccess(*params); + user = temp_access->getUser(); + } + + std::shared_ptr profiles; + if (set_current_profiles_) + profiles = temp_access->getDefaultProfileInfo(); + + std::optional> roles; + if (set_current_roles_) + roles = user->granted_roles.findGranted(user->default_roles); + + String database; + if (set_current_database_) + database = user->default_database; + + /// Apply user's profiles, constraints, settings, roles. auto lock = getLock(); - user_id = user_id_; + setUserID(user_id_); - ContextAccessParams params{ - user_id, - /* full_access= */ false, - /* use_default_roles = */ true, - /* current_roles = */ nullptr, - settings, - current_database, - client_info}; + if (profiles) + { + /// A profile can specify a value and a readonly constraint for same setting at the same time, + /// so we shouldn't check constraints here. + setCurrentProfiles(*profiles, /* check_constraints= */ false); + } - access = getAccessControl().getContextAccess(params); + if (roles) + setCurrentRoles(*roles); - auto user = access->getUser(); - - current_roles = std::make_shared>(user->granted_roles.findGranted(user->default_roles)); - - auto default_profile_info = access->getDefaultProfileInfo(); - settings_constraints_and_current_profiles = default_profile_info->getConstraintsAndProfileIDs(); - applySettingsChanges(default_profile_info->settings); - - if (!user->default_database.empty()) - setCurrentDatabase(user->default_database); + if (!database.empty()) + setCurrentDatabase(database); } std::shared_ptr Context::getUser() const @@ -1096,6 +1117,13 @@ String Context::getUserName() const return getAccess()->getUserName(); } +void Context::setUserID(const UUID & user_id_) +{ + auto lock = getLock(); + user_id = user_id_; + need_recalculate_access = true; +} + std::optional Context::getUserID() const { auto lock = getLock(); @@ -1113,9 +1141,10 @@ void Context::setQuotaKey(String quota_key_) void Context::setCurrentRoles(const std::vector & current_roles_) { auto lock = getLock(); - if (current_roles ? (*current_roles == current_roles_) : current_roles_.empty()) - return; - current_roles = std::make_shared>(current_roles_); + if (current_roles_.empty()) + current_roles = nullptr; + else + current_roles = std::make_shared>(current_roles_); need_recalculate_access = true; } @@ -1208,23 +1237,7 @@ std::shared_ptr Context::getAccess() const RowPolicyFilterPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const { - auto lock = getLock(); - RowPolicyFilterPtr row_filter_of_initial_user; - if (row_policies_of_initial_user) - row_filter_of_initial_user = row_policies_of_initial_user->getFilter(database, table_name, filter_type); - return getAccess()->getRowPolicyFilter(database, table_name, filter_type, row_filter_of_initial_user); -} - -void Context::enableRowPoliciesOfInitialUser() -{ - auto lock = getLock(); - row_policies_of_initial_user = nullptr; - if (client_info.initial_user == client_info.current_user) - return; - auto initial_user_id = getAccessControl().find(client_info.initial_user); - if (!initial_user_id) - return; - row_policies_of_initial_user = getAccessControl().tryGetDefaultRowPolicies(*initial_user_id); + return getAccess()->getRowPolicyFilter(database, table_name, filter_type); } @@ -1240,13 +1253,12 @@ std::optional Context::getQuotaUsage() const } -void Context::setCurrentProfile(const String & profile_name) +void Context::setCurrentProfile(const String & profile_name, bool check_constraints) { - auto lock = getLock(); try { UUID profile_id = getAccessControl().getID(profile_name); - setCurrentProfile(profile_id); + setCurrentProfile(profile_id, check_constraints); } catch (Exception & e) { @@ -1255,15 +1267,20 @@ void Context::setCurrentProfile(const String & profile_name) } } -void Context::setCurrentProfile(const UUID & profile_id) +void Context::setCurrentProfile(const UUID & profile_id, bool check_constraints) { - auto lock = getLock(); auto profile_info = getAccessControl().getSettingsProfileInfo(profile_id); - checkSettingsConstraints(profile_info->settings); - applySettingsChanges(profile_info->settings); - settings_constraints_and_current_profiles = profile_info->getConstraintsAndProfileIDs(settings_constraints_and_current_profiles); + setCurrentProfiles(*profile_info, check_constraints); } +void Context::setCurrentProfiles(const SettingsProfilesInfo & profiles_info, bool check_constraints) +{ + auto lock = getLock(); + if (check_constraints) + checkSettingsConstraints(profiles_info.settings); + applySettingsChanges(profiles_info.settings); + settings_constraints_and_current_profiles = profiles_info.getConstraintsAndProfileIDs(settings_constraints_and_current_profiles); +} std::vector Context::getCurrentProfiles() const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 2c32ad28d01..172f3818dfd 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -50,8 +50,8 @@ struct ContextSharedPart; class ContextAccess; struct User; using UserPtr = std::shared_ptr; +struct SettingsProfilesInfo; struct EnabledRolesInfo; -class EnabledRowPolicies; struct RowPolicyFilter; using RowPolicyFilterPtr = std::shared_ptr; class EnabledQuota; @@ -250,7 +250,6 @@ private: std::shared_ptr settings_constraints_and_current_profiles; mutable std::shared_ptr access; mutable bool need_recalculate_access = true; - std::shared_ptr row_policies_of_initial_user; String current_database; Settings settings; /// Setting for query execution. @@ -530,12 +529,14 @@ public: /// Sets the current user assuming that he/she is already authenticated. /// WARNING: This function doesn't check password! - void setUser(const UUID & user_id_); - + void setUser(const UUID & user_id_, bool set_current_profiles_ = true, bool set_current_roles_ = true, bool set_current_database_ = true); UserPtr getUser() const; - String getUserName() const; + + void setUserID(const UUID & user_id_); std::optional getUserID() const; + String getUserName() const; + void setQuotaKey(String quota_key_); void setCurrentRoles(const std::vector & current_roles_); @@ -544,8 +545,9 @@ public: boost::container::flat_set getEnabledRoles() const; std::shared_ptr getRolesInfo() const; - void setCurrentProfile(const String & profile_name); - void setCurrentProfile(const UUID & profile_id); + void setCurrentProfile(const String & profile_name, bool check_constraints = true); + void setCurrentProfile(const UUID & profile_id, bool check_constraints = true); + void setCurrentProfiles(const SettingsProfilesInfo & profiles_info, bool check_constraints = true); std::vector getCurrentProfiles() const; std::vector getEnabledProfiles() const; @@ -568,13 +570,6 @@ public: RowPolicyFilterPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; - /// Finds and sets extra row policies to be used based on `client_info.initial_user`, - /// if the initial user exists. - /// TODO: we need a better solution here. It seems we should pass the initial row policy - /// because a shard is allowed to not have the initial user or it might be another user - /// with the same name. - void enableRowPoliciesOfInitialUser(); - std::shared_ptr getQuota() const; std::optional getQuotaUsage() const; diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 64f7b4fc934..8571f20b91e 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -515,9 +515,6 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t res_client_info.initial_address = res_client_info.current_address; } - /// Sets that row policies of the initial user should be used too. - query_context->enableRowPoliciesOfInitialUser(); - /// Set user information for the new context: current profiles, roles, access rights. if (user_id && !query_context->getAccess()->tryGetUser()) query_context->setUser(*user_id); From 01cd7a1fd71d8568cfa42f8f3fd5a06fd0cfa059 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 17 Jul 2023 12:59:58 +0000 Subject: [PATCH 0687/2047] Better --- src/Interpreters/Context.cpp | 10 ---------- src/Interpreters/Context.h | 1 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- 3 files changed, 2 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1d274b752e3..e5cf0e89771 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2829,16 +2829,6 @@ std::map Context::getAuxiliaryZooKeepers() const } #if USE_ROCKSDB -MergeTreeMetadataCachePtr Context::getMergeTreeMetadataCache() const -{ - auto cache = tryGetMergeTreeMetadataCache(); - if (!cache) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Merge tree metadata cache is not initialized, please add config merge_tree_metadata_cache in config.xml and restart"); - return cache; -} - MergeTreeMetadataCachePtr Context::tryGetMergeTreeMetadataCache() const { return shared->merge_tree_metadata_cache; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 6cbb0e58911..43526fc5eb2 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -855,7 +855,6 @@ public: void setClientProtocolVersion(UInt64 version); #if USE_ROCKSDB - MergeTreeMetadataCachePtr getMergeTreeMetadataCache() const; MergeTreeMetadataCachePtr tryGetMergeTreeMetadataCache() const; #endif diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index b9591864869..ae63f33a1f7 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1660,8 +1660,8 @@ std::pair IMergeTreeDataPart::canRemovePart() const void IMergeTreeDataPart::initializePartMetadataManager() { #if USE_ROCKSDB - if (use_metadata_cache) - metadata_manager = std::make_shared(this, storage.getContext()->getMergeTreeMetadataCache()); + if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache(); metadata_cache && use_metadata_cache) + metadata_manager = std::make_shared(this, metadata_cache); else metadata_manager = std::make_shared(this); #else From 4da0782e553f90e2cdc742ad51ec30d1786dd422 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Mon, 17 Jul 2023 15:01:06 +0200 Subject: [PATCH 0688/2047] MaterializedMySQL: Add tests to parse db and table names from DDL --- .../MySQL/MaterializedMySQLSyncThread.cpp | 66 +------ .../gtest_try_parse_table_id_from_ddl.cpp | 185 ++++++++++++++++++ .../MySQL/tryParseTableIDFromDDL.cpp | 44 +++++ src/Databases/MySQL/tryParseTableIDFromDDL.h | 11 ++ 4 files changed, 245 insertions(+), 61 deletions(-) create mode 100644 src/Databases/MySQL/tests/gtest_try_parse_table_id_from_ddl.cpp create mode 100644 src/Databases/MySQL/tryParseTableIDFromDDL.cpp create mode 100644 src/Databases/MySQL/tryParseTableIDFromDDL.h diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index a01ab2a15a8..780619ff079 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -3,6 +3,7 @@ #if USE_MYSQL #include +#include #include #include #include @@ -151,61 +152,6 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S } } -static std::tuple tryExtractTableNameFromDDL(const String & ddl) -{ - String table_name; - String database_name; - if (ddl.empty()) return std::make_tuple(database_name, table_name); - - bool parse_failed = false; - Tokens tokens(ddl.data(), ddl.data() + ddl.size()); - IParser::Pos pos(tokens, 0); - Expected expected; - ASTPtr res; - ASTPtr table; - if (ParserKeyword("CREATE TEMPORARY TABLE").ignore(pos, expected) || ParserKeyword("CREATE TABLE").ignore(pos, expected)) - { - ParserKeyword("IF NOT EXISTS").ignore(pos, expected); - if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) - parse_failed = true; - } - else if (ParserKeyword("ALTER TABLE").ignore(pos, expected)) - { - if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) - parse_failed = true; - } - else if (ParserKeyword("DROP TABLE").ignore(pos, expected) || ParserKeyword("DROP TEMPORARY TABLE").ignore(pos, expected)) - { - ParserKeyword("IF EXISTS").ignore(pos, expected); - if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) - parse_failed = true; - } - else if (ParserKeyword("TRUNCATE").ignore(pos, expected)) - { - ParserKeyword("TABLE").ignore(pos, expected); - if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) - parse_failed = true; - } - else if (ParserKeyword("RENAME TABLE").ignore(pos, expected)) - { - if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) - parse_failed = true; - } - else - { - parse_failed = true; - } - if (!parse_failed) - { - if (auto table_id = table->as()->getTableId()) - { - database_name = table_id.database_name; - table_name = table_id.table_name; - } - } - return std::make_tuple(database_name, table_name); -} - MaterializedMySQLSyncThread::MaterializedMySQLSyncThread( ContextPtr context_, const String & database_name_, @@ -868,14 +814,12 @@ void MaterializedMySQLSyncThread::executeDDLAtomic(const QueryEvent & query_even String query = query_event.query; if (!materialized_tables_list.empty()) { - auto [ddl_database_name, ddl_table_name] = tryExtractTableNameFromDDL(query_event.query); - - if (!ddl_table_name.empty()) + auto table_id = tryParseTableIDFromDDL(query, query_event.schema); + if (!table_id.table_name.empty()) { - ddl_database_name = ddl_database_name.empty() ? query_event.schema: ddl_database_name; - if (ddl_database_name != mysql_database_name || !materialized_tables_list.contains(ddl_table_name)) + if (table_id.database_name != mysql_database_name || !materialized_tables_list.contains(table_id.table_name)) { - LOG_DEBUG(log, "Skip MySQL DDL: \n {}", query_event.query); + LOG_DEBUG(log, "Skip MySQL DDL for {}.{}:\n{}", table_id.database_name, table_id.table_name, query); return; } } diff --git a/src/Databases/MySQL/tests/gtest_try_parse_table_id_from_ddl.cpp b/src/Databases/MySQL/tests/gtest_try_parse_table_id_from_ddl.cpp new file mode 100644 index 00000000000..5b60262930f --- /dev/null +++ b/src/Databases/MySQL/tests/gtest_try_parse_table_id_from_ddl.cpp @@ -0,0 +1,185 @@ +#include "config.h" + +#include + +#include + +using namespace DB; + +struct ParseTableIDFromDDLTestCase +{ + String query; + String database_name; + String table_name; + + ParseTableIDFromDDLTestCase( + const String & query_, + const String & database_name_, + const String & table_name_) + : query(query_) + , database_name(database_name_) + , table_name(table_name_) + { + } +}; + +std::ostream & operator<<(std::ostream & ostr, const ParseTableIDFromDDLTestCase & test_case) +{ + return ostr << '"' << test_case.query << "\" extracts `" << test_case.database_name << "`.`" << test_case.table_name << "`"; +} + +class ParseTableIDFromDDLTest : public ::testing::TestWithParam +{ +}; + +TEST_P(ParseTableIDFromDDLTest, parse) +{ + const auto & [query, expected_database_name, expected_table_name] = GetParam(); + auto table_id = tryParseTableIDFromDDL(query, "default"); + EXPECT_EQ(expected_database_name, table_id.database_name); + EXPECT_EQ(expected_table_name, table_id.table_name); +} + +INSTANTIATE_TEST_SUITE_P(MaterializedMySQL, ParseTableIDFromDDLTest, ::testing::ValuesIn(std::initializer_list{ + { + "SELECT * FROM db.table", + "", + "" + }, + { + "CREATE TEMPORARY TABLE db.table", + "db", + "table" + }, + { + "CREATE TEMPORARY TABLE IF NOT EXISTS db.table", + "db", + "table" + }, + { + "CREATE TEMPORARY TABLE table", + "default", + "table" + }, + { + "CREATE TEMPORARY TABLE IF NOT EXISTS table", + "default", + "table" + }, + { + "CREATE TABLE db.table", + "db", + "table" + }, + { + "CREATE TABLE IF NOT EXISTS db.table", + "db", + "table" + }, + { + "CREATE TABLE table", + "default", + "table" + }, + { + "CREATE TABLE IF NOT EXISTS table", + "default", + "table" + }, + { + "ALTER TABLE db.table", + "db", + "table" + }, + { + "ALTER TABLE table", + "default", + "table" + }, + { + "DROP TABLE db.table", + "db", + "table" + }, + { + "DROP TABLE IF EXISTS db.table", + "db", + "table" + }, + { + "DROP TABLE table", + "default", + "table" + }, + { + "DROP TABLE IF EXISTS table", + "default", + "table" + }, + { + "DROP TEMPORARY TABLE db.table", + "db", + "table" + }, + { + "DROP TEMPORARY TABLE IF EXISTS db.table", + "db", + "table" + }, + { + "DROP TEMPORARY TABLE table", + "default", + "table" + }, + { + "DROP TEMPORARY TABLE IF EXISTS table", + "default", + "table" + }, + { + "TRUNCATE db.table", + "db", + "table" + }, + { + "TRUNCATE TABLE db.table", + "db", + "table" + }, + { + "TRUNCATE table1", + "default", + "table1" + }, + { + "TRUNCATE TABLE table", + "default", + "table" + }, + { + "RENAME TABLE db.table", + "db", + "table" + }, + { + "RENAME TABLE table", + "default", + "table" + }, + { + "DROP DATABASE db", + "", + "" + }, + { + "DROP DATA`BASE db", + "", + "" + }, + { + "NOT A SQL", + "", + "" + }, + +})); diff --git a/src/Databases/MySQL/tryParseTableIDFromDDL.cpp b/src/Databases/MySQL/tryParseTableIDFromDDL.cpp new file mode 100644 index 00000000000..a01eb311450 --- /dev/null +++ b/src/Databases/MySQL/tryParseTableIDFromDDL.cpp @@ -0,0 +1,44 @@ +#include +#include +#include +#include + +namespace DB +{ + +StorageID tryParseTableIDFromDDL(const String & query, const String & default_database_name) +{ + bool is_ddl = false; + Tokens tokens(query.data(), query.data() + query.size()); + IParser::Pos pos(tokens, 0); + Expected expected; + if (ParserKeyword("CREATE TEMPORARY TABLE").ignore(pos, expected) || ParserKeyword("CREATE TABLE").ignore(pos, expected)) + { + ParserKeyword("IF NOT EXISTS").ignore(pos, expected); + is_ddl = true; + } + else if (ParserKeyword("ALTER TABLE").ignore(pos, expected) || ParserKeyword("RENAME TABLE").ignore(pos, expected)) + { + is_ddl = true; + } + else if (ParserKeyword("DROP TABLE").ignore(pos, expected) || ParserKeyword("DROP TEMPORARY TABLE").ignore(pos, expected)) + { + ParserKeyword("IF EXISTS").ignore(pos, expected); + is_ddl = true; + } + else if (ParserKeyword("TRUNCATE").ignore(pos, expected)) + { + ParserKeyword("TABLE").ignore(pos, expected); + is_ddl = true; + } + + ASTPtr table; + if (!is_ddl || !ParserCompoundIdentifier(true).parse(pos, table, expected)) + return StorageID::createEmpty(); + auto table_id = table->as()->getTableId(); + if (table_id.database_name.empty()) + table_id.database_name = default_database_name; + return table_id; +} + +} diff --git a/src/Databases/MySQL/tryParseTableIDFromDDL.h b/src/Databases/MySQL/tryParseTableIDFromDDL.h new file mode 100644 index 00000000000..5af733f5e99 --- /dev/null +++ b/src/Databases/MySQL/tryParseTableIDFromDDL.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include + +namespace DB +{ + +StorageID tryParseTableIDFromDDL(const String & query, const String & default_database_name); + +} From 68751b88f3f3b15f5469949000ce9387cff3abdd Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 17 Jul 2023 15:01:48 +0200 Subject: [PATCH 0689/2047] change comments --- src/Processors/Chunk.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 7969cc7cceb..f50e45db644 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -114,7 +114,9 @@ private: using Chunks = std::vector; -/// ChunkOffsets marks offsets of different sub-chunks, which will be used by async inserts. +/// AsyncInsert needs two kinds of information: +/// - offsets of different sub-chunks +/// - tokens of different sub-chunks, which are assigned by setting `insert_deduplication_token`. class AsyncInsertInfo : public ChunkInfo { public: From 9c666552883bff611f6c15a2e7bcf3af3e0127a3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 15:01:58 +0200 Subject: [PATCH 0690/2047] Fix tests --- .../test.py | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_1/test.py b/tests/integration/test_postgresql_replica_database_engine_1/test.py index 1eb2efc73a5..57ee920d49c 100644 --- a/tests/integration/test_postgresql_replica_database_engine_1/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_1/test.py @@ -380,9 +380,9 @@ def test_table_schema_changes(started_cluster): instance.query(f"SELECT count() FROM test_database.{altered_table}") ) - cursor.execute(f"ALTER TABLE {altered_table} DROP COLUMN value2") + pg_manager.execute(f"ALTER TABLE {altered_table} DROP COLUMN value2") for i in range(NUM_TABLES): - cursor.execute(f"INSERT INTO postgresql_replica_{i} VALUES (50, {i}, {i})") + pg_manager.execute(f"INSERT INTO postgresql_replica_{i} VALUES (50, {i}, {i})") assert instance.wait_for_log_line( f"Table postgresql_replica_{altered_idx} is skipped from replication stream" @@ -556,9 +556,8 @@ def test_multiple_databases(started_cluster): port=started_cluster.postgres_port, database=False, ) - cursor = conn.cursor() - pg_manager.create_postgres_db(cursor, "postgres_database_1") - pg_manager.create_postgres_db(cursor, "postgres_database_2") + pg_manager.create_postgres_db("postgres_database_1") + pg_manager.create_postgres_db("postgres_database_2") conn1 = get_postgres_conn( ip=started_cluster.postgres_ip, @@ -577,15 +576,13 @@ def test_multiple_databases(started_cluster): cursor2 = conn2.cursor() pg_manager.create_clickhouse_postgres_db( - cluster.postgres_ip, - cluster.postgres_port, "postgres_database_1", + "", "postgres_database_1", ) pg_manager.create_clickhouse_postgres_db( - cluster.postgres_ip, - cluster.postgres_port, "postgres_database_2", + "", "postgres_database_2", ) From 815a3857de74b92d3071128a6b5fbc5cb0a53c93 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 7 Jul 2023 12:49:50 +0200 Subject: [PATCH 0691/2047] Remove non-const function Context::getClientInfo(). --- programs/client/Client.cpp | 11 +- programs/local/LocalServer.cpp | 5 +- src/Databases/DatabaseReplicated.cpp | 4 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 10 +- .../ClusterProxy/executeQuery.cpp | 2 +- src/Interpreters/Context.cpp | 123 +++++++++++++++ src/Interpreters/Context.h | 26 +++- src/Interpreters/DDLTask.cpp | 6 +- src/Interpreters/DDLWorker.cpp | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 4 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/Session.cpp | 141 +++++++++++++++--- src/Interpreters/Session.h | 17 ++- src/Interpreters/executeQuery.cpp | 5 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 12 +- src/Server/GRPCServer.cpp | 2 +- src/Server/HTTPHandler.cpp | 19 +-- src/Server/MySQLHandler.cpp | 2 +- src/Server/PostgreSQLHandler.cpp | 2 +- src/Server/TCPHandler.cpp | 25 +--- src/Storages/Distributed/DistributedSink.cpp | 2 +- src/Storages/StorageDistributed.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 2 +- 25 files changed, 335 insertions(+), 97 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index f791c39bad1..e1a33231592 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1173,12 +1173,12 @@ void Client::processOptions(const OptionsDescription & options_description, { String traceparent = options["opentelemetry-traceparent"].as(); String error; - if (!global_context->getClientInfo().client_trace_context.parseTraceparentHeader(traceparent, error)) + if (!global_context->getClientTraceContext().parseTraceparentHeader(traceparent, error)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse OpenTelemetry traceparent '{}': {}", traceparent, error); } if (options.count("opentelemetry-tracestate")) - global_context->getClientInfo().client_trace_context.tracestate = options["opentelemetry-tracestate"].as(); + global_context->getClientTraceContext().tracestate = options["opentelemetry-tracestate"].as(); } @@ -1238,10 +1238,9 @@ void Client::processConfig() global_context->getSettingsRef().max_insert_block_size); } - ClientInfo & client_info = global_context->getClientInfo(); - client_info.setInitialQuery(); - client_info.quota_key = config().getString("quota_key", ""); - client_info.query_kind = query_kind; + global_context->setQueryKindInitial(); + global_context->setQuotaClientKey(config().getString("quota_key", "")); + global_context->setQueryKind(query_kind); } diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 96924e3c8d9..3c2a8ae3152 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -737,9 +737,8 @@ void LocalServer::processConfig() for (const auto & [key, value] : prompt_substitutions) boost::replace_all(prompt_by_server_display_name, "{" + key + "}", value); - ClientInfo & client_info = global_context->getClientInfo(); - client_info.setInitialQuery(); - client_info.query_kind = query_kind; + global_context->setQueryKindInitial(); + global_context->setQueryKind(query_kind); } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 661afc6bf1f..25c23e2be17 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -814,8 +814,8 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep { auto query_context = Context::createCopy(getContext()); query_context->makeQueryContext(); - query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - query_context->getClientInfo().is_replicated_database_internal = true; + query_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); + query_context->setQueryKindReplicatedDatabaseInternal(); query_context->setCurrentDatabase(getDatabaseName()); query_context->setCurrentQueryId(""); auto txn = std::make_shared(current_zookeeper, zookeeper_path, false, ""); diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index a01ab2a15a8..379e6ef5097 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -59,7 +59,7 @@ static ContextMutablePtr createQueryContext(ContextPtr context) query_context->setSettings(new_query_settings); query_context->setInternalQuery(true); - query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + query_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); query_context->setCurrentQueryId(""); // generate random query_id return query_context; } diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 0da762699d2..6081919a120 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -421,12 +421,10 @@ try auto insert_query_id = insert_context->getCurrentQueryId(); auto query_start_time = std::chrono::system_clock::now(); Stopwatch start_watch{CLOCK_MONOTONIC}; - ClientInfo & client_info = insert_context->getClientInfo(); - client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; - client_info.initial_query_start_time = timeInSeconds(query_start_time); - client_info.initial_query_start_time_microseconds = timeInMicroseconds(query_start_time); - client_info.current_query_id = insert_query_id; - client_info.initial_query_id = insert_query_id; + insert_context->setQueryKind(ClientInfo::QueryKind::INITIAL_QUERY); + insert_context->setInitialQueryStartTime(query_start_time); + insert_context->setCurrentQueryId(insert_query_id); + insert_context->setInitialQueryId(insert_query_id); size_t log_queries_cut_to_length = insert_context->getSettingsRef().log_queries_cut_to_length; String query_for_logging = insert_query.hasSecretParts() ? insert_query.formatForLogging(log_queries_cut_to_length) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index e2f1dfe8ba7..3dea52faf46 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -171,7 +171,7 @@ void executeQuery( SelectStreamFactory::Shards remote_shards; auto new_context = updateSettingsForCluster(*query_info.getCluster(), context, settings, main_table, &query_info, log); - new_context->getClientInfo().distributed_depth += 1; + new_context->increaseDistributedDepth(); size_t shards = query_info.getCluster()->getShardCount(); for (const auto & shard_info : query_info.getCluster()->getShardsInfo()) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5fae9374705..c097eeb87f1 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3850,6 +3850,129 @@ void Context::resetInputCallbacks() } +void Context::setClientInfo(const ClientInfo & client_info_) +{ + client_info = client_info_; + need_recalculate_access = true; +} + +void Context::setClientName(const String & client_name) +{ + client_info.client_name = client_name; +} + +void Context::setClientInterface(ClientInfo::Interface interface) +{ + client_info.interface = interface; + need_recalculate_access = true; +} + +void Context::setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +{ + client_info.client_version_major = client_version_major; + client_info.client_version_minor = client_version_minor; + client_info.client_version_patch = client_version_patch; + client_info.client_tcp_protocol_version = client_tcp_protocol_version; +} + +void Context::setClientConnectionId(uint32_t connection_id_) +{ + client_info.connection_id = connection_id_; +} + +void Context::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +{ + client_info.http_method = http_method; + client_info.http_user_agent = http_user_agent; + client_info.http_referer = http_referer; + need_recalculate_access = true; +} + +void Context::setForwardedFor(const String & forwarded_for) +{ + client_info.forwarded_for = forwarded_for; + need_recalculate_access = true; +} + +void Context::setQueryKind(ClientInfo::QueryKind query_kind) +{ + client_info.query_kind = query_kind; +} + +void Context::setQueryKindInitial() +{ + /// TODO: Try to combine this function with setQueryKind(). + client_info.setInitialQuery(); +} + +void Context::setQueryKindReplicatedDatabaseInternal() +{ + /// TODO: Try to combine this function with setQueryKind(). + client_info.is_replicated_database_internal = true; +} + +void Context::setCurrentUserName(const String & current_user_name) +{ + /// TODO: Try to combine this function with setUser(). + client_info.current_user = current_user_name; + need_recalculate_access = true; +} + +void Context::setCurrentAddress(const Poco::Net::SocketAddress & current_address) +{ + client_info.current_address = current_address; + need_recalculate_access = true; +} + +void Context::setInitialUserName(const String & initial_user_name) +{ + client_info.initial_user = initial_user_name; + need_recalculate_access = true; +} + +void Context::setInitialAddress(const Poco::Net::SocketAddress & initial_address) +{ + client_info.initial_address = initial_address; +} + +void Context::setInitialQueryId(const String & initial_query_id) +{ + client_info.initial_query_id = initial_query_id; +} + +void Context::setInitialQueryStartTime(std::chrono::time_point initial_query_start_time) +{ + client_info.initial_query_start_time = timeInSeconds(initial_query_start_time); + client_info.initial_query_start_time_microseconds = timeInMicroseconds(initial_query_start_time); +} + +void Context::setQuotaClientKey(const String & quota_key_) +{ + client_info.quota_key = quota_key_; + need_recalculate_access = true; +} + +void Context::setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +{ + client_info.connection_client_version_major = client_version_major; + client_info.connection_client_version_minor = client_version_minor; + client_info.connection_client_version_patch = client_version_patch; + client_info.connection_tcp_protocol_version = client_tcp_protocol_version; +} + +void Context::setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica) +{ + client_info.collaborate_with_initiator = collaborate_with_initiator; + client_info.count_participating_replicas = all_replicas_count; + client_info.number_of_current_replica = number_of_current_replica; +} + +void Context::increaseDistributedDepth() +{ + ++client_info.distributed_depth; +} + + StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where) const { if (storage_id.uuid != UUIDHelpers::Nil) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 172f3818dfd..afc4bfde6a8 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -593,9 +593,33 @@ public: InputBlocksReader getInputBlocksReaderCallback() const; void resetInputCallbacks(); - ClientInfo & getClientInfo() { return client_info; } + /// Returns information about the client executing a query. const ClientInfo & getClientInfo() const { return client_info; } + /// Modify stored in the context information about the client executing a query. + void setClientInfo(const ClientInfo & client_info_); + void setClientName(const String & client_name); + void setClientInterface(ClientInfo::Interface interface); + void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); + void setClientConnectionId(uint32_t connection_id); + void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setForwardedFor(const String & forwarded_for); + void setQueryKind(ClientInfo::QueryKind query_kind); + void setQueryKindInitial(); + void setQueryKindReplicatedDatabaseInternal(); + void setCurrentUserName(const String & current_user_name); + void setCurrentAddress(const Poco::Net::SocketAddress & current_address); + void setInitialUserName(const String & initial_user_name); + void setInitialAddress(const Poco::Net::SocketAddress & initial_address); + void setInitialQueryId(const String & initial_query_id); + void setInitialQueryStartTime(std::chrono::time_point initial_query_start_time); + void setQuotaClientKey(const String & quota_key); + void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); + void setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica); + void increaseDistributedDepth(); + const OpenTelemetry::TracingContext & getClientTraceContext() const { return client_info.client_trace_context; } + OpenTelemetry::TracingContext & getClientTraceContext() { return client_info.client_trace_context; } + enum StorageNamespace { ResolveGlobal = 1u, /// Database name must be specified diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index b24856a6146..4e684f5899f 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -199,7 +199,7 @@ ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const Z auto query_context = Context::createCopy(from_context); query_context->makeQueryContext(); query_context->setCurrentQueryId(""); // generate random query_id - query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + query_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); if (entry.settings) query_context->applySettingsChanges(*entry.settings); return query_context; @@ -439,8 +439,8 @@ void DatabaseReplicatedTask::parseQueryFromEntry(ContextPtr context) ContextMutablePtr DatabaseReplicatedTask::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper) { auto query_context = DDLTaskBase::makeQueryContext(from_context, zookeeper); - query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - query_context->getClientInfo().is_replicated_database_internal = true; + query_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); + query_context->setQueryKindReplicatedDatabaseInternal(); query_context->setCurrentDatabase(database->getDatabaseName()); auto txn = std::make_shared(zookeeper, database->zookeeper_path, is_initial_query, entry_path); diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 81c78000ac3..193bb5b6ab0 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -476,7 +476,7 @@ bool DDLWorker::tryExecuteQuery(DDLTaskBase & task, const ZooKeeperPtr & zookeep query_context->setSetting("implicit_transaction", Field{0}); } - query_context->getClientInfo().initial_query_id = task.entry.initial_query_id; + query_context->setInitialQueryId(task.entry.initial_query_id); if (!task.is_initial_query) query_scope.emplace(query_context); diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 0beb4492aef..616cf80a446 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -451,11 +451,11 @@ void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr auto drop_context = Context::createCopy(global_context); if (ignore_sync_setting) drop_context->setSetting("database_atomic_wait_for_drop_and_detach_synchronously", false); - drop_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + drop_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); if (auto txn = current_context->getZooKeeperMetadataTransaction()) { /// For Replicated database - drop_context->getClientInfo().is_replicated_database_internal = true; + drop_context->setQueryKindReplicatedDatabaseInternal(); drop_context->setQueryContext(std::const_pointer_cast(current_context)); drop_context->initZooKeeperMetadataTransaction(txn, true); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 32812151b59..d07a6521544 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3183,7 +3183,7 @@ void InterpreterSelectQuery::initSettings() if (query.settings()) InterpreterSetQuery(query.settings(), context).executeForCurrentContext(options.ignore_setting_constraints); - auto & client_info = context->getClientInfo(); + const auto & client_info = context->getClientInfo(); auto min_major = DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; auto min_minor = DBMS_MIN_MINOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 8571f20b91e..97b056cfc32 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -299,7 +299,10 @@ Session::~Session() if (notified_session_log_about_login) { if (auto session_log = getSessionLog()) + { + /// TODO: We have to ensure that the same info is added to the session log on a LoginSuccess event and on the corresponding Logout event. session_log->addLogOut(auth_id, user, getClientInfo()); + } } } @@ -368,17 +371,117 @@ void Session::onAuthenticationFailure(const std::optional & user_name, c } } -ClientInfo & Session::getClientInfo() -{ - /// FIXME it may produce different info for LoginSuccess and the corresponding Logout entries in the session log - return session_context ? session_context->getClientInfo() : *prepared_client_info; -} - const ClientInfo & Session::getClientInfo() const { return session_context ? session_context->getClientInfo() : *prepared_client_info; } +void Session::setClientInfo(const ClientInfo & client_info) +{ + if (session_context) + session_context->setClientInfo(client_info); + else + prepared_client_info = client_info; +} + +void Session::setClientName(const String & client_name) +{ + if (session_context) + session_context->setClientName(client_name); + else + prepared_client_info->client_name = client_name; +} + +void Session::setClientInterface(ClientInfo::Interface interface) +{ + if (session_context) + session_context->setClientInterface(interface); + else + prepared_client_info->interface = interface; +} + +void Session::setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +{ + if (session_context) + { + session_context->setClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); + } + else + { + prepared_client_info->client_version_major = client_version_major; + prepared_client_info->client_version_minor = client_version_minor; + prepared_client_info->client_version_patch = client_version_patch; + prepared_client_info->client_tcp_protocol_version = client_tcp_protocol_version; + } +} + +void Session::setClientConnectionId(uint32_t connection_id) +{ + if (session_context) + session_context->setClientConnectionId(connection_id); + else + prepared_client_info->connection_id = connection_id; +} + +void Session::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +{ + if (session_context) + { + session_context->setHttpClientInfo(http_method, http_user_agent, http_referer); + } + else + { + prepared_client_info->http_method = http_method; + prepared_client_info->http_user_agent = http_user_agent; + prepared_client_info->http_referer = http_referer; + } +} + +void Session::setForwardedFor(const String & forwarded_for) +{ + if (session_context) + session_context->setForwardedFor(forwarded_for); + else + prepared_client_info->forwarded_for = forwarded_for; +} + +void Session::setQuotaClientKey(const String & quota_key) +{ + if (session_context) + session_context->setQuotaClientKey(quota_key); + else + prepared_client_info->quota_key = quota_key; +} + +void Session::setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version) +{ + if (session_context) + { + session_context->setConnectionClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); + } + else + { + prepared_client_info->connection_client_version_major = client_version_major; + prepared_client_info->connection_client_version_minor = client_version_minor; + prepared_client_info->connection_client_version_patch = client_version_patch; + prepared_client_info->connection_tcp_protocol_version = client_tcp_protocol_version; + } +} + +const OpenTelemetry::TracingContext & Session::getClientTraceContext() const +{ + if (session_context) + return session_context->getClientTraceContext(); + return prepared_client_info->client_trace_context; +} + +OpenTelemetry::TracingContext & Session::getClientTraceContext() +{ + if (session_context) + return session_context->getClientTraceContext(); + return prepared_client_info->client_trace_context; +} + ContextMutablePtr Session::makeSessionContext() { if (session_context) @@ -396,8 +499,7 @@ ContextMutablePtr Session::makeSessionContext() new_session_context->makeSessionContext(); /// Copy prepared client info to the new session context. - auto & res_client_info = new_session_context->getClientInfo(); - res_client_info = std::move(prepared_client_info).value(); + new_session_context->setClientInfo(*prepared_client_info); prepared_client_info.reset(); /// Set user information for the new context: current profiles, roles, access rights. @@ -436,8 +538,7 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: /// Copy prepared client info to the session context, no matter it's been just created or not. /// If we continue using a previously created session context found by session ID /// it's necessary to replace the client info in it anyway, because it contains actual connection information (client address, etc.) - auto & res_client_info = new_session_context->getClientInfo(); - res_client_info = std::move(prepared_client_info).value(); + new_session_context->setClientInfo(*prepared_client_info); prepared_client_info.reset(); /// Set user information for the new context: current profiles, roles, access rights. @@ -492,27 +593,26 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t } /// Copy the specified client info to the new query context. - auto & res_client_info = query_context->getClientInfo(); if (client_info_to_move) - res_client_info = std::move(*client_info_to_move); + query_context->setClientInfo(*client_info_to_move); else if (client_info_to_copy && (client_info_to_copy != &getClientInfo())) - res_client_info = *client_info_to_copy; + query_context->setClientInfo(*client_info_to_copy); /// Copy current user's name and address if it was authenticated after query_client_info was initialized. if (prepared_client_info && !prepared_client_info->current_user.empty()) { - res_client_info.current_user = prepared_client_info->current_user; - res_client_info.current_address = prepared_client_info->current_address; + query_context->setCurrentUserName(prepared_client_info->current_user); + query_context->setCurrentAddress(prepared_client_info->current_address); } /// Set parameters of initial query. - if (res_client_info.query_kind == ClientInfo::QueryKind::NO_QUERY) - res_client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + if (query_context->getClientInfo().query_kind == ClientInfo::QueryKind::NO_QUERY) + query_context->setQueryKind(ClientInfo::QueryKind::INITIAL_QUERY); - if (res_client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY) + if (query_context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) { - res_client_info.initial_user = res_client_info.current_user; - res_client_info.initial_address = res_client_info.current_address; + query_context->setInitialUserName(query_context->getClientInfo().current_user); + query_context->setInitialAddress(query_context->getClientInfo().current_address); } /// Set user information for the new context: current profiles, roles, access rights. @@ -563,4 +663,3 @@ void Session::closeSession(const String & session_id) } } - diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index d7c06a60464..36f811ccd24 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -54,10 +54,23 @@ public: /// Writes a row about login failure into session log (if enabled) void onAuthenticationFailure(const std::optional & user_name, const Poco::Net::SocketAddress & address_, const Exception & e); - /// Returns a reference to session ClientInfo. - ClientInfo & getClientInfo(); + /// Returns a reference to the session's ClientInfo. const ClientInfo & getClientInfo() const; + /// Modify the session's ClientInfo. + void setClientInfo(const ClientInfo & client_info); + void setClientName(const String & client_name); + void setClientInterface(ClientInfo::Interface interface); + void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); + void setClientConnectionId(uint32_t connection_id); + void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setForwardedFor(const String & forwarded_for); + void setQuotaClientKey(const String & quota_key); + void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); + + const OpenTelemetry::TracingContext & getClientTraceContext() const; + OpenTelemetry::TracingContext & getClientTraceContext(); + /// Makes a session context, can be used one or zero times. /// The function also assigns an user to this context. ContextMutablePtr makeSessionContext(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 4b76d20f31d..2c74039463e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -655,7 +655,7 @@ static std::tuple executeQueryImpl( /// the value passed by the client Stopwatch start_watch{CLOCK_MONOTONIC}; - auto & client_info = context->getClientInfo(); + const auto & client_info = context->getClientInfo(); if (!internal) { @@ -667,8 +667,7 @@ static std::tuple executeQueryImpl( // On the other hand, if it's initialized then take it as the start of the query if (client_info.initial_query_start_time == 0) { - client_info.initial_query_start_time = timeInSeconds(query_start_time); - client_info.initial_query_start_time_microseconds = timeInMicroseconds(query_start_time); + context->setInitialQueryStartTime(query_start_time); } else { diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 9b9cc221ca8..b251eec2d28 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -72,14 +72,10 @@ std::unique_ptr createLocalPlan( if (coordinator) { new_context->parallel_reading_coordinator = coordinator; - new_context->getClientInfo().interface = ClientInfo::Interface::LOCAL; - new_context->getClientInfo().collaborate_with_initiator = true; - new_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - new_context->getClientInfo().count_participating_replicas = replica_count; - new_context->getClientInfo().number_of_current_replica = replica_num; - new_context->getClientInfo().connection_client_version_major = DBMS_VERSION_MAJOR; - new_context->getClientInfo().connection_client_version_minor = DBMS_VERSION_MINOR; - new_context->getClientInfo().connection_tcp_protocol_version = DBMS_TCP_PROTOCOL_VERSION; + new_context->setClientInterface(ClientInfo::Interface::LOCAL); + new_context->setQueryKind(ClientInfo::QueryKind::SECONDARY_QUERY); + new_context->setReplicaInfo(true, replica_count, replica_num); + new_context->setConnectionClientVersion(DBMS_VERSION_MAJOR, DBMS_VERSION_MINOR, DBMS_VERSION_PATCH, DBMS_TCP_PROTOCOL_VERSION); new_context->setParallelReplicasGroupUUID(group_uuid); new_context->setMergeTreeAllRangesCallback([coordinator](InitialAllRangesAnnouncement announcement) { diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index bf9ba20a5cf..67d30012b0e 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -798,7 +798,7 @@ namespace /// Authentication. session.emplace(iserver.context(), ClientInfo::Interface::GRPC); session->authenticate(user, password, user_address); - session->getClientInfo().quota_key = quota_key; + session->setQuotaClientKey(quota_key); ClientInfo client_info = session->getClientInfo(); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index f7cdb905710..069670c84a5 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -474,7 +474,6 @@ bool HTTPHandler::authenticateUser( } /// Set client info. It will be used for quota accounting parameters in 'setUser' method. - ClientInfo & client_info = session->getClientInfo(); ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; if (request.getMethod() == HTTPServerRequest::HTTP_GET) @@ -482,15 +481,13 @@ bool HTTPHandler::authenticateUser( else if (request.getMethod() == HTTPServerRequest::HTTP_POST) http_method = ClientInfo::HTTPMethod::POST; - client_info.http_method = http_method; - client_info.http_user_agent = request.get("User-Agent", ""); - client_info.http_referer = request.get("Referer", ""); - client_info.forwarded_for = request.get("X-Forwarded-For", ""); - client_info.quota_key = quota_key; + session->setHttpClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", "")); + session->setForwardedFor(request.get("X-Forwarded-For", "")); + session->setQuotaClientKey(quota_key); /// Extract the last entry from comma separated list of forwarded_for addresses. /// Only the last proxy can be trusted (if any). - String forwarded_address = client_info.getLastForwardedFor(); + String forwarded_address = session->getClientInfo().getLastForwardedFor(); try { if (!forwarded_address.empty() && server.config().getBool("auth_use_forwarded_address", false)) @@ -988,22 +985,22 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse } // Parse the OpenTelemetry traceparent header. - ClientInfo& client_info = session->getClientInfo(); + auto & client_trace_context = session->getClientTraceContext(); if (request.has("traceparent")) { std::string opentelemetry_traceparent = request.get("traceparent"); std::string error; - if (!client_info.client_trace_context.parseTraceparentHeader(opentelemetry_traceparent, error)) + if (!client_trace_context.parseTraceparentHeader(opentelemetry_traceparent, error)) { LOG_DEBUG(log, "Failed to parse OpenTelemetry traceparent header '{}': {}", opentelemetry_traceparent, error); } - client_info.client_trace_context.tracestate = request.get("tracestate", ""); + client_trace_context.tracestate = request.get("tracestate", ""); } // Setup tracing context for this thread auto context = session->sessionOrGlobalContext(); thread_trace_context = std::make_unique("HTTPHandler", - client_info.client_trace_context, + client_trace_context, context->getSettingsRef(), context->getOpenTelemetrySpanLog()); thread_trace_context->root_span.kind = OpenTelemetry::SERVER; diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 7318b0ad89b..f98b86e6cf8 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -94,7 +94,7 @@ void MySQLHandler::run() session = std::make_unique(server.context(), ClientInfo::Interface::MYSQL); SCOPE_EXIT({ session.reset(); }); - session->getClientInfo().connection_id = connection_id; + session->setClientConnectionId(connection_id); in = std::make_shared(socket()); out = std::make_shared(socket()); diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 36b05932979..7b078154252 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -58,7 +58,7 @@ void PostgreSQLHandler::run() session = std::make_unique(server.context(), ClientInfo::Interface::POSTGRESQL); SCOPE_EXIT({ session.reset(); }); - session->getClientInfo().connection_id = connection_id; + session->setClientConnectionId(connection_id); try { diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 36566832ebc..a747f06f1ce 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1177,21 +1177,12 @@ std::unique_ptr TCPHandler::makeSession() auto res = std::make_unique(server.context(), interface, socket().secure(), certificate); - auto & client_info = res->getClientInfo(); - client_info.forwarded_for = forwarded_for; - client_info.client_name = client_name; - client_info.client_version_major = client_version_major; - client_info.client_version_minor = client_version_minor; - client_info.client_version_patch = client_version_patch; - client_info.client_tcp_protocol_version = client_tcp_protocol_version; - - client_info.connection_client_version_major = client_version_major; - client_info.connection_client_version_minor = client_version_minor; - client_info.connection_client_version_patch = client_version_patch; - client_info.connection_tcp_protocol_version = client_tcp_protocol_version; - - client_info.quota_key = quota_key; - client_info.interface = interface; + res->setForwardedFor(forwarded_for); + res->setClientName(client_name); + res->setClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); + res->setConnectionClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); + res->setQuotaClientKey(quota_key); + res->setClientInterface(interface); return res; } @@ -1253,7 +1244,7 @@ void TCPHandler::receiveHello() } session = makeSession(); - auto & client_info = session->getClientInfo(); + const auto & client_info = session->getClientInfo(); #if USE_SSL /// Authentication with SSL user certificate @@ -1286,7 +1277,7 @@ void TCPHandler::receiveAddendum() { readStringBinary(quota_key, *in); if (!is_interserver_mode) - session->getClientInfo().quota_key = quota_key; + session->setQuotaClientKey(quota_key); } } diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 875764f7633..0dcdae01ba9 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -132,7 +132,7 @@ DistributedSink::DistributedSink( const auto & settings = context->getSettingsRef(); if (settings.max_distributed_depth && context->getClientInfo().distributed_depth >= settings.max_distributed_depth) throw Exception(ErrorCodes::TOO_LARGE_DISTRIBUTED_DEPTH, "Maximum distributed depth exceeded"); - context->getClientInfo().distributed_depth += 1; + context->increaseDistributedDepth(); random_shard_insert = settings.insert_distributed_one_random_shard && !storage.has_sharding_key; } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c46192ab43b..c028cf5ec77 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -914,7 +914,7 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu QueryPipeline pipeline; ContextMutablePtr query_context = Context::createCopy(local_context); - ++query_context->getClientInfo().distributed_depth; + query_context->increaseDistributedDepth(); for (size_t shard_index : collections::range(0, shards_info.size())) { @@ -976,7 +976,7 @@ std::optional StorageDistributed::distributedWriteFromClusterStor QueryPipeline pipeline; ContextMutablePtr query_context = Context::createCopy(local_context); - ++query_context->getClientInfo().distributed_depth; + query_context->increaseDistributedDepth(); /// Here we take addresses from destination cluster and assume source table exists on these nodes for (const auto & replicas : getCluster()->getShardsAddresses()) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6894368841f..066f5a42f46 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5079,7 +5079,7 @@ std::optional StorageReplicatedMergeTree::distributedWriteFromClu QueryPipeline pipeline; ContextMutablePtr query_context = Context::createCopy(local_context); - ++query_context->getClientInfo().distributed_depth; + query_context->increaseDistributedDepth(); for (const auto & replicas : src_cluster->getShardsAddresses()) { diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 242e8e5d570..0f506040cd9 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -992,7 +992,7 @@ void StorageWindowView::cleanup() auto cleanup_context = Context::createCopy(getContext()); cleanup_context->makeQueryContext(); cleanup_context->setCurrentQueryId(""); - cleanup_context->getClientInfo().is_replicated_database_internal = true; + cleanup_context->setQueryKindReplicatedDatabaseInternal(); InterpreterAlterQuery interpreter_alter(alter_query, cleanup_context); interpreter_alter.execute(); From 7a224c252ef9e544638354f0afa15d6276c61e59 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 15:04:39 +0200 Subject: [PATCH 0692/2047] One more fix --- .../test_postgresql_replica_database_engine_2/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 0514c461c5e..97eb5ca7122 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -695,6 +695,7 @@ def test_too_many_parts(started_cluster): break time.sleep(1) print(f"wait sync try {i}") + instance2.query("SYSTEM FLUSH LOGS") if instance2.contains_in_log("DB::Exception: Too many parts"): num = num - 1 break From 2379d8c9d5f73be50c78978c73aea1c2c87044e0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 17 Jul 2023 14:52:17 +0200 Subject: [PATCH 0693/2047] Revert unnecessary improving in ContextAccessCache for now. --- src/Access/AccessControl.cpp | 24 +++++++++--------------- 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 41ac3f42ee2..bf0a2a0fbba 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -72,26 +72,20 @@ public: std::shared_ptr getContextAccess(const ContextAccessParams & params) { + std::lock_guard lock{mutex}; + auto x = cache.get(params); + if (x) { - std::lock_guard lock{mutex}; - auto x = cache.get(params); - if (x) - { - if ((*x)->getUserID() && !(*x)->tryGetUser()) - cache.remove(params); /// The user has been dropped while it was in the cache. - else - return *x; - } + if ((*x)->getUserID() && !(*x)->tryGetUser()) + cache.remove(params); /// The user has been dropped while it was in the cache. + else + return *x; } + /// TODO: There is no need to keep the `ContextAccessCache::mutex` locked while we're calculating access rights. auto res = std::make_shared(access_control, params); res->initialize(); - - { - std::lock_guard lock{mutex}; - cache.add(params, res); - } - + cache.add(params, res); return res; } From d36d69c23cc8d123ac98a8710eb4712a21a30d87 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Mon, 17 Jul 2023 14:27:41 +0200 Subject: [PATCH 0694/2047] MaterializedMySQL: Replace to_string by magic_enum::enum_name --- src/Core/MySQL/MySQLReplication.cpp | 4 +- src/Core/MySQL/MySQLReplication.h | 112 ---------------------------- 2 files changed, 2 insertions(+), 114 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 1ee027b7185..70a8af78c04 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -40,9 +40,9 @@ namespace MySQLReplication void EventHeader::dump(WriteBuffer & out) const { - out << "\n=== " << to_string(this->type) << " ===" << '\n'; + out << "\n=== " << magic_enum::enum_name(this->type) << " ===" << '\n'; out << "Timestamp: " << this->timestamp << '\n'; - out << "Event Type: " << to_string(this->type) << '\n'; + out << "Event Type: " << magic_enum::enum_name(this->type) << '\n'; out << "Server ID: " << this->server_id << '\n'; out << "Event Size: " << this->event_size << '\n'; out << "Log Pos: " << this->log_pos << '\n'; diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 5825924d10b..7e19b0ea11b 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -120,22 +120,6 @@ namespace MySQLReplication BINLOG_CHECKSUM_ALG_UNDEF = 255 }; - inline String to_string(BinlogChecksumAlg type) - { - switch (type) - { - case BINLOG_CHECKSUM_ALG_OFF: - return "BINLOG_CHECKSUM_ALG_OFF"; - case BINLOG_CHECKSUM_ALG_CRC32: - return "BINLOG_CHECKSUM_ALG_CRC32"; - case BINLOG_CHECKSUM_ALG_ENUM_END: - return "BINLOG_CHECKSUM_ALG_ENUM_END"; - case BINLOG_CHECKSUM_ALG_UNDEF: - return "BINLOG_CHECKSUM_ALG_UNDEF"; - } - return std::string("Unknown checksum alg: ") + std::to_string(static_cast(type)); - } - /// http://dev.mysql.com/doc/internals/en/binlog-event-type.html enum EventType { @@ -187,102 +171,6 @@ namespace MySQLReplication MARIA_START_ENCRYPTION_EVENT = 164, }; - inline String to_string(EventType type) - { - switch (type) - { - case START_EVENT_V3: - return "StartEventV3"; - case QUERY_EVENT: - return "QueryEvent"; - case STOP_EVENT: - return "StopEvent"; - case ROTATE_EVENT: - return "RotateEvent"; - case INT_VAR_EVENT: - return "IntVarEvent"; - case LOAD_EVENT: - return "LoadEvent"; - case SLAVE_EVENT: - return "SlaveEvent"; - case CREATE_FILE_EVENT: - return "CreateFileEvent"; - case APPEND_BLOCK_EVENT: - return "AppendBlockEvent"; - case EXEC_LOAD_EVENT: - return "ExecLoadEvent"; - case DELETE_FILE_EVENT: - return "DeleteFileEvent"; - case NEW_LOAD_EVENT: - return "NewLoadEvent"; - case RAND_EVENT: - return "RandEvent"; - case USER_VAR_EVENT: - return "UserVarEvent"; - case FORMAT_DESCRIPTION_EVENT: - return "FormatDescriptionEvent"; - case XID_EVENT: - return "XIDEvent"; - case BEGIN_LOAD_QUERY_EVENT: - return "BeginLoadQueryEvent"; - case EXECUTE_LOAD_QUERY_EVENT: - return "ExecuteLoadQueryEvent"; - case TABLE_MAP_EVENT: - return "TableMapEvent"; - case WRITE_ROWS_EVENT_V0: - return "WriteRowsEventV0"; - case UPDATE_ROWS_EVENT_V0: - return "UpdateRowsEventV0"; - case DELETE_ROWS_EVENT_V0: - return "DeleteRowsEventV0"; - case WRITE_ROWS_EVENT_V1: - return "WriteRowsEventV1"; - case UPDATE_ROWS_EVENT_V1: - return "UpdateRowsEventV1"; - case DELETE_ROWS_EVENT_V1: - return "DeleteRowsEventV1"; - case INCIDENT_EVENT: - return "IncidentEvent"; - case HEARTBEAT_EVENT: - return "HeartbeatEvent"; - case IGNORABLE_EVENT: - return "IgnorableEvent"; - case ROWS_QUERY_EVENT: - return "RowsQueryEvent"; - case WRITE_ROWS_EVENT_V2: - return "WriteRowsEventV2"; - case UPDATE_ROWS_EVENT_V2: - return "UpdateRowsEventV2"; - case DELETE_ROWS_EVENT_V2: - return "DeleteRowsEventV2"; - case GTID_EVENT: - return "GTIDEvent"; - case ANONYMOUS_GTID_EVENT: - return "AnonymousGTIDEvent"; - case PREVIOUS_GTIDS_EVENT: - return "PreviousGTIDsEvent"; - case TRANSACTION_CONTEXT_EVENT: - return "TransactionContextEvent"; - case VIEW_CHANGE_EVENT: - return "ViewChangeEvent"; - case XA_PREPARE_LOG_EVENT: - return "XAPrepareLogEvent"; - case MARIA_ANNOTATE_ROWS_EVENT: - return "MariaAnnotateRowsEvent"; - case MARIA_BINLOG_CHECKPOINT_EVENT: - return "MariaBinlogCheckpointEvent"; - case MARIA_GTID_EVENT: - return "MariaGTIDEvent"; - case MARIA_GTID_LIST_EVENT: - return "MariaGTIDListEvent"; - case MARIA_START_ENCRYPTION_EVENT: - return "MariaStartEncryptionEvent"; - default: - break; - } - return std::string("Unknown event: ") + std::to_string(static_cast(type)); - } - enum MySQLEventType { MYSQL_UNHANDLED_EVENT = 0, From dd74ce3e1cd62444dc8b7c2b09ac7c5bc58f68eb Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 17 Jul 2023 13:44:52 +0000 Subject: [PATCH 0695/2047] Add integration tests check with analyzer to master --- .github/workflows/master.yml | 210 +++++++++++++++++++++++++++++++++++ 1 file changed, 210 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 0fbcb95fc12..4254d899c6f 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -2870,6 +2870,216 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan0: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan1: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan2: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan3: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan4: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=4 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAnalyzerAsan5: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, analyzer) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=5 + RUN_BY_HASH_TOTAL=6 + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" IntegrationTestsTsan0: needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] From 988b46e3f0563376fe981722d59150cfcf0287e5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 15:36:51 +0200 Subject: [PATCH 0696/2047] Fix --- ...etadataStorageFromStaticFilesWebServer.cpp | 49 +++----------- .../MetadataStorageFromStaticFilesWebServer.h | 3 +- .../ObjectStorages/Web/WebObjectStorage.cpp | 64 ++++++++++++------- .../ObjectStorages/Web/WebObjectStorage.h | 11 ++-- 4 files changed, 60 insertions(+), 67 deletions(-) diff --git a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp index 59e66969ec0..fa07ef8590a 100644 --- a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp +++ b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp @@ -33,46 +33,18 @@ const std::string & MetadataStorageFromStaticFilesWebServer::getPath() const bool MetadataStorageFromStaticFilesWebServer::exists(const std::string & path) const { - fs::path fs_path(path); - if (fs_path.has_extension()) - fs_path = fs_path.parent_path(); - - initializeIfNeeded(fs_path); - - if (object_storage.files.empty()) - return false; - - if (object_storage.files.contains(path)) - return true; - - /// `object_storage.files` contains files + directories only inside `metadata_path / uuid_3_digit / uuid /` - /// (specific table files only), but we need to be able to also tell if `exists()`, for example. - auto it = std::lower_bound( - object_storage.files.begin(), - object_storage.files.end(), - path, - [](const auto & file, const std::string & path_) { return file.first < path_; } - ); - if (it == object_storage.files.end()) - return false; - - if (startsWith(it->first, path) - || (it != object_storage.files.begin() && startsWith(std::prev(it)->first, path))) - return true; - - return false; + return object_storage.exists(path); } void MetadataStorageFromStaticFilesWebServer::assertExists(const std::string & path) const { - initializeIfNeeded(path); - if (!exists(path)) #ifdef NDEBUG throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no path {}", path); #else { std::string all_files; + std::shared_lock shared_lock(object_storage.metadata_mutex); for (const auto & [file, _] : object_storage.files) { if (!all_files.empty()) @@ -87,33 +59,40 @@ void MetadataStorageFromStaticFilesWebServer::assertExists(const std::string & p bool MetadataStorageFromStaticFilesWebServer::isFile(const std::string & path) const { assertExists(path); + std::shared_lock shared_lock(object_storage.metadata_mutex); return object_storage.files.at(path).type == WebObjectStorage::FileType::File; } bool MetadataStorageFromStaticFilesWebServer::isDirectory(const std::string & path) const { assertExists(path); + std::shared_lock shared_lock(object_storage.metadata_mutex); return object_storage.files.at(path).type == WebObjectStorage::FileType::Directory; } uint64_t MetadataStorageFromStaticFilesWebServer::getFileSize(const String & path) const { assertExists(path); + std::shared_lock shared_lock(object_storage.metadata_mutex); return object_storage.files.at(path).size; } StoredObjects MetadataStorageFromStaticFilesWebServer::getStorageObjects(const std::string & path) const { assertExists(path); + auto fs_path = fs::path(object_storage.url) / path; std::string remote_path = fs_path.parent_path() / (escapeForFileName(fs_path.stem()) + fs_path.extension().string()); remote_path = remote_path.substr(object_storage.url.size()); + + std::shared_lock shared_lock(object_storage.metadata_mutex); return {StoredObject(remote_path, object_storage.files.at(path).size, path)}; } std::vector MetadataStorageFromStaticFilesWebServer::listDirectory(const std::string & path) const { std::vector result; + std::shared_lock shared_lock(object_storage.metadata_mutex); for (const auto & [file_path, _] : object_storage.files) { if (file_path.starts_with(path)) @@ -122,22 +101,14 @@ std::vector MetadataStorageFromStaticFilesWebServer::listDirectory( return result; } -void MetadataStorageFromStaticFilesWebServer::initializeIfNeeded(const std::string & path) const -{ - if (object_storage.files.find(path) == object_storage.files.end()) - { - object_storage.initialize(fs::path(object_storage.url) / path); - } -} - DirectoryIteratorPtr MetadataStorageFromStaticFilesWebServer::iterateDirectory(const std::string & path) const { std::vector dir_file_paths; - initializeIfNeeded(path); if (!exists(path)) return std::make_unique(std::move(dir_file_paths)); + std::shared_lock shared_lock(object_storage.metadata_mutex); for (const auto & [file_path, _] : object_storage.files) { if (fs::path(parentPath(file_path)) / "" == fs::path(path) / "") diff --git a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h index a04a1359d34..96c749ad80c 100644 --- a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h +++ b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h @@ -13,13 +13,14 @@ class MetadataStorageFromStaticFilesWebServer final : public IMetadataStorage { private: friend class MetadataStorageFromStaticFilesWebServerTransaction; + using FileType = WebObjectStorage::FileType; const WebObjectStorage & object_storage; std::string root_path; void assertExists(const std::string & path) const; - void initializeIfNeeded(const std::string & path) const; + void initializeImpl(const String & uri_path, const std::unique_lock &) const; public: explicit MetadataStorageFromStaticFilesWebServer(const WebObjectStorage & object_storage_); diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 690a0d3372c..755ac0a20f9 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes extern const int NETWORK_ERROR; } -void WebObjectStorage::initialize(const String & uri_path) const +void WebObjectStorage::initialize(const String & uri_path, const std::unique_lock & lock) const { std::vector directories_to_load; LOG_TRACE(log, "Loading metadata for directory: {}", uri_path); @@ -81,8 +81,9 @@ void WebObjectStorage::initialize(const String & uri_path) const } file_path = file_path.substr(url.size()); - files.emplace(std::make_pair(file_path, file_data)); LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Adding file: {}, size: {}", file_path, file_data.size); + + files.emplace(std::make_pair(file_path, file_data)); } files.emplace(std::make_pair(dir_name, FileData({ .type = FileType::Directory }))); @@ -103,7 +104,7 @@ void WebObjectStorage::initialize(const String & uri_path) const } for (const auto & directory_path : directories_to_load) - initialize(directory_path); + initialize(directory_path, lock); } @@ -118,31 +119,50 @@ WebObjectStorage::WebObjectStorage( bool WebObjectStorage::exists(const StoredObject & object) const { - const auto & path = object.remote_path; + return exists(object.remote_path); +} +bool WebObjectStorage::exists(const std::string & path) const +{ LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Checking existence of path: {}", path); - if (files.find(path) != files.end()) + std::shared_lock shared_lock(metadata_mutex); + + if (files.find(path) == files.end()) + { + shared_lock.unlock(); + std::unique_lock unique_lock(metadata_mutex); + if (files.find(path) == files.end()) + { + fs::path index_file_dir = fs::path(url) / path; + if (index_file_dir.has_extension()) + index_file_dir = index_file_dir.parent_path(); + + initialize(index_file_dir, unique_lock); + } + unique_lock.unlock(); + shared_lock.lock(); + } + + if (files.empty()) + return false; + + if (files.contains(path)) return true; - if (path.ends_with(MergeTreeData::FORMAT_VERSION_FILE_NAME) && files.find(fs::path(path).parent_path() / "") == files.end()) - { - try - { - initialize(fs::path(url) / fs::path(path).parent_path()); - return files.find(path) != files.end(); - } - catch (...) - { - const auto message = getCurrentExceptionMessage(false); - bool can_throw = CurrentThread::isInitialized() && CurrentThread::get().getQueryContext(); - if (can_throw) - throw Exception(ErrorCodes::NETWORK_ERROR, "Cannot load disk metadata. Error: {}", message); + /// `object_storage.files` contains files + directories only inside `metadata_path / uuid_3_digit / uuid /` + /// (specific table files only), but we need to be able to also tell if `exists()`, for example. + auto it = std::lower_bound( + files.begin(), files.end(), path, + [](const auto & file, const std::string & path_) { return file.first < path_; } + ); - LOG_TRACE(&Poco::Logger::get("DiskWeb"), "Cannot load disk metadata. Error: {}", message); - return false; - } - } + if (it == files.end()) + return false; + + if (startsWith(it->first, path) + || (it != files.begin() && startsWith(std::prev(it)->first, path))) + return true; return false; } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index e85b7224892..1a21d94e230 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -3,6 +3,7 @@ #include "config.h" #include +#include namespace Poco { @@ -93,9 +94,8 @@ public: bool isReadOnly() const override { return true; } protected: - void initialize(const String & uri_path) const; - [[noreturn]] static void throwNotAllowed(); + bool exists(const std::string & path) const; enum class FileType { @@ -111,12 +111,13 @@ protected: using Files = std::map; /// file path -> file data mutable Files files; - - String url; + mutable std::shared_mutex metadata_mutex; private: - Poco::Logger * log; + void initialize(const String & path, const std::unique_lock &) const; + const String url; + Poco::Logger * log; size_t min_bytes_for_seek; }; From 8f8cb5eb8289af1f720b976158da364227619250 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 15:48:12 +0200 Subject: [PATCH 0697/2047] Update cmake/limit_jobs.cmake MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- cmake/limit_jobs.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 4d81619aa13..39a6f688488 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -43,7 +43,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLE set (PARALLEL_LINK_JOBS 2) endif() -message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") +message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs.") if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) From fc05187e8a61f199c7eb5134edc6f7bbf9a9d981 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 17 Jul 2023 13:50:21 +0000 Subject: [PATCH 0698/2047] Add dependencies to FinishCheck --- .github/workflows/master.yml | 6 ++++++ .github/workflows/pull_request.yml | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 4254d899c6f..e5b797beebd 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -4173,6 +4173,12 @@ jobs: - IntegrationTestsAsan3 - IntegrationTestsAsan4 - IntegrationTestsAsan5 + - IntegrationTestsAnalyzerAsan0 + - IntegrationTestsAnalyzerAsan1 + - IntegrationTestsAnalyzerAsan2 + - IntegrationTestsAnalyzerAsan3 + - IntegrationTestsAnalyzerAsan4 + - IntegrationTestsAnalyzerAsan5 - IntegrationTestsRelease0 - IntegrationTestsRelease1 - IntegrationTestsRelease2 diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index d8030c12128..dd834959578 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5099,6 +5099,12 @@ jobs: - IntegrationTestsAsan3 - IntegrationTestsAsan4 - IntegrationTestsAsan5 + - IntegrationTestsAnalyzerAsan0 + - IntegrationTestsAnalyzerAsan1 + - IntegrationTestsAnalyzerAsan2 + - IntegrationTestsAnalyzerAsan3 + - IntegrationTestsAnalyzerAsan4 + - IntegrationTestsAnalyzerAsan5 - IntegrationTestsRelease0 - IntegrationTestsRelease1 - IntegrationTestsRelease2 From 778efcb95774a3c0c530aff80799feeb054628dc Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 17 Jul 2023 11:49:25 +0000 Subject: [PATCH 0699/2047] Use row_level_filter to restore original header as well --- .../QueryPlan/ReadFromMergeTree.cpp | 35 ++++++++++++------- 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index de00f38bf0c..79d6fd69499 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -141,23 +141,32 @@ static bool checkAllPartsOnRemoteFS(const RangesInDataParts & parts) static void updateSortDescriptionForOutputStream( DataStream & output_stream, const Names & sorting_key_columns, const int sort_direction, InputOrderInfoPtr input_order_info, PrewhereInfoPtr prewhere_info) { + /// Updating sort description can be done after PREWHERE actions are applied to the header. + /// Aftert PREWHERE actions are applied, column names in header can differ from storage column names due to aliases + /// To mitigate it, we're trying to build original header and use it to deduce sorting description + /// TODO: this approach is fragile, it'd be more robust to update sorting description for the whole plan during plan optimization Block original_header = output_stream.header.cloneEmpty(); - /// build original header - if (prewhere_info && prewhere_info->prewhere_actions) + if (prewhere_info) { - FindOriginalNodeForOutputName original_column_finder(prewhere_info->prewhere_actions); - - for (auto & column : original_header) + if (prewhere_info->prewhere_actions) { - const auto * original_node = original_column_finder.find(column.name); - if (original_node) + FindOriginalNodeForOutputName original_column_finder(prewhere_info->prewhere_actions); + for (auto & column : original_header) { - LOG_DEBUG( - &Poco::Logger::get(__PRETTY_FUNCTION__), - "Found original column '{}' for '{}'", - original_node->result_name, - column.name); - column.name = original_node->result_name; + const auto * original_node = original_column_finder.find(column.name); + if (original_node) + column.name = original_node->result_name; + } + } + + if (prewhere_info->row_level_filter) + { + FindOriginalNodeForOutputName original_column_finder(prewhere_info->row_level_filter); + for (auto & column : original_header) + { + const auto * original_node = original_column_finder.find(column.name); + if (original_node) + column.name = original_node->result_name; } } } From 13b493a353780a03f280dbd21df34e48964dc7bf Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 15:53:47 +0200 Subject: [PATCH 0700/2047] Add comment --- src/Disks/ObjectStorages/Web/WebObjectStorage.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 755ac0a20f9..502ca8da8c2 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -140,6 +140,7 @@ bool WebObjectStorage::exists(const std::string & path) const initialize(index_file_dir, unique_lock); } + /// Files are never deleted from `files` as disk is read only, so no worry that we unlock now. unique_lock.unlock(); shared_lock.lock(); } From 2ad1109dbdf37f9a854ea2dbe954312101c127b7 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 16:03:39 +0200 Subject: [PATCH 0701/2047] Avoid zero copy replication more agressively --- src/Storages/StorageReplicatedMergeTree.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6894368841f..40d01215498 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2448,10 +2448,13 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (part_desc->checksum_hex != part_desc->src_table_part->checksums.getTotalChecksumHex()) throw Exception(ErrorCodes::UNFINISHED, "Checksums of {} is suddenly changed", part_desc->src_table_part->name); - bool zero_copy_enabled = dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + /// Don't do hardlinks in case of zero-copy at any side (defensive programming) + bool source_zero_copy_enabled = dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + bool our_zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication; + IDataPartStorage::ClonePartParams clone_params { - .copy_instead_of_hardlink = zero_copy_enabled && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport(), + .copy_instead_of_hardlink = (our_zero_copy_enabled || source_zero_copy_enabled) && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport(), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( @@ -7585,8 +7588,10 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); + /// Don't do hardlinks in case of zero-copy at any side (defensive programming) bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication || dynamic_cast(dest_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + IDataPartStorage::ClonePartParams clone_params { .copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport(), From d11ded952e3e22f576d892ffe46db44920cefea4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 17 Jul 2023 16:09:11 +0200 Subject: [PATCH 0702/2047] Update WebObjectStorage.cpp --- src/Disks/ObjectStorages/Web/WebObjectStorage.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 502ca8da8c2..8a12833281c 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -28,7 +28,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; - extern const int NETWORK_ERROR; } void WebObjectStorage::initialize(const String & uri_path, const std::unique_lock & lock) const From fe53cd3d7464f960c8b824e88ec98183f03b54e8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 16:11:58 +0200 Subject: [PATCH 0703/2047] Minor changes --- .../table-engines/integrations/s3queue.md | 2 +- src/Storages/S3Queue/S3QueueHolder.cpp | 19 +++++++++---------- src/Storages/S3Queue/S3QueueHolder.h | 6 +++--- src/Storages/S3Queue/S3QueueSource.cpp | 12 +++++++----- src/Storages/S3Queue/StorageS3Queue.cpp | 5 ++--- src/Storages/S3Queue/StorageS3Queue.h | 6 +++++- 6 files changed, 27 insertions(+), 23 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index f857df8b1c8..3e6cf88312f 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -58,7 +58,7 @@ Using named collections: ```sql CREATE TABLE s3queue_engine_table (name String, value UInt32) -ENGINE=S3Queue(s3queue_conf, 'CSV', 'gzip') +ENGINE=S3Queue(s3queue_conf, format = 'CSV', compression_method = 'gzip') SETTINGS mode = 'ordred'; ``` diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 03efe6ef097..7a984cd418f 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -29,7 +29,7 @@ void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) while (!in.eof()) { String file_name; - Int64 timestamp; + UInt64 timestamp; UInt64 retries_count; in >> file_name >> "\n"; in >> timestamp >> "\n"; @@ -81,11 +81,10 @@ void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) // Remove old items if (max_age > 0) { - Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - Int64 max_seconds_diff = max_age; + UInt64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + UInt64 max_seconds_diff = max_age; auto new_end = std::remove_if( - files.begin(), - files.end(), + files.begin(), files.end(), [×tamp, &max_seconds_diff](TrackedCollectionItem processed_file) { return (timestamp - processed_file.timestamp) > max_seconds_diff; }); files.erase(new_end, files.end()); @@ -95,7 +94,7 @@ void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) { - Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + UInt64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); TrackedCollectionItem processed_file = {.file_path=file_name, .timestamp=timestamp}; files.push_back(processed_file); @@ -157,11 +156,11 @@ S3QueueHolder::S3QueueHolder( const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, - UInt64 & max_set_age_s_, + UInt64 & max_set_age_sec_, UInt64 & max_loading_retries_) : WithContext(context_) , max_set_size(max_set_size_) - , max_set_age_s(max_set_age_s_) + , max_set_age_sec(max_set_age_sec_) , max_loading_retries(max_loading_retries_) , zookeeper_path(zookeeper_path_) , zookeeper_failed_path(fs::path(zookeeper_path_) / "failed") @@ -191,7 +190,7 @@ void S3QueueHolder::setFileProcessed(const String & file_path) if (mode == S3QueueMode::UNORDERED) { String processed_files = zookeeper->get(zookeeper_processed_path); - auto processed = S3QueueProcessedCollection(max_set_size, max_set_age_s); + auto processed = S3QueueProcessedCollection(max_set_size, max_set_age_sec); processed.parse(processed_files); processed.add(file_path); zookeeper->set(zookeeper_processed_path, processed.toString()); @@ -266,7 +265,7 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::getUnorderedProcessedFiles() auto zookeeper = getZooKeeper(); String processed = zookeeper->get(zookeeper_processed_path); - auto collection = S3QueueProcessedCollection(max_set_size, max_set_age_s); + auto collection = S3QueueProcessedCollection(max_set_size, max_set_age_sec); collection.parse(processed); return collection.getFileNames(); diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index 77719c8b941..1292ebadc60 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -15,7 +15,7 @@ public: struct TrackedCollectionItem { String file_path; - Int64 timestamp = 0; + UInt64 timestamp = 0; UInt64 retries_count = 0; }; @@ -27,7 +27,7 @@ public: const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, - UInt64 & max_set_age_s_, + UInt64 & max_set_age_sec_, UInt64 & max_loading_retries_); void setFileProcessed(const String & file_path); @@ -84,7 +84,7 @@ public: private: const UInt64 max_set_size; - const UInt64 max_set_age_s; + const UInt64 max_set_age_sec; const UInt64 max_loading_retries; zkutil::ZooKeeperPtr current_zookeeper; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 85a1db2ad35..151b63b8051 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -100,7 +100,7 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( auto full_path = bucket + '/' + val.key; if (exclude_keys.find(full_path) != exclude_keys.end()) { - LOG_INFO(log, "Found in exclude keys {}", val.key); + LOG_TRACE(log, "Found in exclude keys {}", val.key); continue; } if ((engine_mode == S3QueueMode::ORDERED) && (full_path.compare(max_file) <= 0)) @@ -130,9 +130,9 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( } Strings keys; - for (const auto & v : processing_keys) + for (const auto & key_info : processing_keys) { - keys.push_back(bucket + '/' + v.key); + keys.push_back(bucket + '/' + key_info.key); } processing_keys.push_back(KeyWithInfo()); @@ -253,8 +253,10 @@ Chunk StorageS3QueueSource::generate() size_t total_size = file_iterator->getTotalSize(); if (num_rows && total_size) { - updateRowsProgressApprox( - *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + size_t chunk_size = reader.getFormat()->getApproxBytesReadForChunk(); + if (!chunk_size) + chunk_size = chunk.bytes(); + updateRowsProgressApprox(*this, num_rows, chunk_size, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); } for (const auto & virtual_column : requested_virtual_columns) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 69c6601365f..71a93884896 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -284,13 +284,12 @@ Pipe StorageS3Queue::read( max_download_threads)); } -SinkToStoragePtr StorageS3Queue::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) +SinkToStoragePtr StorageS3Queue::write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, bool) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); } -void StorageS3Queue::truncate( - const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*local_context*/, TableExclusiveLockHolder &) +void StorageS3Queue::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); } diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 9dd1c6029f5..f9500371489 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -63,7 +63,11 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override; + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + bool async_insert) override; void truncate( const ASTPtr & /*query*/, From 9a30aaa9aa1924d35d79806fb2090570e0d9df49 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 17 Jul 2023 16:14:58 +0200 Subject: [PATCH 0704/2047] Revert "Fixed several issues found by OSS-Fuzz" --- ..._function_state_deserialization_fuzzer.cpp | 1 - .../fuzzers/delta_decompress_fuzzer.cpp | 2 +- .../double_delta_decompress_fuzzer.cpp | 2 +- .../fuzzers/encrypted_decompress_fuzzer.cpp | 4 +- .../fuzzers/lz4_decompress_fuzzer.cpp | 4 +- src/Core/Field.h | 66 +------------------ src/DataTypes/DataTypeFactory.cpp | 2 +- src/Functions/DateTimeTransforms.h | 4 +- .../fuzzers/codegen_fuzzer/CMakeLists.txt | 2 +- 9 files changed, 10 insertions(+), 77 deletions(-) diff --git a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp index 3db1afb7a92..2ea01e1d5bc 100644 --- a/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp +++ b/src/AggregateFunctions/fuzzers/aggregate_function_state_deserialization_fuzzer.cpp @@ -8,7 +8,6 @@ #include #include -#include #include diff --git a/src/Compression/fuzzers/delta_decompress_fuzzer.cpp b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp index 451606843e2..b039777da15 100644 --- a/src/Compression/fuzzers/delta_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/delta_decompress_fuzzer.cpp @@ -34,7 +34,7 @@ try DB::Memory<> memory; memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); - codec->doDecompressData(reinterpret_cast(data), static_cast(size), memory.data(), static_cast(output_buffer_size)); + codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); return 0; } diff --git a/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp b/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp index f7e685d68ad..f9822daa3bd 100644 --- a/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/double_delta_decompress_fuzzer.cpp @@ -34,7 +34,7 @@ try DB::Memory<> memory; memory.resize(output_buffer_size + codec->getAdditionalSizeAtTheEndOfBuffer()); - codec->doDecompressData(reinterpret_cast(data), static_cast(size), memory.data(), static_cast(output_buffer_size)); + codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); return 0; } diff --git a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp index 207cce21e3b..3e3d0e164fe 100644 --- a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp @@ -292,10 +292,10 @@ try DB::Memory<> memory; memory.resize(input.size() + codec_128->getAdditionalSizeAtTheEndOfBuffer()); - codec_128->doDecompressData(input.data(), static_cast(input.size()), memory.data(), static_cast(input.size() - 31)); + codec_128->doDecompressData(input.data(), input.size(), memory.data(), input.size() - 31); memory.resize(input.size() + codec_128->getAdditionalSizeAtTheEndOfBuffer()); - codec_256->doDecompressData(input.data(), static_cast(input.size()), memory.data(), static_cast(input.size() - 31)); + codec_256->doDecompressData(input.data(), input.size(), memory.data(), input.size() - 31); return 0; } catch (...) diff --git a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp index f6d4c51f18b..85c4c9bd329 100644 --- a/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/lz4_decompress_fuzzer.cpp @@ -24,7 +24,7 @@ try return 0; const auto * p = reinterpret_cast(data); - auto codec = DB::getCompressionCodecLZ4(static_cast(p->level)); + auto codec = DB::getCompressionCodecLZ4(p->level); size_t output_buffer_size = p->decompressed_size % 65536; size -= sizeof(AuxiliaryRandomData); @@ -37,7 +37,7 @@ try DB::Memory<> memory; memory.resize(output_buffer_size + LZ4::ADDITIONAL_BYTES_AT_END_OF_BUFFER); - codec->doDecompressData(reinterpret_cast(data), static_cast(size), memory.data(), static_cast(output_buffer_size)); + codec->doDecompressData(reinterpret_cast(data), size, memory.data(), output_buffer_size); return 0; } diff --git a/src/Core/Field.h b/src/Core/Field.h index cc0083e02d8..ef1bd9a895d 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -28,7 +28,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int TOO_DEEP_RECURSION; } constexpr Null NEGATIVE_INFINITY{Null::Value::NegativeInfinity}; @@ -42,13 +41,10 @@ using FieldVector = std::vector>; /// construct a Field of Array or a Tuple type. An alternative approach would be /// to construct both of these types from FieldVector, and have the caller /// specify the desired Field type explicitly. -/// As the result stack overflow on destruction is possible -/// and to avoid it we need to count the depth and have a threshold. #define DEFINE_FIELD_VECTOR(X) \ struct X : public FieldVector \ { \ using FieldVector::FieldVector; \ - uint8_t nested_field_depth = 0; \ } DEFINE_FIELD_VECTOR(Array); @@ -65,7 +61,6 @@ using FieldMap = std::map, AllocatorWithMemoryTrackin struct X : public FieldMap \ { \ using FieldMap::FieldMap; \ - uint8_t nested_field_depth = 0; \ } DEFINE_FIELD_MAP(Object); @@ -296,12 +291,6 @@ decltype(auto) castToNearestFieldType(T && x) */ #define DBMS_MIN_FIELD_SIZE 32 -/// Note: uint8_t is used for storing depth value. -#if defined(SANITIZER) || !defined(NDEBUG) - #define DBMS_MAX_NESTED_FIELD_DEPTH 64 -#else - #define DBMS_MAX_NESTED_FIELD_DEPTH 255 -#endif /** Discriminated union of several types. * Made for replacement of `boost::variant` @@ -682,49 +671,6 @@ private: Types::Which which; - /// StorageType and Original are the same for Array, Tuple, Map, Object - template - uint8_t calculateAndCheckFieldDepth(Original && x) - { - uint8_t result = 0; - - if constexpr (std::is_same_v - || std::is_same_v - || std::is_same_v - || std::is_same_v) - { - result = x.nested_field_depth; - - auto get_depth = [](const Field & elem) - { - switch (elem.which) - { - case Types::Array: - return elem.template get().nested_field_depth; - case Types::Tuple: - return elem.template get().nested_field_depth; - case Types::Map: - return elem.template get().nested_field_depth; - case Types::Object: - return elem.template get().nested_field_depth; - default: - return static_cast(0); - } - }; - - if constexpr (std::is_same_v) - for (auto & [_, value] : x) - result = std::max(get_depth(value), result); - else - for (auto & value : x) - result = std::max(get_depth(value), result); - } - - if (result >= DBMS_MAX_NESTED_FIELD_DEPTH) - throw Exception(ErrorCodes::TOO_DEEP_RECURSION, "Too deep Field"); - - return result; - } /// Assuming there was no allocated state or it was deallocated (see destroy). template @@ -738,17 +684,7 @@ private: // we must initialize the entire wide stored type, and not just the // nominal type. using StorageType = NearestFieldType; - - /// Incrementing the depth since we create a new Field. - auto depth = calculateAndCheckFieldDepth(x); new (&storage) StorageType(std::forward(x)); - - if constexpr (std::is_same_v - || std::is_same_v - || std::is_same_v - || std::is_same_v) - reinterpret_cast(&storage)->nested_field_depth = depth + 1; - which = TypeToEnum::value; } @@ -845,7 +781,7 @@ private: } template - ALWAYS_INLINE void destroy() + void destroy() { T * MAY_ALIAS ptr = reinterpret_cast(&storage); ptr->~T(); diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 89dacae59ff..415f24d8151 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -62,7 +62,7 @@ DataTypePtr DataTypeFactory::getImpl(const String & full_name) const } else { - ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", DBMS_DEFAULT_MAX_QUERY_SIZE, data_type_max_parse_depth); + ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", false, data_type_max_parse_depth); } return getImpl(ast); diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 0d71e802bf0..510a88db2b6 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1521,10 +1521,8 @@ struct Transformer if constexpr (std::is_same_v || std::is_same_v) { -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Wimplicit-const-int-float-conversion" bool is_valid_input = vec_from[i] >= 0 && vec_from[i] <= 0xFFFFFFFFL; -# pragma clang diagnostic pop + if (!is_valid_input) { if constexpr (std::is_same_v) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index 30f0e91a75b..727c49cfc4d 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -42,4 +42,4 @@ clickhouse_add_executable(codegen_select_fuzzer ${FUZZER_SRCS}) set_source_files_properties("${PROTO_SRCS}" "out.cpp" PROPERTIES COMPILE_FLAGS "-Wno-reserved-identifier") target_include_directories(codegen_select_fuzzer SYSTEM BEFORE PRIVATE "${CMAKE_CURRENT_BINARY_DIR}") -target_link_libraries(codegen_select_fuzzer PRIVATE ch_contrib::protobuf ch_contrib::protobuf_mutator ch_contrib::protoc dbms ${LIB_FUZZING_ENGINE}) +target_link_libraries(codegen_select_fuzzer PRIVATE ch_contrib::protobuf_mutator ch_contrib::protoc dbms ${LIB_FUZZING_ENGINE}) From 0caa49d85fc00295b63641bfddb13e48ae499007 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 17 Jul 2023 14:24:52 +0000 Subject: [PATCH 0705/2047] Fix build --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 7a8632f1b0b..6d7b6b39a40 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1674,7 +1674,7 @@ std::pair IMergeTreeDataPart::canRemovePart() const void IMergeTreeDataPart::initializePartMetadataManager() { #if USE_ROCKSDB - if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache(); metadata_cache && use_metadata_cache) + if (auto metadata_cache = storage.getContext()->tryGetMergeTreeMetadataCache(); metadata_cache && use_metadata_cache) metadata_manager = std::make_shared(this, metadata_cache); else metadata_manager = std::make_shared(this); From 2a2ebb1f318ce665885ac2c5f711d6a29fbed582 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 17 Jul 2023 14:26:11 +0000 Subject: [PATCH 0706/2047] Bump From fff1ae73691bc3bbd409b1743a2c85d18412f868 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 17 Jul 2023 17:08:36 +0200 Subject: [PATCH 0707/2047] Use default destructor for ContextAccess. --- src/Access/ContextAccess.cpp | 30 +++++++++--------------------- 1 file changed, 9 insertions(+), 21 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 51bb7794735..9e9d8644539 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -235,20 +235,7 @@ ContextAccess::ContextAccess(const AccessControl & access_control_, const Params } -ContextAccess::~ContextAccess() -{ - enabled_settings.reset(); - enabled_quota.reset(); - enabled_row_policies.reset(); - row_policies_of_initial_user.reset(); - access_with_implicit.reset(); - access.reset(); - roles_info.reset(); - subscription_for_roles_changes.reset(); - enabled_roles.reset(); - subscription_for_user_change.reset(); - user.reset(); -} +ContextAccess::~ContextAccess() = default; void ContextAccess::initialize() @@ -265,12 +252,6 @@ void ContextAccess::initialize() if (!params.user_id) throw Exception(ErrorCodes::LOGICAL_ERROR, "No user in current context, it's a bug"); - if (!params.initial_user.empty()) - { - if (auto initial_user_id = access_control->find(params.initial_user)) - row_policies_of_initial_user = access_control->tryGetDefaultRowPolicies(*initial_user_id); - } - subscription_for_user_change = access_control->subscribeForChanges( *params.user_id, [weak_ptr = weak_from_this()](const UUID &, const AccessEntityPtr & entity) @@ -290,7 +271,8 @@ void ContextAccess::initialize() void ContextAccess::setUser(const UserPtr & user_) const { user = user_; - if (!user) + + if (!user_) { /// User has been dropped. user_was_dropped = true; @@ -301,6 +283,7 @@ void ContextAccess::setUser(const UserPtr & user_) const enabled_roles = nullptr; roles_info = nullptr; enabled_row_policies = nullptr; + row_policies_of_initial_user = nullptr; enabled_quota = nullptr; enabled_settings = nullptr; return; @@ -330,6 +313,11 @@ void ContextAccess::setUser(const UserPtr & user_) const }); setRolesInfo(enabled_roles->getRolesInfo()); + + std::optional initial_user_id; + if (!params.initial_user.empty()) + initial_user_id = access_control->find(params.initial_user); + row_policies_of_initial_user = initial_user_id ? access_control->tryGetDefaultRowPolicies(*initial_user_id) : nullptr; } From 2abd69d9216a62e2e4ac2ff36e45108d89b741db Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 17 Jul 2023 15:14:54 +0000 Subject: [PATCH 0708/2047] Fix data race on Connection::setAsyncCallback --- src/Client/MultiplexedConnections.cpp | 25 +++++++++++-------------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index e5c4a19e02a..1a26c4609c7 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -319,24 +319,21 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac throw Exception(ErrorCodes::NO_AVAILABLE_REPLICA, "Logical error: no available replica"); Packet packet; + try { AsyncCallbackSetter async_setter(current_connection, std::move(async_callback)); - - try + packet = current_connection->receivePacket(); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_SERVER) { - packet = current_connection->receivePacket(); - } - catch (Exception & e) - { - if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_SERVER) - { - /// Exception may happen when packet is received, e.g. when got unknown packet. - /// In this case, invalidate replica, so that we would not read from it anymore. - current_connection->disconnect(); - invalidateReplica(state); - } - throw; + /// Exception may happen when packet is received, e.g. when got unknown packet. + /// In this case, invalidate replica, so that we would not read from it anymore. + current_connection->disconnect(); + invalidateReplica(state); } + throw; } switch (packet.type) From 1dbb86a045a3e505cf7e71de45e992cc1b288bd3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Jun 2023 13:38:46 +0200 Subject: [PATCH 0709/2047] Add _gen to ignore (generated by pytest) Signed-off-by: Azat Khuzhin --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index a04c60d5ca3..39d6f3f9fc8 100644 --- a/.gitignore +++ b/.gitignore @@ -161,6 +161,7 @@ tests/queries/0_stateless/test_* tests/queries/0_stateless/*.binary tests/queries/0_stateless/*.generated-expect tests/queries/0_stateless/*.expect.history +tests/integration/**/_gen # rust /rust/**/target From 6f6562c96bf5fb7157ff582815bae1330d34c50f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 10:23:40 +0200 Subject: [PATCH 0710/2047] Fix running integration tests with spaces in it's names Previously they were simply ignored, and not only them, but all the tests in that group, here is an example [1] [1]: https://s3.amazonaws.com/clickhouse-test-reports/51448/4ed462ac7834a8180f92ca7d7d3c076e687bfca9/integration_tests__asan__[4_6].html Note, that I've replaced PYTEST_OPTS with handled by pytest itself PYTEST_ADDOPTS, since "sh -c $PYTEST_OPTS" does not allows you to preserve the arguments correctly. v5: use double quotes for PYTEST_ADDOPTS value Signed-off-by: Azat Khuzhin --- docker/test/integration/runner/Dockerfile | 3 ++- tests/integration/ci-runner.py | 3 ++- tests/integration/runner | 13 ++++++++++--- 3 files changed, 14 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 0d1fa00b214..8e95d94b6dc 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -135,4 +135,5 @@ ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1' EXPOSE 2375 ENTRYPOINT ["dockerd-entrypoint.sh"] -CMD ["sh", "-c", "pytest $PYTEST_OPTS"] +# To pass additional arguments (i.e. list of tests) use PYTEST_ADDOPTS +CMD ["sh", "-c", "pytest"] diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 31f4a7666c8..98675cb0b34 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -10,6 +10,7 @@ import random import shutil import subprocess import time +import shlex import zlib # for crc32 @@ -646,7 +647,7 @@ class ClickhouseIntegrationTestsRunner: info_basename = test_group_str + "_" + str(i) + ".nfo" info_path = os.path.join(repo_path, "tests/integration", info_basename) - test_cmd = " ".join([test for test in sorted(test_names)]) + test_cmd = " ".join([shlex.quote(test) for test in sorted(test_names)]) parallel_cmd = ( " --parallel {} ".format(num_workers) if num_workers > 0 else "" ) diff --git a/tests/integration/runner b/tests/integration/runner index c124ad46447..d524948dbcf 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -11,6 +11,7 @@ import subprocess import sys import string import random +import shlex def random_str(length=6): @@ -407,8 +408,14 @@ if __name__ == "__main__": if args.analyzer: use_analyzer = "-e CLICKHOUSE_USE_NEW_ANALYZER=1" - pytest_opts = " ".join(args.pytest_args).replace("'", "\\'") - tests_list = " ".join(args.tests_list) + # NOTE: since pytest options is in the argument value already we need to additionally escape '"' + pytest_opts = " ".join( + map(lambda x: shlex.quote(x).replace('"', '\\"'), args.pytest_args) + ) + tests_list = " ".join( + map(lambda x: shlex.quote(x).replace('"', '\\"'), args.tests_list) + ) + cmd_base = ( f"docker run {net} {tty} --rm --name {CONTAINER_NAME} " "--privileged --dns-search='.' " # since recent dns search leaks from host @@ -420,7 +427,7 @@ if __name__ == "__main__": f"--volume={args.src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos " f"--volume=/run:/run/host:ro {dockerd_internal_volume} {env_tags} {env_cleanup} " f"-e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 {use_analyzer} -e PYTHONUNBUFFERED=1 " - f"-e PYTEST_OPTS='{parallel_args} {pytest_opts} {tests_list} {rand_args} -vvv'" + f'-e PYTEST_ADDOPTS="{parallel_args} {pytest_opts} {tests_list} {rand_args} -vvv"' f" {DIND_INTEGRATION_TESTS_IMAGE_NAME}:{args.docker_image_version}" ) From e532d5b376ebadaa8fad5dd5c026e082bf851b7a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Jun 2023 13:48:15 +0200 Subject: [PATCH 0711/2047] tests: use -- over quotes on and on in ci-runner.py Signed-off-by: Azat Khuzhin --- tests/integration/ci-runner.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 98675cb0b34..4f40058b3d9 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -412,7 +412,7 @@ class ClickhouseIntegrationTestsRunner: out_file_full = os.path.join(self.result_path, "runner_get_all_tests.log") cmd = ( "cd {repo_path}/tests/integration && " - "timeout -s 9 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' " + "timeout -s 9 1h ./runner {runner_opts} {image_cmd} -- --setup-plan " "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " "| grep -v 'SKIPPED' | sort -u > {out_file}".format( repo_path=repo_path, @@ -656,7 +656,7 @@ class ClickhouseIntegrationTestsRunner: # -E -- (E)rror # -p -- (p)assed # -s -- (s)kipped - cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner {} {} -t {} {} '-rfEps --run-id={} --color=no --durations=0 {}' | tee {}".format( + cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner {} {} -t {} {} -- -rfEps --run-id={} --color=no --durations=0 {} | tee {}".format( repo_path, self._get_runner_opts(), image_cmd, From acbdbf6ab744ec4659c7ed3ac613c1487a5ab021 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 29 Jun 2023 13:49:03 +0200 Subject: [PATCH 0712/2047] tests: do not use --format if --quiet is set in integration runner Signed-off-by: Azat Khuzhin --- tests/integration/runner | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index d524948dbcf..1b902803741 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -136,9 +136,7 @@ def check_args_and_update_paths(args): def docker_kill_handler_handler(signum, frame): subprocess.check_call( - 'docker ps --all --quiet --filter name={name} --format="{{{{.ID}}}}"'.format( - name=CONTAINER_NAME - ), + "docker ps --all --quiet --filter name={name}".format(name=CONTAINER_NAME), shell=True, ) raise KeyboardInterrupt("Killed by Ctrl+C") @@ -438,7 +436,7 @@ if __name__ == "__main__": ) containers = subprocess.check_output( - f"docker ps --all --quiet --filter name={CONTAINER_NAME} --format={{{{.ID}}}}", + f"docker ps --all --quiet --filter name={CONTAINER_NAME}", shell=True, universal_newlines=True, ).splitlines() From 671f8b34d06364e2c6e26a60ef6489f8c23e4ef7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 17 Jul 2023 17:18:14 +0200 Subject: [PATCH 0713/2047] Added test with all signatures and fixed style check --- .../TableFunctionAzureBlobStorage.h | 2 +- .../test_storage_azure_blob_storage/test.py | 51 +++++++++++++++++++ 2 files changed, 52 insertions(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.h b/src/TableFunctions/TableFunctionAzureBlobStorage.h index efbb6a5b1e0..407b9ac5883 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.h +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.h @@ -19,7 +19,7 @@ class TableFunctionAzureBlobStorage : public ITableFunction { public: static constexpr auto name = "azureBlobStorage"; - + static constexpr auto signature = " - connection_string, container_name, blobpath\n" " - connection_string, container_name, blobpath, structure \n" " - connection_string, container_name, blobpath, format \n" diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 6089466ff5d..de72cbb7bff 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -611,3 +611,54 @@ def test_filter_using_file(cluster): query = f"select count(*) from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_partition_tf_*.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') WHERE _file='test_partition_tf_3.csv'" assert azure_query(node, query) == "1\n" + +def test_function_signatures(cluster): + node = cluster.instances["node"] + connection_string = 'DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;' + storage_account_url = "http://azurite1:10000/devstoreaccount1" + account_name = "devstoreaccount1" + account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32') VALUES (1),(2),(3)", + ) + + #" - connection_string, container_name, blobpath\n" + query_1 = f"select * from azureBlobStorage('{connection_string}', 'cont', 'test_signature.csv')" + assert azure_query(node, query_1) == "1\n2\n3\n" + + #" - connection_string, container_name, blobpath, structure \n" + query_2 = f"select * from azureBlobStorage('{connection_string}', 'cont', 'test_signature.csv', 'column1 UInt32')" + assert azure_query(node, query_2) == "1\n2\n3\n" + + #" - connection_string, container_name, blobpath, format \n" + query_3 = f"select * from azureBlobStorage('{connection_string}', 'cont', 'test_signature.csv', 'CSV')" + assert azure_query(node, query_3) == "1\n2\n3\n" + + #" - connection_string, container_name, blobpath, format, compression \n" + query_4 = f"select * from azureBlobStorage('{connection_string}', 'cont', 'test_signature.csv', 'CSV', 'auto')" + assert azure_query(node, query_4) == "1\n2\n3\n" + + #" - connection_string, container_name, blobpath, format, compression, structure \n" + query_5 = f"select * from azureBlobStorage('{connection_string}', 'cont', 'test_signature.csv', 'CSV', 'auto', 'column1 UInt32')" + assert azure_query(node, query_5) == "1\n2\n3\n" + + #" - storage_account_url, container_name, blobpath, account_name, account_key\n" + query_6 = f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}')" + assert azure_query(node, query_6) == "1\n2\n3\n" + + #" - storage_account_url, container_name, blobpath, account_name, account_key, structure\n" + query_7 = f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'column1 UInt32')" + assert azure_query(node, query_7) == "1\n2\n3\n" + + #" - storage_account_url, container_name, blobpath, account_name, account_key, format\n" + query_8 = f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV')" + assert azure_query(node, query_8) == "1\n2\n3\n" + + #" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n" + query_9 = f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto')" + assert azure_query(node, query_9) == "1\n2\n3\n" + + #" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n" + query_10 = f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32')" + assert azure_query(node, query_10) == "1\n2\n3\n" \ No newline at end of file From 6c986c1fb22e2a5fff2bc281d55eb545868a4645 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 12 Jul 2023 15:21:28 +0200 Subject: [PATCH 0714/2047] Allow test names with spaces in it It had been fixed and now it is allowed. Signed-off-by: Azat Khuzhin --- tests/integration/ci-runner.py | 41 +++++++++++++++++++++------------- 1 file changed, 26 insertions(+), 15 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 4f40058b3d9..f2adea78da7 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -7,6 +7,7 @@ import json import logging import os import random +import re import shutil import subprocess import time @@ -111,16 +112,36 @@ def get_counters(fname): if not (".py::" in line and " " in line): continue - line_arr = line.strip().split(" ") + line = line.strip() + # [gw0] [ 7%] ERROR test_mysql_protocol/test.py::test_golang_client + # ^^^^^^^^^^^^^ + if line.strip().startswith("["): + line = re.sub("^\[[^\[\]]*\] \[[^\[\]]*\] ", "", line) + + line_arr = line.split(" ") if len(line_arr) < 2: logging.debug("Strange line %s", line) continue # Lines like: - # [gw0] [ 7%] ERROR test_mysql_protocol/test.py::test_golang_client - # [gw3] [ 40%] PASSED test_replicated_users/test.py::test_rename_replicated[QUOTA] - state = line_arr[-2] - test_name = line_arr[-1] + # + # ERROR test_mysql_protocol/test.py::test_golang_client + # PASSED test_replicated_users/test.py::test_rename_replicated[QUOTA] + # PASSED test_drop_is_lock_free/test.py::test_query_is_lock_free[detach part] + # + state = line_arr.pop(0) + test_name = " ".join(line_arr) + + # Normalize test names for lines like this: + # + # FAILED test_storage_s3/test.py::test_url_reconnect_in_the_middle - Exception + # FAILED test_distributed_ddl/test.py::test_default_database[configs] - AssertionError: assert ... + # + test_name = re.sub( + r"^(?P[^\[\] ]+)(?P\[[^\[\]]*\]|)(?P - .*|)$", + r"\g\g", + test_name, + ) if state in counters: counters[state].add(test_name) @@ -1000,16 +1021,6 @@ class ClickhouseIntegrationTestsRunner: if "(memory)" in self.params["context_name"]: result_state = "success" - for res in test_result: - # It's not easy to parse output of pytest - # Especially when test names may contain spaces - # Do not allow it to avoid obscure failures - if " " not in res[0]: - continue - logging.warning("Found invalid test name with space: %s", res[0]) - status_text = "Found test with invalid name, see main log" - result_state = "failure" - return result_state, status_text, test_result, [] From 32311bf28f148e32836bfdc168b8f312330f3a5a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 17 Jul 2023 15:30:31 +0000 Subject: [PATCH 0715/2047] Automatic style fix --- .../test_storage_azure_blob_storage/test.py | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index c7e0b4b75b0..9beb2039b81 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -612,9 +612,10 @@ def test_filter_using_file(cluster): query = f"select count(*) from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_partition_tf_*.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') WHERE _file='test_partition_tf_3.csv'" assert azure_query(node, query) == "1\n" + def test_function_signatures(cluster): node = cluster.instances["node"] - connection_string = 'DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;' + connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;" storage_account_url = "http://azurite1:10000/devstoreaccount1" account_name = "devstoreaccount1" account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" @@ -623,42 +624,42 @@ def test_function_signatures(cluster): f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32') VALUES (1),(2),(3)", ) - #" - connection_string, container_name, blobpath\n" + # " - connection_string, container_name, blobpath\n" query_1 = f"select * from azureBlobStorage('{connection_string}', 'cont', 'test_signature.csv')" assert azure_query(node, query_1) == "1\n2\n3\n" - #" - connection_string, container_name, blobpath, structure \n" + # " - connection_string, container_name, blobpath, structure \n" query_2 = f"select * from azureBlobStorage('{connection_string}', 'cont', 'test_signature.csv', 'column1 UInt32')" assert azure_query(node, query_2) == "1\n2\n3\n" - #" - connection_string, container_name, blobpath, format \n" + # " - connection_string, container_name, blobpath, format \n" query_3 = f"select * from azureBlobStorage('{connection_string}', 'cont', 'test_signature.csv', 'CSV')" assert azure_query(node, query_3) == "1\n2\n3\n" - #" - connection_string, container_name, blobpath, format, compression \n" + # " - connection_string, container_name, blobpath, format, compression \n" query_4 = f"select * from azureBlobStorage('{connection_string}', 'cont', 'test_signature.csv', 'CSV', 'auto')" assert azure_query(node, query_4) == "1\n2\n3\n" - #" - connection_string, container_name, blobpath, format, compression, structure \n" + # " - connection_string, container_name, blobpath, format, compression, structure \n" query_5 = f"select * from azureBlobStorage('{connection_string}', 'cont', 'test_signature.csv', 'CSV', 'auto', 'column1 UInt32')" assert azure_query(node, query_5) == "1\n2\n3\n" - #" - storage_account_url, container_name, blobpath, account_name, account_key\n" + # " - storage_account_url, container_name, blobpath, account_name, account_key\n" query_6 = f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}')" assert azure_query(node, query_6) == "1\n2\n3\n" - #" - storage_account_url, container_name, blobpath, account_name, account_key, structure\n" + # " - storage_account_url, container_name, blobpath, account_name, account_key, structure\n" query_7 = f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'column1 UInt32')" assert azure_query(node, query_7) == "1\n2\n3\n" - #" - storage_account_url, container_name, blobpath, account_name, account_key, format\n" + # " - storage_account_url, container_name, blobpath, account_name, account_key, format\n" query_8 = f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV')" assert azure_query(node, query_8) == "1\n2\n3\n" - #" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n" + # " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n" query_9 = f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto')" assert azure_query(node, query_9) == "1\n2\n3\n" - #" - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n" + # " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n" query_10 = f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_signature.csv', '{account_name}', '{account_key}', 'CSV', 'auto', 'column1 UInt32')" - assert azure_query(node, query_10) == "1\n2\n3\n" \ No newline at end of file + assert azure_query(node, query_10) == "1\n2\n3\n" From ee5e639ce9ea4ed07f9fb309c9dbc6c1ad2193a7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 17:29:47 +0200 Subject: [PATCH 0716/2047] Update analyzer integration broken tests list After proper parsing more failed tests pops up: - https://s3.amazonaws.com/clickhouse-test-reports/51514/aa90e11241a97e0fd3f36c5ea0b57db50e299d2d/integration_tests__asan__analyzer__[4_6].html - https://s3.amazonaws.com/clickhouse-test-reports/51514/aa90e11241a97e0fd3f36c5ea0b57db50e299d2d/integration_tests__asan__analyzer__[5_6].html Signed-off-by: Azat Khuzhin --- tests/analyzer_integration_broken_tests.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 46b9ade43de..02f70c8a6df 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -196,3 +196,8 @@ test_quota/test.py::test_tracking_quota test_quota/test.py::test_users_xml_is_readonly test_replicating_constants/test.py::test_different_versions test_merge_tree_s3/test.py::test_heavy_insert_select_check_memory[node] +test_drop_is_lock_free/test.py::test_query_is_lock_free[detach table] +test_backward_compatibility/test_data_skipping_indices.py::test_index +test_backward_compatibility/test_convert_ordinary.py::test_convert_ordinary_to_atomic +test_backward_compatibility/test_memory_bound_aggregation.py::test_backward_compatability +test_odbc_interaction/test.py::test_postgres_insert From 76abbce91829c8ec47b952d5d26a0d9df8ce2f85 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 17 Jul 2023 16:33:04 +0000 Subject: [PATCH 0717/2047] Trying to add methods to queue --- src/Common/SystemLogBase.cpp | 102 ++++++++++++++++++++++----------- src/Common/SystemLogBase.h | 28 ++++----- src/Interpreters/SystemLog.cpp | 28 ++------- src/Interpreters/SystemLog.h | 4 +- 4 files changed, 85 insertions(+), 77 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 7373786e514..c979613b146 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -42,11 +42,8 @@ template SystemLogBase::SystemLogBase( const String & name_, std::shared_ptr> queue_) + : queue(queue_ ? queue_ : std::make_shared>(name_)) { - if (queue_) - queue = queue_; - else - queue = std::make_shared>(name_); } template @@ -153,27 +150,40 @@ void SystemLogQueue::add(const LogElement & element) } template -void SystemLogBase::add(const LogElement & element) +uint64_t SystemLogQueue::notifyFlush(bool force) { - queue->add(element); + uint64_t this_thread_requested_offset; + + { + std::lock_guard lock(mutex); + if (is_shutdown) + return uint64_t(-1); + + this_thread_requested_offset = queue_front_index + queue.size(); + + // Publish our flush request, taking care not to overwrite the requests + // made by other threads. + is_force_prepare_tables |= force; + requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); + + flush_event.notify_all(); + } + + LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); + return this_thread_requested_offset; } template -void SystemLogBase::flush(bool force) +void SystemLogQueue::waitFlush(uint64_t this_thread_requested_offset_) { - uint64_t this_thread_requested_offset = notifyFlushImpl(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; - - // Use an arbitrary timeout to avoid endless waiting. 60s proved to be // too fast for our parallel functional tests, probably because they // heavily load the disk. const int timeout_seconds = 180; - std::unique_lock lock(queue->mutex); - bool result = queue->flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] + std::unique_lock lock(mutex); + bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] { - return flushed_up_to >= this_thread_requested_offset && !is_force_prepare_tables; + return flushed_up_to >= this_thread_requested_offset_ && !is_force_prepare_tables; }); if (!result) @@ -183,33 +193,57 @@ void SystemLogBase::flush(bool force) } } -template -void SystemLogBase::notifyFlush(bool force) { notifyFlushImpl(force); } +constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; template -uint64_t SystemLogBase::notifyFlushImpl(bool force) +void SystemLogQueue::pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread) { - uint64_t this_thread_requested_offset; + std::unique_lock lock(queue->mutex); + flush_event.wait_for(lock, + std::chrono::milliseconds(flush_interval_milliseconds), + [&] () + { + return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables; + } + ); - { - std::lock_guard lock(queue->mutex); - if (is_shutdown) - return uint64_t(-1); + queue_front_index += queue->size(); + to_flush_end = queue->queue_front_index; + // Swap with existing array from previous flush, to save memory + // allocations. + output.resize(0); + queue.swap(to_flush); - this_thread_requested_offset = queue->queue_front_index + queue->queue.size(); + should_prepare_tables_anyway = is_force_prepare_tables; - // Publish our flush request, taking care not to overwrite the requests - // made by other threads. - is_force_prepare_tables |= force; - queue->requested_flush_up_to = std::max(queue->requested_flush_up_to, this_thread_requested_offset); - - queue->flush_event.notify_all(); - } - - LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); - return this_thread_requested_offset; + exit_this_thread = is_shutdown; } +template +void SystemLogBase::add(const LogElement & element) +{ + queue->add(element); +} + +template +void SystemLogBase::flush(bool force) +{ + uint64_t this_thread_requested_offset = queue->notifyFlush(force); + if (this_thread_requested_offset == uint64_t(-1)) + return; + + queue->waitFlush(this_thread_requested_offset); +} + +template +void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } + +// template +// uint64_t SystemLogBase::notifyFlushImpl(bool force) +// { +// return queue->notifyFlush(force); +// } + #define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 109334964d2..0c1f32672a9 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -81,6 +81,10 @@ public: //void push_back(const LogElement & element) { queue.push_back(element); } void shutdown() { is_shutdown = true; } + uint64_t notifyFlush(bool force); + void waitFlush(uint64_t this_thread_requested_offset_); + void pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread) + // Queue is bounded. But its size is quite large to not block in all normal cases. std::vector queue; // An always-incrementing index of the first message currently in the queue. @@ -96,12 +100,16 @@ public: // Requested to flush logs up to this index, exclusive uint64_t requested_flush_up_to = 0; - // Logged overflow message at this queue front index - uint64_t logged_queue_full_at_index = -1; - + // A flag that says we must create the tables even if the queue is empty. + bool is_force_prepare_tables = false; + + // Flushed log up to this index, exclusive + uint64_t flushed_up_to = 0; private: Poco::Logger * log; bool is_shutdown = false; + // Logged overflow message at this queue front index + uint64_t logged_queue_full_at_index = -1; }; template @@ -138,21 +146,7 @@ protected: std::shared_ptr> queue; - // A flag that says we must create the tables even if the queue is empty. - bool is_force_prepare_tables = false; - - // Flushed log up to this index, exclusive - uint64_t flushed_up_to = 0; - bool is_shutdown = false; - - // Logged overflow message at this queue front index - uint64_t logged_queue_full_at_index = -1; - -private: - uint64_t notifyFlushImpl(bool force); - - }; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index b77cb2311d5..983c9f87c45 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -372,27 +372,7 @@ void SystemLog::savingThreadFunction() // Should we prepare table even if there are no new messages. bool should_prepare_tables_anyway = false; - { - std::unique_lock lock(queue->mutex); - queue->flush_event.wait_for(lock, - std::chrono::milliseconds(flush_interval_milliseconds), - [&] () - { - return queue->requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables; - } - ); - - queue->queue_front_index += queue->size(); - to_flush_end = queue->queue_front_index; - // Swap with existing array from previous flush, to save memory - // allocations. - to_flush.resize(0); - queue->queue.swap(to_flush); - - should_prepare_tables_anyway = is_force_prepare_tables; - - exit_this_thread = is_shutdown; - } + queue->pop(to_flush, to_flush_end, should_prepare_tables_anyway, exit_this_thread); if (to_flush.empty()) { @@ -402,7 +382,7 @@ void SystemLog::savingThreadFunction() LOG_TRACE(log, "Table created (force)"); std::lock_guard lock(queue->mutex); - is_force_prepare_tables = false; + queue->is_force_prepare_tables = false; queue->flush_event.notify_all(); } } @@ -477,8 +457,8 @@ void SystemLog::flushImpl(const std::vector & to_flush, { std::lock_guard lock(queue->mutex); - flushed_up_to = to_flush_end; - is_force_prepare_tables = false; + queue->flushed_up_to = to_flush_end; + queue->is_force_prepare_tables = false; queue->flush_event.notify_all(); } diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 3f5ad8a5b6c..c730aa23319 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -121,9 +121,9 @@ protected: using Base::log; using Base::queue; // using Base::queue_front_index; - using Base::is_force_prepare_tables; + // using Base::is_force_prepare_tables; //using Base::requested_flush_up_to; - using Base::flushed_up_to; + // using Base::flushed_up_to; // using Base::logged_queue_full_at_index; private: From 918e3f7712a65554d4b5d0ae279c812aca3298d2 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 18:41:12 +0200 Subject: [PATCH 0718/2047] Fix flaky test --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../0_stateless/01076_parallel_alter_replicated_zookeeper.sh | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6894368841f..7934ba7d8b5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1385,7 +1385,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) } const UInt64 parts_to_fetch_blocks = std::accumulate(parts_to_fetch.cbegin(), parts_to_fetch.cend(), 0, - [&](UInt64 acc, const String& part_name) + [&](UInt64 acc, const String & part_name) { if (const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, format_version)) return acc + part_info->getBlocksCount(); diff --git a/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh b/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh index 5f69427c0cd..79d2c736793 100755 --- a/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh @@ -138,8 +138,13 @@ while true ; do done for i in $(seq $REPLICAS); do + $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_mutate_mt_$i" + $CLICKHOUSE_CLIENT --query "CHECK TABLE concurrent_mutate_mt_$i" &> /dev/null # if we will remove something the output of select will be wrong $CLICKHOUSE_CLIENT --query "SELECT SUM(toUInt64(value1)) > $INITIAL_SUM FROM concurrent_mutate_mt_$i" $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE table='concurrent_mutate_mt_$i' and is_done=0" # all mutations have to be done $CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE table='concurrent_mutate_mt_$i' and is_done=0" # for verbose output +done + +for i in $(seq $REPLICAS); do $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_mt_$i" done From b70349852c4f5b37a0c8b5db172b9ac6b901fb24 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 19:02:29 +0200 Subject: [PATCH 0719/2047] Better message on timeout and fix test --- src/Storages/IStorage.cpp | 19 ++++++++++++++----- src/Storages/IStorage.h | 1 + src/Storages/StorageMergeTree.cpp | 8 +++++++- .../02543_alter_rename_modify_stuck.sh | 15 +++------------ 4 files changed, 25 insertions(+), 18 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 8cf708acd8b..d4f796f71ca 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -71,15 +71,12 @@ TableLockHolder IStorage::tryLockForShare(const String & query_id, const std::ch return result; } -IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds & acquire_timeout) +std::optional IStorage::tryLockForAlter(const std::chrono::milliseconds & acquire_timeout) { AlterLockHolder lock{alter_lock, std::defer_lock}; if (!lock.try_lock_for(acquire_timeout)) - throw Exception(ErrorCodes::DEADLOCK_AVOIDED, - "Locking attempt for ALTER on \"{}\" has timed out! ({} ms) " - "Possible deadlock avoided. Client should retry.", - getStorageID().getFullTableName(), acquire_timeout.count()); + return {} if (is_dropped || is_detached) throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is dropped or detached", getStorageID()); @@ -87,6 +84,18 @@ IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds return lock; } +IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds & acquire_timeout) +{ + + if (auto lock = tryLockForAlter(acquire_timeout); lock == std::nullopt) + throw Exception(ErrorCodes::DEADLOCK_AVOIDED, + "Locking attempt for ALTER on \"{}\" has timed out! ({} ms) " + "Possible deadlock avoided. Client should retry.", + getStorageID().getFullTableName(), acquire_timeout.count()); + else + return *lock; +} + TableExclusiveLockHolder IStorage::lockExclusively(const String & query_id, const std::chrono::milliseconds & acquire_timeout) { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index b262d88db57..76641b656a2 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -283,6 +283,7 @@ public: /// sure, that we execute only one simultaneous alter. Doesn't affect share lock. using AlterLockHolder = std::unique_lock; AlterLockHolder lockForAlter(const std::chrono::milliseconds & acquire_timeout); + std::optional tryLockForAlter(const std::chrono::milliseconds & acquire_timeout); /// Lock table exclusively. This lock must be acquired if you want to be /// sure, that no other thread (SELECT, merge, ALTER, etc.) doing something diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 8cc2fbc2911..a20f9ae12fa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -603,7 +603,13 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer { /// It's important to serialize order of mutations with alter queries because /// they can depend on each other. - auto alter_lock = lockForAlter(query_context->getSettings().lock_acquire_timeout); + if (auto alter_lock = tryLockForAlter(query_context->getSettings().lock_acquire_timeout); alter_lock == std::nullopt) + { + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, + "Cannot start mutation in {}ms because some metadata-changing ALTER (MODIFY|RENAME|ADD|DROP) is currently executing. " + "You can change this timeout with `lock_acquire_timeout` setting", + query_context->getSettings().lock_acquire_timeout) + } version = startMutation(commands, query_context); } diff --git a/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh index adaf1846552..8aa04c97937 100755 --- a/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh +++ b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh @@ -34,19 +34,10 @@ done $CLICKHOUSE_CLIENT --query="ALTER TABLE table_to_rename UPDATE v2 = 77 WHERE 1 = 1 SETTINGS mutations_sync = 2" & -counter=0 retries=60 - -I=0 -while [[ $counter -lt $retries ]]; do - I=$((I + 1)) - result=$($CLICKHOUSE_CLIENT --query "SELECT count() from system.mutations where database='${CLICKHOUSE_DATABASE}' and table='table_to_rename'") - if [[ $result == "2" ]]; then - break; - fi - sleep 0.1 - ((++counter)) -done +# we cannot wait in the same way like we do for previous alter +# because it's metadata alter and this one will wait for it +sleep 1 $CLICKHOUSE_CLIENT --query="SYSTEM START MERGES table_to_rename" From bfbf9f292efd5bc35952f91e920f8ae2bd85e5e2 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 19:03:04 +0200 Subject: [PATCH 0720/2047] Fix --- tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh index 8aa04c97937..1f517913076 100755 --- a/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh +++ b/tests/queries/0_stateless/02543_alter_rename_modify_stuck.sh @@ -37,7 +37,7 @@ $CLICKHOUSE_CLIENT --query="ALTER TABLE table_to_rename UPDATE v2 = 77 WHERE 1 = # we cannot wait in the same way like we do for previous alter # because it's metadata alter and this one will wait for it -sleep 1 +sleep 3 $CLICKHOUSE_CLIENT --query="SYSTEM START MERGES table_to_rename" From 1dbc88630d56f2b815c2c5746f6706d2ff76a6bc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 17:09:13 +0000 Subject: [PATCH 0721/2047] Cosmetics --- cmake/limit_jobs.cmake | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 39a6f688488..acc38b6fa2a 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -1,7 +1,11 @@ +# Limit compiler/linker job concurrency to avoid OOMs on subtrees where compilation/linking is memory-intensive. +# # Usage from CMake: -# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # megabyte -# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte -# include (cmake/limit_jobs.cmake) +# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # megabyte +# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte +# include (cmake/limit_jobs.cmake) +# +# (bigger values mean fewer jobs) cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) cmake_host_system_information(RESULT NUMBER_OF_LOGICAL_CORES QUERY NUMBER_OF_LOGICAL_CORES) @@ -17,7 +21,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") + message(WARNING "The auto-calculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") endif() endif () @@ -28,7 +32,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") + message(WARNING "The auto-calculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") endif() endif () @@ -43,7 +47,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLE set (PARALLEL_LINK_JOBS 2) endif() -message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs.") +message(STATUS "Building sub-tree with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs (system: ${NUMBER_OF_LOGICAL_CORES} cores, ${TOTAL_PHYSICAL_MEMORY} MB DRAM, 'OFF' means the native core count).") if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) From 2db14e36b4d187ed485891d862c8365a4ce7bf17 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 17 Jul 2023 19:15:07 +0200 Subject: [PATCH 0722/2047] Fix build --- src/Storages/IStorage.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index d4f796f71ca..ae7659e074f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -76,7 +76,7 @@ std::optional IStorage::tryLockForAlter(const std::ch AlterLockHolder lock{alter_lock, std::defer_lock}; if (!lock.try_lock_for(acquire_timeout)) - return {} + return {}; if (is_dropped || is_detached) throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {} is dropped or detached", getStorageID()); @@ -93,7 +93,7 @@ IStorage::AlterLockHolder IStorage::lockForAlter(const std::chrono::milliseconds "Possible deadlock avoided. Client should retry.", getStorageID().getFullTableName(), acquire_timeout.count()); else - return *lock; + return std::move(*lock); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a20f9ae12fa..353a647704e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -608,7 +608,7 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Cannot start mutation in {}ms because some metadata-changing ALTER (MODIFY|RENAME|ADD|DROP) is currently executing. " "You can change this timeout with `lock_acquire_timeout` setting", - query_context->getSettings().lock_acquire_timeout) + query_context->getSettings().lock_acquire_timeout.totalMilliseconds()); } version = startMutation(commands, query_context); } From e8b3310319793250e7c019d690fa3763ff91e8ea Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Mon, 17 Jul 2023 14:35:07 -0400 Subject: [PATCH 0723/2047] touch buffer in advance to avoid page fault --- .../building_and_benchmarking_deflate_qpl.md | 11 ++++++++--- src/Compression/CompressionCodecDeflateQpl.cpp | 7 +++++++ 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl.md b/docs/en/development/building_and_benchmarking_deflate_qpl.md index 0501c1cbdcb..baad1f3ea2a 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl.md +++ b/docs/en/development/building_and_benchmarking_deflate_qpl.md @@ -7,11 +7,16 @@ description: How to build Clickhouse and run benchmark with DEFLATE_QPL Codec # Build Clickhouse with DEFLATE_QPL -- Make sure your target machine meet the QPL required [prerequisites](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#prerequisites) -- Pass the following flag to CMake when building ClickHouse: +- Make sure your host machine meet the QPL required [prerequisites](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#prerequisites) +- For Clickhouse version [v23.5.1.3174-stable](docs/changelogs/v23.5.1.3174-stable.md) and later, deflate_qpl is enabled by default which means no additional flags required pass to cmake build. +- For Clickhouse version between [v22.8.1.2097-lts](docs/changelogs/v22.8.1.2097-lts.md) and [v23.5.1.3174-stable](docs/changelogs/v23.5.1.3174-stable.md), you need pass the following flag to CMake when building ClickHouse: ``` bash -cmake -DENABLE_QPL=1 .. +cmake -DAVX512=1 -DENABLE_QPL=1 .. +``` +or +``` bash +cmake -DAVX2=1 -DENABLE_QPL=1 .. ``` - For generic requirements, please refer to Clickhouse generic [build instructions](/docs/en/development/build.md) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 859a9c2463a..41e6a365b6c 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -405,6 +405,13 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 so #if defined(MEMORY_SANITIZER) __msan_unpoison(dest, uncompressed_size); #endif +/// Device IOTLB miss has big perf. impact for IAA accelerators. +/// To avoid page fault, we need touch buffers related to accelerator in advance. + for (char * p = dest; p < dest + uncompressed_size; p += ::getPageSize()/(sizeof(*p))) + { + *p = 0; + } + switch (getDecompressMode()) { case CodecMode::Synchronous: From a955d5621d611dd1f1159a008141a8538575b6db Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jul 2023 18:53:12 +0000 Subject: [PATCH 0724/2047] Rename constants in KeyCondition analysis. --- src/Storages/MergeTree/KeyCondition.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index f909d854cf6..3f02a6b197e 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -564,7 +564,17 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } case (ActionsDAG::ActionType::COLUMN): { - res = &inverted_dag.addColumn({node.column, node.result_type, node.result_name}); + String name; + if (const auto * column_const = typeid_cast(node.column.get())) + /// Re-generate column name for constant. + /// DAG form query (with enabled analyzer) uses suffixes for constants, like 1_UInt8. + /// DAG from PK does not use it. This is breakig match by column name sometimes. + /// Ideally, we should not compare manes, but DAG subtrees instead. + name = ASTLiteral(column_const->getDataColumn()[0]).getColumnName(); + else + name = node.result_name; + + res = &inverted_dag.addColumn({node.column, node.result_type, name}); break; } case (ActionsDAG::ActionType::ALIAS): From eb8629db5138b79cedc4096ecaa96c58e228d307 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 23:17:47 +0300 Subject: [PATCH 0725/2047] Update 02815_logical_error_cannot_get_column_name_of_set.sql --- .../02815_logical_error_cannot_get_column_name_of_set.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql index 98a8a8992b8..aa659165940 100644 --- a/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql +++ b/tests/queries/0_stateless/02815_logical_error_cannot_get_column_name_of_set.sql @@ -1,3 +1,3 @@ SELECT * FROM numbers(SETTINGS x = 1); -- { serverError BAD_ARGUMENTS } -SELECT * FROM numbers(numbers(SETTINGS x = 1)); -- { serverError UNKNOWN_FUNCTION } -SELECT * FROM numbers(numbers(SETTINGS x = 1), SETTINGS x = 1); -- { serverError UNKNOWN_FUNCTION } +SELECT * FROM numbers(numbers(SETTINGS x = 1)); -- { serverError UNKNOWN_FUNCTION, UNSUPPORTED_METHOD } +SELECT * FROM numbers(numbers(SETTINGS x = 1), SETTINGS x = 1); -- { serverError UNKNOWN_FUNCTION, UNSUPPORTED_METHOD } From 4043174a81f03d0ad272a5c5c5b3e1dc46a6968f Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 17 Jul 2023 23:46:45 +0200 Subject: [PATCH 0726/2047] fix special build --- src/Storages/MergeTree/MergeTreeDataWriter.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index d6c980d83d8..2fb6b1f22d4 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -52,7 +52,7 @@ public: * (split rows by partition) * Works deterministically: if same block was passed, function will return same result in same order. */ - static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, AsyncInsertInfoPtr chunk_offsets = nullptr); + static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, AsyncInsertInfoPtr async_insert_info = nullptr); /// This structure contains not completely written temporary part. /// Some writes may happen asynchronously, e.g. for blob storages. From 0902ee1ec7c191cce834467384ab8b4ce4db3ac5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 01:36:44 +0200 Subject: [PATCH 0727/2047] Remove useless header --- src/Common/DateLUTImpl.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index d7bdd0bb3d9..8146b35cc5f 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include From b5f19371339ae1b98efae9421c9d6d9bd94a75fd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 01:48:12 +0200 Subject: [PATCH 0728/2047] Update more tests --- .../0_stateless/00753_system_columns_and_system_tables_long.sql | 2 +- .../0_stateless/02241_filesystem_cache_on_write_operations.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql index f4c4110cd5b..16085c8a995 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql @@ -12,7 +12,7 @@ CREATE TABLE check_system_tables ORDER BY name1 PARTITION BY name2 SAMPLE BY name1 - SETTINGS min_bytes_for_wide_part = 0, compress_marks=false, compress_primary_key=false; + SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; SELECT name, partition_key, sorting_key, primary_key, sampling_key, storage_policy, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase() diff --git a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh index e65bf9cb35f..3a22ddfd7f5 100755 --- a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh +++ b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh @@ -11,7 +11,7 @@ for STORAGE_POLICY in 's3_cache' 'local_cache'; do echo "Using storage policy: $STORAGE_POLICY" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_02241" - $CLICKHOUSE_CLIENT --query "CREATE TABLE test_02241 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='$STORAGE_POLICY', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false" + $CLICKHOUSE_CLIENT --query "CREATE TABLE test_02241 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='$STORAGE_POLICY', min_bytes_for_wide_part = 10485760, compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization = 1" $CLICKHOUSE_CLIENT --query "SYSTEM STOP MERGES test_02241" $CLICKHOUSE_CLIENT --query "SYSTEM DROP FILESYSTEM CACHE" From a4b92652da148bef3cf13d0182378ba8ca366c24 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 01:50:43 +0200 Subject: [PATCH 0729/2047] Update more tests --- tests/integration/test_multiple_disks/test.py | 2 +- tests/integration/test_partition/test.py | 20 +++++++++---------- .../test_polymorphic_parts/test.py | 2 +- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index bc7ac6683af..0724791c940 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -1718,7 +1718,7 @@ def test_freeze(start_cluster): ) ENGINE = MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(d) - SETTINGS storage_policy='small_jbod_with_external', compress_marks=false, compress_primary_key=false + SETTINGS storage_policy='small_jbod_with_external', compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1 """ ) diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index 93f03f4420e..2517b2d1ae6 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -38,7 +38,7 @@ def partition_table_simple(started_cluster): q( "CREATE TABLE test.partition_simple (date MATERIALIZED toDate(0), x UInt64, sample_key MATERIALIZED intHash64(x)) " "ENGINE=MergeTree PARTITION BY date SAMPLE BY sample_key ORDER BY (date,x,sample_key) " - "SETTINGS index_granularity=8192, index_granularity_bytes=0, compress_marks=false, compress_primary_key=false" + "SETTINGS index_granularity=8192, index_granularity_bytes=0, compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1" ) q("INSERT INTO test.partition_simple ( x ) VALUES ( now() )") q("INSERT INTO test.partition_simple ( x ) VALUES ( now()+1 )") @@ -150,7 +150,7 @@ def partition_table_complex(started_cluster): q("DROP TABLE IF EXISTS test.partition_complex") q( "CREATE TABLE test.partition_complex (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) " - "ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0, compress_marks=false, compress_primary_key=false" + "ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0, compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1" ) q("INSERT INTO test.partition_complex (p, k) VALUES(toDate(31), 1)") q("INSERT INTO test.partition_complex (p, k) VALUES(toDate(1), 2)") @@ -188,7 +188,7 @@ def test_partition_complex(partition_table_complex): def cannot_attach_active_part_table(started_cluster): q("DROP TABLE IF EXISTS test.attach_active") q( - "CREATE TABLE test.attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n SETTINGS compress_marks=false, compress_primary_key=false" + "CREATE TABLE test.attach_active (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 4) ORDER BY n SETTINGS compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1" ) q("INSERT INTO test.attach_active SELECT number FROM system.numbers LIMIT 16") @@ -217,7 +217,7 @@ def attach_check_all_parts_table(started_cluster): q("DROP TABLE IF EXISTS test.attach_partition") q( "CREATE TABLE test.attach_partition (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n " - "SETTINGS compress_marks=false, compress_primary_key=false, old_parts_lifetime=0" + "SETTINGS compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1, old_parts_lifetime=0" ) q( "INSERT INTO test.attach_partition SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8" @@ -299,7 +299,7 @@ def drop_detached_parts_table(started_cluster): q("SYSTEM STOP MERGES") q("DROP TABLE IF EXISTS test.drop_detached") q( - "CREATE TABLE test.drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n SETTINGS compress_marks=false, compress_primary_key=false" + "CREATE TABLE test.drop_detached (n UInt64) ENGINE = MergeTree() PARTITION BY intDiv(n, 8) ORDER BY n SETTINGS compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1" ) q( "INSERT INTO test.drop_detached SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 8" @@ -370,13 +370,13 @@ def test_drop_detached_parts(drop_detached_parts_table): def test_system_detached_parts(drop_detached_parts_table): q( - "create table sdp_0 (n int, x int) engine=MergeTree order by n SETTINGS compress_marks=false, compress_primary_key=false" + "create table sdp_0 (n int, x int) engine=MergeTree order by n SETTINGS compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1" ) q( - "create table sdp_1 (n int, x int) engine=MergeTree order by n partition by x SETTINGS compress_marks=false, compress_primary_key=false" + "create table sdp_1 (n int, x int) engine=MergeTree order by n partition by x SETTINGS compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1" ) q( - "create table sdp_2 (n int, x String) engine=MergeTree order by n partition by x SETTINGS compress_marks=false, compress_primary_key=false" + "create table sdp_2 (n int, x String) engine=MergeTree order by n partition by x SETTINGS compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1" ) q( "create table sdp_3 (n int, x Enum('broken' = 0, 'all' = 1)) engine=MergeTree order by n partition by x" @@ -497,7 +497,7 @@ def test_system_detached_parts(drop_detached_parts_table): def test_detached_part_dir_exists(started_cluster): q( "create table detached_part_dir_exists (n int) engine=MergeTree order by n " - "SETTINGS compress_marks=false, compress_primary_key=false, old_parts_lifetime=0" + "SETTINGS compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1, old_parts_lifetime=0" ) q("insert into detached_part_dir_exists select 1") # will create all_1_1_0 q( @@ -549,7 +549,7 @@ def test_detached_part_dir_exists(started_cluster): def test_make_clone_in_detached(started_cluster): q( - "create table clone_in_detached (n int, m String) engine=ReplicatedMergeTree('/clone_in_detached', '1') order by n SETTINGS compress_marks=false, compress_primary_key=false" + "create table clone_in_detached (n int, m String) engine=ReplicatedMergeTree('/clone_in_detached', '1') order by n SETTINGS compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1" ) path = path_to_data + "data/default/clone_in_detached/" diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index fb1f363b825..c5859146fe9 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -498,7 +498,7 @@ def test_polymorphic_parts_index(start_cluster): """ CREATE TABLE test_index.index_compact(a UInt32, s String) ENGINE = MergeTree ORDER BY a - SETTINGS min_rows_for_wide_part = 1000, index_granularity = 128, merge_max_block_size = 100, compress_marks=false, compress_primary_key=false""" + SETTINGS min_rows_for_wide_part = 1000, index_granularity = 128, merge_max_block_size = 100, compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1""" ) node1.query( From 806caea2821480f04cb26a07842ce06ed3d0f319 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 01:56:38 +0200 Subject: [PATCH 0730/2047] Update more tests --- .../test_merge_tree_s3/configs/config.d/storage_conf.xml | 1 + .../configs/config.d/storage_conf.xml | 1 + .../configs/config.d/storage_conf.xml | 1 + tests/integration/test_s3_zero_copy_ttl/configs/s3.xml | 1 + 4 files changed, 4 insertions(+) diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index 504280e4bed..4f0e2db9b08 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -152,6 +152,7 @@ 0 + 1.0 0 diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml index 00aa03b1a92..829bf16fdfb 100644 --- a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml @@ -35,6 +35,7 @@ 0 0 + 1.0 diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml index 96d59d5633e..f78256bdb26 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/configs/config.d/storage_conf.xml @@ -29,6 +29,7 @@ 0 true + 1.0 diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml index e179c848be1..7bb7fa875e4 100644 --- a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml +++ b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml @@ -32,6 +32,7 @@ true + 1.0 true From e4f798fcda5bd912d9e5a14bb087fb32ea8a842e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 01:36:44 +0200 Subject: [PATCH 0731/2047] Remove useless header --- src/Common/DateLUTImpl.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index d7bdd0bb3d9..8146b35cc5f 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include From bc9bf009af1423a2528b169bbbdc13bc9bb04340 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 02:27:39 +0200 Subject: [PATCH 0732/2047] Inhibit settings randomization in the test `json_ghdata`. --- tests/queries/0_stateless/01825_type_json_ghdata.sh | 2 +- .../0_stateless/01825_type_json_ghdata_insert_select.sh | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01825_type_json_ghdata.sh b/tests/queries/0_stateless/01825_type_json_ghdata.sh index bdb439f756f..cea02131d86 100755 --- a/tests/queries/0_stateless/01825_type_json_ghdata.sh +++ b/tests/queries/0_stateless/01825_type_json_ghdata.sh @@ -6,7 +6,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_object_type 1 cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata FORMAT JSONAsObject" diff --git a/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh b/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh index 487c95137ae..498e1db1f69 100755 --- a/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh +++ b/tests/queries/0_stateless/01825_type_json_ghdata_insert_select.sh @@ -9,9 +9,9 @@ ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2_string" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS ghdata_2_from_string" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2 (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1 -${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_string (data String) ENGINE = MergeTree ORDER BY tuple()" -${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_from_string (data JSON) ENGINE = MergeTree ORDER BY tuple()" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2 (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_string (data String) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE ghdata_2_from_string (data JSON) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'" --allow_experimental_object_type 1 cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata_2 FORMAT JSONAsObject" cat $CUR_DIR/data_json/ghdata_sample.json | ${CLICKHOUSE_CLIENT} -q "INSERT INTO ghdata_2_string FORMAT JSONAsString" From 182d8b8f081d73f07877ab1ce7269288b1cf70b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 01:36:44 +0200 Subject: [PATCH 0733/2047] Remove useless header --- src/Common/DateLUTImpl.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index d7bdd0bb3d9..8146b35cc5f 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include From 8a4f939e1af9337c99ade273b7ba4e992a7c6eea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 02:37:50 +0200 Subject: [PATCH 0734/2047] Slightly better diagnostics in tests --- tests/queries/0_stateless/02725_memory-for-merges.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02725_memory-for-merges.sql b/tests/queries/0_stateless/02725_memory-for-merges.sql index b6ae7af7f1a..347c8b2a8d3 100644 --- a/tests/queries/0_stateless/02725_memory-for-merges.sql +++ b/tests/queries/0_stateless/02725_memory-for-merges.sql @@ -22,6 +22,6 @@ OPTIMIZE TABLE 02725_memory_for_merges FINAL; SYSTEM FLUSH LOGS; WITH (SELECT uuid FROM system.tables WHERE table='02725_memory_for_merges' and database=currentDatabase()) as uuid -SELECT sum(peak_memory_usage) < 1024 * 1024 * 200 from system.part_log where table_uuid=uuid and event_type='MergeParts'; +SELECT (sum(peak_memory_usage) < 1024 * 1024 * 200 AS x) ? x : sum(peak_memory_usage) from system.part_log where table_uuid=uuid and event_type='MergeParts'; DROP TABLE IF EXISTS 02725_memory_for_merges SYNC; From d8dd480cd78834841a1173b67e1cdf44201898d7 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 17 Jul 2023 00:15:11 +0800 Subject: [PATCH 0735/2047] Disable implicit projections by default --- src/Core/Settings.h | 1 + src/Processors/QueryPlan/Optimizations/Optimizations.h | 2 +- .../Optimizations/QueryPlanOptimizationSettings.cpp | 1 + .../Optimizations/QueryPlanOptimizationSettings.h | 1 + .../QueryPlan/Optimizations/optimizeTree.cpp | 3 ++- .../Optimizations/optimizeUseAggregateProjection.cpp | 10 ++++++---- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++- 7 files changed, 14 insertions(+), 7 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8f304f0aab6..dcac91eaa75 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -577,6 +577,7 @@ class IColumn; M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) ALIAS(allow_experimental_projection_optimization) \ + M(Bool, optimize_use_implicit_projections, false, "Automatically choose implicit projections to perform SELECT query", 0) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \ diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index bc47413cbb5..6ecec1359c5 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -111,7 +111,7 @@ void optimizePrimaryKeyCondition(const Stack & stack); void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); -bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes); +bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections); bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes); diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index 21c7d362b17..e011fb8ecbe 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -19,6 +19,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.remove_redundant_distinct = from.query_plan_remove_redundant_distinct; settings.optimize_projection = from.optimize_use_projections && from.query_plan_optimize_projection; settings.force_use_projection = settings.optimize_projection && from.force_optimize_projection; + settings.optimize_use_implicit_projections = settings.optimize_projection && from.optimize_use_implicit_projections; return settings; } diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 967cfdaca7f..d98c34ce226 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -41,6 +41,7 @@ struct QueryPlanOptimizationSettings /// If reading from projection can be applied bool optimize_projection = false; bool force_use_projection = false; + bool optimize_use_implicit_projections = false; static QueryPlanOptimizationSettings fromSettings(const Settings & from); static QueryPlanOptimizationSettings fromContext(ContextPtr from); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index e788918703e..6cb76d540f7 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -126,7 +126,8 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizeReadInOrder(*frame.node, nodes); if (optimization_settings.optimize_projection) - num_applied_projection += optimizeUseAggregateProjections(*frame.node, nodes); + num_applied_projection + += optimizeUseAggregateProjections(*frame.node, nodes, optimization_settings.optimize_use_implicit_projections); if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 6e76d6e67a7..f183bdca7a9 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -433,7 +433,8 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( QueryPlan::Node & node, AggregatingStep & aggregating, ReadFromMergeTree & reading, - const std::shared_ptr & max_added_blocks) + const std::shared_ptr & max_added_blocks, + bool allow_implicit_projections) { const auto & keys = aggregating.getParams().keys; const auto & aggregates = aggregating.getParams().aggregates; @@ -453,7 +454,8 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( if (projection.type == ProjectionDescription::Type::Aggregate) agg_projections.push_back(&projection); - bool can_use_minmax_projection = metadata->minmax_count_projection && !reading.getMergeTreeData().has_lightweight_delete_parts.load(); + bool can_use_minmax_projection = allow_implicit_projections && metadata->minmax_count_projection + && !reading.getMergeTreeData().has_lightweight_delete_parts.load(); if (!can_use_minmax_projection && agg_projections.empty()) return candidates; @@ -543,7 +545,7 @@ static QueryPlan::Node * findReadingStep(QueryPlan::Node & node) return nullptr; } -bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections) { if (node.children.size() != 1) return false; @@ -568,7 +570,7 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & std::shared_ptr max_added_blocks = getMaxAddedBlocks(reading); - auto candidates = getAggregateProjectionCandidates(node, *aggregating, *reading, max_added_blocks); + auto candidates = getAggregateProjectionCandidates(node, *aggregating, *reading, max_added_blocks, allow_implicit_projections); AggregateProjectionCandidate * best_candidate = nullptr; if (candidates.minmax_projection) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ec8ce3f5e3d..9c5e45aa488 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6994,7 +6994,8 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg ProjectionCandidate * selected_candidate = nullptr; size_t min_sum_marks = std::numeric_limits::max(); - if (metadata_snapshot->minmax_count_projection && !has_lightweight_delete_parts.load(std::memory_order_relaxed)) /// Disable ReadFromStorage for parts with lightweight. + if (settings.optimize_use_implicit_projections && metadata_snapshot->minmax_count_projection + && !has_lightweight_delete_parts.load(std::memory_order_relaxed)) /// Disable ReadFromStorage for parts with lightweight. add_projection_candidate(*metadata_snapshot->minmax_count_projection, true); std::optional minmax_count_projection_candidate; if (!candidates.empty()) From 796a92d9ab49298c208b3809f20640f87ac3a08d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 17 Jul 2023 10:17:33 +0800 Subject: [PATCH 0736/2047] Fix tests --- .../01505_trivial_count_with_partition_predicate.sql | 1 + tests/queries/0_stateless/01710_minmax_count_projection.sql | 2 +- tests/queries/0_stateless/01739_index_hint.reference | 2 +- tests/queries/0_stateless/01739_index_hint.sql | 2 +- .../0_stateless/02680_illegal_type_of_filter_projection.sql | 2 +- 5 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql b/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql index 030db421683..aaf88f95f0c 100644 --- a/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql +++ b/tests/queries/0_stateless/01505_trivial_count_with_partition_predicate.sql @@ -6,6 +6,7 @@ create table test1(p DateTime, k int) engine MergeTree partition by toDate(p) or insert into test1 values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); set max_rows_to_read = 1; +set optimize_use_implicit_projections = 1; -- non-optimized select count() from test1 settings max_parallel_replicas = 3; -- optimized (toYear is monotonic and we provide the partition expr as is) diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index c17f0e1e1fb..bc8327e3631 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -4,7 +4,7 @@ create table d (i int, j int) engine MergeTree partition by i % 2 order by tuple insert into d select number, number from numbers(10000); -set max_rows_to_read = 2, optimize_use_projections = 1; +set max_rows_to_read = 2, optimize_use_projections = 1, optimize_use_implicit_projections = 1; select min(i), max(i), count() from d; select min(i), max(i), count() from d group by _partition_id order by _partition_id; diff --git a/tests/queries/0_stateless/01739_index_hint.reference b/tests/queries/0_stateless/01739_index_hint.reference index 71dfab29154..3a4b380de65 100644 --- a/tests/queries/0_stateless/01739_index_hint.reference +++ b/tests/queries/0_stateless/01739_index_hint.reference @@ -30,6 +30,6 @@ SELECT sum(t) FROM XXXX WHERE indexHint(t = 42); drop table if exists XXXX; create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=8192; insert into XXXX select number*60, 0 from numbers(100000); -SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)); +SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_use_implicit_projections = 1; 100000 drop table XXXX; diff --git a/tests/queries/0_stateless/01739_index_hint.sql b/tests/queries/0_stateless/01739_index_hint.sql index 30dfa43d334..e1e66c630e1 100644 --- a/tests/queries/0_stateless/01739_index_hint.sql +++ b/tests/queries/0_stateless/01739_index_hint.sql @@ -30,6 +30,6 @@ create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings inde insert into XXXX select number*60, 0 from numbers(100000); -SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)); +SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_use_implicit_projections = 1; drop table XXXX; diff --git a/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql b/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql index 3ef3b8a4fe6..d20e4deee27 100644 --- a/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql +++ b/tests/queries/0_stateless/02680_illegal_type_of_filter_projection.sql @@ -1,3 +1,3 @@ CREATE TABLE test_tuple (`p` DateTime, `i` int, `j` int) ENGINE = MergeTree PARTITION BY (toDate(p), i) ORDER BY j SETTINGS index_granularity = 1; insert into test_tuple values (1, 1, 1); -SELECT count() FROM test_tuple PREWHERE sipHash64(sipHash64(p, toString(toDate(p))), toString(toDate(p))) % -0. WHERE i > NULL settings optimize_trivial_count_query=0; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } +SELECT count() FROM test_tuple PREWHERE sipHash64(sipHash64(p, toString(toDate(p))), toString(toDate(p))) % -0. WHERE i > NULL settings optimize_trivial_count_query=0, optimize_use_implicit_projections=1; -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } From ef521fd6ce79798a2661bce13050d4bc350ecc41 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 17 Jul 2023 14:36:18 +0800 Subject: [PATCH 0737/2047] Fix more tests --- tests/queries/0_stateless/01848_partition_value_column.sql | 2 ++ .../02675_profile_events_from_query_log_and_client.reference | 4 ++-- .../02675_profile_events_from_query_log_and_client.sh | 4 ++-- tests/queries/1_stateful/00172_early_constant_folding.sql | 1 + 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01848_partition_value_column.sql b/tests/queries/0_stateless/01848_partition_value_column.sql index d13e4508789..28d842af3e9 100644 --- a/tests/queries/0_stateless/01848_partition_value_column.sql +++ b/tests/queries/0_stateless/01848_partition_value_column.sql @@ -5,6 +5,8 @@ create table tbl(dt DateTime, i int, j String, v Float64) engine MergeTree parti insert into tbl values ('2021-04-01 00:01:02', 1, '123', 4), ('2021-04-01 01:01:02', 1, '12', 4), ('2021-04-01 02:11:02', 2, '345', 4), ('2021-04-01 04:31:02', 2, '2', 4), ('2021-04-02 00:01:02', 1, '1234', 4), ('2021-04-02 00:01:02', 2, '123', 4), ('2021-04-02 00:01:02', 3, '12', 4), ('2021-04-02 00:01:02', 4, '1', 4); +set optimize_use_implicit_projections = 1; + select count() from tbl where _partition_value = ('2021-04-01', 1, 2) settings max_rows_to_read = 1; select count() from tbl where _partition_value.1 = '2021-04-01' settings max_rows_to_read = 4; select count() from tbl where _partition_value.2 = 0 settings max_rows_to_read = 4; diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference index 3f34d5e2c79..a97879eaca8 100644 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.reference @@ -17,7 +17,7 @@ INSERT and READ INSERT DROP CHECK with query_log QueryFinish INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0; FileOpen 8 -QueryFinish SELECT \'1\', min(t) FROM times; FileOpen 0 +QueryFinish SELECT \'1\', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; FileOpen 0 QueryFinish INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0; FileOpen 8 -QueryFinish SELECT \'2\', min(t) FROM times; FileOpen 0 +QueryFinish SELECT \'2\', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; FileOpen 0 QueryFinish INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0; FileOpen 8 diff --git a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh index adc9525ef81..288f1129b53 100755 --- a/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh +++ b/tests/queries/0_stateless/02675_profile_events_from_query_log_and_client.sh @@ -44,13 +44,13 @@ INSERT INTO times SELECT now() + INTERVAL 1 day SETTINGS optimize_on_insert = 0; echo "READ" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " -SELECT '1', min(t) FROM times; +SELECT '1', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; " 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' echo "INSERT and READ INSERT" $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -nq " INSERT INTO times SELECT now() + INTERVAL 2 day SETTINGS optimize_on_insert = 0; -SELECT '2', min(t) FROM times; +SELECT '2', min(t) FROM times SETTINGS optimize_use_implicit_projections = 1; INSERT INTO times SELECT now() + INTERVAL 3 day SETTINGS optimize_on_insert = 0; " 2>&1 | grep -o -e '\ \[\ .*\ \]\ FileOpen:\ .*\ ' diff --git a/tests/queries/1_stateful/00172_early_constant_folding.sql b/tests/queries/1_stateful/00172_early_constant_folding.sql index 1ed7b8719b4..19f99f107ac 100644 --- a/tests/queries/1_stateful/00172_early_constant_folding.sql +++ b/tests/queries/1_stateful/00172_early_constant_folding.sql @@ -1,4 +1,5 @@ -- Tags: no-parallel-replicas set max_threads=10; +set optimize_use_implicit_projections=1; EXPLAIN PIPELINE SELECT count(JavaEnable) FROM test.hits WHERE WatchID = 1 OR Title = 'next' OR URL = 'prev' OR URL = '???' OR 1; From c782689c2d64e0a3bfdc97bcd4c9f53a00622d7f Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 18 Jul 2023 09:49:45 +0800 Subject: [PATCH 0738/2047] added to compatibility setting --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 2886cdd288d..3e58750e1d2 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,6 +80,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"23.7", {{"optimize_use_implicit_projections", true, false, "Disable implicit projections due to unexpected results."}}}, {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, From 56a5446af3958d37c0f2cbc8e879beb3876b8857 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 18 Jul 2023 10:28:51 +0800 Subject: [PATCH 0739/2047] format Signed-off-by: Lloyd-Pottiger --- docs/en/sql-reference/functions/date-time-functions.md | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 9a42c3ec835..66e4f6bc9eb 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1130,10 +1130,12 @@ Result: └──────────────────────────────────────────────────────────────┘ ``` -## now/current_timestamp +## now Returns the current date and time at the moment of query analysis. The function is a constant expression. +Alias: `current_timestamp`. + **Syntax** ``` sql @@ -1259,11 +1261,13 @@ Result: └─────────────────────┴─────────────────────┴──────────┘ ``` -## today/curdate/current_date +## today Accepts zero arguments and returns the current date at one of the moments of query analysis. The same as ‘toDate(now())’. +Aliases: `curdate`, `current_date`. + ## yesterday Accepts zero arguments and returns yesterday’s date at one of the moments of query analysis. From f3dcf4db3a30f552932aa08b72116935de9d519d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 18 Jul 2023 11:00:39 +0800 Subject: [PATCH 0740/2047] update 02815_range_dict_no_direct_join.sql --- tests/queries/0_stateless/02815_range_dict_no_direct_join.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql index 249ffdf8089..f4e5ccccf78 100644 --- a/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql +++ b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql @@ -30,4 +30,4 @@ RANGE(MIN discount_start_date MAX discount_end_date); CREATE TABLE ids (id UInt64) ENGINE = Memory; INSERT INTO ids SELECT * FROM numbers(10); -SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id SETTINGS join_algorithm = 'direct'; \ No newline at end of file +SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id ORDER BY id, amount SETTINGS join_algorithm = 'direct'; From 9b7df9e12e9e85e1a2ecb2788c56dff74b75b309 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 18 Jul 2023 05:51:55 +0000 Subject: [PATCH 0741/2047] fix for analyzer --- src/Interpreters/TableJoin.h | 4 ++-- src/Planner/PlannerJoins.cpp | 3 ++- .../0_stateless/02815_range_dict_no_direct_join.reference | 6 ++++++ .../queries/0_stateless/02815_range_dict_no_direct_join.sql | 1 + 4 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index ba3befab59b..5d14a57759f 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -223,10 +223,10 @@ public: { /// When join_algorithm = 'default' (not specified by user) we use hash or direct algorithm. /// It's behaviour that was initially supported by clickhouse. - bool is_enbaled_by_default = val == JoinAlgorithm::DEFAULT + bool is_enabled_by_default = val == JoinAlgorithm::DEFAULT || val == JoinAlgorithm::HASH || val == JoinAlgorithm::DIRECT; - if (join_algorithm.isSet(JoinAlgorithm::DEFAULT) && is_enbaled_by_default) + if (join_algorithm.isSet(JoinAlgorithm::DEFAULT) && is_enabled_by_default) return true; return join_algorithm.isSet(val); } diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 7da10a8523b..e495b0967e9 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -542,7 +542,8 @@ void trySetStorageInTableJoin(const QueryTreeNodePtr & table_expression, std::sh if (!table_join->isEnabledAlgorithm(JoinAlgorithm::DIRECT)) return; - if (auto storage_dictionary = std::dynamic_pointer_cast(storage); storage_dictionary) + if (auto storage_dictionary = std::dynamic_pointer_cast(storage); + storage_dictionary && storage_dictionary->getDictionary()->getSpecialKeyType() != DictionarySpecialKeyType::Range) table_join->setStorageJoin(std::dynamic_pointer_cast(storage_dictionary->getDictionary())); else if (auto storage_key_value = std::dynamic_pointer_cast(storage); storage_key_value) table_join->setStorageJoin(storage_key_value); diff --git a/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference b/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference index 3576991f7a4..051716a791e 100644 --- a/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference +++ b/tests/queries/0_stateless/02815_range_dict_no_direct_join.reference @@ -4,3 +4,9 @@ 2 0.4 3 0.5 3 0.6 +1 0.1 +1 0.2 +2 0.3 +2 0.4 +3 0.5 +3 0.6 diff --git a/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql index f4e5ccccf78..e3af53fa335 100644 --- a/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql +++ b/tests/queries/0_stateless/02815_range_dict_no_direct_join.sql @@ -31,3 +31,4 @@ CREATE TABLE ids (id UInt64) ENGINE = Memory; INSERT INTO ids SELECT * FROM numbers(10); SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id ORDER BY id, amount SETTINGS join_algorithm = 'direct'; +SELECT id, amount FROM ids INNER JOIN discounts_dict ON id = advertiser_id ORDER BY id, amount SETTINGS allow_experimental_analyzer = 1; From 2f6bf4590b465e84641f090cfa1ee80c3cb28602 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 7 Jul 2023 17:13:41 +0200 Subject: [PATCH 0742/2047] Filter databases list before querying potentially slow fields --- .../System/StorageSystemDatabases.cpp | 38 ++++++++++++++++++- 1 file changed, 36 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index a3d05281b28..ee8a1de9b24 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include @@ -69,20 +71,52 @@ static String getEngineFull(const ContextPtr & ctx, const DatabasePtr & database return engine_full; } -void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +static ColumnPtr getFilteredDatabases(const Databases & databases, const SelectQueryInfo & query_info, ContextPtr context) +{ + MutableColumnPtr name_column = ColumnString::create(); + MutableColumnPtr engine_column = ColumnString::create(); + MutableColumnPtr uuid_column = ColumnUUID::create(); + + for (const auto & [database_name, database] : databases) + { + if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) + continue; /// We don't want to show the internal database for temporary tables in system.tables + + name_column->insert(database_name); + engine_column->insert(database->getEngineName()); + uuid_column->insert(database->getUUID()); + } + + Block block + { + ColumnWithTypeAndName(std::move(name_column), std::make_shared(), "name"), + ColumnWithTypeAndName(std::move(engine_column), std::make_shared(), "engine"), + ColumnWithTypeAndName(std::move(uuid_column), std::make_shared(), "uuid") + }; + VirtualColumnUtils::filterBlockWithQuery(query_info.query, block, context); + return block.getByPosition(0).column; +} + +void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const { const auto access = context->getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_DATABASES); const auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & [database_name, database] : databases) + ColumnPtr filtered_databases_column = getFilteredDatabases(databases, query_info, context); + + for (size_t i = 0; i < filtered_databases_column->size(); ++i) { + auto database_name = filtered_databases_column->getDataAt(i).toString(); + if (check_access_for_databases && !access->isGranted(AccessType::SHOW_DATABASES, database_name)) continue; if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) continue; /// filter out the internal database for temporary tables in system.databases, asynchronous metric "NumberOfDatabases" behaves the same way + const auto database = databases.at(database_name); + res_columns[0]->insert(database_name); res_columns[1]->insert(database->getEngineName()); res_columns[2]->insert(context->getPath() + database->getDataPath()); From 44595b98d0c7bd25bd92da69d4b80907d81b25db Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 17 Jul 2023 21:20:33 +0200 Subject: [PATCH 0743/2047] Update src/Storages/System/StorageSystemDatabases.cpp Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- src/Storages/System/StorageSystemDatabases.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index ee8a1de9b24..2fcc91e49bb 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -115,7 +115,7 @@ void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr c if (database_name == DatabaseCatalog::TEMPORARY_DATABASE) continue; /// filter out the internal database for temporary tables in system.databases, asynchronous metric "NumberOfDatabases" behaves the same way - const auto database = databases.at(database_name); + const auto & database = databases.at(database_name); res_columns[0]->insert(database_name); res_columns[1]->insert(database->getEngineName()); From 562ea03ce70c5fe0b44298b18f898e87d9ab44a5 Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Mon, 17 Jul 2023 12:54:34 +0300 Subject: [PATCH 0744/2047] FunctionsURL.h to StringHelpers.h and move out of URL/ --- src/Functions/{URL/FunctionsURL.h => StringHelpers.h} | 8 ++++---- src/Functions/URL/FirstSignificantSubdomainCustomImpl.h | 2 +- src/Functions/URL/basename.cpp | 2 +- src/Functions/URL/fragment.h | 2 +- src/Functions/URL/netloc.cpp | 3 +-- src/Functions/URL/path.cpp | 2 +- src/Functions/URL/path.h | 2 +- src/Functions/URL/pathFull.cpp | 2 +- src/Functions/URL/protocol.h | 3 +-- src/Functions/URL/queryString.h | 2 +- src/Functions/URL/queryStringAndFragment.h | 3 +-- 11 files changed, 14 insertions(+), 17 deletions(-) rename src/Functions/{URL/FunctionsURL.h => StringHelpers.h} (93%) diff --git a/src/Functions/URL/FunctionsURL.h b/src/Functions/StringHelpers.h similarity index 93% rename from src/Functions/URL/FunctionsURL.h rename to src/Functions/StringHelpers.h index 362042e31e1..a0f4d18aa80 100644 --- a/src/Functions/URL/FunctionsURL.h +++ b/src/Functions/StringHelpers.h @@ -7,8 +7,8 @@ namespace DB { -/** URL processing functions. See implementation in separate .cpp files. - * All functions are not strictly follow RFC, instead they are maximally simplified for performance reasons. +/** These helpers are used by URL processing functions. See implementation in separate .cpp files. + * All functions do not strictly follow RFC, instead they are maximally simplified for performance reasons. * * Functions for extraction parts of URL. * If URL has nothing like, then empty string is returned. @@ -101,7 +101,7 @@ struct ExtractSubstringImpl static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by URL functions"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by this function"); } }; @@ -156,7 +156,7 @@ struct CutSubstringImpl static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by URL functions"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported this function"); } }; diff --git a/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h b/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h index 5862265ce7d..93691e35741 100644 --- a/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h +++ b/src/Functions/URL/FirstSignificantSubdomainCustomImpl.h @@ -1,8 +1,8 @@ #pragma once #include -#include #include +#include #include #include #include diff --git a/src/Functions/URL/basename.cpp b/src/Functions/URL/basename.cpp index 6992f924ef2..bc747e2595f 100644 --- a/src/Functions/URL/basename.cpp +++ b/src/Functions/URL/basename.cpp @@ -1,7 +1,7 @@ #include #include +#include #include -#include "FunctionsURL.h" namespace DB { diff --git a/src/Functions/URL/fragment.h b/src/Functions/URL/fragment.h index 6c11d0fbb34..0414c4f64a2 100644 --- a/src/Functions/URL/fragment.h +++ b/src/Functions/URL/fragment.h @@ -1,7 +1,7 @@ #pragma once -#include "FunctionsURL.h" #include +#include namespace DB { diff --git a/src/Functions/URL/netloc.cpp b/src/Functions/URL/netloc.cpp index bc34e34a40d..bf3e8a471ef 100644 --- a/src/Functions/URL/netloc.cpp +++ b/src/Functions/URL/netloc.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace DB @@ -154,4 +154,3 @@ REGISTER_FUNCTION(Netloc) } } - diff --git a/src/Functions/URL/path.cpp b/src/Functions/URL/path.cpp index ccc7dedb724..8d609f43191 100644 --- a/src/Functions/URL/path.cpp +++ b/src/Functions/URL/path.cpp @@ -1,6 +1,6 @@ #include #include -#include "FunctionsURL.h" +#include #include "path.h" #include diff --git a/src/Functions/URL/path.h b/src/Functions/URL/path.h index 89244659088..a0dd5eea2d3 100644 --- a/src/Functions/URL/path.h +++ b/src/Functions/URL/path.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/src/Functions/URL/pathFull.cpp b/src/Functions/URL/pathFull.cpp index 002770e812a..9aacee21fed 100644 --- a/src/Functions/URL/pathFull.cpp +++ b/src/Functions/URL/pathFull.cpp @@ -1,6 +1,6 @@ #include #include -#include "FunctionsURL.h" +#include #include "path.h" #include diff --git a/src/Functions/URL/protocol.h b/src/Functions/URL/protocol.h index 74c0bb820b4..c1d83192835 100644 --- a/src/Functions/URL/protocol.h +++ b/src/Functions/URL/protocol.h @@ -1,7 +1,7 @@ #pragma once -#include "FunctionsURL.h" #include +#include namespace DB @@ -54,4 +54,3 @@ struct ExtractProtocol }; } - diff --git a/src/Functions/URL/queryString.h b/src/Functions/URL/queryString.h index a0777a5c9a1..603450d102b 100644 --- a/src/Functions/URL/queryString.h +++ b/src/Functions/URL/queryString.h @@ -1,7 +1,7 @@ #pragma once -#include "FunctionsURL.h" #include +#include namespace DB diff --git a/src/Functions/URL/queryStringAndFragment.h b/src/Functions/URL/queryStringAndFragment.h index ed19cd14b74..27008388e4d 100644 --- a/src/Functions/URL/queryStringAndFragment.h +++ b/src/Functions/URL/queryStringAndFragment.h @@ -1,7 +1,7 @@ #pragma once -#include "FunctionsURL.h" #include +#include namespace DB @@ -34,4 +34,3 @@ struct ExtractQueryStringAndFragment }; } - From 14ab1f2a98ad50ea08d8e089d4c426c353bd686b Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 12 Jul 2023 10:18:39 +0200 Subject: [PATCH 0745/2047] Fix SipHash doc Changes originally made by Robert Schulze (@rschu1ze) --- docs/en/sql-reference/functions/hash-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 06097d92480..556fe622c27 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -51,7 +51,7 @@ Calculates the MD5 from a string and returns the resulting set of bytes as Fixed If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead. If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))). -## sipHash64 (#hash_functions-siphash64) +## sipHash64 {#hash_functions-siphash64} Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. @@ -63,9 +63,9 @@ This is a cryptographic hash function. It works at least three times faster than The function [interprets](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. It then combines the hashes by the following algorithm: -1. The first and the second hash value are concatenated to an array which is hashed. -2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. -3. This calculation is repeated for all remaining hash values of the original input. +1. The first and the second hash value are concatenated to an array which is hashed. +2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. +3. This calculation is repeated for all remaining hash values of the original input. **Arguments** From 693892801ce2a56dac7a55694e90e216ee3c5bcc Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Mon, 17 Jul 2023 10:18:32 +0200 Subject: [PATCH 0746/2047] Re-add keyed SipHash and fix issues with current keyed hashes framwork --- src/Functions/FunctionsHashing.h | 289 +++++++++++++++++++------ src/Functions/FunctionsHashingMisc.cpp | 5 + 2 files changed, 222 insertions(+), 72 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 279294b367c..20612e91a62 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -79,28 +79,49 @@ namespace impl UInt64 key1 = 0; }; - static SipHashKey parseSipHashKey(const ColumnWithTypeAndName & key) + struct SipHashKeyColumns { - SipHashKey ret{}; + ColumnPtr key0; + ColumnPtr key1; + bool is_const; - const auto * tuple = checkAndGetColumn(key.column.get()); + size_t size() const + { + assert(key0 && key1); + assert(key0->size() == key1->size()); + return key0->size(); + } + SipHashKey getKey(size_t i) const + { + if (is_const) + i = 0; + return {key0->get64(i), key1->get64(i)}; + } + }; + + static SipHashKeyColumns parseSipHashKeyColumns(const ColumnWithTypeAndName & key) + { + const ColumnTuple * tuple = nullptr; + const auto * column = key.column.get(); + bool is_const = false; + if (isColumnConst(*column)) + { + is_const = true; + tuple = checkAndGetColumnConstData(column); + } + else + tuple = checkAndGetColumn(column); if (!tuple) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "key must be a tuple"); - if (tuple->tupleSize() != 2) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "wrong tuple size: key must be a tuple of 2 UInt64"); - if (tuple->empty()) - return ret; - - if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) - ret.key0 = key0col->get64(0); - else + SipHashKeyColumns ret{tuple->getColumnPtr(0), tuple->getColumnPtr(1), is_const}; + assert(ret.key0); + if (!checkColumn(*ret.key0)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); - - if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) - ret.key1 = key1col->get64(0); - else + assert(ret.key1); + if (!checkColumn(*ret.key1)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); return ret; @@ -329,8 +350,10 @@ struct SipHash64KeyedImpl static constexpr auto name = "sipHash64Keyed"; using ReturnType = UInt64; using Key = impl::SipHashKey; + using KeyColumns = impl::SipHashKeyColumns; - static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); } + static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } + static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } static UInt64 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash64Keyed(key.key0, key.key1, begin, size); } @@ -371,8 +394,10 @@ struct SipHash128KeyedImpl static constexpr auto name = "sipHash128Keyed"; using ReturnType = UInt128; using Key = impl::SipHashKey; + using KeyColumns = impl::SipHashKeyColumns; - static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); } + static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } + static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash128Keyed(key.key0, key.key1, begin, size); } @@ -405,6 +430,36 @@ struct SipHash128ReferenceImpl static constexpr bool use_int_hash_for_pods = false; }; +struct SipHash128ReferenceKeyedImpl +{ + static constexpr auto name = "sipHash128ReferenceKeyed"; + using ReturnType = UInt128; + using Key = impl::SipHashKey; + using KeyColumns = impl::SipHashKeyColumns; + + static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } + static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } + + static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) + { + return sipHash128ReferenceKeyed(key.key0, key.key1, begin, size); + } + + static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2) + { +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + UInt128 tmp; + reverseMemcpy(&tmp, &h1, sizeof(UInt128)); + h1 = tmp; + reverseMemcpy(&tmp, &h2, sizeof(UInt128)); + h2 = tmp; +#endif + UInt128 hashes[] = {h1, h2}; + return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt128)); + } + + static constexpr bool use_int_hash_for_pods = false; +}; /** Why we need MurmurHash2? * MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash. @@ -1023,7 +1078,7 @@ private: DECLARE_MULTITARGET_CODE( -template +template class FunctionAnyHash : public IFunction { public: @@ -1033,9 +1088,12 @@ private: using ToType = typename Impl::ReturnType; template - void executeIntType(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeIntType(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const { using ColVecType = ColumnVectorOrDecimal; + KeyType key{}; + if constexpr (Keyed) + key = Impl::getKey(key_cols, 0); if (const ColVecType * col_from = checkAndGetColumn(column)) { @@ -1044,6 +1102,9 @@ private: for (size_t i = 0; i < size; ++i) { ToType hash; + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); if constexpr (Impl::use_int_hash_for_pods) { @@ -1077,6 +1138,14 @@ private: } else if (auto col_from_const = checkAndGetColumnConst(column)) { + if constexpr (Keyed) + { + if (!key_cols.is_const) + { + ColumnPtr full_column = col_from_const->convertToFullColumn(); + return executeIntType(key_cols, full_column.get(), vec_to); + } + } auto value = col_from_const->template getValue(); ToType hash; @@ -1107,8 +1176,15 @@ private: if constexpr (first) vec_to.assign(size, hash); else + { for (size_t i = 0; i < size; ++i) + { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); + } + } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1116,9 +1192,12 @@ private: } template - void executeBigIntType(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeBigIntType(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const { using ColVecType = ColumnVectorOrDecimal; + KeyType key{}; + if constexpr (Keyed) + key = Impl::getKey(key_cols, 0); if (const ColVecType * col_from = checkAndGetColumn(column)) { @@ -1127,6 +1206,9 @@ private: for (size_t i = 0; i < size; ++i) { ToType hash; + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); if constexpr (std::endian::native == std::endian::little) hash = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); else @@ -1143,6 +1225,14 @@ private: } else if (auto col_from_const = checkAndGetColumnConst(column)) { + if constexpr (Keyed) + { + if (!key_cols.is_const) + { + ColumnPtr full_column = col_from_const->convertToFullColumn(); + return executeBigIntType(key_cols, full_column.get(), vec_to); + } + } auto value = col_from_const->template getValue(); ToType hash; @@ -1158,8 +1248,15 @@ private: if constexpr (first) vec_to.assign(size, hash); else + { for (size_t i = 0; i < size; ++i) + { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); + } + } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1167,10 +1264,16 @@ private: } template - void executeGeneric(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeGeneric(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const { + KeyType key{}; + if constexpr (Keyed) + key = Impl::getKey(key_cols, 0); for (size_t i = 0, size = column->size(); i < size; ++i) { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); StringRef bytes = column->getDataAt(i); const ToType hash = apply(key, bytes.data, bytes.size); if constexpr (first) @@ -1181,8 +1284,11 @@ private: } template - void executeString(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeString(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const { + KeyType key{}; + if constexpr (Keyed) + key = Impl::getKey(key_cols, 0); if (const ColumnString * col_from = checkAndGetColumn(column)) { const typename ColumnString::Chars & data = col_from->getChars(); @@ -1192,6 +1298,9 @@ private: ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); const ToType hash = apply(key, reinterpret_cast(&data[current_offset]), offsets[i] - current_offset - 1); @@ -1212,6 +1321,9 @@ private: for (size_t i = 0; i < size; ++i) { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); const ToType hash = apply(key, reinterpret_cast(&data[i * n]), n); if constexpr (first) vec_to[i] = hash; @@ -1221,6 +1333,14 @@ private: } else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column)) { + if constexpr (Keyed) + { + if (!key_cols.is_const) + { + ColumnPtr full_column = col_from_const->convertToFullColumn(); + return executeString(key_cols, full_column.get(), vec_to); + } + } String value = col_from_const->getValue(); const ToType hash = apply(key, value.data(), value.size()); const size_t size = vec_to.size(); @@ -1228,8 +1348,15 @@ private: if constexpr (first) vec_to.assign(size, hash); else + { for (size_t i = 0; i < size; ++i) + { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); + } + } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", @@ -1237,7 +1364,7 @@ private: } template - void executeArray(const KeyType & key, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeArray(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to) const { const IDataType * nested_type = typeid_cast(*type).getNestedType().get(); @@ -1249,13 +1376,19 @@ private: typename ColumnVector::Container vec_temp(nested_size); bool nested_is_first = true; - executeForArgument(key, nested_type, nested_column, vec_temp, nested_is_first); + executeForArgument(key_cols, nested_type, nested_column, vec_temp, nested_is_first); const size_t size = offsets.size(); ColumnArray::Offset current_offset = 0; + KeyType key{}; + if constexpr (Keyed) + key = Impl::getKey(key_cols, 0); for (size_t i = 0; i < size; ++i) { + if constexpr (Keyed) + if (!key_cols.is_const && i != 0) + key = Impl::getKey(key_cols, i); ColumnArray::Offset next_offset = offsets[i]; ToType hash; @@ -1279,7 +1412,7 @@ private: { /// NOTE: here, of course, you can do without the materialization of the column. ColumnPtr full_column = col_from_const->convertToFullColumn(); - executeArray(key, type, full_column.get(), vec_to); + executeArray(key_cols, type, full_column.get(), vec_to); } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", @@ -1287,7 +1420,7 @@ private: } template - void executeAny(const KeyType & key, const IDataType * from_type, const IColumn * icolumn, typename ColumnVector::Container & vec_to) const + void executeAny(const KeyColumnsType & key_cols, const IDataType * from_type, const IColumn * icolumn, typename ColumnVector::Container & vec_to) const { WhichDataType which(from_type); @@ -1295,40 +1428,45 @@ private: throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument column '{}' size {} doesn't match result column size {} of function {}", icolumn->getName(), icolumn->size(), vec_to.size(), getName()); - if (which.isUInt8()) executeIntType(key, icolumn, vec_to); - else if (which.isUInt16()) executeIntType(key, icolumn, vec_to); - else if (which.isUInt32()) executeIntType(key, icolumn, vec_to); - else if (which.isUInt64()) executeIntType(key, icolumn, vec_to); - else if (which.isUInt128()) executeBigIntType(key, icolumn, vec_to); - else if (which.isUInt256()) executeBigIntType(key, icolumn, vec_to); - else if (which.isInt8()) executeIntType(key, icolumn, vec_to); - else if (which.isInt16()) executeIntType(key, icolumn, vec_to); - else if (which.isInt32()) executeIntType(key, icolumn, vec_to); - else if (which.isInt64()) executeIntType(key, icolumn, vec_to); - else if (which.isInt128()) executeBigIntType(key, icolumn, vec_to); - else if (which.isInt256()) executeBigIntType(key, icolumn, vec_to); - else if (which.isUUID()) executeBigIntType(key, icolumn, vec_to); - else if (which.isIPv4()) executeIntType(key, icolumn, vec_to); - else if (which.isIPv6()) executeBigIntType(key, icolumn, vec_to); - else if (which.isEnum8()) executeIntType(key, icolumn, vec_to); - else if (which.isEnum16()) executeIntType(key, icolumn, vec_to); - else if (which.isDate()) executeIntType(key, icolumn, vec_to); - else if (which.isDate32()) executeIntType(key, icolumn, vec_to); - else if (which.isDateTime()) executeIntType(key, icolumn, vec_to); + if constexpr (Keyed) + if ((!key_cols.is_const && key_cols.size() != vec_to.size()) + || (key_cols.is_const && key_cols.size() != 1)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Key column size {} doesn't match result column size {} of function {}", key_cols.size(), vec_to.size(), getName()); + + if (which.isUInt8()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isUInt16()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isUInt32()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isUInt64()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isUInt128()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isUInt256()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isInt8()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isInt16()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isInt32()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isInt64()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isInt128()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isInt256()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isUUID()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isIPv4()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isIPv6()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isEnum8()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isEnum16()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isDate()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isDate32()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isDateTime()) executeIntType(key_cols, icolumn, vec_to); /// TODO: executeIntType() for Decimal32/64 leads to incompatible result - else if (which.isDecimal32()) executeBigIntType(key, icolumn, vec_to); - else if (which.isDecimal64()) executeBigIntType(key, icolumn, vec_to); - else if (which.isDecimal128()) executeBigIntType(key, icolumn, vec_to); - else if (which.isDecimal256()) executeBigIntType(key, icolumn, vec_to); - else if (which.isFloat32()) executeIntType(key, icolumn, vec_to); - else if (which.isFloat64()) executeIntType(key, icolumn, vec_to); - else if (which.isString()) executeString(key, icolumn, vec_to); - else if (which.isFixedString()) executeString(key, icolumn, vec_to); - else if (which.isArray()) executeArray(key, from_type, icolumn, vec_to); - else executeGeneric(key, icolumn, vec_to); + else if (which.isDecimal32()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isDecimal64()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isDecimal128()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isDecimal256()) executeBigIntType(key_cols, icolumn, vec_to); + else if (which.isFloat32()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isFloat64()) executeIntType(key_cols, icolumn, vec_to); + else if (which.isString()) executeString(key_cols, icolumn, vec_to); + else if (which.isFixedString()) executeString(key_cols, icolumn, vec_to); + else if (which.isArray()) executeArray(key_cols, from_type, icolumn, vec_to); + else executeGeneric(key_cols, icolumn, vec_to); } - void executeForArgument(const KeyType & key, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) const + void executeForArgument(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) const { /// Flattening of tuples. if (const ColumnTuple * tuple = typeid_cast(column)) @@ -1337,7 +1475,7 @@ private: const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); for (size_t i = 0; i < tuple_size; ++i) - executeForArgument(key, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); + executeForArgument(key_cols, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); } else if (const ColumnTuple * tuple_const = checkAndGetColumnConstData(column)) { @@ -1347,24 +1485,24 @@ private: for (size_t i = 0; i < tuple_size; ++i) { auto tmp = ColumnConst::create(tuple_columns[i], column->size()); - executeForArgument(key, tuple_types[i].get(), tmp.get(), vec_to, is_first); + executeForArgument(key_cols, tuple_types[i].get(), tmp.get(), vec_to, is_first); } } else if (const auto * map = checkAndGetColumn(column)) { const auto & type_map = assert_cast(*type); - executeForArgument(key, type_map.getNestedType().get(), map->getNestedColumnPtr().get(), vec_to, is_first); + executeForArgument(key_cols, type_map.getNestedType().get(), map->getNestedColumnPtr().get(), vec_to, is_first); } else if (const auto * const_map = checkAndGetColumnConst(column)) { - executeForArgument(key, type, const_map->convertToFullColumnIfConst().get(), vec_to, is_first); + executeForArgument(key_cols, type, const_map->convertToFullColumnIfConst().get(), vec_to, is_first); } else { if (is_first) - executeAny(key, type, column, vec_to); + executeAny(key_cols, type, column, vec_to); else - executeAny(key, type, column, vec_to); + executeAny(key_cols, type, column, vec_to); } is_first = false; @@ -1395,6 +1533,9 @@ public: { auto col_to = ColumnVector::create(input_rows_count); + if (input_rows_count == 0) + return col_to; + typename ColumnVector::Container & vec_to = col_to->getData(); /// If using a "keyed" algorithm, the first argument is the key and @@ -1408,17 +1549,17 @@ public: vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); } - KeyType key{}; + KeyColumnsType key_cols{}; if constexpr (Keyed) if (!arguments.empty()) - key = Impl::parseKey(arguments[0]); + key_cols = Impl::parseKeyColumns(arguments[0]); /// The function supports arbitrary number of arguments of arbitrary types. bool is_first_argument = true; for (size_t i = first_data_argument; i < arguments.size(); ++i) { const auto & col = arguments[i]; - executeForArgument(key, col.type.get(), col.column.get(), vec_to, is_first_argument); + executeForArgument(key_cols, col.type.get(), col.column.get(), vec_to, is_first_argument); } if constexpr (std::is_same_v) /// backward-compatible @@ -1450,17 +1591,19 @@ public: ) // DECLARE_MULTITARGET_CODE -template -class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash +template +class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash { public: explicit FunctionAnyHash(ContextPtr context) : selector(context) { - selector.registerImplementation>(); + selector + .registerImplementation>(); #if USE_MULTITARGET_CODE - selector.registerImplementation>(); - selector.registerImplementation>(); + selector.registerImplementation>(); + selector + .registerImplementation>(); #endif } @@ -1696,7 +1839,7 @@ struct NameIntHash32 { static constexpr auto name = "intHash32"; }; struct NameIntHash64 { static constexpr auto name = "intHash64"; }; using FunctionSipHash64 = FunctionAnyHash; -using FunctionSipHash64Keyed = FunctionAnyHash; +using FunctionSipHash64Keyed = FunctionAnyHash; using FunctionIntHash32 = FunctionIntHash; using FunctionIntHash64 = FunctionIntHash; #if USE_SSL @@ -1710,8 +1853,10 @@ using FunctionSHA384 = FunctionStringHashFixedString; using FunctionSHA512 = FunctionStringHashFixedString; #endif using FunctionSipHash128 = FunctionAnyHash; -using FunctionSipHash128Keyed = FunctionAnyHash; +using FunctionSipHash128Keyed = FunctionAnyHash; using FunctionSipHash128Reference = FunctionAnyHash; +using FunctionSipHash128ReferenceKeyed + = FunctionAnyHash; using FunctionCityHash64 = FunctionAnyHash; using FunctionFarmFingerprint64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; diff --git a/src/Functions/FunctionsHashingMisc.cpp b/src/Functions/FunctionsHashingMisc.cpp index 56c3c1ed00c..f56568b2508 100644 --- a/src/Functions/FunctionsHashingMisc.cpp +++ b/src/Functions/FunctionsHashingMisc.cpp @@ -20,6 +20,11 @@ REGISTER_FUNCTION(Hashing) .examples{{"hash", "SELECT hex(sipHash128Reference('foo', '\\x01', 3))", ""}}, .categories{"Hash"} }); + factory.registerFunction(FunctionDocumentation{ + .description = "Same as [sipHash128Reference](#hash_functions-siphash128reference) but additionally takes an explicit key argument " + "instead of using a fixed key.", + .examples{{"hash", "SELECT hex(sipHash128ReferenceKeyed((506097522914230528, 1084818905618843912),'foo', '\\x01', 3));", ""}}, + .categories{"Hash"}}); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); From fb23ece79730545debd90ca2032279f243ea1208 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 12 Jul 2023 10:24:55 +0200 Subject: [PATCH 0747/2047] Add tests for memsan bug Changes originally made by Robert Schulze (@rschu1ze) --- tests/queries/0_stateless/02534_keyed_siphash.reference | 5 +++++ tests/queries/0_stateless/02534_keyed_siphash.sql | 6 ++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index ccc514e7ea2..697b81a1390 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -197,3 +197,8 @@ E28DBDE7FE22E41C Check bug with hashing of const integer values 11862823756610506724 11862823756610506724 +Check memsan bug +1CE422FEE7BD8DE20000000000000000 +7766709361750702608 +20AF99D3A87829E0 +12489502208762728797 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 900b99f548a..bc7dda92e2a 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -281,3 +281,9 @@ INSERT INTO tab VALUES ((2, 2), 4); SELECT sipHash64Keyed(key, val) FROM tab; SELECT sipHash64Keyed(key, 4::UInt64) FROM tab; DROP TABLE tab; + +SELECT 'Check memsan bug'; +SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); +SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); +SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); +SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); From 425d9c17c7501b7ad7f9302312af59e6d5a84c95 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Mon, 17 Jul 2023 10:15:19 +0200 Subject: [PATCH 0748/2047] Fix memsan test --- tests/queries/0_stateless/02534_keyed_siphash.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 697b81a1390..4f6978e7ec7 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -199,6 +199,6 @@ Check bug with hashing of const integer values 11862823756610506724 Check memsan bug 1CE422FEE7BD8DE20000000000000000 -7766709361750702608 +18096612095653370192 20AF99D3A87829E0 12489502208762728797 From e61426f7ae9295ffa2801a8a0bfb754347c9cb86 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Mon, 17 Jul 2023 10:15:54 +0200 Subject: [PATCH 0749/2047] Add keyed SipHash tests --- .../0_stateless/02534_keyed_siphash.reference | 21 ++ .../0_stateless/02534_keyed_siphash.sql | 65 ++++++ .../02552_siphash128_reference.reference | 129 +++++++++++ .../02552_siphash128_reference.sql | 206 ++++++++++++++++++ 4 files changed, 421 insertions(+) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 4f6978e7ec7..ea77e5c3963 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -202,3 +202,24 @@ Check memsan bug 18096612095653370192 20AF99D3A87829E0 12489502208762728797 +Check const columns +15080046610211022027 +15080046610211022027 +15080046610211022027 +15080046610211022027 +2E779C73D13981AA1AE19AFF9617EA49 +2E779C73D13981AA1AE19AFF9617EA49 +Check multiple keys +11862823756610506724 +9357996107237883963 +Check multiple keys 2 +11862823756610506724 +9357996107237883963 +11862823756610506724 +11862823756610506724 +Check multiple keys 3 +11862823756610506724 +9357996107237883963 +- +11862823756610506724 +11862823756610506724 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index bc7dda92e2a..70ec8470ca3 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -287,3 +287,68 @@ SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807) SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); + +SELECT 'Check const columns'; +DROP TABLE IF EXISTS sipHashKeyed_test; + +CREATE TABLE sipHashKeyed_test +ENGINE = Memory() +AS +SELECT +1 a, +'test' b; + +SELECT +sipHash64Keyed((toUInt64(0), toUInt64(0)), 1, 'test'); + +SELECT +sipHash64(tuple(*)) +FROM +sipHashKeyed_test; + +SELECT +sipHash64Keyed((toUInt64(0), toUInt64(0)), tuple(*)) +FROM +sipHashKeyed_test; + +SELECT +sipHash64Keyed((toUInt64(0), toUInt64(0)), a, b) +FROM +sipHashKeyed_test; + +SELECT +hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), tuple(*))) +FROM +sipHashKeyed_test; + +SELECT +hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), a, b)) +FROM +sipHashKeyed_test; + +DROP TABLE sipHashKeyed_test; + +SELECT 'Check multiple keys'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); +INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); +SELECT sipHash64Keyed(key, val) FROM sipHashKeyed_keys ORDER by key; +DROP TABLE sipHashKeyed_keys; +SELECT 'Check multiple keys 2'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); +INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); +SELECT sipHash64Keyed((key0, key1), val) FROM sipHashKeyed_keys ORDER by key0; +SELECT sipHash64Keyed((2::UInt64, 2::UInt64), val) FROM sipHashKeyed_keys ORDER by val; +DROP TABLE sipHashKeyed_keys; +SELECT 'Check multiple keys 3'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES (2, 2); +INSERT INTO sipHashKeyed_keys VALUES (4, 4); +SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; +SELECT '-'; +SELECT sipHash64Keyed((2::UInt64, 2::UInt64), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; +DROP TABLE sipHashKeyed_keys; diff --git a/tests/queries/0_stateless/02552_siphash128_reference.reference b/tests/queries/0_stateless/02552_siphash128_reference.reference index d00491fd7e5..452e9910660 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.reference +++ b/tests/queries/0_stateless/02552_siphash128_reference.reference @@ -1 +1,130 @@ +A3817F04BA25A8E66DF67214C7550293 +DA87C1D86B99AF44347659119B22FC45 +8177228DA4A45DC7FCA38BDEF60AFFE4 +9C70B60C5267A94E5F33B6B02985ED51 +F88164C12D9C8FAF7D0F6E7C7BCD5579 +1368875980776F8854527A07690E9627 +14EECA338B208613485EA0308FD7A15E +A1F1EBBED8DBC153C0B84AA61FF08239 +3B62A9BA6258F5610F83E264F31497B4 +264499060AD9BAABC47F8B02BB6D71ED +00110DC378146956C95447D3F3D0FBBA +0151C568386B6677A2B4DC6F81E5DC18 +D626B266905EF35882634DF68532C125 +9869E247E9C08B10D029934FC4B952F7 +31FCEFAC66D7DE9C7EC7485FE4494902 +5493E99933B0A8117E08EC0F97CFC3D9 +6EE2A4CA67B054BBFD3315BF85230577 +473D06E8738DB89854C066C47AE47740 +A426E5E423BF4885294DA481FEAEF723 +78017731CF65FAB074D5208952512EB1 +9E25FC833F2290733E9344A5E83839EB +568E495ABE525A218A2214CD3E071D12 +4A29B54552D16B9A469C10528EFF0AAE +C9D184DDD5A9F5E0CF8CE29A9ABF691C +2DB479AE78BD50D8882A8A178A6132AD +8ECE5F042D5E447B5051B9EACB8D8F6F +9C0B53B4B3C307E87EAEE08678141F66 +ABF248AF69A6EAE4BFD3EB2F129EEB94 +0664DA1668574B88B935F3027358AEF4 +AA4B9DC4BF337DE90CD4FD3C467C6AB7 +EA5C7F471FAF6BDE2B1AD7D4686D2287 +2939B0183223FAFC1723DE4F52C43D35 +7C3956CA5EEAFC3E363E9D556546EB68 +77C6077146F01C32B6B69D5F4EA9FFCF +37A6986CB8847EDF0925F0F1309B54DE +A705F0E69DA9A8F907241A2E923C8CC8 +3DC47D1F29C448461E9E76ED904F6711 +0D62BF01E6FC0E1A0D3C4751C5D3692B +8C03468BCA7C669EE4FD5E084BBEE7B5 +528A5BB93BAF2C9C4473CCE5D0D22BD9 +DF6A301E95C95DAD97AE0CC8C6913BD8 +801189902C857F39E73591285E70B6DB +E617346AC9C231BB3650AE34CCCA0C5B +27D93437EFB721AA401821DCEC5ADF89 +89237D9DED9C5E78D8B1C9B166CC7342 +4A6D8091BF5E7D651189FA94A250B14C +0E33F96055E7AE893FFC0E3DCF492902 +E61C432B720B19D18EC8D84BDC63151B +F7E5AEF549F782CF379055A608269B16 +438D030FD0B7A54FA837F2AD201A6403 +A590D3EE4FBF04E3247E0D27F286423F +5FE2C1A172FE93C4B15CD37CAEF9F538 +2C97325CBD06B36EB2133DD08B3A017C +92C814227A6BCA949FF0659F002AD39E +DCE850110BD8328CFBD50841D6911D87 +67F14984C7DA791248E32BB5922583DA +1938F2CF72D54EE97E94166FA91D2A36 +74481E9646ED49FE0F6224301604698E +57FCA5DE98A9D6D8006438D0583D8A1D +9FECDE1CEFDC1CBED4763674D9575359 +E3040C00EB28F15366CA73CBD872E740 +7697009A6A831DFECCA91C5993670F7A +5853542321F567A005D547A4F04759BD +5150D1772F50834A503E069A973FBD7C +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 1 diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql index 200954c3b57..c238e51b690 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.sql +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -1 +1,207 @@ +-- Test Vectors from the SipHash reference C implementation: +-- Written by +-- Jean-Philippe Aumasson +-- Daniel J. Bernstein +-- Released under CC0 +-- https://github.com/veorq/SipHash/blob/eee7d0d84dc7731df2359b243aa5e75d85f6eaef/vectors.h#L645 + +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + '')); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62))); + +-- CH tests +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0)) == sipHash128Reference(char(0)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1)) == sipHash128Reference(char(0, 1)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2)) == sipHash128Reference(char(0, 1, 2)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3)) == sipHash128Reference(char(0, 1, 2, 3)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4)) == sipHash128Reference(char(0, 1, 2, 3, 4)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); + +select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError 48 } +select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError 48 } + SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; +SELECT hex(sipHash128ReferenceKeyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; From ed4c560e330194c7a8f9ffd6b676ac6ac256830e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 18 Jul 2023 11:00:13 +0200 Subject: [PATCH 0750/2047] Accept paths without ancestors --- src/Common/ZooKeeper/ZooKeeper.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index e3e89969c17..f23663f9c25 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -357,6 +357,8 @@ void ZooKeeper::createAncestors(const std::string & path) std::vector pending_nodes; size_t last_pos = path.rfind('/'); + if (last_pos == std::string::npos || last_pos == 0) + return; std::string current_node = path.substr(0, last_pos); while (true) From 4f7fd69883fe01c9e6fe5661051cd1bc9ffc49f6 Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Mon, 17 Jul 2023 14:58:27 +0300 Subject: [PATCH 0751/2047] Added function firstLine to extract the first line from a multiline string Fixes #51172 --- .../functions/string-functions.md | 33 +++++++++++++++ .../functions/string-functions.md | 36 ++++++++++++++++ src/Functions/StringHelpers.h | 2 +- src/Functions/firstLine.cpp | 42 +++++++++++++++++++ .../0_stateless/02815_first_line.reference | 9 ++++ .../queries/0_stateless/02815_first_line.sql | 12 ++++++ .../aspell-ignore/en/aspell-dict.txt | 1 + 7 files changed, 134 insertions(+), 1 deletion(-) create mode 100644 src/Functions/firstLine.cpp create mode 100644 tests/queries/0_stateless/02815_first_line.reference create mode 100644 tests/queries/0_stateless/02815_first_line.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 4f174a53ad6..9890d257e84 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -1267,3 +1267,36 @@ Like [initcap](#initcap), assuming that the string contains valid UTF-8 encoded Does not detect the language, e.g. for Turkish the result might not be exactly correct (i/İ vs. i/I). If the length of the UTF-8 byte sequence is different for upper and lower case of a code point, the result may be incorrect for this code point. + +## firstLine + +Returns the first line from a multi-line string. + +**Syntax** + +```sql +firstLine(val) +``` + +**Arguments** + +- `val` - Input value. [String](../data-types/string.md) + +**Returned value** + +- The first line of the input value or the whole value if there is no line + separators. [String](../data-types/string.md) + +**Example** + +```sql +select firstLine('foo\nbar\nbaz'); +``` + +Result: + +```result +┌─firstLine('foo\nbar\nbaz')─┐ +│ foo │ +└────────────────────────────┘ +``` diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index b872200f99b..276dfc2ef20 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -1124,3 +1124,39 @@ Do Nothing for 2 Minutes 2:00   Не учитывает язык. То есть, для турецкого языка, результат может быть не совсем верным. Если длина UTF-8 последовательности байтов различна для верхнего и нижнего регистра кодовой точки, то для этой кодовой точки результат работы может быть некорректным. Если строка содержит набор байтов, не являющийся UTF-8, то поведение не определено. + +## firstLine + +Возвращает первую строку в многострочном тексте. + +**Синтаксис** + +```sql +firstLine(val) +``` + +**Аргументы** + +- `val` - текст для обработки. [String](../data-types/string.md) + +**Returned value** + +- Первая строка текста или весь текст, если переносы строк отсутствуют. + +Тип: [String](../data-types/string.md) + +**Пример** + +Запрос: + +```sql +select firstLine('foo\nbar\nbaz'); +``` + +Результат: + +```result +┌─firstLine('foo\nbar\nbaz')─┐ +│ foo │ +└────────────────────────────┘ +``` diff --git a/src/Functions/StringHelpers.h b/src/Functions/StringHelpers.h index a0f4d18aa80..8f3a87d5d0e 100644 --- a/src/Functions/StringHelpers.h +++ b/src/Functions/StringHelpers.h @@ -156,7 +156,7 @@ struct CutSubstringImpl static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported this function"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Column of type FixedString is not supported by this function"); } }; diff --git a/src/Functions/firstLine.cpp b/src/Functions/firstLine.cpp new file mode 100644 index 00000000000..20b47361d58 --- /dev/null +++ b/src/Functions/firstLine.cpp @@ -0,0 +1,42 @@ +#include +#include +#include +#include + +namespace DB +{ + +struct FirstLine +{ + static size_t getReserveLengthForElement() { return 16; } + + static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) + { + res_data = data; + + const Pos end = data + size; + const Pos pos = find_first_symbols<'\r', '\n'>(data, end); + res_size = pos - data; + } +}; + +struct NameFirstLine +{ + static constexpr auto name = "firstLine"; +}; + +using FunctionFirstLine = FunctionStringToString, NameFirstLine>; + +REGISTER_FUNCTION(FirstLine) +{ + factory.registerFunction(FunctionDocumentation{ + .description = "Returns first line of a multi-line string.", + .syntax = "firstLine(string)", + .arguments = {{.name = "string", .description = "The string to process."}}, + .returned_value = {"The first line of the string or the whole string if there is no line separators."}, + .examples = { + {.name = "Return first line", .query = "firstLine('Hello\\nWorld')", .result = "'Hello'"}, + {.name = "Return whole string", .query = "firstLine('Hello World')", .result = "'Hello World'"}, + }}); +} +} diff --git a/tests/queries/0_stateless/02815_first_line.reference b/tests/queries/0_stateless/02815_first_line.reference new file mode 100644 index 00000000000..cdc86229cc8 --- /dev/null +++ b/tests/queries/0_stateless/02815_first_line.reference @@ -0,0 +1,9 @@ +foo +foo +foo +foobarbaz +== vector +1 foo +2 quux +3 single line +4 windows diff --git a/tests/queries/0_stateless/02815_first_line.sql b/tests/queries/0_stateless/02815_first_line.sql new file mode 100644 index 00000000000..8c0affaebd3 --- /dev/null +++ b/tests/queries/0_stateless/02815_first_line.sql @@ -0,0 +1,12 @@ +select firstLine('foo\nbar\nbaz'); +select firstLine('foo\rbar\rbaz'); +select firstLine('foo\r\nbar\r\nbaz'); +select firstLine('foobarbaz'); + +select '== vector'; + +drop table if exists 02815_first_line_vector; +create table 02815_first_line_vector (n Int32, text String) engine = MergeTree order by n; + +insert into 02815_first_line_vector values (1, 'foo\nbar\nbaz'), (2, 'quux\n'), (3, 'single line'), (4, 'windows\r\nline breaks'); +select n, firstLine(text) from 02815_first_line_vector order by n; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 6c88d63be49..57934b28728 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1428,6 +1428,7 @@ filesystemFree filesystems finalizeAggregation fips +firstLine firstSignificantSubdomain firstSignificantSubdomainCustom fixedstring From dc55c8ed8eeb4891f8ee7161eb962ee991bacd3d Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 18 Jul 2023 09:54:33 +0000 Subject: [PATCH 0752/2047] Enable no-upgrade-check for 02273_full_sort_join --- tests/queries/0_stateless/02273_full_sort_join.sql.j2 | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 8b739330364..43f7354017c 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -1,4 +1,6 @@ --- Tags: long +-- Tags: long, no-upgrade-check + +-- TODO(@vdimir): remove no-upgrade-check tag after https://github.com/ClickHouse/ClickHouse/pull/51737 is released DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From 6b00a38aa45d79b72088f608a993579a0fe31d9a Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 10:06:44 +0000 Subject: [PATCH 0753/2047] Remove SystemLogBase (wip) --- src/Common/SystemLogBase.cpp | 96 +++++++++------------------------- src/Common/SystemLogBase.h | 75 ++++++++------------------ src/Interpreters/SystemLog.cpp | 62 +++++++++++++++++----- src/Interpreters/SystemLog.h | 38 +++++++++----- src/Interpreters/TextLog.cpp | 2 +- src/Interpreters/TextLog.h | 4 +- src/Loggers/Loggers.cpp | 5 +- 7 files changed, 127 insertions(+), 155 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index c979613b146..dc6a4a08bb7 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -38,46 +38,13 @@ namespace ISystemLog::~ISystemLog() = default; + template -SystemLogBase::SystemLogBase( +SystemLogQueue::SystemLogQueue( const String & name_, - std::shared_ptr> queue_) - : queue(queue_ ? queue_ : std::make_shared>(name_)) -{ -} - -template -void SystemLogBase::stopFlushThread() -{ - { - std::lock_guard lock(queue->mutex); - - if (!saving_thread || !saving_thread->joinable()) - return; - - if (is_shutdown) - return; - - is_shutdown = true; - queue->shutdown(); - - /// Tell thread to shutdown. - queue->flush_event.notify_all(); - } - - saving_thread->join(); -} - -template -void SystemLogBase::startup() -{ - std::lock_guard lock(queue->mutex); - saving_thread = std::make_unique([this] { savingThreadFunction(); }); -} - -template -SystemLogQueue::SystemLogQueue(const String & name_) + size_t flush_interval_milliseconds_) : log(&Poco::Logger::get(name_)) + , flush_interval_milliseconds(flush_interval_milliseconds_) {} static thread_local bool recursive_add_call = false; @@ -149,6 +116,14 @@ void SystemLogQueue::add(const LogElement & element) LOG_INFO(log, "Queue is half full for system log '{}'.", demangle(typeid(*this).name())); } +template +void SystemLogQueue::shutdown() +{ + is_shutdown = true; + /// Tell thread to shutdown. + flush_event.notify_all(); +} + template uint64_t SystemLogQueue::notifyFlush(bool force) { @@ -193,12 +168,19 @@ void SystemLogQueue::waitFlush(uint64_t this_thread_requested_offset } } -constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; +template +void SystemLogQueue::confirm(uint64_t to_flush_end) +{ + std::lock_guard lock(mutex); + flushed_up_to = to_flush_end; + is_force_prepare_tables = false; + flush_event.notify_all(); +} template void SystemLogQueue::pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread) { - std::unique_lock lock(queue->mutex); + std::unique_lock lock(mutex); flush_event.wait_for(lock, std::chrono::milliseconds(flush_interval_milliseconds), [&] () @@ -207,47 +189,19 @@ void SystemLogQueue::pop(std::vector& output, uint64_t& } ); - queue_front_index += queue->size(); - to_flush_end = queue->queue_front_index; + queue_front_index += queue.size(); + to_flush_end = queue_front_index; // Swap with existing array from previous flush, to save memory // allocations. output.resize(0); - queue.swap(to_flush); + queue.swap(output); should_prepare_tables_anyway = is_force_prepare_tables; exit_this_thread = is_shutdown; } -template -void SystemLogBase::add(const LogElement & element) -{ - queue->add(element); -} - -template -void SystemLogBase::flush(bool force) -{ - uint64_t this_thread_requested_offset = queue->notifyFlush(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; - - queue->waitFlush(this_thread_requested_offset); -} - -template -void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } - -// template -// uint64_t SystemLogBase::notifyFlushImpl(bool force) -// { -// return queue->notifyFlush(force); -// } - -#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; +#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogQueue; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) -#define INSTANTIATE_SYSTEM_LOG_BASE2(ELEMENT) template class SystemLogQueue; -SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE2) - } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 0c1f32672a9..4cdb07c2cab 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -68,22 +68,33 @@ public: protected: std::unique_ptr saving_thread; + + bool is_shutdown = false; }; template class SystemLogQueue { public: - SystemLogQueue(const String & name_); + SystemLogQueue( + const String & name_, + size_t flush_interval_milliseconds_); + // producer methods void add(const LogElement & element); - size_t size() const { return queue.size(); } - //void push_back(const LogElement & element) { queue.push_back(element); } - void shutdown() { is_shutdown = true; } - + void shutdown(); uint64_t notifyFlush(bool force); void waitFlush(uint64_t this_thread_requested_offset_); - void pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread) + + // consumer methods + void pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread); + void confirm(uint64_t to_flush_end); + + /// Data shared between callers of add()/flush()/shutdown(), and the saving thread + std::mutex mutex; + +private: + Poco::Logger * log; // Queue is bounded. But its size is quite large to not block in all normal cases. std::vector queue; @@ -92,61 +103,19 @@ public: // can wait until a particular message is flushed. This is used to implement // synchronous log flushing for SYSTEM FLUSH LOGS. uint64_t queue_front_index = 0; - - /// Data shared between callers of add()/flush()/shutdown(), and the saving thread - std::mutex mutex; - std::condition_variable flush_event; - - // Requested to flush logs up to this index, exclusive - uint64_t requested_flush_up_to = 0; - // A flag that says we must create the tables even if the queue is empty. bool is_force_prepare_tables = false; - + // Requested to flush logs up to this index, exclusive + uint64_t requested_flush_up_to = 0; + std::condition_variable flush_event; // Flushed log up to this index, exclusive uint64_t flushed_up_to = 0; -private: - Poco::Logger * log; - bool is_shutdown = false; // Logged overflow message at this queue front index uint64_t logged_queue_full_at_index = -1; -}; - -template -class SystemLogBase : public ISystemLog -{ -public: - using Self = SystemLogBase; - - SystemLogBase( - const String & name_, - std::shared_ptr> queue_ = nullptr); - - /** Append a record into log. - * Writing to table will be done asynchronously and in case of failure, record could be lost. - */ - void add(const LogElement & element); - - /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. - void flush(bool force) override; - - void startup() override; - - void stopFlushThread() override; - - /// Non-blocking flush data in the buffer to disk. - void notifyFlush(bool force); - - String getName() const override { return LogElement::name(); } - - static const char * getDefaultOrderBy() { return "event_date, event_time"; } - -protected: - Poco::Logger * log; - - std::shared_ptr> queue; bool is_shutdown = false; + + const size_t flush_interval_milliseconds; }; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 983c9f87c45..cb4578689a2 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -334,15 +334,21 @@ SystemLog::SystemLog( const String & storage_def_, size_t flush_interval_milliseconds_, std::shared_ptr> queue_) - : Base(database_name_ + "." + table_name_, queue_) - , WithContext(context_) + : WithContext(context_) + , log(&Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")")) , table_id(database_name_, table_name_) , storage_def(storage_def_) , create_query(serializeAST(*getCreateTableQuery())) - , flush_interval_milliseconds(flush_interval_milliseconds_) + , queue(queue_ ? queue_ : std::make_shared>(database_name_ + "." + table_name_, flush_interval_milliseconds_)) { assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); - log = &Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"); +} + +template +void SystemLog::startup() +{ + std::lock_guard lock(queue->mutex); + saving_thread = std::make_unique([this] { savingThreadFunction(); }); } template @@ -355,6 +361,26 @@ void SystemLog::shutdown() table->flushAndShutdown(); } +template +void SystemLog::stopFlushThread() +{ + { + std::lock_guard lock(queue->mutex); + + if (!saving_thread || !saving_thread->joinable()) + return; + + if (is_shutdown) + return; + + is_shutdown = true; + queue->shutdown(); + } + + saving_thread->join(); +} + + template void SystemLog::savingThreadFunction() { @@ -381,9 +407,7 @@ void SystemLog::savingThreadFunction() prepareTable(); LOG_TRACE(log, "Table created (force)"); - std::lock_guard lock(queue->mutex); - queue->is_force_prepare_tables = false; - queue->flush_event.notify_all(); + queue->confirm(to_flush_end); } } else @@ -455,12 +479,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, tryLogCurrentException(__PRETTY_FUNCTION__); } - { - std::lock_guard lock(queue->mutex); - queue->flushed_up_to = to_flush_end; - queue->is_force_prepare_tables = false; - queue->flush_event.notify_all(); - } + queue->confirm(to_flush_end); LOG_TRACE(log, "Flushed system log up to offset {}", to_flush_end); } @@ -599,7 +618,24 @@ ASTPtr SystemLog::getCreateTableQuery() return create; } +template +void SystemLog::add(const LogElement & element) +{ + queue->add(element); +} +template +void SystemLog::flush(bool force) +{ + uint64_t this_thread_requested_offset = queue->notifyFlush(force); + if (this_thread_requested_offset == uint64_t(-1)) + return; + + queue->waitFlush(this_thread_requested_offset); +} + +template +void SystemLog::notifyFlush(bool force) { queue->notifyFlush(force); } #define INSTANTIATE_SYSTEM_LOG(ELEMENT) template class SystemLog; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index c730aa23319..fe9e4aa35d2 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -89,11 +89,10 @@ struct SystemLogs template -class SystemLog : public SystemLogBase, private boost::noncopyable, WithContext +class SystemLog : public ISystemLog, private boost::noncopyable, WithContext { public: using Self = SystemLog; - using Base = SystemLogBase; /** Parameter: table name where to write log. * If table is not exists, then it get created with specified engine. @@ -111,30 +110,41 @@ public: size_t flush_interval_milliseconds_, std::shared_ptr> queue_ = nullptr); + void startup() override; + /** Append a record into log. + * Writing to table will be done asynchronously and in case of failure, record could be lost. + */ + void add(const LogElement & element); + void shutdown() override; + String getName() const override { return LogElement::name(); } + static const char * getDefaultOrderBy() { return "event_date, event_time"; } + + /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. + void flush(bool force) override; + + /// Non-blocking flush data in the buffer to disk. + void notifyFlush(bool force); + + void stopFlushThread() override; + protected: - //using ISystemLog::mutex; - using Base::is_shutdown; - // using ISystemLog::flush_event; - using Base::stopFlushThread; - using Base::log; - using Base::queue; - // using Base::queue_front_index; - // using Base::is_force_prepare_tables; - //using Base::requested_flush_up_to; - // using Base::flushed_up_to; - // using Base::logged_queue_full_at_index; + Poco::Logger * log; + + using ISystemLog::is_shutdown; + using ISystemLog::saving_thread; private: + /* Saving thread data */ const StorageID table_id; const String storage_def; String create_query; String old_create_query; bool is_prepared = false; - const size_t flush_interval_milliseconds; + std::shared_ptr> queue; /** Creates new table if it does not exist. * Renames old table if its structure is not suitable. diff --git a/src/Interpreters/TextLog.cpp b/src/Interpreters/TextLog.cpp index a66092c1c2b..108135c78b3 100644 --- a/src/Interpreters/TextLog.cpp +++ b/src/Interpreters/TextLog.cpp @@ -84,7 +84,7 @@ TextLog::TextLog(ContextPtr context_, const String & database_name_, const String & table_name_, const String & storage_def_, size_t flush_interval_milliseconds_) : SystemLog(context_, database_name_, table_name_, - storage_def_, flush_interval_milliseconds_, getLogQueue()) + storage_def_, flush_interval_milliseconds_, getLogQueue(flush_interval_milliseconds_)) { // SystemLog methods may write text logs, so we disable logging for the text // log table to avoid recursion. diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 8390259e147..0febce03abc 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -47,9 +47,9 @@ public: const String & storage_def_, size_t flush_interval_milliseconds_); - static std::shared_ptr> getLogQueue() + static std::shared_ptr> getLogQueue(size_t flush_interval_milliseconds) { - static std::shared_ptr> queue = std::make_shared>("text_log"); + static std::shared_ptr> queue = std::make_shared>("text_log", flush_interval_milliseconds); return queue; } }; diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index fa143440cc2..a9869847f65 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -34,6 +34,7 @@ static std::string createDirectory(const std::string & file) return path; } +constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger /*_root*/, const std::string & cmd_name) { @@ -254,7 +255,9 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log { String text_log_level_str = config.getString("text_log.level", ""); int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); - split->addTextLog(DB::TextLog::getLogQueue(), text_log_level); + size_t flush_interval_milliseconds = config.getUInt64("text_log.flush_interval_milliseconds", + DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); + split->addTextLog(DB::TextLog::getLogQueue(flush_interval_milliseconds), text_log_level); } #endif } From e9b7a84aa1c7d4a5ccabed137ed1859ba87ea676 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 18 Jul 2023 12:09:58 +0200 Subject: [PATCH 0754/2047] Fix network manager for integration tests Sometimes you may get: > raise subprocess.CalledProcessError(exit_code, cmd) E subprocess.CalledProcessError: Command '['iptables', '--wait', '-D', 'DOCKER-USER', '-p', 'tcp', '-s', '172.16.2.3', '-d', '172.16.2.2', '-j', 'DROP']' returned non-zero exit status 137. And only sometimes you may get the reason: OCI runtime exec failed: exec failed: cannot exec in a stopped container: unknown So this means that container for iptables does not exists anymore, and the reason is the timeout. And the fact that container_exit_timeout was equal to container_expire_timeout and was 120. From the docker logs: time="2023-07-16T15:46:52.513673446Z" level=debug msg="form data: {\"AttachStderr\":false,\"AttachStdin\":false,\"AttachStdout\":false,\"Cmd\":[\"sleep\",\"120\"],\"HostConfig\":{\"AutoRemove\":true,\"NetworkMode\":\"host\"},\"Image\":\"clickhouse/integration-helper:latest\",\"NetworkDisabled\":false,\"OpenStdin\":false,\"StdinOnce\":false,\"Tty\":false}" time="2023-07-16T15:48:57.611857183Z" level=debug msg="form data: {\"AttachStderr\":false,\"AttachStdin\":false,\"AttachStdout\":false,\"Cmd\":[\"sleep\",\"120\"],\"HostConfig\":{\"AutoRemove\":true,\"NetworkMode\":\"host\"},\"Image\":\"clickhouse/integration-helper:latest\",\"NetworkDisabled\":false,\"OpenStdin\":false,\"StdinOnce\":false,\"Tty\":false}" And then pytest will try to execute the iptables command: time="2023-07-16T15:50:57.698705244Z" level=debug msg="starting exec command 860920ab2aa07e8d285050f200ac92423a3cf8ec3fb2f57683541e62cf6bc20e in container 66d6c96671b5e987345290ddd260727d96b99789b512d40f333f6263f42fd2f1" Signed-off-by: Azat Khuzhin --- tests/integration/helpers/network.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 60b46926589..e6e79dc7947 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -219,10 +219,15 @@ class _NetworkManager: def __init__( self, - container_expire_timeout=120, - container_exit_timeout=120, + container_expire_timeout=600, + container_exit_timeout=660, docker_api_version=os.environ.get("DOCKER_API_VERSION"), ): + # container should be alive for at least 15 seconds then the expiration + # timeout, this is the protection from the case when the container will + # be destroyed just when some test will try to use it. + assert container_exit_timeout >= container_expire_timeout + 15 + self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout From ed60bb90b28e9b325031db6cfcf438fa6a7ca51b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 10:22:07 +0000 Subject: [PATCH 0755/2047] Move code to be more beautiful in pr --- src/Common/SystemLogBase.cpp | 51 ++++++++++++++++++------------------ 1 file changed, 26 insertions(+), 25 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index dc6a4a08bb7..38d675e4b5f 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -58,7 +58,8 @@ void SystemLogQueue::add(const LogElement & element) return; recursive_add_call = true; SCOPE_EXIT({ recursive_add_call = false; }); - /// Memory can be allocated while resizing on queue.push_back. + + /// Memory can be allocated while resizing on queue.push_back. /// The size of allocation can be in order of a few megabytes. /// But this should not be accounted for query memory usage. /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky. @@ -124,30 +125,6 @@ void SystemLogQueue::shutdown() flush_event.notify_all(); } -template -uint64_t SystemLogQueue::notifyFlush(bool force) -{ - uint64_t this_thread_requested_offset; - - { - std::lock_guard lock(mutex); - if (is_shutdown) - return uint64_t(-1); - - this_thread_requested_offset = queue_front_index + queue.size(); - - // Publish our flush request, taking care not to overwrite the requests - // made by other threads. - is_force_prepare_tables |= force; - requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); - - flush_event.notify_all(); - } - - LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); - return this_thread_requested_offset; -} - template void SystemLogQueue::waitFlush(uint64_t this_thread_requested_offset_) { @@ -201,6 +178,30 @@ void SystemLogQueue::pop(std::vector& output, uint64_t& exit_this_thread = is_shutdown; } +template +uint64_t SystemLogQueue::notifyFlush(bool force) +{ + uint64_t this_thread_requested_offset; + + { + std::lock_guard lock(mutex); + if (is_shutdown) + return uint64_t(-1); + + this_thread_requested_offset = queue_front_index + queue.size(); + + // Publish our flush request, taking care not to overwrite the requests + // made by other threads. + is_force_prepare_tables |= force; + requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); + + flush_event.notify_all(); + } + + LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); + return this_thread_requested_offset; +} + #define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogQueue; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) From 92ca2b0bdd4e2160831d14f1c855a55dbac1a108 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 18 Jul 2023 11:34:28 +0200 Subject: [PATCH 0756/2047] Make psql tests in test_odbc_interaction more robust to other test failures Signed-off-by: Azat Khuzhin --- .../integration/test_odbc_interaction/test.py | 295 +++++++++--------- 1 file changed, 155 insertions(+), 140 deletions(-) diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 14f5de17870..e8b3ba3fcf3 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -582,75 +582,83 @@ def test_sqlite_odbc_cached_dictionary(started_cluster): def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(started_cluster) - cursor = conn.cursor() - cursor.execute( - "insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')" - ) - node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed") - node1.exec_in_container( - ["ss", "-K", "dport", "postgresql"], privileged=True, user="root" - ) - node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed") - assert_eq_with_retry( - node1, - "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))", - "hello", - ) - assert_eq_with_retry( - node1, - "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))", - "world", - ) - cursor.execute("truncate table clickhouse.test_table") + try: + conn = get_postgres_conn(started_cluster) + cursor = conn.cursor() + cursor.execute( + "insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')" + ) + node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed") + node1.exec_in_container( + ["ss", "-K", "dport", "postgresql"], privileged=True, user="root" + ) + node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed") + assert_eq_with_retry( + node1, + "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))", + "hello", + ) + assert_eq_with_retry( + node1, + "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))", + "world", + ) + finally: + cursor.execute("truncate table clickhouse.test_table") def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(started_cluster) - cursor = conn.cursor() - cursor.execute("insert into clickhouse.test_table values(3, 3, 'xxx')") - for i in range(100): - try: - node1.query("system reload dictionary postgres_odbc_hashed", timeout=15) - except Exception as ex: - assert False, "Exception occured -- odbc-bridge hangs: " + str(ex) + try: + conn = get_postgres_conn(started_cluster) + cursor = conn.cursor() + cursor.execute("insert into clickhouse.test_table values(3, 3, 'xxx')") + for i in range(100): + try: + node1.query("system reload dictionary postgres_odbc_hashed", timeout=15) + except Exception as ex: + assert False, "Exception occured -- odbc-bridge hangs: " + str(ex) - assert_eq_with_retry( - node1, - "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(3))", - "xxx", - ) - cursor.execute("truncate table clickhouse.test_table") + assert_eq_with_retry( + node1, + "select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(3))", + "xxx", + ) + finally: + cursor.execute("truncate table clickhouse.test_table") def test_no_connection_pooling(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(started_cluster) - cursor = conn.cursor() - cursor.execute( - "insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')" - ) - node1.exec_in_container(["ss", "-K", "dport", "5432"], privileged=True, user="root") - node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_nopool") - assert_eq_with_retry( - node1, - "select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(1))", - "hello", - ) - assert_eq_with_retry( - node1, - "select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(2))", - "world", - ) + try: + conn = get_postgres_conn(started_cluster) + cursor = conn.cursor() + cursor.execute( + "insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')" + ) + node1.exec_in_container( + ["ss", "-K", "dport", "5432"], privileged=True, user="root" + ) + node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_nopool") + assert_eq_with_retry( + node1, + "select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(1))", + "hello", + ) + assert_eq_with_retry( + node1, + "select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(2))", + "world", + ) - # No open connections should be left because we don't use connection pooling. - assert "" == node1.exec_in_container( - ["ss", "-H", "dport", "5432"], privileged=True, user="root" - ) - cursor.execute("truncate table clickhouse.test_table") + # No open connections should be left because we don't use connection pooling. + assert "" == node1.exec_in_container( + ["ss", "-H", "dport", "5432"], privileged=True, user="root" + ) + finally: + cursor.execute("truncate table clickhouse.test_table") def test_postgres_insert(started_cluster): @@ -662,112 +670,119 @@ def test_postgres_insert(started_cluster): # postgres .yml file). This is needed to check parsing, validation and # reconstruction of connection string. - node1.query( - "create table pg_insert (id UInt64, column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" - ) - node1.query("insert into pg_insert values (1, 1, 'hello'), (2, 2, 'world')") - assert node1.query("select * from pg_insert") == "1\t1\thello\n2\t2\tworld\n" - node1.query( - "insert into table function odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table') format CSV 3,3,test" - ) - node1.query( - "insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" - " select number, number, 's' || toString(number) from numbers (4, 7)" - ) - assert ( - node1.query("select sum(column1), count(column1) from pg_insert") == "55\t10\n" - ) - assert ( + try: node1.query( - "select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table')))" + "create table pg_insert (id UInt64, column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" ) - == "55\t10\n" - ) - node1.query("DROP TABLE pg_insert") - conn.cursor().execute("truncate table clickhouse.test_table") + node1.query("insert into pg_insert values (1, 1, 'hello'), (2, 2, 'world')") + assert node1.query("select * from pg_insert") == "1\t1\thello\n2\t2\tworld\n" + node1.query( + "insert into table function odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table') format CSV 3,3,test" + ) + node1.query( + "insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" + " select number, number, 's' || toString(number) from numbers (4, 7)" + ) + assert ( + node1.query("select sum(column1), count(column1) from pg_insert") + == "55\t10\n" + ) + assert ( + node1.query( + "select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table')))" + ) + == "55\t10\n" + ) + finally: + node1.query("DROP TABLE IF EXISTS pg_insert") + conn.cursor().execute("truncate table clickhouse.test_table") def test_odbc_postgres_date_data_type(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(started_cluster) - cursor = conn.cursor() - cursor.execute( - "CREATE TABLE clickhouse.test_date (id integer, column1 integer, column2 date)" - ) + try: + conn = get_postgres_conn(started_cluster) + cursor = conn.cursor() + cursor.execute( + "CREATE TABLE clickhouse.test_date (id integer, column1 integer, column2 date)" + ) - cursor.execute("INSERT INTO clickhouse.test_date VALUES (1, 1, '2020-12-01')") - cursor.execute("INSERT INTO clickhouse.test_date VALUES (2, 2, '2020-12-02')") - cursor.execute("INSERT INTO clickhouse.test_date VALUES (3, 3, '2020-12-03')") - conn.commit() + cursor.execute("INSERT INTO clickhouse.test_date VALUES (1, 1, '2020-12-01')") + cursor.execute("INSERT INTO clickhouse.test_date VALUES (2, 2, '2020-12-02')") + cursor.execute("INSERT INTO clickhouse.test_date VALUES (3, 3, '2020-12-03')") + conn.commit() - node1.query( - """ - CREATE TABLE test_date (id UInt64, column1 UInt64, column2 Date) - ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_date')""" - ) + node1.query( + """ + CREATE TABLE test_date (id UInt64, column1 UInt64, column2 Date) + ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_date')""" + ) - expected = "1\t1\t2020-12-01\n2\t2\t2020-12-02\n3\t3\t2020-12-03\n" - result = node1.query("SELECT * FROM test_date") - assert result == expected - cursor.execute("DROP TABLE clickhouse.test_date") - node1.query("DROP TABLE test_date") + expected = "1\t1\t2020-12-01\n2\t2\t2020-12-02\n3\t3\t2020-12-03\n" + result = node1.query("SELECT * FROM test_date") + assert result == expected + finally: + cursor.execute("DROP TABLE clickhouse.test_date") + node1.query("DROP TABLE IF EXISTS test_date") def test_odbc_postgres_conversions(started_cluster): skip_test_msan(node1) - conn = get_postgres_conn(started_cluster) - cursor = conn.cursor() + try: + conn = get_postgres_conn(started_cluster) + cursor = conn.cursor() - cursor.execute( - """CREATE TABLE clickhouse.test_types ( - a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, - h timestamp)""" - ) + cursor.execute( + """CREATE TABLE clickhouse.test_types ( + a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, + h timestamp)""" + ) - node1.query( - """ - INSERT INTO TABLE FUNCTION - odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types') - VALUES (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12')""" - ) + node1.query( + """ + INSERT INTO TABLE FUNCTION + odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types') + VALUES (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12')""" + ) - result = node1.query( - """ - SELECT a, b, c, d, e, f, g, h - FROM odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types') - """ - ) + result = node1.query( + """ + SELECT a, b, c, d, e, f, g, h + FROM odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types') + """ + ) - assert ( - result - == "-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\n" - ) - cursor.execute("DROP TABLE IF EXISTS clickhouse.test_types") + assert ( + result + == "-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\n" + ) + cursor.execute("DROP TABLE IF EXISTS clickhouse.test_types") - cursor.execute( - """CREATE TABLE clickhouse.test_types (column1 Timestamp, column2 Numeric)""" - ) + cursor.execute( + """CREATE TABLE clickhouse.test_types (column1 Timestamp, column2 Numeric)""" + ) - node1.query( - """ - CREATE TABLE test_types (column1 DateTime64, column2 Decimal(5, 1)) - ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types')""" - ) + node1.query( + """ + CREATE TABLE test_types (column1 DateTime64, column2 Decimal(5, 1)) + ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types')""" + ) - node1.query( - """INSERT INTO test_types - SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)""" - ) + node1.query( + """INSERT INTO test_types + SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)""" + ) - expected = node1.query( - "SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)" - ) - result = node1.query("SELECT * FROM test_types") - cursor.execute("DROP TABLE clickhouse.test_types") - node1.query("DROP TABLE test_types") - assert result == expected + expected = node1.query( + "SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)" + ) + result = node1.query("SELECT * FROM test_types") + assert result == expected + finally: + cursor.execute("DROP TABLE IF EXISTS clickhouse.test_types") + node1.query("DROP TABLE IF EXISTS test_types") def test_odbc_cyrillic_with_varchar(started_cluster): From baf6d182d63dddf9dfb42d687d21ed2835f536a2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 18 Jul 2023 11:49:34 +0200 Subject: [PATCH 0757/2047] Fix reporting some BROKEN tests as ERROR Signed-off-by: Azat Khuzhin --- tests/integration/ci-runner.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index f2adea78da7..a6e9716dc20 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -788,6 +788,7 @@ class ClickhouseIntegrationTestsRunner: and test not in counters["ERROR"] and test not in counters["SKIPPED"] and test not in counters["FAILED"] + and test not in counters["BROKEN"] and "::" in test ): counters["ERROR"].append(test) From 0cfd12aba46030b2147def929fada009a05eb07c Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 18 Jul 2023 12:20:56 +0200 Subject: [PATCH 0758/2047] Fix --- .../MaterializedPostgreSQLConsumer.cpp | 55 +++++++++++++------ .../test.py | 11 ++-- 2 files changed, 45 insertions(+), 21 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index ea7009fc082..ff4ccd97c0d 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -11,6 +11,7 @@ #include #include #include +#include "Common/Exception.h" #include @@ -22,6 +23,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int POSTGRESQL_REPLICATION_INTERNAL_ERROR; extern const int BAD_ARGUMENTS; + extern const int TOO_MANY_PARTS; } MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( @@ -556,34 +558,49 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl void MaterializedPostgreSQLConsumer::syncTables() { - for (const auto & table_name : tables_to_sync) + while (!tables_to_sync.empty()) { + auto table_name = *tables_to_sync.begin(); auto & storage_data = storages.find(table_name)->second; Block result_rows = storage_data.buffer.description.sample_block.cloneWithColumns(std::move(storage_data.buffer.columns)); storage_data.buffer.columns = storage_data.buffer.description.sample_block.cloneEmptyColumns(); - if (result_rows.rows()) + try { - auto storage = storage_data.storage; + if (result_rows.rows()) + { + auto storage = storage_data.storage; - auto insert_context = Context::createCopy(context); - insert_context->setInternalQuery(true); + auto insert_context = Context::createCopy(context); + insert_context->setInternalQuery(true); - auto insert = std::make_shared(); - insert->table_id = storage->getStorageID(); - insert->columns = storage_data.buffer.columns_ast; + auto insert = std::make_shared(); + insert->table_id = storage->getStorageID(); + insert->columns = storage_data.buffer.columns_ast; - InterpreterInsertQuery interpreter(insert, insert_context, true); - auto io = interpreter.execute(); - auto input = std::make_shared( - result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); + InterpreterInsertQuery interpreter(insert, insert_context, true); + auto io = interpreter.execute(); + auto input = std::make_shared( + result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); - assertBlocksHaveEqualStructure(input->getPort().getHeader(), io.pipeline.getHeader(), "postgresql replica table sync"); - io.pipeline.complete(Pipe(std::move(input))); + assertBlocksHaveEqualStructure(input->getPort().getHeader(), io.pipeline.getHeader(), "postgresql replica table sync"); + io.pipeline.complete(Pipe(std::move(input))); - CompletedPipelineExecutor executor(io.pipeline); - executor.execute(); + CompletedPipelineExecutor executor(io.pipeline); + executor.execute(); + } } + catch (DB::Exception & e) + { + if (e.code() == ErrorCodes::TOO_MANY_PARTS) + { + /// Retry this buffer later. + storage_data.buffer.columns = result_rows.mutateColumns(); + } + throw; + } + + tables_to_sync.erase(tables_to_sync.begin()); } LOG_DEBUG(log, "Table sync end for {} tables, last lsn: {} = {}, (attempted lsn {})", tables_to_sync.size(), current_lsn, getLSNValue(current_lsn), getLSNValue(final_lsn)); @@ -735,8 +752,12 @@ void MaterializedPostgreSQLConsumer::setSetting(const SettingChange & setting) /// Read binary changes from replication slot via COPY command (starting from current lsn in a slot). bool MaterializedPostgreSQLConsumer::consume() { - bool slot_empty = true; + if (!tables_to_sync.empty()) + { + syncTables(); + } + bool slot_empty = true; try { auto tx = std::make_shared(connection->getRef()); diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 2b17024f417..51e980f0423 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -545,7 +545,9 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): clickhouse_postgres_db = f"clickhouse_postgres_db{i}" create_postgres_schema(cursor, schema_name) pg_manager.create_clickhouse_postgres_db( - database_name=clickhouse_postgres_db, schema_name=schema_name, postgres_database="postgres_database", + database_name=clickhouse_postgres_db, + schema_name=schema_name, + postgres_database="postgres_database", ) for ti in range(NUM_TABLES): table_name = f"postgresql_replica_{ti}" @@ -695,15 +697,16 @@ def test_too_many_parts(started_cluster): time.sleep(1) print(f"wait sync try {i}") if instance2.contains_in_log("DB::Exception: Too many parts"): - num = num - 1 break assert num == int( instance2.query("SELECT count() FROM test_database.test_table") - ) + ) or num - 1 == int(instance2.query("SELECT count() FROM test_database.test_table")) assert instance2.contains_in_log("DB::Exception: Too many parts") print(num) - assert num == int(instance2.query("SELECT count() FROM test_database.test_table")) + assert num == int( + instance2.query("SELECT count() FROM test_database.test_table") + ) or num - 1 == int(instance2.query("SELECT count() FROM test_database.test_table")) instance2.query("SYSTEM START MERGES") check_tables_are_synchronized( From 722b3859083daf255f7dcde1d188cb9521d37962 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 18 Jul 2023 12:26:42 +0200 Subject: [PATCH 0759/2047] List replication queue only for current test database --- .../0_stateless/00834_kill_mutation_replicated_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh index 6797e9cac12..2e917f67fe8 100755 --- a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh @@ -36,7 +36,7 @@ ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA kill_mutation_r1" ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA kill_mutation_r2" # Should be empty, but in case of problems we will see some diagnostics -${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.replication_queue WHERE table like 'kill_mutation_r%'" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.replication_queue WHERE database = '$CLICKHOUSE_DATABASE' AND table like 'kill_mutation_r%'" ${CLICKHOUSE_CLIENT} --query="ALTER TABLE kill_mutation_r1 DELETE WHERE toUInt32(s) = 1" From f3bfc6e5cf8476734110e9a0037c82120ca64eb4 Mon Sep 17 00:00:00 2001 From: timfursov <126660039+timfursov@users.noreply.github.com> Date: Tue, 18 Jul 2023 11:40:57 +0100 Subject: [PATCH 0760/2047] Update with.md I propose changes to make it clear that CTEs do not guarantee the same results in all places they are used, since the query will be re-executed in each place they are used. --- docs/en/sql-reference/statements/select/with.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/with.md b/docs/en/sql-reference/statements/select/with.md index 4654f249548..f841d497dbb 100644 --- a/docs/en/sql-reference/statements/select/with.md +++ b/docs/en/sql-reference/statements/select/with.md @@ -5,7 +5,9 @@ sidebar_label: WITH # WITH Clause -ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)), that is provides to use results of `WITH` clause in the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. +ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)) and substitutes the code defined in the `WITH` clause in all places of use for the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. + +Note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case. ## Syntax From a74a56b0d7026d187783da861ef8eb4840947764 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Jul 2023 11:06:16 +0000 Subject: [PATCH 0761/2047] Attempt to fix assert in tsan with fibers --- src/Common/AsyncTaskExecutor.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Common/AsyncTaskExecutor.cpp b/src/Common/AsyncTaskExecutor.cpp index b824a0a5b31..923dad1d346 100644 --- a/src/Common/AsyncTaskExecutor.cpp +++ b/src/Common/AsyncTaskExecutor.cpp @@ -5,7 +5,6 @@ namespace DB AsyncTaskExecutor::AsyncTaskExecutor(std::unique_ptr task_) : task(std::move(task_)) { - createFiber(); } void AsyncTaskExecutor::resume() @@ -13,6 +12,10 @@ void AsyncTaskExecutor::resume() if (routine_is_finished) return; + /// Create fiber lazily on first resume() call. + if (!fiber) + createFiber(); + if (!checkBeforeTaskResume()) return; @@ -22,6 +25,11 @@ void AsyncTaskExecutor::resume() return; resumeUnlocked(); + + /// Destroy fiber when it's finished. + if (routine_is_finished) + destroyFiber(); + if (exception) processException(exception); } @@ -46,9 +54,8 @@ void AsyncTaskExecutor::cancel() void AsyncTaskExecutor::restart() { std::lock_guard guard(fiber_lock); - if (fiber) + if (!routine_is_finished) destroyFiber(); - createFiber(); routine_is_finished = false; } From f163953b277b6ab1bc3d0aa0c68d8f7a65ce28c0 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 13:52:59 +0200 Subject: [PATCH 0762/2047] Use error name instead of number in tests --- tests/queries/0_stateless/02534_keyed_siphash.sql | 8 ++++---- tests/queries/0_stateless/02552_siphash128_reference.sql | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 70ec8470ca3..e780e1df167 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -263,10 +263,10 @@ select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); -select sipHash64Keyed((0, 0), '1'); -- { serverError 48 } -select sipHash128Keyed((0, 0), '1'); -- { serverError 48 } -select sipHash64Keyed(toUInt64(0), '1'); -- { serverError 48 } -select sipHash128Keyed(toUInt64(0), '1'); -- { serverError 48 } +select sipHash64Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash128Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash64Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash128Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } select hex(sipHash64()); SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql index c238e51b690..8b27ecbe61c 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.sql +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -200,8 +200,8 @@ select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); -select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError 48 } -select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError 48 } +select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; SELECT hex(sipHash128ReferenceKeyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; From 00d4a0790905360eae940d08becf01176e272ef1 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 14:02:48 +0200 Subject: [PATCH 0763/2047] Improve tests formatting --- .../0_stateless/02534_keyed_siphash.reference | 10 ++-- .../0_stateless/02534_keyed_siphash.sql | 54 +++++-------------- 2 files changed, 17 insertions(+), 47 deletions(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index ea77e5c3963..b6dd2a1c9a3 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -209,17 +209,15 @@ Check const columns 15080046610211022027 2E779C73D13981AA1AE19AFF9617EA49 2E779C73D13981AA1AE19AFF9617EA49 -Check multiple keys +Check multiple keys as tuple from a table 11862823756610506724 9357996107237883963 -Check multiple keys 2 +Check multiple keys as separate ints from a table 11862823756610506724 9357996107237883963 +Check constant key and data from a table 11862823756610506724 11862823756610506724 -Check multiple keys 3 +Check multiple keys as separate ints from a table with constant data 11862823756610506724 9357996107237883963 -- -11862823756610506724 -11862823756610506724 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index e780e1df167..5c6789bc305 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -290,65 +290,37 @@ SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt6 SELECT 'Check const columns'; DROP TABLE IF EXISTS sipHashKeyed_test; - -CREATE TABLE sipHashKeyed_test -ENGINE = Memory() -AS -SELECT -1 a, -'test' b; - -SELECT -sipHash64Keyed((toUInt64(0), toUInt64(0)), 1, 'test'); - -SELECT -sipHash64(tuple(*)) -FROM -sipHashKeyed_test; - -SELECT -sipHash64Keyed((toUInt64(0), toUInt64(0)), tuple(*)) -FROM -sipHashKeyed_test; - -SELECT -sipHash64Keyed((toUInt64(0), toUInt64(0)), a, b) -FROM -sipHashKeyed_test; - -SELECT -hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), tuple(*))) -FROM -sipHashKeyed_test; - -SELECT -hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), a, b)) -FROM -sipHashKeyed_test; - +CREATE TABLE sipHashKeyed_test ENGINE = Memory() AS SELECT 1 a, 'test' b; +SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), 1, 'test'); +SELECT sipHash64(tuple(*)) FROM sipHashKeyed_test; +SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), tuple(*)) FROM sipHashKeyed_test; +SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), a, b) FROM sipHashKeyed_test; +SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test; +SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test; DROP TABLE sipHashKeyed_test; -SELECT 'Check multiple keys'; +SELECT 'Check multiple keys as tuple from a table'; DROP TABLE IF EXISTS sipHashKeyed_keys; CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); SELECT sipHash64Keyed(key, val) FROM sipHashKeyed_keys ORDER by key; DROP TABLE sipHashKeyed_keys; -SELECT 'Check multiple keys 2'; + +SELECT 'Check multiple keys as separate ints from a table'; DROP TABLE IF EXISTS sipHashKeyed_keys; CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory; INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); SELECT sipHash64Keyed((key0, key1), val) FROM sipHashKeyed_keys ORDER by key0; +SELECT 'Check constant key and data from a table'; SELECT sipHash64Keyed((2::UInt64, 2::UInt64), val) FROM sipHashKeyed_keys ORDER by val; DROP TABLE sipHashKeyed_keys; -SELECT 'Check multiple keys 3'; + +SELECT 'Check multiple keys as separate ints from a table with constant data'; DROP TABLE IF EXISTS sipHashKeyed_keys; CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; INSERT INTO sipHashKeyed_keys VALUES (2, 2); INSERT INTO sipHashKeyed_keys VALUES (4, 4); SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; -SELECT '-'; -SELECT sipHash64Keyed((2::UInt64, 2::UInt64), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; DROP TABLE sipHashKeyed_keys; From 5a398f9a9324be4bdc126f782c01abbe4a9f5b29 Mon Sep 17 00:00:00 2001 From: timfursov <126660039+timfursov@users.noreply.github.com> Date: Tue, 18 Jul 2023 13:39:53 +0100 Subject: [PATCH 0764/2047] Update with.md Added an example --- .../en/sql-reference/statements/select/with.md | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/with.md b/docs/en/sql-reference/statements/select/with.md index f841d497dbb..3a74378008a 100644 --- a/docs/en/sql-reference/statements/select/with.md +++ b/docs/en/sql-reference/statements/select/with.md @@ -7,7 +7,23 @@ sidebar_label: WITH ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)) and substitutes the code defined in the `WITH` clause in all places of use for the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. -Note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case. +Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case.
+An example of such behavior is below +``` sql +with cte_numbers as +( + select + num + from generateRandom('num UInt64', NULL) + limit 1000000 +) +select + count() +from cte_numbers +where num in (select num from cte_numbers) +``` +If CTEs were to pass exactly the results and not just a piece of code, you would always see `1000000`
+However, due to the fact that we are referring `cte_numbers` twice, random numbers are generated each time and, accordingly, we see different random results, `280501, 392454, 261636, 196227` and so on... ## Syntax From 47370136e0f3b86a03120d962d8fe43dbcfca9dc Mon Sep 17 00:00:00 2001 From: "pedro.riera" Date: Wed, 14 Jun 2023 10:45:39 +0200 Subject: [PATCH 0765/2047] new alias for current_database and added current_schemas --- src/Functions/currentDatabase.cpp | 3 +- src/Functions/currentSchemas.cpp | 88 +++++++++++++++++++ .../02788_current_schemas_function.reference | 2 + .../02788_current_schemas_function.sql | 4 + 4 files changed, 96 insertions(+), 1 deletion(-) create mode 100644 src/Functions/currentSchemas.cpp create mode 100644 tests/queries/0_stateless/02788_current_schemas_function.reference create mode 100644 tests/queries/0_stateless/02788_current_schemas_function.sql diff --git a/src/Functions/currentDatabase.cpp b/src/Functions/currentDatabase.cpp index b1a3cbf5856..b7fd6c4fecc 100644 --- a/src/Functions/currentDatabase.cpp +++ b/src/Functions/currentDatabase.cpp @@ -54,7 +54,8 @@ public: REGISTER_FUNCTION(CurrentDatabase) { factory.registerFunction(); - factory.registerAlias("DATABASE", "currentDatabase", FunctionFactory::CaseInsensitive); + factory.registerAlias("DATABASE", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("current_database", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive); } } diff --git a/src/Functions/currentSchemas.cpp b/src/Functions/currentSchemas.cpp new file mode 100644 index 00000000000..322e719eb17 --- /dev/null +++ b/src/Functions/currentSchemas.cpp @@ -0,0 +1,88 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +class FunctionCurrentSchemas : public IFunction +{ + const String db_name; + +public: + static constexpr auto name = "currentSchemas"; + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context->getCurrentDatabase()); + } + + explicit FunctionCurrentSchemas(const String & db_name_) : + db_name{db_name_} + { + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + // For compatibility, function implements the same signature as Postgres' + const bool argument_is_valid = arguments.size() == 1 && isBool(arguments.front()); + if (!argument_is_valid) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument for function {} must be bool", getName()); + + return std::make_shared(std::make_shared()); + } + + bool isDeterministic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + return DataTypeArray(std::make_shared()) + .createColumnConst(input_rows_count, Array { db_name }); + } +}; + +} + +REGISTER_FUNCTION(CurrentSchema) +{ + factory.registerFunction(FunctionDocumentation + { + .description=R"( +Returns a single-element array with the name of the current database + +Requires a boolean parameter, but it is ignored actually. It is required just for compatibility with the implementation of this function in other DB engines. + +[example:common] +)", + .examples{ + {"common", "SELECT current_schemas(true);", "['default']"} + } + }, + FunctionFactory::CaseInsensitive); + factory.registerAlias("current_schemas", FunctionCurrentSchemas::name, FunctionFactory::CaseInsensitive); + +} + +} diff --git a/tests/queries/0_stateless/02788_current_schemas_function.reference b/tests/queries/0_stateless/02788_current_schemas_function.reference new file mode 100644 index 00000000000..999c306b3ac --- /dev/null +++ b/tests/queries/0_stateless/02788_current_schemas_function.reference @@ -0,0 +1,2 @@ +['default'] +['default'] diff --git a/tests/queries/0_stateless/02788_current_schemas_function.sql b/tests/queries/0_stateless/02788_current_schemas_function.sql new file mode 100644 index 00000000000..408b21c0e34 --- /dev/null +++ b/tests/queries/0_stateless/02788_current_schemas_function.sql @@ -0,0 +1,4 @@ +SELECT current_schemas(true) AS result; +SELECT current_schemas(false) AS result; +SELECT current_schemas(1); -- { serverError 43 } +SELECT current_schemas(); -- { serverError 42 } \ No newline at end of file From 7466ef58376810d7a81a1e48ac14937d336cd97f Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 13:05:20 +0000 Subject: [PATCH 0766/2047] Add index --- src/Common/SystemLogBase.cpp | 70 ++++++++++++++++----------------- src/Common/SystemLogBase.h | 27 +++++++------ src/Interpreters/SystemLog.cpp | 4 +- src/Loggers/OwnSplitChannel.cpp | 2 +- 4 files changed, 53 insertions(+), 50 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 38d675e4b5f..e9442617acd 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -47,17 +47,17 @@ SystemLogQueue::SystemLogQueue( , flush_interval_milliseconds(flush_interval_milliseconds_) {} -static thread_local bool recursive_add_call = false; +static thread_local bool recursive_push_call = false; template -void SystemLogQueue::add(const LogElement & element) +void SystemLogQueue::push(const LogElement & element) { /// It is possible that the method will be called recursively. /// Better to drop these events to avoid complications. - if (recursive_add_call) + if (recursive_push_call) return; - recursive_add_call = true; - SCOPE_EXIT({ recursive_add_call = false; }); + recursive_push_call = true; + SCOPE_EXIT({ recursive_push_call = false; }); /// Memory can be allocated while resizing on queue.push_back. /// The size of allocation can be in order of a few megabytes. @@ -118,15 +118,31 @@ void SystemLogQueue::add(const LogElement & element) } template -void SystemLogQueue::shutdown() -{ - is_shutdown = true; - /// Tell thread to shutdown. - flush_event.notify_all(); +uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) +{ + uint64_t this_thread_requested_offset; + + { + std::lock_guard lock(mutex); + if (is_shutdown) + return uint64_t(-1); + + this_thread_requested_offset = queue_front_index + queue.size(); + + // Publish our flush request, taking care not to overwrite the requests + // made by other threads. + is_force_prepare_tables |= should_prepare_tables_anyway; + requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); + + flush_event.notify_all(); + } + + LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); + return this_thread_requested_offset; } template -void SystemLogQueue::waitFlush(uint64_t this_thread_requested_offset_) +void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) { // Use an arbitrary timeout to avoid endless waiting. 60s proved to be // too fast for our parallel functional tests, probably because they @@ -135,7 +151,7 @@ void SystemLogQueue::waitFlush(uint64_t this_thread_requested_offset std::unique_lock lock(mutex); bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] { - return flushed_up_to >= this_thread_requested_offset_ && !is_force_prepare_tables; + return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables; }); if (!result) @@ -155,7 +171,7 @@ void SystemLogQueue::confirm(uint64_t to_flush_end) } template -void SystemLogQueue::pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread) +SystemLogQueue::Index SystemLogQueue::pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread) { std::unique_lock lock(mutex); flush_event.wait_for(lock, @@ -167,7 +183,6 @@ void SystemLogQueue::pop(std::vector& output, uint64_t& ); queue_front_index += queue.size(); - to_flush_end = queue_front_index; // Swap with existing array from previous flush, to save memory // allocations. output.resize(0); @@ -176,30 +191,15 @@ void SystemLogQueue::pop(std::vector& output, uint64_t& should_prepare_tables_anyway = is_force_prepare_tables; exit_this_thread = is_shutdown; + return queue_front_index; } template -uint64_t SystemLogQueue::notifyFlush(bool force) -{ - uint64_t this_thread_requested_offset; - - { - std::lock_guard lock(mutex); - if (is_shutdown) - return uint64_t(-1); - - this_thread_requested_offset = queue_front_index + queue.size(); - - // Publish our flush request, taking care not to overwrite the requests - // made by other threads. - is_force_prepare_tables |= force; - requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset); - - flush_event.notify_all(); - } - - LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset); - return this_thread_requested_offset; +void SystemLogQueue::shutdown() +{ + is_shutdown = true; + /// Tell thread to shutdown. + flush_event.notify_all(); } #define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogQueue; diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 4cdb07c2cab..281cd06354a 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -75,20 +75,23 @@ protected: template class SystemLogQueue { + using Index = uint64_t; + public: SystemLogQueue( const String & name_, size_t flush_interval_milliseconds_); - // producer methods - void add(const LogElement & element); void shutdown(); - uint64_t notifyFlush(bool force); - void waitFlush(uint64_t this_thread_requested_offset_); + + // producer methods + void push(const LogElement & element); + Index notifyFlush(bool should_prepare_tables_anyway); + void waitFlush(Index expected_flushed_up_to); // consumer methods - void pop(std::vector& output, uint64_t& to_flush_end, bool& should_prepare_tables_anyway, bool& exit_this_thread); - void confirm(uint64_t to_flush_end); + Index pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread); + void confirm(Index to_flush_end); /// Data shared between callers of add()/flush()/shutdown(), and the saving thread std::mutex mutex; @@ -102,19 +105,19 @@ private: // We use it to give a global sequential index to every message, so that we // can wait until a particular message is flushed. This is used to implement // synchronous log flushing for SYSTEM FLUSH LOGS. - uint64_t queue_front_index = 0; + Index queue_front_index = 0; // A flag that says we must create the tables even if the queue is empty. bool is_force_prepare_tables = false; // Requested to flush logs up to this index, exclusive - uint64_t requested_flush_up_to = 0; - std::condition_variable flush_event; + Index requested_flush_up_to = 0; // Flushed log up to this index, exclusive - uint64_t flushed_up_to = 0; + Index flushed_up_to = 0; // Logged overflow message at this queue front index - uint64_t logged_queue_full_at_index = -1; - + Index logged_queue_full_at_index = -1; + bool is_shutdown = false; + std::condition_variable flush_event; const size_t flush_interval_milliseconds; }; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index cb4578689a2..3193baa551f 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -398,7 +398,7 @@ void SystemLog::savingThreadFunction() // Should we prepare table even if there are no new messages. bool should_prepare_tables_anyway = false; - queue->pop(to_flush, to_flush_end, should_prepare_tables_anyway, exit_this_thread); + to_flush_end = queue->pop(to_flush, should_prepare_tables_anyway, exit_this_thread); if (to_flush.empty()) { @@ -621,7 +621,7 @@ ASTPtr SystemLog::getCreateTableQuery() template void SystemLog::add(const LogElement & element) { - queue->add(element); + queue->push(element); } template diff --git a/src/Loggers/OwnSplitChannel.cpp b/src/Loggers/OwnSplitChannel.cpp index 17806153905..b5ac42d6041 100644 --- a/src/Loggers/OwnSplitChannel.cpp +++ b/src/Loggers/OwnSplitChannel.cpp @@ -138,7 +138,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) std::shared_ptr> text_log_locked{}; text_log_locked = text_log.lock(); if (text_log_locked) - text_log_locked->add(elem); + text_log_locked->push(elem); } #endif } From 681bf10c9319722dd32eaf11abd2f42b07452ab4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 18 Jul 2023 13:06:02 +0000 Subject: [PATCH 0767/2047] Check nullptr in fuzzer --- src/Client/QueryFuzzer.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 39ca7a5ed88..5ce95c82528 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -848,6 +848,9 @@ ASTs QueryFuzzer::getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query) void QueryFuzzer::notifyQueryFailed(ASTPtr ast) { + if (ast == nullptr) + return; + auto remove_fuzzed_table = [this](const auto & table_name) { auto pos = table_name.find("__fuzz_"); From cfcfb83ab739a9271a5da8618ee372005f76c357 Mon Sep 17 00:00:00 2001 From: timfursov <126660039+timfursov@users.noreply.github.com> Date: Tue, 18 Jul 2023 14:41:43 +0100 Subject: [PATCH 0768/2047] Update with.md replaced
to \ --- docs/en/sql-reference/statements/select/with.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/select/with.md b/docs/en/sql-reference/statements/select/with.md index 3a74378008a..662262efb2b 100644 --- a/docs/en/sql-reference/statements/select/with.md +++ b/docs/en/sql-reference/statements/select/with.md @@ -7,7 +7,7 @@ sidebar_label: WITH ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)) and substitutes the code defined in the `WITH` clause in all places of use for the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. -Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case.
+Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case.\ An example of such behavior is below ``` sql with cte_numbers as @@ -22,7 +22,7 @@ select from cte_numbers where num in (select num from cte_numbers) ``` -If CTEs were to pass exactly the results and not just a piece of code, you would always see `1000000`
+If CTEs were to pass exactly the results and not just a piece of code, you would always see `1000000`\ However, due to the fact that we are referring `cte_numbers` twice, random numbers are generated each time and, accordingly, we see different random results, `280501, 392454, 261636, 196227` and so on... ## Syntax From 36822543423f7812a3a80b842fa549e32337e45c Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Jul 2023 13:49:23 +0000 Subject: [PATCH 0769/2047] Fix hdfs tests --- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/StorageS3.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index e583d2e30b7..12fd12d73d5 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -314,7 +314,7 @@ public: hdfs_info = hdfsGetPathInfo(fs.get(), path_and_uri.first.c_str()); } /// Skip non-existed files. - while (String(hdfsGetLastError()).find("FileNotFoundException") != std::string::npos); + while (!hdfs_info && String(hdfsGetLastError()).find("FileNotFoundException") != std::string::npos); std::optional info; if (hdfs_info) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 0bb840c8d5d..22b1037ceef 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -418,7 +418,7 @@ public: std::function file_progress_callback_) : WithContext(context_) , keys(keys_) - , client(S3::Client::create(client_)) + , client(client_.clone()) , version_id(version_id_) , bucket(bucket_) , request_settings(request_settings_) From 6ae5207819b84dfd63c92cbd848995eaba6586f2 Mon Sep 17 00:00:00 2001 From: Song Liyong Date: Thu, 13 Jul 2023 13:27:23 +0200 Subject: [PATCH 0770/2047] MaterializedMySQL: Introduce charset conversion --- src/Core/MySQL/IMySQLReadPacket.cpp | 12 +- src/Core/MySQL/IMySQLReadPacket.h | 1 + src/Core/MySQL/MySQLCharset.cpp | 301 +++++++++++++++ src/Core/MySQL/MySQLCharset.h | 26 ++ src/Core/MySQL/MySQLReplication.cpp | 143 ++++++- src/Core/MySQL/MySQLReplication.h | 61 ++- src/Core/tests/gtest_charset_conv.cpp | 351 ++++++++++++++++++ .../materialized_with_ddl.py | 83 +++++ .../test_materialized_mysql_database/test.py | 6 + utils/check-mysql-binlog/main.cpp | 4 +- 10 files changed, 979 insertions(+), 9 deletions(-) create mode 100644 src/Core/MySQL/MySQLCharset.cpp create mode 100644 src/Core/MySQL/MySQLCharset.h create mode 100644 src/Core/tests/gtest_charset_conv.cpp diff --git a/src/Core/MySQL/IMySQLReadPacket.cpp b/src/Core/MySQL/IMySQLReadPacket.cpp index 39b2e5bbfb5..bb00444c6b3 100644 --- a/src/Core/MySQL/IMySQLReadPacket.cpp +++ b/src/Core/MySQL/IMySQLReadPacket.cpp @@ -43,11 +43,12 @@ void LimitedReadPacket::readPayloadWithUnpacked(ReadBuffer & in) IMySQLReadPacket::readPayloadWithUnpacked(limited); } -uint64_t readLengthEncodedNumber(ReadBuffer & buffer) +uint64_t readLengthEncodedNumber(ReadBuffer & buffer, UInt16 & bytes_read) { char c{}; uint64_t buf = 0; buffer.readStrict(c); + bytes_read = 1; auto cc = static_cast(c); switch (cc) { @@ -56,12 +57,15 @@ uint64_t readLengthEncodedNumber(ReadBuffer & buffer) break; case 0xfc: buffer.readStrict(reinterpret_cast(&buf), 2); + bytes_read += 2; break; case 0xfd: buffer.readStrict(reinterpret_cast(&buf), 3); + bytes_read += 3; break; case 0xfe: buffer.readStrict(reinterpret_cast(&buf), 8); + bytes_read += 8; break; default: return cc; @@ -69,6 +73,12 @@ uint64_t readLengthEncodedNumber(ReadBuffer & buffer) return buf; } +uint64_t readLengthEncodedNumber(ReadBuffer & buffer) +{ + UInt16 bytes_read = 0; + return readLengthEncodedNumber(buffer, bytes_read); +} + void readLengthEncodedString(String & s, ReadBuffer & buffer) { uint64_t len = readLengthEncodedNumber(buffer); diff --git a/src/Core/MySQL/IMySQLReadPacket.h b/src/Core/MySQL/IMySQLReadPacket.h index eab31889091..b6c3d59f5ee 100644 --- a/src/Core/MySQL/IMySQLReadPacket.h +++ b/src/Core/MySQL/IMySQLReadPacket.h @@ -34,6 +34,7 @@ public: }; uint64_t readLengthEncodedNumber(ReadBuffer & buffer); +uint64_t readLengthEncodedNumber(ReadBuffer & buffer, UInt16 & bytes_read); void readLengthEncodedString(String & s, ReadBuffer & buffer); } diff --git a/src/Core/MySQL/MySQLCharset.cpp b/src/Core/MySQL/MySQLCharset.cpp new file mode 100644 index 00000000000..869941ebd84 --- /dev/null +++ b/src/Core/MySQL/MySQLCharset.cpp @@ -0,0 +1,301 @@ +#include "MySQLCharset.h" +#include "config.h" +#include +#include + +#if USE_ICU +#include +#define CHUNK_SIZE 1024 +static const char * TARGET_CHARSET = "utf8"; +#endif + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_EXCEPTION; +} + +const std::unordered_map MySQLCharset::charsets + = { + {1, "big5"}, + {2, "latin2"}, + {3, "dec8"}, + {4, "cp850"}, + {5, "latin1"}, + {6, "hp8"}, + {7, "koi8r"}, + {8, "latin1"}, + {9, "latin2"}, + {10, "swe7"}, + {11, "ascii"}, + {12, "ujis"}, + {13, "sjis"}, + {14, "cp1251"}, + {15, "latin1"}, + {16, "hebrew"}, + {18, "tis620"}, + {19, "euckr"}, + {20, "latin7"}, + {21, "latin2"}, + {22, "koi8u"}, + {23, "cp1251"}, + {24, "gb2312"}, + {25, "greek"}, + {26, "cp1250"}, + {27, "latin2"}, + {28, "gbk"}, + {29, "cp1257"}, + {30, "latin5"}, + {31, "latin1"}, + {32, "armscii8"}, + {34, "cp1250"}, + {35, "ucs2"}, + {36, "cp866"}, + {37, "keybcs2"}, + {38, "macce"}, + {39, "macroman"}, + {40, "cp852"}, + {41, "latin7"}, + {42, "latin7"}, + {43, "macce"}, + {44, "cp1250"}, + {47, "latin1"}, + {48, "latin1"}, + {49, "latin1"}, + {50, "cp1251"}, + {51, "cp1251"}, + {52, "cp1251"}, + {53, "macroman"}, + {54, "utf16"}, + {55, "utf16"}, + {56, "utf16le"}, + {57, "cp1256"}, + {58, "cp1257"}, + {59, "cp1257"}, + {60, "utf32"}, + {61, "utf32"}, + {62, "utf16le"}, + {64, "armscii8"}, + {65, "ascii"}, + {66, "cp1250"}, + {67, "cp1256"}, + {68, "cp866"}, + {69, "dec8"}, + {70, "greek"}, + {71, "hebrew"}, + {72, "hp8"}, + {73, "keybcs2"}, + {74, "koi8r"}, + {75, "koi8u"}, + {77, "latin2"}, + {78, "latin5"}, + {79, "latin7"}, + {80, "cp850"}, + {81, "cp852"}, + {82, "swe7"}, + {84, "big5"}, + {85, "euckr"}, + {86, "gb2312"}, + {87, "gbk"}, + {88, "sjis"}, + {89, "tis620"}, + {90, "ucs2"}, + {91, "ujis"}, + {92, "geostd8"}, + {93, "geostd8"}, + {94, "latin1"}, + {95, "cp932"}, + {96, "cp932"}, + {97, "eucjpms"}, + {98, "eucjpms"}, + {99, "cp1250"}, + {101, "utf16"}, + {102, "utf16"}, + {103, "utf16"}, + {104, "utf16"}, + {105, "utf16"}, + {106, "utf16"}, + {107, "utf16"}, + {108, "utf16"}, + {109, "utf16"}, + {110, "utf16"}, + {111, "utf16"}, + {112, "utf16"}, + {113, "utf16"}, + {114, "utf16"}, + {115, "utf16"}, + {116, "utf16"}, + {117, "utf16"}, + {118, "utf16"}, + {119, "utf16"}, + {120, "utf16"}, + {121, "utf16"}, + {122, "utf16"}, + {123, "utf16"}, + {124, "utf16"}, + {128, "ucs2"}, + {129, "ucs2"}, + {130, "ucs2"}, + {131, "ucs2"}, + {132, "ucs2"}, + {133, "ucs2"}, + {134, "ucs2"}, + {135, "ucs2"}, + {136, "ucs2"}, + {137, "ucs2"}, + {138, "ucs2"}, + {139, "ucs2"}, + {140, "ucs2"}, + {141, "ucs2"}, + {142, "ucs2"}, + {143, "ucs2"}, + {144, "ucs2"}, + {145, "ucs2"}, + {146, "ucs2"}, + {147, "ucs2"}, + {148, "ucs2"}, + {149, "ucs2"}, + {150, "ucs2"}, + {151, "ucs2"}, + {159, "ucs2"}, + {160, "utf32"}, + {161, "utf32"}, + {162, "utf32"}, + {163, "utf32"}, + {164, "utf32"}, + {165, "utf32"}, + {166, "utf32"}, + {167, "utf32"}, + {168, "utf32"}, + {169, "utf32"}, + {170, "utf32"}, + {171, "utf32"}, + {172, "utf32"}, + {173, "utf32"}, + {174, "utf32"}, + {175, "utf32"}, + {176, "utf32"}, + {177, "utf32"}, + {178, "utf32"}, + {179, "utf32"}, + {180, "utf32"}, + {181, "utf32"}, + {182, "utf32"}, + {183, "utf32"}, + {248, "gb18030"}, + {249, "gb18030"}, + {250, "gb18030"} + }; + +MySQLCharset::~MySQLCharset() +{ +#if USE_ICU + std::lock_guard lock(mutex); + for (auto & conv : conv_cache) + { + ucnv_close(conv.second); + } + conv_cache.clear(); +#endif +} + +bool MySQLCharset::needConvert(UInt32 id) +{ + return charsets.contains(id); +} + +String MySQLCharset::getCharsetFromId(UInt32 id) +{ + return charsets.at(id); +} + +UConverter * MySQLCharset::getCachedConverter(const String & charset [[maybe_unused]]) +{ + UConverter * conv = nullptr; +#if USE_ICU + UErrorCode error = U_ZERO_ERROR; + /// Get conv from cache + auto result = conv_cache.find(charset); + if (result != conv_cache.end()) + { + conv = result->second; + //reset to init state + ucnv_reset(conv); + } + else + { + conv = ucnv_open(charset.c_str(), &error); + if (error != U_ZERO_ERROR) + { + throw Exception( + ErrorCodes::UNKNOWN_EXCEPTION, "MySQLCharset::getCachedConveter: ucnv_open failed, error={}", std::to_string(error)); + } + conv_cache[charset.c_str()] = conv; + } +#endif + return conv; +} + +Int32 MySQLCharset::convertFromId(UInt32 id [[maybe_unused]], String & to, const String & from) +{ +#if USE_ICU + std::lock_guard lock(mutex); + UErrorCode error = U_ZERO_ERROR; + String source_charset = getCharsetFromId(id); + to.clear(); + if (source_charset.empty()) + { + return U_ILLEGAL_ARGUMENT_ERROR; + } + + UChar pivot_buf[CHUNK_SIZE]; // stream mode must use this buf + char target_buf[CHUNK_SIZE]; + UChar * pivot; + UChar * pivot2; + UConverter * in_conv; + UConverter * out_conv; + char * cur_target; + const char * source_end; + const char * target_end; + + size_t source_len = from.size(); + const char * source = from.data(); + source_end = source + source_len; + + out_conv = getCachedConverter(TARGET_CHARSET); + in_conv = getCachedConverter(source_charset); + pivot = pivot_buf; + pivot2 = pivot_buf; + + target_end = target_buf + CHUNK_SIZE; + do + { + error = U_ZERO_ERROR; + cur_target = target_buf; + ucnv_convertEx( + out_conv, + in_conv, + &cur_target, + target_end, + &source, + source_end, + pivot_buf, + &pivot, + &pivot2, + pivot_buf + CHUNK_SIZE, + false, + true, + &error); + to.append(target_buf, cur_target - target_buf); + } while (error == U_BUFFER_OVERFLOW_ERROR); + + return error; +#else + to = from; + return 0; +#endif +} + +} diff --git a/src/Core/MySQL/MySQLCharset.h b/src/Core/MySQL/MySQLCharset.h new file mode 100644 index 00000000000..4371a2853ed --- /dev/null +++ b/src/Core/MySQL/MySQLCharset.h @@ -0,0 +1,26 @@ +#pragma once +#include +#include +#include +#include + +struct UConverter; + +namespace DB +{ +class MySQLCharset final : boost::noncopyable +{ +public: + ~MySQLCharset(); + String getCharsetFromId(UInt32 id); + Int32 convertFromId(UInt32 id, String & to, const String & from); + bool needConvert(UInt32 id); +private: + std::mutex mutex; + std::unordered_map conv_cache; + UConverter * getCachedConverter(const String & charset); + static const std::unordered_map charsets; +}; + +using MySQLCharsetPtr = std::shared_ptr; +} diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 1ee027b7185..1b0f36714f1 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -176,9 +176,9 @@ namespace MySQLReplication size_t null_bitmap_size = (column_count + 7) / 8; readBitmap(payload, null_bitmap, null_bitmap_size); - /// Ignore MySQL 8.0 optional metadata fields. + /// Parse MySQL 8.0 optional metadata fields. /// https://mysqlhighavailability.com/more-metadata-is-written-into-binary-log/ - payload.ignoreAll(); + parseOptionalMetaField(payload); } /// Types that do not used in the binlog event: @@ -252,6 +252,118 @@ namespace MySQLReplication } } + void TableMapEvent::parseOptionalMetaField(ReadBuffer & payload) + { + char type = 0; + while (payload.read(type)) + { + UInt64 len = readLengthEncodedNumber(payload); + if (len == 0) + { + payload.ignoreAll(); + return; + } + switch (type) + { + /// It may be useful, parse later + case SIGNEDNESS: + payload.ignore(len); + break; + case DEFAULT_CHARSET: + { + UInt32 total_read = 0; + UInt16 once_read = 0; + default_charset = static_cast(readLengthEncodedNumber(payload, once_read)); + total_read += once_read; + while (total_read < len) + { + UInt32 col_index = static_cast(readLengthEncodedNumber(payload, once_read)); + total_read += once_read; + UInt32 col_charset = static_cast(readLengthEncodedNumber(payload, once_read)); + total_read += once_read; + default_charset_pairs.emplace(col_index, col_charset); + } + break; + } + case COLUMN_CHARSET: + { + UInt32 total_read = 0; + UInt16 once_read = 0; + while (total_read < len) + { + UInt32 collation_id = static_cast(readLengthEncodedNumber(payload, once_read)); + column_charset.emplace_back(collation_id); + total_read += once_read; + } + break; + } + case COLUMN_NAME: + payload.ignore(len); + break; + case SET_STR_VALUE: + case GEOMETRY_TYPE: + case SIMPLE_PRIMARY_KEY: + case PRIMARY_KEY_WITH_PREFIX: + case ENUM_AND_SET_DEFAULT_CHARSET: + case COLUMN_VISIBILITY: + default: + payload.ignore(len); + break; + } + } + } + + UInt32 TableMapEvent::getColumnCharsetId(UInt32 column_index) + { + if (!column_charset.empty()) + { + UInt32 str_index = 0xFFFFFFFF; + /// Calc the index in the column_charset + for (UInt32 i = 0; i <= column_index; ++i) + { + switch (column_type[i]) + { + case MYSQL_TYPE_STRING: + case MYSQL_TYPE_VAR_STRING: + case MYSQL_TYPE_VARCHAR: + case MYSQL_TYPE_BLOB: + ++str_index; + break; + default: + break; + } + } + + if (str_index != 0xFFFFFFFF && str_index < column_charset.size()) + { + return column_charset[str_index]; + } + } + else if (!default_charset_pairs.empty()) + { + UInt32 str_index = 0xFFFFFFFF; + for (UInt32 i = 0; i <= column_index; ++i) + { + switch (column_type[i]) + { + case MYSQL_TYPE_STRING: + case MYSQL_TYPE_VAR_STRING: + case MYSQL_TYPE_VARCHAR: + case MYSQL_TYPE_BLOB: + ++str_index; + break; + default: + break; + } + } + if (default_charset_pairs.contains(str_index)) + { + return default_charset_pairs[str_index]; + } + } + return default_charset; + } + void TableMapEvent::dump(WriteBuffer & out) const { header.dump(out); @@ -308,6 +420,22 @@ namespace MySQLReplication } } + static inline String convertCharsetIfNeeded( + const std::shared_ptr & table_map, + UInt32 i, + const String & val) + { + const auto collation_id = table_map->getColumnCharsetId(i); + if (table_map->charset_ptr->needConvert(collation_id)) + { + String target; + auto err = table_map->charset_ptr->convertFromId(collation_id, target, val); + if (err == 0) + return target; + } + return val; + } + /// Types that do not used in the binlog event: /// MYSQL_TYPE_SET /// MYSQL_TYPE_TINY_BLOB @@ -716,7 +844,7 @@ namespace MySQLReplication String val; val.resize(size); payload.readStrict(reinterpret_cast(val.data()), size); - row.push_back(Field{String{val}}); + row.emplace_back(Field{convertCharsetIfNeeded(table_map, i, val)}); break; } case MYSQL_TYPE_STRING: @@ -734,7 +862,7 @@ namespace MySQLReplication String val; val.resize(size); payload.readStrict(reinterpret_cast(val.data()), size); - row.push_back(Field{String{val}}); + row.emplace_back(Field{convertCharsetIfNeeded(table_map, i, val)}); break; } case MYSQL_TYPE_GEOMETRY: @@ -766,7 +894,10 @@ namespace MySQLReplication String val; val.resize(size); payload.readStrict(reinterpret_cast(val.data()), size); - row.push_back(Field{String{val}}); + row.emplace_back(Field{ + field_type == MYSQL_TYPE_BLOB + ? convertCharsetIfNeeded(table_map, i, val) + : val}); break; } default: @@ -966,7 +1097,7 @@ namespace MySQLReplication map_event_header.parse(event_payload); if (doReplicate(map_event_header.schema, map_event_header.table)) { - event = std::make_shared(std::move(event_header), map_event_header); + event = std::make_shared(std::move(event_header), map_event_header, flavor_charset); event->parseEvent(event_payload); auto table_map = std::static_pointer_cast(event); table_maps[table_map->table_id] = table_map; diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 5825924d10b..190a2e8484d 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -436,9 +437,24 @@ namespace MySQLReplication UInt32 column_count; std::vector column_type; std::vector column_meta; + /// Character set of string columns + std::vector column_charset; + /// Character set of string columns, + /// optimized to minimize space when many + /// columns have the same charset + UInt32 default_charset = 255; /// utf8mb4_0900_ai_ci + std::unordered_map default_charset_pairs; + /// Points to flavor_charset object + MySQLCharsetPtr charset_ptr; Bitmap null_bitmap; - TableMapEvent(EventHeader && header_, const TableMapEventHeader & map_event_header) : EventBase(std::move(header_)), column_count(0) + TableMapEvent( + EventHeader && header_, + const TableMapEventHeader & map_event_header, + const MySQLCharsetPtr & charset_ptr_) + : EventBase(std::move(header_)) + , column_count(0) + , charset_ptr(charset_ptr_) { table_id = map_event_header.table_id; flags = map_event_header.flags; @@ -448,10 +464,52 @@ namespace MySQLReplication table = map_event_header.table; } void dump(WriteBuffer & out) const override; + UInt32 getColumnCharsetId(UInt32 column_index); + /// https://mysqlhighavailability.com/more-metadata-is-written-into-binary-log/ + /// https://github.com/mysql/mysql-server/blob/8.0/libbinlogevents/include/rows_event.h#L50 + /// DEFAULT_CHARSET and COLUMN_CHARSET don't appear together, and + /// ENUM_AND_SET_DEFAULT_CHARSET and ENUM_AND_SET_COLUMN_CHARSET don't appear together. + enum OptionalMetaType : char + { + /// UNSIGNED flag of numeric columns + SIGNEDNESS = 1, + /// Character set of string columns, optimized to + /// minimize space when many columns have the + /// same charset + DEFAULT_CHARSET, + /// Character set of string columns, optimized to + /// minimize space when columns have many + /// different charsets + COLUMN_CHARSET, + COLUMN_NAME, + /// String value of SET columns + SET_STR_VALUE, + /// String value of ENUM columns + ENUM_STR_VALUE, + /// Real type of geometry columns + GEOMETRY_TYPE, + /// Primary key without prefix + SIMPLE_PRIMARY_KEY, + /// Primary key with prefix + PRIMARY_KEY_WITH_PREFIX, + /// Character set of enum and set + /// columns, optimized to minimize + /// space when many columns have the + /// same charset + ENUM_AND_SET_DEFAULT_CHARSET, + /// Character set of enum and set + /// columns, optimized to minimize + /// space when many columns have the + /// same charset + ENUM_AND_SET_COLUMN_CHARSET, + /// Flag to indicate column visibility attribute + COLUMN_VISIBILITY + }; protected: void parseImpl(ReadBuffer & payload) override; void parseMeta(String meta); + void parseOptionalMetaField(ReadBuffer & payload); }; enum RowsEventFlags @@ -598,6 +656,7 @@ namespace MySQLReplication std::unordered_set replicate_tables; std::map > table_maps; size_t checksum_signature_length = 4; + MySQLCharsetPtr flavor_charset = std::make_shared(); bool doReplicate(UInt64 table_id); bool doReplicate(const String & db, const String & table_name); diff --git a/src/Core/tests/gtest_charset_conv.cpp b/src/Core/tests/gtest_charset_conv.cpp new file mode 100644 index 00000000000..073b0dd74b4 --- /dev/null +++ b/src/Core/tests/gtest_charset_conv.cpp @@ -0,0 +1,351 @@ +#include +#include +#include + +namespace DB +{ + +struct CheckResult +{ + Int32 id; + String name; + bool need_convert; +}; + +TEST(CharsetTest, CharsetTest) +{ + MySQLCharset charset; + UInt32 big5_id = 1; + UInt32 gbk_id = 28; + UInt32 gb2312_id = 24; + UInt32 utf8mb4_ai_ci_id = 255; + EXPECT_TRUE(charset.needConvert(big5_id)); + EXPECT_TRUE(charset.needConvert(gbk_id)); + EXPECT_TRUE(charset.needConvert(gb2312_id)); + EXPECT_FALSE(charset.needConvert(utf8mb4_ai_ci_id)); + EXPECT_FALSE(charset.needConvert(0)); + EXPECT_FALSE(charset.needConvert(1000)); + + EXPECT_EQ(charset.getCharsetFromId(big5_id), String("big5")); + EXPECT_EQ(charset.getCharsetFromId(gbk_id), String("gbk")); + EXPECT_EQ(charset.getCharsetFromId(gb2312_id), String("gb2312")); +} + +TEST(CharsetTest, ConvTest) +{ + MySQLCharset charset; + UInt32 big5_id = 1; + UInt32 gbk_id = 28; + UInt32 gb2312_id = 24; + Int32 error = 0; + String source("\xc4\xe3\xba\xc3"); // gbk "你好" + String target; + String expect("\xe4\xbd\xa0\xe5\xa5\xbd"); + + error = charset.convertFromId(gbk_id, target, source); + EXPECT_EQ(error, 0); + EXPECT_TRUE(target == expect); + + error = charset.convertFromId(gb2312_id, target, source); + EXPECT_EQ(error, 0); + EXPECT_TRUE(target == expect); + + source.assign("\xa7\x41\xa6\x6e"); // big5 "你好" + error = charset.convertFromId(big5_id, target, source); + EXPECT_EQ(error, 0); + EXPECT_TRUE(target == expect); +} + +TEST(CharsetTest, FullCharsetCheck) +{ + CheckResult result[] = + { + {1, "big5", true}, // "big5_chinese_ci", + {2, "latin2", true}, // "latin2_czech_cs", + {3, "dec8", true}, // "dec8_swedish_ci", + {4, "cp850", true}, // "cp850_general_ci", + {5, "latin1", true}, // "latin1_german1_ci", + {6, "hp8", true}, // "hp8_english_ci", + {7, "koi8r", true}, // "koi8r_general_ci", + {8, "latin1", true}, // "latin1_swedish_ci", + {9, "latin2", true}, // "latin2_general_ci", + {10, "swe7", true}, // "swe7_swedish_ci", + {11, "ascii", true}, // "ascii_general_ci", + {12, "ujis", true}, // "ujis_japanese_ci", + {13, "sjis", true}, // "sjis_japanese_ci", + {14, "cp1251", true}, // "cp1251_bulgarian_ci", + {15, "latin1", true}, // "latin1_danish_ci", + {16, "hebrew", true}, // "hebrew_general_ci", + {18, "tis620", true}, // "tis620_thai_ci", + {19, "euckr", true}, // "euckr_korean_ci", + {20, "latin7", true}, // "latin7_estonian_cs", + {21, "latin2", true}, // "latin2_hungarian_ci", + {22, "koi8u", true}, // "koi8u_general_ci", + {23, "cp1251", true}, // "cp1251_ukrainian_ci", + {24, "gb2312", true}, // "gb2312_chinese_ci", + {25, "greek", true}, // "greek_general_ci", + {26, "cp1250", true}, // "cp1250_general_ci", + {27, "latin2", true}, // "latin2_croatian_ci", + {28, "gbk", true}, // "gbk_chinese_ci", + {29, "cp1257", true}, // "cp1257_lithuanian_ci", + {30, "latin5", true}, // "latin5_turkish_ci", + {31, "latin1", true}, // "latin1_german2_ci", + {32, "armscii8", true}, // "armscii8_general_ci", + {33, "utf8", false}, // "utf8_general_ci", + {34, "cp1250", true}, // "cp1250_czech_cs", + {35, "ucs2", true}, // "ucs2_general_ci", + {36, "cp866", true}, // "cp866_general_ci", + {37, "keybcs2", true}, // "keybcs2_general_ci", + {38, "macce", true}, // "macce_general_ci", + {39, "macroman", true}, // "macroman_general_ci", + {40, "cp852", true}, // "cp852_general_ci", + {41, "latin7", true}, // "latin7_general_ci", + {42, "latin7", true}, // "latin7_general_cs", + {43, "macce", true}, // "macce_bin", + {44, "cp1250", true}, // "cp1250_croatian_ci", + {45, "utf8mb4", false}, // "utf8mb4_general_ci", + {46, "utf8mb4", false}, // "utf8mb4_bin", + {47, "latin1", true}, // "latin1_bin", + {48, "latin1", true}, // "latin1_general_ci", + {49, "latin1", true}, // "latin1_general_cs", + {50, "cp1251", true}, // "cp1251_bin", + {51, "cp1251", true}, // "cp1251_general_ci", + {52, "cp1251", true}, // "cp1251_general_cs", + {53, "macroman", true}, // "macroman_bin", + {54, "utf16", true}, // "utf16_general_ci", + {55, "utf16", true}, // "utf16_bin", + {56, "utf16le", true}, // "utf16le_general_ci", + {57, "cp1256", true}, // "cp1256_general_ci", + {58, "cp1257", true}, // "cp1257_bin", + {59, "cp1257", true}, // "cp1257_general_ci", + {60, "utf32", true}, // "utf32_general_ci", + {61, "utf32", true}, // "utf32_bin", + {62, "utf16le", true}, // "utf16le_bin", + {64, "armscii8", true}, // "armscii8_bin", + {65, "ascii", true}, // "ascii_bin", + {66, "cp1250", true}, // "cp1250_bin", + {67, "cp1256", true}, // "cp1256_bin", + {68, "cp866", true}, // "cp866_bin", + {69, "dec8", true}, // "dec8_bin", + {70, "greek", true}, // "greek_bin", + {71, "hebrew", true}, // "hebrew_bin", + {72, "hp8", true}, // "hp8_bin", + {73, "keybcs2", true}, // "keybcs2_bin", + {74, "koi8r", true}, // "koi8r_bin", + {75, "koi8u", true}, // "koi8u_bin", + {77, "latin2", true}, // "latin2_bin", + {78, "latin5", true}, // "latin5_bin", + {79, "latin7", true}, // "latin7_bin", + {80, "cp850", true}, // "cp850_bin", + {81, "cp852", true}, // "cp852_bin", + {82, "swe7", true}, // "swe7_bin", + {83, "utf8", false}, // "utf8_bin", + {84, "big5", true}, // "big5_bin", + {85, "euckr", true}, // "euckr_bin", + {86, "gb2312", true}, // "gb2312_bin", + {87, "gbk", true}, // "gbk_bin", + {88, "sjis", true}, // "sjis_bin", + {89, "tis620", true}, // "tis620_bin", + {90, "ucs2", true}, // "ucs2_bin", + {91, "ujis", true}, // "ujis_bin", + {92, "geostd8", true}, // "geostd8_general_ci", + {93, "geostd8", true}, // "geostd8_bin", + {94, "latin1", true}, // "latin1_spanish_ci", + {95, "cp932", true}, // "cp932_japanese_ci", + {96, "cp932", true}, // "cp932_bin", + {97, "eucjpms", true}, // "eucjpms_japanese_ci", + {98, "eucjpms", true}, // "eucjpms_bin", + {99, "cp1250", true}, // "cp1250_polish_ci", + {101, "utf16", true}, // "utf16_unicode_ci", + {102, "utf16", true}, // "utf16_icelandic_ci", + {103, "utf16", true}, // "utf16_latvian_ci", + {104, "utf16", true}, // "utf16_romanian_ci", + {105, "utf16", true}, // "utf16_slovenian_ci", + {106, "utf16", true}, // "utf16_polish_ci", + {107, "utf16", true}, // "utf16_estonian_ci", + {108, "utf16", true}, // "utf16_spanish_ci", + {109, "utf16", true}, // "utf16_swedish_ci", + {110, "utf16", true}, // "utf16_turkish_ci", + {111, "utf16", true}, // "utf16_czech_ci", + {112, "utf16", true}, // "utf16_danish_ci", + {113, "utf16", true}, // "utf16_lithuanian_ci", + {114, "utf16", true}, // "utf16_slovak_ci", + {115, "utf16", true}, // "utf16_spanish2_ci", + {116, "utf16", true}, // "utf16_roman_ci", + {117, "utf16", true}, // "utf16_persian_ci", + {118, "utf16", true}, // "utf16_esperanto_ci", + {119, "utf16", true}, // "utf16_hungarian_ci", + {120, "utf16", true}, // "utf16_sinhala_ci", + {121, "utf16", true}, // "utf16_german2_ci", + {122, "utf16", true}, // "utf16_croatian_ci", + {123, "utf16", true}, // "utf16_unicode_520_ci", + {124, "utf16", true}, // "utf16_vietnamese_ci", + {128, "ucs2", true}, // "ucs2_unicode_ci", + {129, "ucs2", true}, // "ucs2_icelandic_ci", + {130, "ucs2", true}, // "ucs2_latvian_ci", + {131, "ucs2", true}, // "ucs2_romanian_ci", + {132, "ucs2", true}, // "ucs2_slovenian_ci", + {133, "ucs2", true}, // "ucs2_polish_ci", + {134, "ucs2", true}, // "ucs2_estonian_ci", + {135, "ucs2", true}, // "ucs2_spanish_ci", + {136, "ucs2", true}, // "ucs2_swedish_ci", + {137, "ucs2", true}, // "ucs2_turkish_ci", + {138, "ucs2", true}, // "ucs2_czech_ci", + {139, "ucs2", true}, // "ucs2_danish_ci", + {140, "ucs2", true}, // "ucs2_lithuanian_ci", + {141, "ucs2", true}, // "ucs2_slovak_ci", + {142, "ucs2", true}, // "ucs2_spanish2_ci", + {143, "ucs2", true}, // "ucs2_roman_ci", + {144, "ucs2", true}, // "ucs2_persian_ci", + {145, "ucs2", true}, // "ucs2_esperanto_ci", + {146, "ucs2", true}, // "ucs2_hungarian_ci", + {147, "ucs2", true}, // "ucs2_sinhala_ci", + {148, "ucs2", true}, // "ucs2_german2_ci", + {149, "ucs2", true}, // "ucs2_croatian_ci", + {150, "ucs2", true}, // "ucs2_unicode_520_ci", + {151, "ucs2", true}, // "ucs2_vietnamese_ci", + {159, "ucs2", true}, // "ucs2_general_mysql500_ci", + {160, "utf32", true}, // "utf32_unicode_ci", + {161, "utf32", true}, // "utf32_icelandic_ci", + {162, "utf32", true}, // "utf32_latvian_ci", + {163, "utf32", true}, // "utf32_romanian_ci", + {164, "utf32", true}, // "utf32_slovenian_ci", + {165, "utf32", true}, // "utf32_polish_ci", + {166, "utf32", true}, // "utf32_estonian_ci", + {167, "utf32", true}, // "utf32_spanish_ci", + {168, "utf32", true}, // "utf32_swedish_ci", + {169, "utf32", true}, // "utf32_turkish_ci", + {170, "utf32", true}, // "utf32_czech_ci", + {171, "utf32", true}, // "utf32_danish_ci", + {172, "utf32", true}, // "utf32_lithuanian_ci", + {173, "utf32", true}, // "utf32_slovak_ci", + {174, "utf32", true}, // "utf32_spanish2_ci", + {175, "utf32", true}, // "utf32_roman_ci", + {176, "utf32", true}, // "utf32_persian_ci", + {177, "utf32", true}, // "utf32_esperanto_ci", + {178, "utf32", true}, // "utf32_hungarian_ci", + {179, "utf32", true}, // "utf32_sinhala_ci", + {180, "utf32", true}, // "utf32_german2_ci", + {181, "utf32", true}, // "utf32_croatian_ci", + {182, "utf32", true}, // "utf32_unicode_520_ci", + {183, "utf32", true}, // "utf32_vietnamese_ci", + {192, "utf8", false}, // "utf8_unicode_ci", + {193, "utf8", false}, // "utf8_icelandic_ci", + {194, "utf8", false}, // "utf8_latvian_ci", + {195, "utf8", false}, // "utf8_romanian_ci", + {196, "utf8", false}, // "utf8_slovenian_ci", + {197, "utf8", false}, // "utf8_polish_ci", + {198, "utf8", false}, // "utf8_estonian_ci", + {199, "utf8", false}, // "utf8_spanish_ci", + {200, "utf8", false}, // "utf8_swedish_ci", + {201, "utf8", false}, // "utf8_turkish_ci", + {202, "utf8", false}, // "utf8_czech_ci", + {203, "utf8", false}, // "utf8_danish_ci", + {204, "utf8", false}, // "utf8_lithuanian_ci", + {205, "utf8", false}, // "utf8_slovak_ci", + {206, "utf8", false}, // "utf8_spanish2_ci", + {207, "utf8", false}, // "utf8_roman_ci", + {208, "utf8", false}, // "utf8_persian_ci", + {209, "utf8", false}, // "utf8_esperanto_ci", + {210, "utf8", false}, // "utf8_hungarian_ci", + {211, "utf8", false}, // "utf8_sinhala_ci", + {212, "utf8", false}, // "utf8_german2_ci", + {213, "utf8", false}, // "utf8_croatian_ci", + {214, "utf8", false}, // "utf8_unicode_520_ci", + {215, "utf8", false}, // "utf8_vietnamese_ci", + {223, "utf8", false}, // "utf8_general_mysql500_ci", + {224, "utf8mb4", false}, // "utf8mb4_unicode_ci", + {225, "utf8mb4", false}, // "utf8mb4_icelandic_ci", + {226, "utf8mb4", false}, // "utf8mb4_latvian_ci", + {227, "utf8mb4", false}, // "utf8mb4_romanian_ci", + {228, "utf8mb4", false}, // "utf8mb4_slovenian_ci", + {229, "utf8mb4", false}, // "utf8mb4_polish_ci", + {230, "utf8mb4", false}, // "utf8mb4_estonian_ci", + {231, "utf8mb4", false}, // "utf8mb4_spanish_ci", + {232, "utf8mb4", false}, // "utf8mb4_swedish_ci", + {233, "utf8mb4", false}, // "utf8mb4_turkish_ci", + {234, "utf8mb4", false}, // "utf8mb4_czech_ci", + {235, "utf8mb4", false}, // "utf8mb4_danish_ci", + {236, "utf8mb4", false}, // "utf8mb4_lithuanian_ci", + {237, "utf8mb4", false}, // "utf8mb4_slovak_ci", + {238, "utf8mb4", false}, // "utf8mb4_spanish2_ci", + {239, "utf8mb4", false}, // "utf8mb4_roman_ci", + {240, "utf8mb4", false}, // "utf8mb4_persian_ci", + {241, "utf8mb4", false}, // "utf8mb4_esperanto_ci", + {242, "utf8mb4", false}, // "utf8mb4_hungarian_ci", + {243, "utf8mb4", false}, // "utf8mb4_sinhala_ci", + {244, "utf8mb4", false}, // "utf8mb4_german2_ci", + {245, "utf8mb4", false}, // "utf8mb4_croatian_ci", + {246, "utf8mb4", false}, // "utf8mb4_unicode_520_ci", + {247, "utf8mb4", false}, // "utf8mb4_vietnamese_ci", + {248, "gb18030", true}, // "gb18030_chinese_ci", + {249, "gb18030", true}, // "gb18030_bin", + {250, "gb18030", true}, // "gb18030_unicode_520_ci", + {255, "utf8mb4", false}, // "utf8mb4_0900_ai_ci", + {256, "utf8mb4", false}, // "utf8mb4_de_pb_0900_ai_ci", + {257, "utf8mb4", false}, // "utf8mb4_is_0900_ai_ci", + {258, "utf8mb4", false}, // "utf8mb4_lv_0900_ai_ci", + {259, "utf8mb4", false}, // "utf8mb4_ro_0900_ai_ci", + {260, "utf8mb4", false}, // "utf8mb4_sl_0900_ai_ci", + {261, "utf8mb4", false}, // "utf8mb4_pl_0900_ai_ci", + {262, "utf8mb4", false}, // "utf8mb4_et_0900_ai_ci", + {263, "utf8mb4", false}, // "utf8mb4_es_0900_ai_ci", + {264, "utf8mb4", false}, // "utf8mb4_is_0900_ai_ci", + {265, "utf8mb4", false}, // "utf8mb4_tr_0900_ai_ci", + {266, "utf8mb4", false}, // "utf8mb4_cs_0900_ai_ci", + {267, "utf8mb4", false}, // "utf8mb4_da_0900_ai_ci", + {268, "utf8mb4", false}, // "utf8mb4_lt_0900_ai_ci", + {269, "utf8mb4", false}, // "utf8mb4_sk_0900_ai_ci", + {270, "utf8mb4", false}, // "utf8mb4_es_trad_0900_ai_ci", + {271, "utf8mb4", false}, // "utf8mb4_la_0900_ai_ci", + {272, "utf8mb4", false}, // "utf8mb4_fa_0900_ai_ci", + {273, "utf8mb4", false}, // "utf8mb4_eo_0900_ai_ci", + {274, "utf8mb4", false}, // "utf8mb4_hu_0900_ai_ci", + {275, "utf8mb4", false}, // "utf8mb4_hr_0900_ai_ci", + {276, "utf8mb4", false}, // "utf8mb4_si_0900_ai_ci", + {277, "utf8mb4", false}, // "utf8mb4_vi_0900_ai_ci", + {278, "utf8mb4", false}, // "utf8mb4_0900_as_cs", + {279, "utf8mb4", false}, // "utf8mb4_de_pb_0900_as_cs", + {280, "utf8mb4", false}, // "utf8mb4_is_0900_as_cs", + {281, "utf8mb4", false}, // "utf8mb4_lv_0900_as_cs", + {282, "utf8mb4", false}, // "utf8mb4_ro_0900_as_cs", + {283, "utf8mb4", false}, // "utf8mb4_sl_0900_as_cs", + {284, "utf8mb4", false}, // "utf8mb4_pl_0900_as_cs", + {285, "utf8mb4", false}, // "utf8mb4_et_0900_as_cs", + {286, "utf8mb4", false}, // "utf8mb4_es_0900_as_cs", + {287, "utf8mb4", false}, // "utf8mb4_sv_0900_as_cs", + {288, "utf8mb4", false}, // "utf8mb4_tr_0900_as_cs", + {289, "utf8mb4", false}, // "utf8mb4_cs_0900_as_cs", + {290, "utf8mb4", false}, // "utf8mb4_da_0900_as_cs" + {291, "utf8mb4", false}, // "utf8mb4_lt_0900_as_cs" + {292, "utf8mb4", false}, // "utf8mb4_sk_0900_as_cs" + {293, "utf8mb4", false}, // "utf8mb4_es_trad_0900_as_cs" + {294, "utf8mb4", false}, // "utf8mb4_la_0900_as_cs" + {295, "utf8mb4", false}, // "utf8mb4_fa_0900_as_cs" + {296, "utf8mb4", false}, // "utf8mb4_eo_0900_as_cs" + {297, "utf8mb4", false}, // "utf8mb4_hu_0900_as_cs" + {298, "utf8mb4", false}, // "utf8mb4_hr_0900_as_cs" + {299, "utf8mb4", false}, // "utf8mb4_si_0900_as_cs" + {300, "utf8mb4", false}, // "utf8mb4_vi_0900_as_cs" + {303, "utf8mb4", false}, // "utf8mb4_ja_0900_as_cs_ks" + {304, "utf8mb4", false}, // "utf8mb4_la_0900_as_cs" + {305, "utf8mb4", false}, // "utf8mb4_0900_as_ci" + {306, "utf8mb4", false}, // "utf8mb4_ru_0900_ai_ci" + {307, "utf8mb4", false}, // "utf8mb4_ru_0900_as_cs" + {308, "utf8mb4", false}, // "utf8mb4_zh_0900_as_cs" + {309, "utf8mb4", false} // "utf8mb4_0900_bin" + }; + + MySQLCharset charset; + + for (auto & item : result) + { + EXPECT_TRUE(charset.needConvert(item.id) == item.need_convert); + if (charset.needConvert(item.id)) + { + EXPECT_TRUE(charset.getCharsetFromId(item.id) == item.name); + } + } +} + +} diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 8cf9e67bf63..bc19101efb8 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -980,6 +980,89 @@ def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name mysql_node.query("DROP DATABASE test_database_event") +def text_blob_with_charset_test(clickhouse_node, mysql_node, service_name): + db = "text_blob_with_charset_test" + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"CREATE DATABASE {db} DEFAULT CHARACTER SET 'utf8'") + + mysql_node.query( + f"CREATE TABLE {db}.test_table_1 (a INT NOT NULL PRIMARY KEY, b text CHARACTER SET gbk, c tinytext CHARSET big5, d longtext, e varchar(256), f char(4)) ENGINE = InnoDB DEFAULT CHARSET=gbk" + ) + mysql_node.query( + f"CREATE TABLE {db}.test_table_2 (a INT NOT NULL PRIMARY KEY, b blob, c longblob) ENGINE = InnoDB DEFAULT CHARSET=gbk" + ) + mysql_node.query( + f"CREATE TABLE {db}.test_table_3 (a INT NOT NULL PRIMARY KEY, b text CHARACTER SET gbk, c tinytext CHARSET gbk, d tinytext CHARSET big5, e varchar(256), f char(4)) ENGINE = InnoDB" + ) + + mysql_node.query( + f"INSERT INTO {db}.test_table_1 VALUES (1, '你好', '世界', '哈罗', '您Hi您', '您Hi您')" + ) + mysql_node.query( + f"INSERT INTO {db}.test_table_2 VALUES (1, '你好', 0xFAAA00000000000DDCC)" + ) + mysql_node.query( + f"INSERT INTO {db}.test_table_3 VALUES (1, '你好', '世界', 'hello', '您Hi您', '您Hi您')" + ) + + clickhouse_node.query( + f"CREATE DATABASE {db} ENGINE = MaterializedMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')" + ) + assert db in clickhouse_node.query("SHOW DATABASES") + + # from full replication + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "test_table_1\ntest_table_2\ntest_table_3\n", + ) + check_query( + clickhouse_node, + f"SELECT b, c, d, e, f FROM {db}.test_table_1 WHERE a = 1 FORMAT TSV", + "你好\t世界\t哈罗\t您Hi您\t您Hi您\n", + ) + check_query( + clickhouse_node, + f"SELECT hex(b), hex(c) FROM {db}.test_table_2 WHERE a = 1 FORMAT TSV", + "E4BDA0E5A5BD\t0FAAA00000000000DDCC\n", + ) + check_query( + clickhouse_node, + f"SELECT b, c, d, e, f FROM {db}.test_table_3 WHERE a = 1 FORMAT TSV", + "你好\t世界\thello\t您Hi您\t您Hi您\n", + ) + + # from increment replication + mysql_node.query( + f"INSERT INTO {db}.test_table_1 VALUES (2, '你好', '世界', '哈罗', '您Hi您', '您Hi您')" + ) + mysql_node.query( + f"INSERT INTO {db}.test_table_2 VALUES (2, '你好', 0xFAAA00000000000DDCC)" + ) + mysql_node.query( + f"INSERT INTO {db}.test_table_3 VALUES (2, '你好', '世界', 'hello', '您Hi您', '您Hi您')" + ) + + check_query( + clickhouse_node, + f"SELECT b, c, d, e, f FROM {db}.test_table_1 WHERE a = 2 FORMAT TSV", + "你好\t世界\t哈罗\t您Hi您\t您Hi您\n", + ) + check_query( + clickhouse_node, + f"SELECT hex(b), hex(c) FROM {db}.test_table_2 WHERE a = 2 FORMAT TSV", + "E4BDA0E5A5BD\t0FAAA00000000000DDCC\n", + ) + check_query( + clickhouse_node, + f"SELECT b, c, d, e, f FROM {db}.test_table_3 WHERE a = 2 FORMAT TSV", + "你好\t世界\thello\t您Hi您\t您Hi您\n", + ) + clickhouse_node.query(f"DROP DATABASE {db}") + mysql_node.query(f"DROP DATABASE {db}") + + def select_without_columns(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS db") clickhouse_node.query("DROP DATABASE IF EXISTS db") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 21316d1a474..df670d6e84d 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -262,6 +262,12 @@ def test_materialized_database_ddl_with_empty_transaction_8_0( ) +def test_text_blob_charset(started_cluster, started_mysql_8_0, clickhouse_node): + materialized_with_ddl.text_blob_with_charset_test( + clickhouse_node, started_mysql_8_0, "mysql80" + ) + + def test_select_without_columns_5_7( started_cluster, started_mysql_5_7, clickhouse_node ): diff --git a/utils/check-mysql-binlog/main.cpp b/utils/check-mysql-binlog/main.cpp index 68558340180..d1f868eba97 100644 --- a/utils/check-mysql-binlog/main.cpp +++ b/utils/check-mysql-binlog/main.cpp @@ -11,7 +11,9 @@ #include #include #include +#include +static DB::MySQLCharsetPtr charset = std::make_shared(); static DB::MySQLReplication::BinlogEventPtr parseSingleEventBody( DB::MySQLReplication::EventHeader & header, DB::ReadBuffer & payload, std::shared_ptr & last_table_map_event, bool exist_checksum) @@ -64,7 +66,7 @@ static DB::MySQLReplication::BinlogEventPtr parseSingleEventBody( { DB::MySQLReplication::TableMapEventHeader map_event_header; map_event_header.parse(*event_payload); - event = std::make_shared(std::move(header), map_event_header); + event = std::make_shared(std::move(header), map_event_header, charset); event->parseEvent(*event_payload); last_table_map_event = std::static_pointer_cast(event); break; From 9b7ecbaa277bc29962cccd1e0faf379fbbef36af Mon Sep 17 00:00:00 2001 From: Chen768959 <67011523+Chen768959@users.noreply.github.com> Date: Tue, 18 Jul 2023 22:00:47 +0800 Subject: [PATCH 0771/2047] fix issue#50582 The buildInputOrderInfo() method in optimizeReadInOrder.cpp adds constant columns to order_key_prefix_descr. However, since header_without_constants does not contain constant columns, the constant columns in description_sorted_ also need to be removed in advance. --- .../Transforms/FinishSortingTransform.cpp | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index 05fddc35e15..dd61472bc37 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -35,9 +35,25 @@ FinishSortingTransform::FinishSortingTransform( "Can't finish sorting. SortDescription " "of already sorted stream is not prefix of SortDescription needed to sort"); + /// Remove constants from description_sorted_. + SortDescription description_sorted_without_constants; + description_sorted_without_constants.reserve(description_sorted_.size()); + size_t num_columns = header.columns(); + ColumnNumbers map(num_columns, num_columns); + for (const auto & column_description : description_sorted_) + { + auto old_pos = header.getPositionByName(column_description.column_name); + auto new_pos = map[old_pos]; + + if (new_pos < num_columns) + { + description_sorted_without_constants.push_back(column_description); + } + } + /// The target description is modified in SortingTransform constructor. /// To avoid doing the same actions with description_sorted just copy it from prefix of target description. - for (const auto & column_sort_desc : description_sorted_) + for (const auto & column_sort_desc : description_sorted_without_constants) description_with_positions.emplace_back(column_sort_desc, header_without_constants.getPositionByName(column_sort_desc.column_name)); } From 90df922f502289e17f29c6314cd0be578741335a Mon Sep 17 00:00:00 2001 From: timfursov <126660039+timfursov@users.noreply.github.com> Date: Tue, 18 Jul 2023 15:02:59 +0100 Subject: [PATCH 0772/2047] Update with.md Removed new line chars completely just in case --- docs/en/sql-reference/statements/select/with.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/select/with.md b/docs/en/sql-reference/statements/select/with.md index 662262efb2b..a59ef463419 100644 --- a/docs/en/sql-reference/statements/select/with.md +++ b/docs/en/sql-reference/statements/select/with.md @@ -7,7 +7,8 @@ sidebar_label: WITH ClickHouse supports Common Table Expressions ([CTE](https://en.wikipedia.org/wiki/Hierarchical_and_recursive_queries_in_SQL)) and substitutes the code defined in the `WITH` clause in all places of use for the rest of `SELECT` query. Named subqueries can be included to the current and child query context in places where table objects are allowed. Recursion is prevented by hiding the current level CTEs from the WITH expression. -Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case.\ +Please note that CTEs do not guarantee the same results in all places they are called because the query will be re-executed for each use case. + An example of such behavior is below ``` sql with cte_numbers as @@ -22,7 +23,8 @@ select from cte_numbers where num in (select num from cte_numbers) ``` -If CTEs were to pass exactly the results and not just a piece of code, you would always see `1000000`\ +If CTEs were to pass exactly the results and not just a piece of code, you would always see `1000000` + However, due to the fact that we are referring `cte_numbers` twice, random numbers are generated each time and, accordingly, we see different random results, `280501, 392454, 261636, 196227` and so on... ## Syntax From 5282fc2c3450535b2fe2854e4cb1176faed3cba3 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 15:04:17 +0200 Subject: [PATCH 0773/2047] Fix class name --- src/Functions/FunctionsHashing.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 20612e91a62..ef63f5827fc 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -423,7 +423,7 @@ struct SipHash128ReferenceImpl using ReturnType = UInt128; - static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc(h1, h2); } + static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc(h1, h2); } static UInt128 apply(const char * data, const size_t size) { return sipHash128Reference(data, size); } @@ -1853,10 +1853,10 @@ using FunctionSHA384 = FunctionStringHashFixedString; using FunctionSHA512 = FunctionStringHashFixedString; #endif using FunctionSipHash128 = FunctionAnyHash; -using FunctionSipHash128Keyed = FunctionAnyHash; +using FunctionSipHash128Keyed = FunctionAnyHash; using FunctionSipHash128Reference = FunctionAnyHash; using FunctionSipHash128ReferenceKeyed - = FunctionAnyHash; + = FunctionAnyHash; using FunctionCityHash64 = FunctionAnyHash; using FunctionFarmFingerprint64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; From be0096e536140f62c7ac9600fc6f637a5638ace5 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 15:05:13 +0200 Subject: [PATCH 0774/2047] Add more sipHash128 tests --- .../0_stateless/02534_keyed_siphash.reference | 13 ++++++++++++- tests/queries/0_stateless/02534_keyed_siphash.sql | 9 ++++++++- 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index b6dd2a1c9a3..de783d7dddf 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -197,8 +197,9 @@ E28DBDE7FE22E41C Check bug with hashing of const integer values 11862823756610506724 11862823756610506724 +86AE90BB6A238D3F6221457630142C9B +86AE90BB6A238D3F6221457630142C9B Check memsan bug -1CE422FEE7BD8DE20000000000000000 18096612095653370192 20AF99D3A87829E0 12489502208762728797 @@ -209,15 +210,25 @@ Check const columns 15080046610211022027 2E779C73D13981AA1AE19AFF9617EA49 2E779C73D13981AA1AE19AFF9617EA49 +2E779C73D13981AA1AE19AFF9617EA49 +2E779C73D13981AA1AE19AFF9617EA49 Check multiple keys as tuple from a table 11862823756610506724 9357996107237883963 +86AE90BB6A238D3F6221457630142C9B +F6D93D8FEA6D7DECCDD95A7A0A2AA36D Check multiple keys as separate ints from a table 11862823756610506724 9357996107237883963 +86AE90BB6A238D3F6221457630142C9B +F6D93D8FEA6D7DECCDD95A7A0A2AA36D Check constant key and data from a table 11862823756610506724 11862823756610506724 +86AE90BB6A238D3F6221457630142C9B +86AE90BB6A238D3F6221457630142C9B Check multiple keys as separate ints from a table with constant data 11862823756610506724 9357996107237883963 +86AE90BB6A238D3F6221457630142C9B +F6D93D8FEA6D7DECCDD95A7A0A2AA36D diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 5c6789bc305..14b422ac713 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -280,10 +280,11 @@ INSERT INTO tab VALUES ((2, 2), 4); -- these two statements must produce the same result SELECT sipHash64Keyed(key, val) FROM tab; SELECT sipHash64Keyed(key, 4::UInt64) FROM tab; +SELECT hex(sipHash128Keyed(key, val)) FROM tab; +SELECT hex(sipHash128Keyed(key, 4::UInt64)) FROM tab; DROP TABLE tab; SELECT 'Check memsan bug'; -SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); @@ -295,6 +296,8 @@ SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), 1, 'test'); SELECT sipHash64(tuple(*)) FROM sipHashKeyed_test; SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), tuple(*)) FROM sipHashKeyed_test; SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), a, b) FROM sipHashKeyed_test; +SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), 1, 'test')); +SELECT hex(sipHash128(tuple(*))) FROM sipHashKeyed_test; SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test; SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test; DROP TABLE sipHashKeyed_test; @@ -305,6 +308,7 @@ CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Me INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); SELECT sipHash64Keyed(key, val) FROM sipHashKeyed_keys ORDER by key; +SELECT hex(sipHash128Keyed(key, val)) FROM sipHashKeyed_keys ORDER by key; DROP TABLE sipHashKeyed_keys; SELECT 'Check multiple keys as separate ints from a table'; @@ -313,8 +317,10 @@ CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Mem INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); SELECT sipHash64Keyed((key0, key1), val) FROM sipHashKeyed_keys ORDER by key0; +SELECT hex(sipHash128Keyed((key0, key1), val)) FROM sipHashKeyed_keys ORDER by key0; SELECT 'Check constant key and data from a table'; SELECT sipHash64Keyed((2::UInt64, 2::UInt64), val) FROM sipHashKeyed_keys ORDER by val; +SELECT hex(sipHash128Keyed((2::UInt64, 2::UInt64), val)) FROM sipHashKeyed_keys ORDER by val; DROP TABLE sipHashKeyed_keys; SELECT 'Check multiple keys as separate ints from a table with constant data'; @@ -323,4 +329,5 @@ CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; INSERT INTO sipHashKeyed_keys VALUES (2, 2); INSERT INTO sipHashKeyed_keys VALUES (4, 4); SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; +SELECT hex(sipHash128Keyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0; DROP TABLE sipHashKeyed_keys; From 388d090e538dd6e5f229f2c036bece7720097f69 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 15:05:53 +0200 Subject: [PATCH 0775/2047] Add more sipHash128Reference tests --- .../02552_siphash128_reference.reference | 22 +++++++++ .../02552_siphash128_reference.sql | 47 +++++++++++++++++++ 2 files changed, 69 insertions(+) diff --git a/tests/queries/0_stateless/02552_siphash128_reference.reference b/tests/queries/0_stateless/02552_siphash128_reference.reference index 452e9910660..ece9f6a4615 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.reference +++ b/tests/queries/0_stateless/02552_siphash128_reference.reference @@ -128,3 +128,25 @@ E3040C00EB28F15366CA73CBD872E740 1 1 1 +Check bug with hashing of const integer values +E940B12600C844966162FF8FE7A16AAE +E940B12600C844966162FF8FE7A16AAE +Check memsan bug +1CE422FEE7BD8DE20000000000000000 +Check const columns +B66B53476BDBEB8549A257E3B1766C30 +B66B53476BDBEB8549A257E3B1766C30 +B66B53476BDBEB8549A257E3B1766C30 +B66B53476BDBEB8549A257E3B1766C30 +Check multiple keys as tuple from a table +E940B12600C844966162FF8FE7A16AAE +EC58946A98A0D37F4E3FAC02FBBA9480 +Check multiple keys as separate ints from a table +E940B12600C844966162FF8FE7A16AAE +EC58946A98A0D37F4E3FAC02FBBA9480 +Check constant key and data from a table +E940B12600C844966162FF8FE7A16AAE +E940B12600C844966162FF8FE7A16AAE +Check multiple keys as separate ints from a table with constant data +E940B12600C844966162FF8FE7A16AAE +EC58946A98A0D37F4E3FAC02FBBA9480 diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql index 8b27ecbe61c..f7324ed0ee4 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.sql +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -205,3 +205,50 @@ select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMEN SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; SELECT hex(sipHash128ReferenceKeyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; + +SELECT 'Check bug with hashing of const integer values'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; +INSERT INTO tab VALUES ((2, 2), 4); +-- these two statements must produce the same result +SELECT hex(sipHash128ReferenceKeyed(key, val)) FROM tab; +SELECT hex(sipHash128ReferenceKeyed(key, 4::UInt64)) FROM tab; +DROP TABLE tab; + +SELECT 'Check memsan bug'; +SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); + +SELECT 'Check const columns'; +DROP TABLE IF EXISTS sipHashKeyed_test; +CREATE TABLE sipHashKeyed_test ENGINE = Memory() AS SELECT 1 a, 'test' b; +SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), 1, 'test')); +SELECT hex(sipHash128Reference(tuple(*))) FROM sipHashKeyed_test; +SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test; +SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test; +DROP TABLE sipHashKeyed_test; + +SELECT 'Check multiple keys as tuple from a table'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); +INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); +SELECT hex(sipHash128ReferenceKeyed(key, val)) FROM sipHashKeyed_keys ORDER by key; +DROP TABLE sipHashKeyed_keys; + +SELECT 'Check multiple keys as separate ints from a table'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); +INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); +SELECT hex(sipHash128ReferenceKeyed((key0, key1), val)) FROM sipHashKeyed_keys ORDER by key0; +SELECT 'Check constant key and data from a table'; +SELECT hex(sipHash128ReferenceKeyed((2::UInt64, 2::UInt64), val)) FROM sipHashKeyed_keys ORDER by val; +DROP TABLE sipHashKeyed_keys; + +SELECT 'Check multiple keys as separate ints from a table with constant data'; +DROP TABLE IF EXISTS sipHashKeyed_keys; +CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; +INSERT INTO sipHashKeyed_keys VALUES (2, 2); +INSERT INTO sipHashKeyed_keys VALUES (4, 4); +SELECT hex(sipHash128ReferenceKeyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0; +DROP TABLE sipHashKeyed_keys; From f6bbc4bce926f378ec07655022e897182bdcd4c4 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Tue, 18 Jul 2023 15:37:10 +0200 Subject: [PATCH 0776/2047] Use getData instead of get64 --- src/Functions/FunctionsHashing.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index ef63f5827fc..4965d1f7b49 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -95,7 +95,9 @@ namespace impl { if (is_const) i = 0; - return {key0->get64(i), key1->get64(i)}; + const auto & key0data = assert_cast(*key0).getData(); + const auto & key1data = assert_cast(*key1).getData(); + return {key0data[i], key1data[i]}; } }; From 8be1454b28974f7d70f332913eca23e592675fe2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 Jul 2023 16:06:03 +0200 Subject: [PATCH 0777/2047] Fix race --- programs/server/Server.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 91c9a456156..d1c1a1d200f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -739,11 +739,12 @@ try [&]() -> std::vector { std::vector metrics; - metrics.reserve(servers_to_start_before_tables.size()); + metrics.reserve(servers_to_start_before_tables.size() + servers.size()); + + std::lock_guard lock(servers_lock); for (const auto & server : servers_to_start_before_tables) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); - std::lock_guard lock(servers_lock); for (const auto & server : servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); return metrics; From a06631f0dbaa7402966d69c8b4cac69761627b2d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 18 Jul 2023 16:26:10 +0200 Subject: [PATCH 0778/2047] Better detecting invalid figures --- src/Functions/geoToS2.cpp | 7 +++++++ src/Functions/s2RectAdd.cpp | 13 +++++++++---- src/Functions/s2RectContains.cpp | 13 +++++++++---- src/Functions/s2RectIntersection.cpp | 12 ++++++------ src/Functions/s2RectUnion.cpp | 12 ++++++------ src/Functions/s2ToGeo.cpp | 2 +- src/Interpreters/Cache/Metadata.cpp | 2 +- .../0_stateless/02816_s2_invalid_point.reference | 0 .../queries/0_stateless/02816_s2_invalid_point.sql | 1 + 9 files changed, 40 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02816_s2_invalid_point.reference create mode 100644 tests/queries/0_stateless/02816_s2_invalid_point.sql diff --git a/src/Functions/geoToS2.cpp b/src/Functions/geoToS2.cpp index c1f333f63e6..8d065b01c34 100644 --- a/src/Functions/geoToS2.cpp +++ b/src/Functions/geoToS2.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; } namespace @@ -108,6 +109,12 @@ public: /// S2 acceptes point as (latitude, longitude) S2LatLng lat_lng = S2LatLng::FromDegrees(lat, lon); + + if (!lat_lng.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "and the longitude is between -180 and 180 degrees inclusive."); + S2CellId id(lat_lng); dst_data[row] = id.id(); diff --git a/src/Functions/s2RectAdd.cpp b/src/Functions/s2RectAdd.cpp index e086fdd6b3a..9266f4ae1a7 100644 --- a/src/Functions/s2RectAdd.cpp +++ b/src/Functions/s2RectAdd.cpp @@ -114,13 +114,18 @@ public: const auto hi = S2CellId(data_hi[row]); const auto point = S2CellId(data_point[row]); - if (!lo.is_valid() || !hi.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Rectangle is not valid"); + S2LatLngRect rect(lo.ToLatLng(), hi.ToLatLng()); if (!point.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Point is not valid"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "and the longitude is between -180 and 180 degrees inclusive."); - S2LatLngRect rect(lo.ToLatLng(), hi.ToLatLng()); + if (!rect.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." + "Also, if either the latitude or longitude bound is empty then both must be."); rect.AddPoint(point.ToPoint()); diff --git a/src/Functions/s2RectContains.cpp b/src/Functions/s2RectContains.cpp index e4d74ee2545..aed123ce8ee 100644 --- a/src/Functions/s2RectContains.cpp +++ b/src/Functions/s2RectContains.cpp @@ -107,13 +107,18 @@ public: const auto hi = S2CellId(data_hi[row]); const auto point = S2CellId(data_point[row]); - if (!lo.is_valid() || !hi.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Rectangle is not valid"); + S2LatLngRect rect(lo.ToLatLng(), hi.ToLatLng()); if (!point.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Point is not valid"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "and the longitude is between -180 and 180 degrees inclusive."); - S2LatLngRect rect(lo.ToLatLng(), hi.ToLatLng()); + if (!rect.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." + "Also, if either the latitude or longitude bound is empty then both must be."); dst_data.emplace_back(rect.Contains(point.ToLatLng())); } diff --git a/src/Functions/s2RectIntersection.cpp b/src/Functions/s2RectIntersection.cpp index 072c7147809..ffe26d171d0 100644 --- a/src/Functions/s2RectIntersection.cpp +++ b/src/Functions/s2RectIntersection.cpp @@ -128,15 +128,15 @@ public: const auto lo2 = S2CellId(data_lo2[row]); const auto hi2 = S2CellId(data_hi2[row]); - if (!lo1.is_valid() || !hi1.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "First rectangle is not valid"); - - if (!lo2.is_valid() || !hi2.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second rectangle is not valid"); - S2LatLngRect rect1(lo1.ToLatLng(), hi1.ToLatLng()); S2LatLngRect rect2(lo2.ToLatLng(), hi2.ToLatLng()); + if (!rect1.is_valid() || !rect2.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." + "Also, if either the latitude or longitude bound is empty then both must be."); + S2LatLngRect rect_intersection = rect1.Intersection(rect2); vec_res_first.emplace_back(S2CellId(rect_intersection.lo()).id()); diff --git a/src/Functions/s2RectUnion.cpp b/src/Functions/s2RectUnion.cpp index bb63229b484..472b30c2d55 100644 --- a/src/Functions/s2RectUnion.cpp +++ b/src/Functions/s2RectUnion.cpp @@ -126,15 +126,15 @@ public: const auto lo2 = S2CellId(data_lo2[row]); const auto hi2 = S2CellId(data_hi2[row]); - if (!lo1.is_valid() || !hi1.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "First rectangle is not valid"); - - if (!lo2.is_valid() || !hi2.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second rectangle is not valid"); - S2LatLngRect rect1(lo1.ToLatLng(), hi1.ToLatLng()); S2LatLngRect rect2(lo2.ToLatLng(), hi2.ToLatLng()); + if (!rect1.is_valid() || !rect2.is_valid()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." + "Also, if either the latitude or longitude bound is empty then both must be."); + S2LatLngRect rect_union = rect1.Union(rect2); vec_res_first.emplace_back(S2CellId(rect_union.lo()).id()); diff --git a/src/Functions/s2ToGeo.cpp b/src/Functions/s2ToGeo.cpp index 5c1dbfa0382..61cc8212b39 100644 --- a/src/Functions/s2ToGeo.cpp +++ b/src/Functions/s2ToGeo.cpp @@ -97,7 +97,7 @@ public: const auto id = S2CellId(data_id[row]); if (!id.is_valid()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Point is not valid"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "CellId is invalid."); S2Point point = id.ToPoint(); S2LatLng ll(point); diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index bfaa00eac2c..6a68d0f21f7 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -337,7 +337,7 @@ public: { { std::lock_guard lock(mutex); - queue.emplace(file_segment->key(), file_segment->offset(), file_segment); + queue.push(DownloadInfo{file_segment->key(), file_segment->offset(), file_segment}); } CurrentMetrics::add(CurrentMetrics::FilesystemCacheDownloadQueueElements); diff --git a/tests/queries/0_stateless/02816_s2_invalid_point.reference b/tests/queries/0_stateless/02816_s2_invalid_point.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02816_s2_invalid_point.sql b/tests/queries/0_stateless/02816_s2_invalid_point.sql new file mode 100644 index 00000000000..6f7620d98da --- /dev/null +++ b/tests/queries/0_stateless/02816_s2_invalid_point.sql @@ -0,0 +1 @@ +SELECT geoToS2(toFloat64(toUInt64(-1)), toFloat64(toUInt64(-1))); -- { serverError BAD_ARGUMENTS } From f091baaa0faf7fbd63460c486a4853bb66b34b5b Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 14:33:26 +0000 Subject: [PATCH 0779/2047] Remove whitespaces --- src/Common/SystemLogBase.cpp | 4 ++-- src/Common/SystemLogBase.h | 2 +- src/Interpreters/SystemLog.h | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index e9442617acd..1d0673e30dd 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -196,8 +196,8 @@ SystemLogQueue::Index SystemLogQueue::pop(std::vector void SystemLogQueue::shutdown() -{ - is_shutdown = true; +{ + is_shutdown = true; /// Tell thread to shutdown. flush_event.notify_all(); } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 281cd06354a..9436137d4a8 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -114,7 +114,7 @@ private: Index flushed_up_to = 0; // Logged overflow message at this queue front index Index logged_queue_full_at_index = -1; - + bool is_shutdown = false; std::condition_variable flush_event; diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index fe9e4aa35d2..6f61e075b49 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -128,10 +128,10 @@ public: void notifyFlush(bool force); void stopFlushThread() override; - + protected: Poco::Logger * log; - + using ISystemLog::is_shutdown; using ISystemLog::saving_thread; From ea3d9e92f1cd08cfee01832051e304b5f6c59d72 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 18 Jul 2023 14:44:47 +0000 Subject: [PATCH 0780/2047] Add support of YAML configs for decryption --- src/Common/Config/ConfigProcessor.cpp | 4 +++- src/Common/Config/YAMLParser.cpp | 15 ++++++++++++--- .../test_config_decryption/configs/config.yaml | 11 +++++++++++ tests/integration/test_config_decryption/test.py | 13 +++++++++++-- 4 files changed, 37 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_config_decryption/configs/config.yaml diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 0d40f782be2..faeb5678953 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -28,6 +28,7 @@ #include #if USE_SSL +#include #include #include #include @@ -845,7 +846,8 @@ void ConfigProcessor::savePreprocessedConfig(LoadedConfig & loaded_config, std:: } #if USE_SSL - if (fs::path(preprocessed_path).filename() == "config.xml") + std::string preprocessed_file_name = fs::path(preprocessed_path).filename(); + if (preprocessed_file_name == "config.xml" || preprocessed_file_name == std::format("config{}.xml", PREPROCESSED_SUFFIX)) decryptEncryptedElements(loaded_config); #endif } diff --git a/src/Common/Config/YAMLParser.cpp b/src/Common/Config/YAMLParser.cpp index d9604919726..a1de14afc13 100644 --- a/src/Common/Config/YAMLParser.cpp +++ b/src/Common/Config/YAMLParser.cpp @@ -110,9 +110,18 @@ namespace } else { - Poco::AutoPtr xml_key = xml_document->createElement(key); - parent_xml_node.appendChild(xml_key); - processNode(value_node, *xml_key); + if (key == "#text" && value_node.IsScalar()) + { + std::string value = value_node.as(); + Poco::AutoPtr xml_value = xml_document->createTextNode(value); + parent_xml_node.appendChild(xml_value); + } + else + { + Poco::AutoPtr xml_key = xml_document->createElement(key); + parent_xml_node.appendChild(xml_key); + processNode(value_node, *xml_key); + } } } break; diff --git a/tests/integration/test_config_decryption/configs/config.yaml b/tests/integration/test_config_decryption/configs/config.yaml new file mode 100644 index 00000000000..ab4391be3c5 --- /dev/null +++ b/tests/integration/test_config_decryption/configs/config.yaml @@ -0,0 +1,11 @@ +encryption_codecs: + aes_128_gcm_siv: + key_hex: 00112233445566778899aabbccddeeff + aes_256_gcm_siv: + key_hex: 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff +max_table_size_to_drop: + '#text': 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + '@encryption_codec': AES_128_GCM_SIV +max_partition_size_to_drop: + '@encryption_codec': AES_256_GCM_SIV + '#text': 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 diff --git a/tests/integration/test_config_decryption/test.py b/tests/integration/test_config_decryption/test.py index a3cb1bb57f3..dd8cdc2e4e1 100644 --- a/tests/integration/test_config_decryption/test.py +++ b/tests/integration/test_config_decryption/test.py @@ -3,7 +3,8 @@ import os from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", main_configs=["configs/config.xml"]) +node1 = cluster.add_instance("node1", main_configs=["configs/config.xml"]) +node2 = cluster.add_instance("node2", main_configs=["configs/config.yaml"]) @pytest.fixture(scope="module") @@ -16,7 +17,7 @@ def started_cluster(): cluster.shutdown() -def test_successful_decryption(started_cluster): +def check_node(started_cluster, node): assert ( node.query( "select value from system.server_settings where name ='max_table_size_to_drop'" @@ -29,3 +30,11 @@ def test_successful_decryption(started_cluster): ) == "40000000000\n" ) + + +def test_successful_decryption_xml(started_cluster): + check_node(started_cluster, node1) + + +def test_successful_decryption_yaml(started_cluster): + check_node(started_cluster, node2) From 12df1b2d7f21088e2cdc16e399e97ec0c092e88f Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 18 Jul 2023 14:52:38 +0000 Subject: [PATCH 0781/2047] Fix MD docs style --- docs/en/operations/configuration-files.md | 6 +++--- docs/ru/operations/configuration-files.md | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index 8cd34cc6c30..d1d9fa542ab 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -67,9 +67,9 @@ Substitutions can also be performed from ZooKeeper. To do this, specify the attr ## Encrypting Configuration {#encryption} -You can use symmetric encryption to encrypt a configuration element, for example, a password field. To do so, first configure the [encryption codec](../sql-reference/statements/create/table.md#encryption-codecs), then add attribute `encryption_codec`` with the name of the encryption codec as value to the element to encrypt. +You can use symmetric encryption to encrypt a configuration element, for example, a password field. To do so, first configure the [encryption codec](../sql-reference/statements/create/table.md#encryption-codecs), then add attribute `encryption_codec` with the name of the encryption codec as value to the element to encrypt. -Unlike attributes `from_zk`, `from_env` and `incl`` (or element `include``), no substitution, i.e. decryption of the encrypted value, is performed in the preprocessed file. Decryption happens only at runtime in the server process. +Unlike attributes `from_zk`, `from_env` and `incl` (or element `include`), no substitution, i.e. decryption of the encrypted value, is performed in the preprocessed file. Decryption happens only at runtime in the server process. Example: @@ -82,7 +82,7 @@ Example: admin - 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 ``` diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index 91877bdef95..01a91bd41c6 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -87,9 +87,9 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ## Шифрование {#encryption} -Вы можете использовать симметричное шифрование для зашифровки элемента конфигурации, например, поля password. Чтобы это сделать, сначала настройте [кодек шифрования](../sql-reference/statements/create/table.md#encryption-codecs), затем добавьте аттибут`encryption_codec`` с именем кодека шифрования как значение к элементу, который надо зашифровать. +Вы можете использовать симметричное шифрование для зашифровки элемента конфигурации, например, поля password. Чтобы это сделать, сначала настройте [кодек шифрования](../sql-reference/statements/create/table.md#encryption-codecs), затем добавьте аттибут`encryption_codec` с именем кодека шифрования как значение к элементу, который надо зашифровать. -В отличии от аттрибутов `from_zk`, `from_env` и `incl`` (или элемента `include``), подстановка, т.е. расшифровка зашифрованного значения, не выподняется в файле предобработки. Расшифровка происходит только во время исполнения в серверном процессе. +В отличии от аттрибутов `from_zk`, `from_env` и `incl` (или элемента `include`), подстановка, т.е. расшифровка зашифрованного значения, не выподняется в файле предобработки. Расшифровка происходит только во время исполнения в серверном процессе. Пример: @@ -102,7 +102,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml admin - 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 ``` From 680903e89bb6923f5b2fd49f6111d8506f5e5c16 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 18 Jul 2023 16:54:33 +0200 Subject: [PATCH 0782/2047] Fix style check --- .../test_postgresql_replica_database_engine_2/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index a6b2b7c498c..3f2ec74180b 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -700,7 +700,9 @@ def test_too_many_parts(started_cluster): break assert num == int( instance2.query("SELECT count() FROM test_database.test_table") - ) or num - 1 == int(instance2.query("SELECT count() FROM test_database.test_table")) + ) or num - 1 == int( + instance2.query("SELECT count() FROM test_database.test_table") + ) assert instance2.contains_in_log("DB::Exception: Too many parts") print(num) From 91b1dd6f7881b2a6291f9426cdfb257191566eda Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 18 Jul 2023 11:00:03 -0400 Subject: [PATCH 0783/2047] define a function call for zero filling --- src/Compression/CompressionCodecDeflateQpl.cpp | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 41e6a365b6c..5dcd6008b51 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -398,6 +398,14 @@ UInt32 CompressionCodecDeflateQpl::doCompressData(const char * source, UInt32 so return res; } +inline void touchBufferWithZeroFilling(char * buffer, UInt32 buffer_size) +{ + for (char * p = buffer; p < buffer + buffer_size; p += ::getPageSize()/(sizeof(*p))) + { + *p = 0; + } +} + void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const { /// QPL library is using AVX-512 with some shuffle operations. @@ -407,10 +415,7 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 so #endif /// Device IOTLB miss has big perf. impact for IAA accelerators. /// To avoid page fault, we need touch buffers related to accelerator in advance. - for (char * p = dest; p < dest + uncompressed_size; p += ::getPageSize()/(sizeof(*p))) - { - *p = 0; - } + touchBufferWithZeroFilling(dest, uncompressed_size); switch (getDecompressMode()) { From 97b2893f78257b0bee60c688666c343be3de0620 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 18:03:56 +0300 Subject: [PATCH 0784/2047] Update 02816_s2_invalid_point.sql --- tests/queries/0_stateless/02816_s2_invalid_point.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02816_s2_invalid_point.sql b/tests/queries/0_stateless/02816_s2_invalid_point.sql index 6f7620d98da..590eb8b5ec2 100644 --- a/tests/queries/0_stateless/02816_s2_invalid_point.sql +++ b/tests/queries/0_stateless/02816_s2_invalid_point.sql @@ -1 +1,3 @@ +-- Tags: no-fasttest + SELECT geoToS2(toFloat64(toUInt64(-1)), toFloat64(toUInt64(-1))); -- { serverError BAD_ARGUMENTS } From 59570b7e2066442f508ad1ff37824f1925b9d3e9 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 18 Jul 2023 15:07:02 +0000 Subject: [PATCH 0785/2047] Make encryptValue and decryptValue static --- src/Common/Config/ConfigProcessor.cpp | 4 ++-- src/Common/Config/ConfigProcessor.h | 16 ++++++++-------- src/Compression/CompressionCodecEncrypted.h | 2 +- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index faeb5678953..6529e94a41d 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -184,7 +184,7 @@ static void mergeAttributes(Element & config_element, Element & with_element) #if USE_SSL -std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) const +std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) { EncryptionMethod method = getEncryptionMethod(codec_name); CompressionCodecEncrypted codec(method); @@ -198,7 +198,7 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const return hex_value; } -std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) const +std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) { EncryptionMethod method = getEncryptionMethod(codec_name); CompressionCodecEncrypted codec(method); diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 0da304bc0d2..5901101a410 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -92,14 +92,6 @@ public: const zkutil::EventPtr & zk_changed_event, bool fallback_to_preprocessed = false); -#if USE_SSL - /// Encrypt text value - std::string encryptValue(const std::string & codec_name, const std::string & value) const; - - /// Decrypt value - std::string decryptValue(const std::string & codec_name, const std::string & value) const; -#endif - /// Save preprocessed config to specified directory. /// If preprocessed_dir is empty - calculate from loaded_config.path + /preprocessed_configs/ void savePreprocessedConfig(LoadedConfig & loaded_config, std::string preprocessed_dir); @@ -114,6 +106,14 @@ public: /// Is the file named as result of config preprocessing, not as original files. static bool isPreprocessedFile(const std::string & config_path); +#if USE_SSL + /// Encrypt text value + static std::string encryptValue(const std::string & codec_name, const std::string & value); + + /// Decrypt value + static std::string decryptValue(const std::string & codec_name, const std::string & value); +#endif + static inline const auto SUBSTITUTION_ATTRS = {"incl", "from_zk", "from_env"}; private: diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h index a8faf3d0b6c..fafcf4af507 100644 --- a/src/Compression/CompressionCodecEncrypted.h +++ b/src/Compression/CompressionCodecEncrypted.h @@ -18,7 +18,7 @@ enum EncryptionMethod MAX_ENCRYPTION_METHOD }; -/// Get method for string name. Throw exception for wrong name +/// Get method for string name. Throw exception for wrong name. EncryptionMethod getEncryptionMethod(const std::string & name); /** This codec encrypts and decrypts blocks with AES-128 in From c8c6c3138488f69d59579712aef10f7d51596431 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 18 Jul 2023 15:33:50 +0000 Subject: [PATCH 0786/2047] Change Method into method in exceptions --- src/Compression/CompressionCodecEncrypted.cpp | 10 +++++----- .../test_config_decryption/test_wrong_settings.py | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 01bf6281a0e..fb870ababa3 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -38,7 +38,7 @@ EncryptionMethod getEncryptionMethod(const std::string & name) else if (name == "AES_256_GCM_SIV") return AES_256_GCM_SIV; else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", name); } namespace @@ -74,7 +74,7 @@ uint8_t getMethodCode(EncryptionMethod Method) } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", getMethodName(Method)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); } } @@ -114,7 +114,7 @@ UInt64 methodKeySize(EncryptionMethod Method) } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", getMethodName(Method)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); } } @@ -139,7 +139,7 @@ auto getMethod(EncryptionMethod Method) } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", getMethodName(Method)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); } } @@ -215,7 +215,7 @@ auto getMethod(EncryptionMethod Method) } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption Method. Got {}", getMethodName(Method)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); } } diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index c01f5050b00..a180c37870d 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -14,7 +14,7 @@ def start_clickhouse(config, err_msg): def test_wrong_method(): - start_clickhouse("configs/config_wrong_method.xml", "Wrong encryption Method") + start_clickhouse("configs/config_wrong_method.xml", "Wrong encryption method. Got WRONG") def test_invalid_chars(): From 50734017ae8c4e05cd37c525b53c46e38846c2c8 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 18 Jul 2023 15:45:43 +0000 Subject: [PATCH 0787/2047] Fix test style with black --- .../integration/test_config_decryption/test_wrong_settings.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index a180c37870d..e86f7fa9b39 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -14,7 +14,9 @@ def start_clickhouse(config, err_msg): def test_wrong_method(): - start_clickhouse("configs/config_wrong_method.xml", "Wrong encryption method. Got WRONG") + start_clickhouse( + "configs/config_wrong_method.xml", "Wrong encryption method. Got WRONG" + ) def test_invalid_chars(): From 6a104cc3f748b4366f0f19df9d47a27daf232368 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 18 Jul 2023 16:02:56 +0000 Subject: [PATCH 0788/2047] Add checked waits to improve kafka tests It might take a few minutes to receive all the messages for the last materialized view. By waiting to the number of expected results the happy path of execution takes minimal time while becoming more stable, while the erroneous path might take a bit longer. --- tests/integration/test_storage_kafka/test.py | 91 +++++++++++--------- 1 file changed, 52 insertions(+), 39 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 9a6d3e0513c..27a315b53bb 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -843,24 +843,7 @@ def test_kafka_formats(kafka_cluster): extra_settings=format_opts.get("extra_settings") or "", ) ) - - instance.wait_for_log_line( - "kafka.*Committed offset [0-9]+.*format_tests_", - repetitions=len(all_formats.keys()), - look_behind_lines=12000, - ) - - for format_name, format_opts in list(all_formats.items()): - logging.debug(("Checking {}".format(format_name))) - topic_name = f"format_tests_{format_name}" - # shift offsets by 1 if format supports empty value - offsets = ( - [1, 2, 3] if format_opts.get("supports_empty_value", False) else [0, 1, 2] - ) - result = instance.query( - "SELECT * FROM test.kafka_{format_name}_mv;".format(format_name=format_name) - ) - expected = """\ + raw_expected = """\ 0 0 AM 0.5 1 {topic_name} 0 {offset_0} 1 0 AM 0.5 1 {topic_name} 0 {offset_1} 2 0 AM 0.5 1 {topic_name} 0 {offset_1} @@ -878,7 +861,27 @@ def test_kafka_formats(kafka_cluster): 14 0 AM 0.5 1 {topic_name} 0 {offset_1} 15 0 AM 0.5 1 {topic_name} 0 {offset_1} 0 0 AM 0.5 1 {topic_name} 0 {offset_2} -""".format( +""" + + expected_rows_count = raw_expected.count("\n") + instance.query_with_retry( + f"SELECT * FROM test.kafka_data_{list(all_formats.keys())[-1]}_mv;", + retry_count=30, + sleep_time=1, + check_callback=lambda res: res.count("\n") == expected_rows_count, + ) + + for format_name, format_opts in list(all_formats.items()): + logging.debug(("Checking {}".format(format_name))) + topic_name = f"format_tests_{format_name}" + # shift offsets by 1 if format supports empty value + offsets = ( + [1, 2, 3] if format_opts.get("supports_empty_value", False) else [0, 1, 2] + ) + result = instance.query( + "SELECT * FROM test.kafka_{format_name}_mv;".format(format_name=format_name) + ) + expected = raw_expected.format( topic_name=topic_name, offset_0=offsets[0], offset_1=offsets[1], @@ -3755,19 +3758,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): ) ) - for format_name, format_opts in list(all_formats.items()): - logging.debug("Checking {format_name}") - topic_name = f"{topic_name_prefix}{format_name}" - # shift offsets by 1 if format supports empty value - offsets = ( - [1, 2, 3] if format_opts.get("supports_empty_value", False) else [0, 1, 2] - ) - result = instance.query( - "SELECT * FROM test.kafka_data_{format_name}_mv;".format( - format_name=format_name - ) - ) - expected = """\ + raw_expected = """\ 0 0 AM 0.5 1 {topic_name} 0 {offset_0} 1 0 AM 0.5 1 {topic_name} 0 {offset_1} 2 0 AM 0.5 1 {topic_name} 0 {offset_1} @@ -3785,7 +3776,29 @@ def test_kafka_formats_with_broken_message(kafka_cluster): 14 0 AM 0.5 1 {topic_name} 0 {offset_1} 15 0 AM 0.5 1 {topic_name} 0 {offset_1} 0 0 AM 0.5 1 {topic_name} 0 {offset_2} -""".format( +""" + + expected_rows_count = raw_expected.count("\n") + instance.query_with_retry( + f"SELECT * FROM test.kafka_data_{list(all_formats.keys())[-1]}_mv;", + retry_count=30, + sleep_time=1, + check_callback=lambda res: res.count("\n") == expected_rows_count, + ) + + for format_name, format_opts in list(all_formats.items()): + logging.debug(f"Checking {format_name}") + topic_name = f"{topic_name_prefix}{format_name}" + # shift offsets by 1 if format supports empty value + offsets = ( + [1, 2, 3] if format_opts.get("supports_empty_value", False) else [0, 1, 2] + ) + result = instance.query( + "SELECT * FROM test.kafka_data_{format_name}_mv;".format( + format_name=format_name + ) + ) + expected = pre_formatted_expected.format( topic_name=topic_name, offset_0=offsets[0], offset_1=offsets[1], @@ -4339,7 +4352,7 @@ def test_row_based_formats(kafka_cluster): f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.kafka; - + CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', @@ -4347,10 +4360,10 @@ def test_row_based_formats(kafka_cluster): kafka_group_name = '{format_name}', kafka_format = '{format_name}', kafka_max_rows_per_message = 5; - + CREATE MATERIALIZED VIEW test.view Engine=Log AS SELECT key, value FROM test.kafka; - + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); """ ) @@ -4459,17 +4472,17 @@ def test_block_based_formats_2(kafka_cluster): f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.kafka; - + CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = '{format_name}', kafka_group_name = '{format_name}', kafka_format = '{format_name}'; - + CREATE MATERIALIZED VIEW test.view Engine=Log AS SELECT key, value FROM test.kafka; - + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; """ ) From 342af6442822386adff831cc6565c088f6f884e4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 18 Jul 2023 18:20:03 +0200 Subject: [PATCH 0789/2047] Fixed issue when multiple requests attempt to create same container --- src/Storages/StorageAzureBlob.cpp | 55 +++++++++++++++++++++++-------- 1 file changed, 41 insertions(+), 14 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 6855585291a..7db95a25712 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -345,7 +345,17 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co "AzureBlobStorage container does not exist '{}'", configuration.container); - result->CreateIfNotExists(); + try + { + result->CreateIfNotExists(); + } catch (const Azure::Storage::StorageException & e) + { + if (!(e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict + && e.ReasonPhrase == "The specified container already exists.")) + { + throw; + } + } } } else @@ -369,20 +379,20 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co bool container_exists = containerExists(blob_service_client,configuration.container); + std::string final_url; + size_t pos = configuration.connection_url.find('?'); + if (pos != std::string::npos) + { + auto url_without_sas = configuration.connection_url.substr(0, pos); + final_url = url_without_sas + (url_without_sas.back() == '/' ? "" : "/") + configuration.container + + configuration.connection_url.substr(pos); + } + else + final_url + = configuration.connection_url + (configuration.connection_url.back() == '/' ? "" : "/") + configuration.container; + if (container_exists) { - std::string final_url; - size_t pos = configuration.connection_url.find('?'); - if (pos != std::string::npos) - { - auto url_without_sas = configuration.connection_url.substr(0, pos); - final_url = url_without_sas + (url_without_sas.back() == '/' ? "" : "/") + configuration.container - + configuration.connection_url.substr(pos); - } - else - final_url - = configuration.connection_url + (configuration.connection_url.back() == '/' ? "" : "/") + configuration.container; - if (storage_shared_key_credential) result = std::make_unique(final_url, storage_shared_key_credential); else @@ -395,7 +405,24 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co ErrorCodes::DATABASE_ACCESS_DENIED, "AzureBlobStorage container does not exist '{}'", configuration.container); - result = std::make_unique(blob_service_client->CreateBlobContainer(configuration.container).Value); + try + { + result = std::make_unique(blob_service_client->CreateBlobContainer(configuration.container).Value); + } catch (const Azure::Storage::StorageException & e) + { + if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict + && e.ReasonPhrase == "The specified container already exists.") + { + if (storage_shared_key_credential) + result = std::make_unique(final_url, storage_shared_key_credential); + else + result = std::make_unique(final_url); + } + else + { + throw; + } + } } } From 25eee81746609cead61f6cf929bc595102893915 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 18 Jul 2023 16:29:21 +0000 Subject: [PATCH 0790/2047] Fix temp data deletion on startup, add test --- src/Disks/TemporaryFileOnDisk.cpp | 4 -- src/Disks/TemporaryFileOnDisk.h | 3 +- .../test_temporary_data/__init__.py | 0 tests/integration/test_temporary_data/test.py | 57 +++++++++++++++++++ 4 files changed, 58 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_temporary_data/__init__.py create mode 100644 tests/integration/test_temporary_data/test.py diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index 6fe6fd5a1c9..8e5c8bcebbd 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -23,10 +23,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_) - : TemporaryFileOnDisk(disk_, "") -{} - TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Metric metric_scope) : TemporaryFileOnDisk(disk_) { diff --git a/src/Disks/TemporaryFileOnDisk.h b/src/Disks/TemporaryFileOnDisk.h index 4c376383087..bd82b9744ea 100644 --- a/src/Disks/TemporaryFileOnDisk.h +++ b/src/Disks/TemporaryFileOnDisk.h @@ -16,9 +16,8 @@ using DiskPtr = std::shared_ptr; class TemporaryFileOnDisk { public: - explicit TemporaryFileOnDisk(const DiskPtr & disk_); explicit TemporaryFileOnDisk(const DiskPtr & disk_, CurrentMetrics::Metric metric_scope); - explicit TemporaryFileOnDisk(const DiskPtr & disk_, const String & prefix); + explicit TemporaryFileOnDisk(const DiskPtr & disk_, const String & prefix = "tmp"); ~TemporaryFileOnDisk(); diff --git a/tests/integration/test_temporary_data/__init__.py b/tests/integration/test_temporary_data/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_temporary_data/test.py b/tests/integration/test_temporary_data/test.py new file mode 100644 index 00000000000..9228da0698f --- /dev/null +++ b/tests/integration/test_temporary_data/test.py @@ -0,0 +1,57 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name + +import pytest +import time + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_tmp_data_no_leftovers(start_cluster): + q = node.get_query_request + + settings = { + "max_bytes_before_external_group_by": "10K", + "max_bytes_before_external_sort": "10K", + "join_algorithm": "grace_hash", + "max_bytes_in_join": "10K", + "grace_hash_join_initial_buckets": "16", + } + + # Run some queries in the background to generate temporary data + q( + "SELECT ignore(*) FROM numbers(10 * 1024 * 1024) ORDER BY sipHash64(number)", + settings=settings, + ) + q("SELECT * FROM system.numbers GROUP BY ALL", settings=settings) + q( + "SELECT * FROM system.numbers as t1 JOIN system.numbers as t2 USING (number)", + settings=settings, + ) + + # Wait a bit to make sure the temporary data is written to disk + time.sleep(5) + + # Hard restart the node + node.restart_clickhouse(kill=True) + path_to_data = "/var/lib/clickhouse/" + + # Check that there are no temporary files left + result = node.exec_in_container(["ls", path_to_data + "tmp/"]) + assert result == "" From 89552bbde5f7f452bb07c6b5c9fb1da324524620 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 18 Jul 2023 18:39:20 +0200 Subject: [PATCH 0791/2047] Pull logs for first replica before optimize (#52032) Co-authored-by: Alexander Tokmakov --- .../00502_custom_partitioning_replicated_zookeeper_long.sql | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql index 0ee8ba07006..a5e33bffb0d 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql @@ -15,6 +15,7 @@ INSERT INTO not_partitioned_replica1_00502 VALUES (4), (5); SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned_replica1_00502' AND active ORDER BY name; +SYSTEM SYNC REPLICA not_partitioned_replica1_00502 PULL; SYSTEM SYNC REPLICA not_partitioned_replica2_00502; OPTIMIZE TABLE not_partitioned_replica1_00502 PARTITION tuple() FINAL; SELECT 'Parts after OPTIMIZE:'; @@ -42,6 +43,7 @@ INSERT INTO partitioned_by_week_replica1 VALUES ('2000-01-03', 4), ('2000-01-03' SELECT 'Parts before OPTIMIZE:'; -- Select parts on the first replica to avoid waiting for replication. SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week_replica1' AND active ORDER BY name; +SYSTEM SYNC REPLICA partitioned_by_week_replica1 PULL; SYSTEM SYNC REPLICA partitioned_by_week_replica2; OPTIMIZE TABLE partitioned_by_week_replica1 PARTITION '2000-01-03' FINAL; SELECT 'Parts after OPTIMIZE:'; -- After OPTIMIZE with replication_alter_partitions_sync=2 replicas must be in sync. @@ -68,6 +70,7 @@ INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-02', 1, 4), ('2 SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_tuple_replica1_00502' AND active ORDER BY name; +SYSTEM SYNC REPLICA partitioned_by_tuple_replica1_00502 PULL; SYSTEM SYNC REPLICA partitioned_by_tuple_replica2_00502; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-01', 1) FINAL; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-02', 1) FINAL; @@ -95,6 +98,7 @@ INSERT INTO partitioned_by_string_replica1 VALUES ('bbb', 4), ('aaa', 5); SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_string_replica1' AND active ORDER BY name; +SYSTEM SYNC REPLICA partitioned_by_string_replica1 PULL; SYSTEM SYNC REPLICA partitioned_by_string_replica2; OPTIMIZE TABLE partitioned_by_string_replica2 PARTITION 'aaa' FINAL; SELECT 'Parts after OPTIMIZE:'; @@ -119,6 +123,7 @@ CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMerg INSERT INTO without_fixed_size_columns_replica1 VALUES ('a'), ('aa'), ('b'), ('cc'); -- Wait for replication. +SYSTEM SYNC REPLICA without_fixed_size_columns_replica1 PULL; SYSTEM SYNC REPLICA without_fixed_size_columns_replica2; OPTIMIZE TABLE without_fixed_size_columns_replica2 PARTITION 1 FINAL; From 0502a4c73e11f4377ead87f00a60d6806e15d489 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 18 Jul 2023 19:09:01 +0200 Subject: [PATCH 0792/2047] Set startup event during shutdown --- .../ReplicatedMergeTreeRestartingThread.cpp | 6 ++++++ .../ReplicatedMergeTreeRestartingThread.h | 2 ++ src/Storages/StorageReplicatedMergeTree.cpp | 16 +++++++++++++++- 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index e43cc879e93..79054ef46da 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -339,9 +339,15 @@ void ReplicatedMergeTreeRestartingThread::shutdown(bool part_of_full_shutdown) /// Stop restarting_thread before stopping other tasks - so that it won't restart them again. need_stop = true; task->deactivate(); + + /// Explicitly set the event, because the restarting thread will not set it again + if (part_of_full_shutdown) + storage.startup_event.set(); + LOG_TRACE(log, "Restarting thread finished"); setReadonly(part_of_full_shutdown); + } void ReplicatedMergeTreeRestartingThread::setReadonly(bool on_shutdown) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 01a877a07e5..02103272a1f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -25,6 +26,7 @@ public: void start(bool schedule = true) { + LOG_TRACE(log, "Starting restating thread, schedule: {}", schedule); if (schedule) task->activateAndSchedule(); else diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 861f981f760..53fac578fca 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -186,6 +186,7 @@ namespace ErrorCodes extern const int CHECKSUM_DOESNT_MATCH; extern const int NOT_INITIALIZED; extern const int TOO_LARGE_DISTRIBUTED_DEPTH; + extern const int TABLE_IS_DROPPED; } namespace ActionLocks @@ -3918,7 +3919,10 @@ void StorageReplicatedMergeTree::startBeingLeader() void StorageReplicatedMergeTree::stopBeingLeader() { if (!is_leader) + { + LOG_TRACE(log, "stopBeingLeader called but we are not a leader already"); return; + } LOG_INFO(log, "Stopped being leader"); is_leader = false; @@ -4774,6 +4778,7 @@ MergeTreeData::MutableDataPartPtr StorageReplicatedMergeTree::fetchExistsPart( void StorageReplicatedMergeTree::startup() { + LOG_TRACE(log, "Starting up table"); startOutdatedDataPartsLoadingTask(); if (attach_thread) { @@ -4795,6 +4800,8 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) since_metadata_err_incr_readonly_metric = true; CurrentMetrics::add(CurrentMetrics::ReadonlyReplica); } + + LOG_TRACE(log, "No connection to ZooKeeper or no metadata in ZooKeeper, will not startup"); return; } @@ -4829,6 +4836,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) if (from_attach_thread) { + LOG_TRACE(log, "Trying to startup table from right now"); /// Try activating replica in current thread. restarting_thread.run(); } @@ -4838,9 +4846,13 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) /// NOTE It does not mean that replication is actually started after receiving this event. /// It only means that an attempt to startup replication was made. /// Table may be still in readonly mode if this attempt failed for any reason. - startup_event.wait(); + while (!startup_event.tryWait(10 * 1000)) + LOG_TRACE(log, "Waiting for RestartingThread to startup table"); } + if (shutdown_prepared_called.load() || shutdown_called.load()) + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot startup table because it is dropped"); + /// And this is just a callback session_expired_callback_handler = EventNotifier::instance().subscribe(Coordination::Error::ZSESSIONEXPIRED, [this]() { @@ -4898,6 +4910,8 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() attach_thread->shutdown(); restarting_thread.shutdown(/* part_of_full_shutdown */true); + /// Explicetly set the event, because the restarting thread will not set it again + startup_event.set(); shutdown_deadline.emplace(std::chrono::system_clock::now() + std::chrono::milliseconds(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds())); } From b300781fd8995a4e8feac0e58a9b756c17fe93f7 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Jul 2023 17:48:39 +0000 Subject: [PATCH 0793/2047] Make better, add tests --- src/Formats/StructureToFormatSchemaUtils.cpp | 1 + src/Functions/generateRandomStructure.cpp | 507 ------------------ src/Functions/structureToFormatSchema.cpp | 20 +- .../02817_structure_to_schema.reference | 488 +++++++++++++++++ .../0_stateless/02817_structure_to_schema.sh | 89 +++ 5 files changed, 588 insertions(+), 517 deletions(-) delete mode 100644 src/Functions/generateRandomStructure.cpp create mode 100644 tests/queries/0_stateless/02817_structure_to_schema.reference create mode 100755 tests/queries/0_stateless/02817_structure_to_schema.sh diff --git a/src/Formats/StructureToFormatSchemaUtils.cpp b/src/Formats/StructureToFormatSchemaUtils.cpp index 02c9af2c65f..fd187c033ec 100644 --- a/src/Formats/StructureToFormatSchemaUtils.cpp +++ b/src/Formats/StructureToFormatSchemaUtils.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp deleted file mode 100644 index 5f2153ff89f..00000000000 --- a/src/Functions/generateRandomStructure.cpp +++ /dev/null @@ -1,507 +0,0 @@ -#include "config.h" - -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int BAD_ARGUMENTS; -} - -class FunctionGenerateRandomStructure : public IFunction -{ -private: - enum class Type - { - Int8, - UInt8, - Bool, - Int16, - UInt16, - Int32, - UInt32, - Int64, - UInt64, - Float32, - Float64, - DateTime64, - Decimal32, - Decimal64, - Date, - Date32, - DateTime, - String, - FixedString, - IPv4, - IPv6, - Int128, - UInt128, - Int256, - UInt256, - Decimal128, - Decimal256, - Enum8, - Enum16, - Nullable, - LowCardinality, - Array, - Tuple, - Map, - Nested, - }; - - static constexpr std::array simple_types - { - Type::Int8, - Type::UInt8, - Type::Bool, - Type::Int16, - Type::UInt16, - Type::Int32, - Type::UInt32, - Type::Int64, - Type::UInt64, - Type::Float32, - Type::Float64, - Type::Date, - Type::Date32, - Type::DateTime, - Type::String, - Type::FixedString, - }; - - static constexpr std::array big_integer_types - { - Type::Int128, - Type::UInt128, - Type::Int256, - Type::UInt256, - }; - - static constexpr std::array decimal_types - { - Type::DateTime64, - Type::Decimal32, - Type::Decimal64, - }; - - static constexpr std::array big_decimal_types - { - Type::Decimal128, - Type::Decimal256, - }; - - static constexpr std::array enum_types - { - Type::Enum8, - Type::Enum16, - }; - - static constexpr std::array ip_types - { - Type::IPv4, - Type::IPv6, - }; - - static constexpr std::array complex_types - { - Type::Nullable, - Type::LowCardinality, - Type::Array, - Type::Tuple, - Type::Map, - Type::Nested, - }; - - static constexpr std::array map_key_types - { - Type::Int8, - Type::UInt8, - Type::Bool, - Type::Int16, - Type::UInt16, - Type::Int32, - Type::UInt32, - Type::Int64, - Type::UInt64, - Type::Date, - Type::Date32, - Type::DateTime, - Type::String, - Type::FixedString, - }; - - static constexpr std::array map_key_string_types - { - Type::String, - Type::FixedString - }; - - static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128; - static constexpr size_t MAX_TUPLE_ELEMENTS = 16; - static constexpr size_t MAX_DATETIME64_PRECISION = 9; - static constexpr size_t MAX_DECIMAL32_PRECISION = 9; - static constexpr size_t MAX_DECIMAL64_PRECISION = 18; - static constexpr size_t MAX_DECIMAL128_PRECISION = 38; - static constexpr size_t MAX_DECIMAL256_PRECISION = 76; - static constexpr size_t MAX_DEPTH = 32; - -public: - static constexpr auto name = "generateRandomStructure"; - - static FunctionPtr create(ContextPtr /*context*/) - { - return std::make_shared(); - } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 0; } - - bool isVariadic() const override { return true; } - bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1, 2, 3, 4, 5, 6}; } - bool useDefaultImplementationForConstants() const override { return false; } - bool useDefaultImplementationForNulls() const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments.size() > 7) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, expected from 0 to 7", - getName(), arguments.size()); - - for (size_t i = 0; i != 2; ++i) - { - if (arguments.size() == i) - break; - - if (!isUnsignedInteger(arguments[i]) && !arguments[i]->onlyNull()) - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the {} argument of function {}, expected unsigned integer or Null", - i + 1, - arguments[i]->getName(), - getName()); - } - } - - for (size_t i = 2; i != 7; ++i) - { - if (arguments.size() <= i) - break; - - if (!isUInt8(arguments[i])) - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of the {} argument of function {}, expected UInt8", - i + 1, - arguments[i]->getName(), - getName()); - } - } - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override - { - size_t seed = randomSeed(); - size_t number_of_columns = 0; - - if (!arguments.empty() && !arguments[0].column->onlyNull()) - { - number_of_columns = arguments[0].column->getUInt(0); - if (number_of_columns > MAX_NUMBER_OF_COLUMNS) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Maximum allowed number of columns is {}, got {}", - MAX_NUMBER_OF_COLUMNS, - number_of_columns); - } - - if (arguments.size() > 1 && !arguments[1].column->onlyNull()) - seed = arguments[1].column->getUInt(0); - - bool allow_big_numbers = true; - if (arguments.size() > 2) - allow_big_numbers = arguments[2].column->getBool(0); - - bool allow_enums = true; - if (arguments.size() > 3) - allow_enums = arguments[3].column->getBool(0); - - bool allow_decimals = true; - if (arguments.size() > 4) - allow_decimals = arguments[4].column->getBool(0); - - bool allow_ip = true; - if (arguments.size() > 5) - allow_ip = arguments[5].column->getBool(0); - - bool only_string_map_key = false; - if (arguments.size() > 6) - only_string_map_key = arguments[6].column->getBool(0); - - pcg64 rng(seed); - if (number_of_columns == 0) - number_of_columns = generateNumberOfColumns(rng); - - auto col_res = ColumnString::create(); - String generated_structure; - for (size_t i = 0; i != number_of_columns; ++i) - { - if (i != 0) - generated_structure += ", "; - String column_name = "c" + std::to_string(i + 1); - auto type = generateRandomType(column_name, rng, allow_big_numbers, allow_enums, allow_decimals, allow_ip, only_string_map_key); - generated_structure += column_name + " " + type; - } - col_res->insert(generated_structure); - return ColumnConst::create(std::move(col_res), input_rows_count); - } - -private: - - size_t generateNumberOfColumns(pcg64 & rng) const - { - return rng() % MAX_NUMBER_OF_COLUMNS + 1; - } - - /// Helper struct to call generateRandomTypeImpl with lots of bool template arguments without writing big if/else over all bool variables. - template - struct Dispatcher - { - static auto call(const FunctionGenerateRandomStructure * f, const String & column_name, pcg64 & rng) - { - return f->generateRandomTypeImpl(column_name, rng); - } - - template - static auto call(const FunctionGenerateRandomStructure * f, const String & column_name, pcg64 & rng, bool b, Args1... ar1) - { - if (b) - return Dispatcher::call(f, column_name, rng, ar1...); - else - return Dispatcher::call(f, column_name, rng, ar1...); - } - - friend FunctionGenerateRandomStructure; - }; - - String generateRandomType(const String & column_name, pcg64 & rng, bool allow_big_numbers, bool allow_enums, bool allow_decimals, bool allow_ip, bool allow_only_string_map_keys) const - { - return Dispatcher<>::call(this, column_name, rng, allow_big_numbers, allow_enums, allow_decimals, allow_ip, allow_only_string_map_keys, true); - } - - template - String generateRandomTypeImpl(const String & column_name, pcg64 & rng, size_t depth = 0) const - { - constexpr auto all_types = getAllTypes(); - auto type = all_types[rng() % all_types.size()]; - - switch (type) - { - case Type::FixedString: - return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - case Type::DateTime64: - return "DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")"; - case Type::Decimal32: - return "Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")"; - case Type::Decimal64: - return "Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")"; - case Type::Decimal128: - return "Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")"; - case Type::Decimal256: - return "Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")"; - case Type::Enum8: - return "Enum8(" + generateEnumValues(column_name, rng) + ")"; - case Type::Enum16: - return "Enum16(" + generateEnumValues(column_name, rng) + ")"; - case Type::LowCardinality: - return "LowCardinality(" + generateLowCardinalityNestedType(rng) + ")"; - case Type::Nullable: - { - auto nested_type = generateRandomTypeImpl(column_name, rng, depth + 1); - return "Nullable(" + nested_type + ")"; - } - case Type::Array: - { - auto nested_type = generateRandomTypeImpl(column_name, rng, depth + 1); - return "Array(" + nested_type + ")"; - } - case Type::Map: - { - auto key_type = generateMapKeyType(rng); - auto value_type = generateRandomTypeImpl(column_name, rng, depth + 1); - return "Map(" + key_type + ", " + value_type + ")"; - } - case Type::Tuple: - { - size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; - bool named_tuple = rng() % 2; - String tuple_type = "Tuple("; - for (size_t i = 0; i != elements; ++i) - { - if (i != 0) - tuple_type += ", "; - - String element_name = "e" + std::to_string(i + 1); - if (named_tuple) - tuple_type += element_name + " "; - tuple_type += generateRandomTypeImpl(element_name, rng, depth + 1); - } - return tuple_type + ")"; - } - case Type::Nested: - { - size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1; - String nested_type = "Nested("; - for (size_t i = 0; i != elements; ++i) - { - if (i != 0) - nested_type += ", "; - String element_name = "e" + std::to_string(i + 1); - auto element_type = generateRandomTypeImpl(element_name, rng, depth + 1); - nested_type += element_name + " " + element_type; - } - return nested_type + ")"; - } - default: - return String(magic_enum::enum_name(type)); - } - } - - template - String generateMapKeyType(pcg64 & rng) const - { - Type type; - if constexpr (allow_only_string_map_keys) - type = map_key_string_types[rng() % map_key_string_types.size()]; - else - type = map_key_types[rng() % map_key_types.size()]; - - if (type == Type::FixedString) - return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - return String(magic_enum::enum_name(type)); - } - - String generateLowCardinalityNestedType(pcg64 & rng) const - { - /// Support only String and FixedString. - String nested_type; - if (rng() % 2) - nested_type = "String"; - else - nested_type = "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")"; - return rng() % 2 ? nested_type : "Nullable(" + nested_type + ")"; - } - - String generateEnumValues(const String & column_name, pcg64 & rng) const - { - /// Don't generate big enums, because it will lead to really big strings - /// and slowness of this function, and it can lead to `Max query size exceeded` - /// while using this function with generateRandom. - ssize_t num_values = rng() % 16 + 1; - String result; - for (ssize_t i = 0; i != num_values; ++i) - { - if (i != 0) - result += ", "; - result += "'" + column_name + "V" + std::to_string(i) + "' = " + std::to_string(i); - } - return result; - } - - template - static constexpr auto getAllTypes() - { - constexpr size_t big_integer_types_size = big_integer_types.size() * allow_big_numbers; - constexpr size_t enum_types_size = enum_types.size() * allow_enums; - constexpr size_t decimal_types_size = decimal_types.size() * allow_decimals; - constexpr size_t big_decimal_types_size = big_decimal_types.size() * allow_big_numbers * allow_decimals; - constexpr size_t ip_types_size = ip_types.size() * allow_ip; - constexpr size_t complex_types_size = complex_types.size() * allow_complex_types; - - constexpr size_t result_size = simple_types.size() + big_integer_types_size + enum_types_size + decimal_types_size - + big_decimal_types_size + ip_types_size + complex_types_size; - std::array result; - size_t index = 0; - - for (size_t i = 0; i != simple_types.size(); ++i, ++index) - result[index] = simple_types[i]; - - for (size_t i = 0; i != big_integer_types_size; ++i, ++index) - result[index] = big_integer_types[i]; - - for (size_t i = 0; i != enum_types_size; ++i, ++index) - result[index] = enum_types[i]; - - for (size_t i = 0; i != decimal_types_size; ++i, ++index) - result[index] = decimal_types[i]; - - for (size_t i = 0; i != big_decimal_types_size; ++i, ++index) - result[index] = big_decimal_types[i]; - - for (size_t i = 0; i != ip_types_size; ++i, ++index) - result[index] = ip_types[i]; - - for (size_t i = 0; i != complex_types_size; ++i, ++index) - result[index] = complex_types[i]; - - return result; - } -}; - - -REGISTER_FUNCTION(GenerateRandomStructure) -{ - factory.registerFunction( - { - R"( -Generates a random table structure. -This function takes 4 optional constant arguments: -1) the number of column in the result structure (random by default) -2) random seed (random by default) -3) flag that indicates if big number types can be used (true by default) -4) flag that indicates if enum types can be used (true by default) -5) flag that indicates if decimal types can be used (true by default) -6) flag that indicates if ip types (IPv4, IPv6) can be used (true by default) -7) flag that indicates if map keys should be only String or FixedString (false by default) -The maximum number of columns is 128. -The function returns a value of type String. -)", - Documentation::Examples{ - {"random", "SELECT generateRandomStructure()"}, - {"with specified number of arguments", "SELECT generateRandomStructure(10)"}, - {"with specified seed", "SELECT generateRandomStructure(10, 42)"}, - {"without big number types", "SELECT generateRandomStructure(10, NULL, false)"}, - {"without enum types", "SELECT generateRandomStructure(10, NULL, true, false)"}, - {"without decimal types", "SELECT generateRandomStructure(10, NULL, true, true, false)"}, - {"without ip types", "SELECT generateRandomStructure(10, NULL, true, true, true, false)"}, - {"with only string mak key types", "SELECT generateRandomStructure(10, NULL, true, true, true, true, true)"}, - }, - Documentation::Categories{"Random"} - }, - FunctionFactory::CaseSensitive); -} - -} diff --git a/src/Functions/structureToFormatSchema.cpp b/src/Functions/structureToFormatSchema.cpp index 332db44e06e..8c561595504 100644 --- a/src/Functions/structureToFormatSchema.cpp +++ b/src/Functions/structureToFormatSchema.cpp @@ -105,15 +105,15 @@ private: REGISTER_FUNCTION(StructureToCapnProtoSchema) { - factory.registerFunction>( + factory.registerFunction>(FunctionDocumentation { - R"( + .description=R"( )", - Documentation::Examples{ - {"random", "SELECT structureToCapnProtoSchema()"}, + .examples{ + {"random", "SELECT structureToCapnProtoSchema('s String, x UInt32', 'MessageName')", ""}, }, - Documentation::Categories{"Other"} + .categories{"Other"} }, FunctionFactory::CaseSensitive); } @@ -121,15 +121,15 @@ REGISTER_FUNCTION(StructureToCapnProtoSchema) REGISTER_FUNCTION(StructureToProtobufSchema) { - factory.registerFunction>( + factory.registerFunction>(FunctionDocumentation { - R"( + .description=R"( )", - Documentation::Examples{ - {"random", "SELECT structureToCapnProtoSchema()"}, + .examples{ + {"random", "SELECT structureToCapnProtoSchema()", ""}, }, - Documentation::Categories{"Other"} + .categories{"Other"} }, FunctionFactory::CaseSensitive); } diff --git a/tests/queries/0_stateless/02817_structure_to_schema.reference b/tests/queries/0_stateless/02817_structure_to_schema.reference new file mode 100644 index 00000000000..a1aed3f171f --- /dev/null +++ b/tests/queries/0_stateless/02817_structure_to_schema.reference @@ -0,0 +1,488 @@ +CapnProto +Numbers +@0xfcfddd851150d4a7; + +struct Message +{ + int8 @0 : Int8; + uint8 @1 : UInt8; + int16 @2 : Int16; + uint16 @3 : UInt16; + int32 @4 : Int32; + uint32 @5 : UInt32; + int64 @6 : Int64; + uint64 @7 : UInt64; + int128 @8 : Data; + uint128 @9 : Data; + int256 @10 : Data; + uint256 @11 : Data; + float32 @12 : Float32; + float64 @13 : Float64; + decimal32 @14 : Int32; + decimal64 @15 : Int64; + decimal128 @16 : Data; + decimal256 @17 : Data; +} +Dates +@0xf6c4d408fe41b545; + +struct Message +{ + data @0 : UInt16; + date32 @1 : Int32; + datetime @2 : UInt32; + datatime64 @3 : Int64; +} +Strings +@0xaa8865faa0622d35; + +struct Message +{ + string @0 : Data; + fixedstring @1 : Data; +} +Special +@0xc4a88da5454bdc56; + +struct Message +{ + ipv4 @0 : UInt32; + ipv6 @1 : Data; + uuid @2 : Data; +} +Nullable +@0xec817ea81ba08bcf; + +struct Message +{ + struct Nullable + { + union + { + value @0 : UInt32; + null @1 : Void; + } + } + nullable @0 : Nullable; +} +Enums +@0x9d57ed1a12d87f28; + +struct Message +{ + enum Enum8 + { + v1 @0; + v2 @1; + v3 @2; + v4 @3; + } + enum8 @0 : Enum8; + enum Enum16 + { + v5 @0; + v6 @1; + v7 @2; + v8 @3; + v9 @4; + } + enum16 @1 : Enum16; +} +Arrays +@0xc10c035fde2e533a; + +struct Message +{ + arr1 @0 : List(UInt32); + arr2 @1 : List(List(List(UInt32))); +} +Tuples +@0xfbb3d48432e54b68; + +struct Message +{ + struct Tuple1 + { + e1 @0 : UInt32; + e2 @1 : Data; + e3 @2 : UInt32; + } + tuple1 @0 : Tuple1; + struct Tuple2 + { + struct E1 + { + e1 @0 : UInt32; + struct E2 + { + e1 @0 : Data; + e2 @1 : UInt32; + } + e2 @1 : E2; + e3 @2 : Data; + } + e1 @0 : E1; + struct E2 + { + e1 @0 : Data; + e2 @1 : UInt32; + } + e2 @1 : E2; + } + tuple2 @1 : Tuple2; +} +Maps +@0xfdb6e3129839665e; + +struct Message +{ + struct Map1 + { + struct Entry + { + key @0 : Data; + value @1 : UInt32; + } + entries @0 : List(Entry); + } + map1 @0 : Map1; + struct Map2 + { + struct Entry + { + struct Value + { + struct Entry + { + struct Value + { + struct Entry + { + key @0 : Data; + value @1 : UInt32; + } + entries @0 : List(Entry); + } + key @0 : Data; + value @1 : Value; + } + entries @0 : List(Entry); + } + key @0 : Data; + value @1 : Value; + } + entries @0 : List(Entry); + } + map2 @1 : Map2; +} +Complex +@0x8bfcfe1141c8e85c; + +struct Message +{ + struct C1 + { + struct E1 + { + struct Entry + { + struct Value + { + union + { + value @0 : UInt32; + null @1 : Void; + } + } + key @0 : Data; + value @1 : List(List(Value)); + } + entries @0 : List(Entry); + } + e1 @0 : List(E1); + struct E2 + { + struct Entry + { + struct Value + { + struct E1 + { + union + { + value @0 : Data; + null @1 : Void; + } + } + e1 @0 : List(List(E1)); + struct E2 + { + e1 @0 : UInt32; + struct E2 + { + struct E1 + { + union + { + value @0 : Data; + null @1 : Void; + } + } + e1 @0 : List(List(E1)); + e2 @1 : UInt32; + } + e2 @1 : E2; + } + e2 @1 : List(E2); + } + key @0 : Data; + value @1 : Value; + } + entries @0 : List(Entry); + } + e2 @1 : List(E2); + } + c1 @0 : C1; +} +Read/write with no schema +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +Output schema +@0xa43428a60b94a646; + +struct Message +{ + number @0 : UInt64; +} +Bad output schema path +2 +2 +Protobuf +Numbers +syntax = "proto3"; + +message Message +{ + int32 int8 = 1; + uint32 uint8 = 2; + int32 int16 = 3; + uint32 uint16 = 4; + int32 int32 = 5; + uint32 uint32 = 6; + int64 int64 = 7; + uint64 uint64 = 8; + bytes int128 = 9; + bytes uint128 = 10; + bytes int256 = 11; + bytes uint256 = 12; + float float32 = 13; + double float64 = 14; + bytes decimal32 = 15; + bytes decimal64 = 16; + bytes decimal128 = 17; + bytes decimal256 = 18; +} +Dates +syntax = "proto3"; + +message Message +{ + uint32 data = 1; + int32 date32 = 2; + uint32 datetime = 3; + uint64 datatime64 = 4; +} +Strings +syntax = "proto3"; + +message Message +{ + bytes string = 1; + bytes fixedstring = 2; +} +Special +syntax = "proto3"; + +message Message +{ + uint32 ipv4 = 1; + bytes ipv6 = 2; + bytes uuid = 3; +} +Nullable +syntax = "proto3"; + +message Message +{ + uint32 nullable = 1; +} +Enums +syntax = "proto3"; + +message Message +{ + enum Enum8 + { + v1 = 0; + v2 = 1; + v3 = 2; + v4 = 3; + } + Enum8 enum8 = 1; + enum Enum16 + { + v5 = 0; + v6 = 1; + v7 = 2; + v8 = 3; + v9 = 4; + } + Enum16 enum16 = 2; +} +Arrays +syntax = "proto3"; + +message Message +{ + repeated uint32 arr1 = 1; + message Arr2 + { + message Arr2 + { + repeated uint32 arr2 = 1; + } + repeated Arr2 arr2 = 1; + } + repeated Arr2 arr2 = 2; +} +Tuples +syntax = "proto3"; + +message Message +{ + message Tuple1 + { + uint32 e1 = 1; + bytes e2 = 2; + uint32 e3 = 3; + } + Tuple1 tuple1 = 1; + message Tuple2 + { + message E1 + { + uint32 e1 = 1; + message E2 + { + bytes e1 = 1; + uint32 e2 = 2; + } + E2 e2 = 2; + bytes e3 = 3; + } + E1 e1 = 1; + message E2 + { + bytes e1 = 1; + uint32 e2 = 2; + } + E2 e2 = 2; + } + Tuple2 tuple2 = 2; +} +Maps +syntax = "proto3"; + +message Message +{ + map map1 = 1; + message Map2Value + { + message Map2ValueValue + { + map map2ValueValue = 1; + } + map map2Value = 1; + } + map map2 = 2; +} +Complex +syntax = "proto3"; + +message Message +{ + message C1 + { + message E1 + { + message E1Value + { + message E1Value + { + repeated uint32 e1Value = 1; + } + repeated E1Value e1Value = 1; + } + map e1 = 1; + } + repeated E1 e1 = 1; + message E2 + { + message E2Value + { + message E1 + { + repeated bytes e1 = 1; + } + repeated E1 e1 = 1; + message E2 + { + uint32 e1 = 1; + message E2 + { + message E1 + { + repeated bytes e1 = 1; + } + repeated E1 e1 = 1; + uint32 e2 = 2; + } + E2 e2 = 2; + } + repeated E2 e2 = 2; + } + map e2 = 1; + } + repeated E2 e2 = 2; + } + C1 c1 = 1; +} +Read/write with no schema +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +Output schema +syntax = "proto3"; + +message Message +{ + uint64 number = 1; +} +Bad output schema path +2 +2 diff --git a/tests/queries/0_stateless/02817_structure_to_schema.sh b/tests/queries/0_stateless/02817_structure_to_schema.sh new file mode 100755 index 00000000000..8aaf35acb33 --- /dev/null +++ b/tests/queries/0_stateless/02817_structure_to_schema.sh @@ -0,0 +1,89 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +DATA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME-data +SCHEMA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME-schema + +function test_structure() +{ + format=$1 + ext=$2 + structure=$3 + + $CLICKHOUSE_LOCAL -q "select structureTo${format}Schema('$structure') format TSVRaw" > $SCHEMA_FILE.$ext + cat $SCHEMA_FILE.$ext + + $CLICKHOUSE_LOCAL -q "select * from generateRandom('$structure', 42) limit 10 format $format settings format_schema='$SCHEMA_FILE:Message', format_capn_proto_enum_comparising_mode='by_names'" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', $format, '$structure') format Null settings format_schema='$SCHEMA_FILE:Message', format_capn_proto_enum_comparising_mode='by_names'" + +} + +function test_format() +{ + format=$1 + ext=$2 + + echo $format + + echo Numbers + numbers='int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, int128 Int128, uint128 UInt128, int256 Int256, uint256 UInt256, float32 Float32, float64 Float64, decimal32 Decimal32(3), decimal64 Decimal64(10), decimal128 Decimal128(20), decimal256 Decimal256(40)' + test_structure $format $ext "$numbers" + + echo Dates + dates='data Date, date32 Date32, datetime DateTime, datatime64 DateTime64(9)' + test_structure $format $ext "$dates" + + echo Strings + strings='string String, fixedstring FixedString(42)' + test_structure $format $ext "$strings" + + echo Special + special='ipv4 IPv4, ipv6 IPv6, uuid UUID' + test_structure $format $ext "$special" + + echo Nullable + nullable='nullable Nullable(UInt32)' + test_structure $format $ext "$nullable" + + echo Enums + enums="enum8 Enum8(''v1'' = -100, ''v2'' = -10, ''v3'' = 0, ''v4'' = 42), enum16 Enum16(''v5'' = -2000, ''v6'' = -1000, ''v7'' = 0, ''v8'' = 1000, ''v9'' = 2000)" + test_structure $format $ext "$enums" + + echo Arrays + arrays='arr1 Array(UInt32), arr2 Array(Array(Array(UInt32)))' + test_structure $format $ext "$arrays" + + echo Tuples + tuples='tuple1 Tuple(e1 UInt32, e2 String, e3 DateTime), tuple2 Tuple(e1 Tuple(e1 UInt32, e2 Tuple(e1 String, e2 DateTime), e3 String), e2 Tuple(e1 String, e2 UInt32))' + test_structure $format $ext "$tuples" + + echo Maps + maps='map1 Map(String, UInt32), map2 Map(String, Map(String, Map(String, UInt32)))' + test_structure $format $ext "$maps" + + echo Complex + complex='c1 Array(Tuple(e1 Map(String, Array(Array(Nullable(UInt32)))), e2 Map(String, Tuple(e1 Array(Array(Nullable(String))), e2 Nested(e1 UInt32, e2 Tuple(e1 Array(Array(Nullable(String))), e2 UInt32))))))' + test_structure $format $ext "$complex" + + echo "Read/write with no schema" + $CLICKHOUSE_LOCAL -q "select * from numbers(10) format $format" > $DATA_FILE + $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', $format, 'number UInt64')" + + echo "Output schema" + $CLICKHOUSE_LOCAL -q "select * from numbers(10) format $format settings output_format_schema='$SCHEMA_FILE.$ext'" > $DATA_FILE + cat $SCHEMA_FILE.$ext + + echo "Bad output schema path" + $CLICKHOUSE_CLIENT -q "insert into function file('$DATA_FILE', $format) select * from numbers(10) settings output_format_schema='/tmp/schema.$ext'" 2>&1 | grep "BAD_ARGUMENTS" -c + $CLICKHOUSE_CLIENT -q "insert into function file('$DATA_FILE', $format) select * from numbers(10) settings output_format_schema='../../schema.$ext'" 2>&1 | grep "BAD_ARGUMENTS" -c +} + +test_format CapnProto capnp +test_format Protobuf proto + +rm $DATA_FILE +rm $SCHEMA_FILE* + From 7c69449a0371a907404c263f90fa0414ea6dcb78 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Jul 2023 17:58:42 +0000 Subject: [PATCH 0794/2047] Clean --- src/TableFunctions/TableFunctionGenerateRandom.cpp | 3 --- src/TableFunctions/TableFunctionS3.cpp | 1 - 2 files changed, 4 deletions(-) diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 8d02b9bca51..08059796660 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -56,9 +56,6 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co args[0] = evaluateConstantExpressionAsLiteral(args[0], context); } - /// Allow constant expression for structure argument, it can be generated using generateRandomStructure function. - args[0] = evaluateConstantExpressionAsLiteral(args[0], context); - // All the arguments must be literals. for (const auto & arg : args) { diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 5b81aa72914..c8cc0cddd30 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -306,7 +306,6 @@ ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) bool TableFunctionS3::supportsReadingSubsetOfColumns() { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); } From 8ed2feeb043210070fbd1d3a89e07f7eaa4e9073 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 18 Jul 2023 14:04:31 -0400 Subject: [PATCH 0795/2047] modify docs for build clickhouse with qpl --- .../building_and_benchmarking_deflate_qpl.md | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/docs/en/development/building_and_benchmarking_deflate_qpl.md b/docs/en/development/building_and_benchmarking_deflate_qpl.md index baad1f3ea2a..4e01b41ab3c 100644 --- a/docs/en/development/building_and_benchmarking_deflate_qpl.md +++ b/docs/en/development/building_and_benchmarking_deflate_qpl.md @@ -8,16 +8,7 @@ description: How to build Clickhouse and run benchmark with DEFLATE_QPL Codec # Build Clickhouse with DEFLATE_QPL - Make sure your host machine meet the QPL required [prerequisites](https://intel.github.io/qpl/documentation/get_started_docs/installation.html#prerequisites) -- For Clickhouse version [v23.5.1.3174-stable](docs/changelogs/v23.5.1.3174-stable.md) and later, deflate_qpl is enabled by default which means no additional flags required pass to cmake build. -- For Clickhouse version between [v22.8.1.2097-lts](docs/changelogs/v22.8.1.2097-lts.md) and [v23.5.1.3174-stable](docs/changelogs/v23.5.1.3174-stable.md), you need pass the following flag to CMake when building ClickHouse: - -``` bash -cmake -DAVX512=1 -DENABLE_QPL=1 .. -``` -or -``` bash -cmake -DAVX2=1 -DENABLE_QPL=1 .. -``` +- deflate_qpl is enabled by default during cmake build. In case you accidentally change it, please double-check build flag: ENABLE_QPL=1 - For generic requirements, please refer to Clickhouse generic [build instructions](/docs/en/development/build.md) From 9d0391e6fbc4c88dd68a414f84ae285cb13f53f3 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 18:13:57 +0000 Subject: [PATCH 0796/2047] Add ifndef to fix build --- src/Interpreters/SystemLog.cpp | 1 - src/Loggers/Loggers.cpp | 2 ++ 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 3193baa551f..efeb22ba370 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -101,7 +101,6 @@ namespace namespace { -constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; constexpr size_t DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS = 1000; /// Creates a system log with MergeTree engine using parameters from config diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index a9869847f65..4cc74902ee1 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -34,7 +34,9 @@ static std::string createDirectory(const std::string & file) return path; } +#ifndef WITHOUT_TEXT_LOG constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; +#endif void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger /*_root*/, const std::string & cmd_name) { From e0f737f363e08de8c028b0d3eac1908ed3193758 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 18 Jul 2023 19:28:35 +0000 Subject: [PATCH 0797/2047] Remove timeout --- .../02497_trace_events_stress_long.sh | 35 ++++++------------- 1 file changed, 10 insertions(+), 25 deletions(-) diff --git a/tests/queries/0_stateless/02497_trace_events_stress_long.sh b/tests/queries/0_stateless/02497_trace_events_stress_long.sh index 3ec729079b8..91f6a9bb541 100755 --- a/tests/queries/0_stateless/02497_trace_events_stress_long.sh +++ b/tests/queries/0_stateless/02497_trace_events_stress_long.sh @@ -10,7 +10,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function thread1() { thread_id=$1 - while true; do + local TIMELIMIT=$((SECONDS+$2)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do query_id="02497_$CLICKHOUSE_DATABASE-$RANDOM-$thread_id" $CLICKHOUSE_CLIENT --query_id=$query_id --query " SELECT count() FROM numbers_mt(100000) SETTINGS @@ -25,7 +26,8 @@ function thread1() function thread2() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" done } @@ -35,29 +37,12 @@ export -f thread2 TIMEOUT=10 -timeout $TIMEOUT bash -c "thread1 0" >/dev/null & -timeout $TIMEOUT bash -c "thread1 1" >/dev/null & -timeout $TIMEOUT bash -c "thread1 2" >/dev/null & -timeout $TIMEOUT bash -c "thread1 3" >/dev/null & -timeout $TIMEOUT bash -c thread2 >/dev/null & +thread1 0 $TIMEOUT >/dev/null & +thread1 1 $TIMEOUT >/dev/null & +thread1 2 $TIMEOUT >/dev/null & +thread1 3 $TIMEOUT >/dev/null & +thread2 $TIMEOUT >/dev/null & wait -for _ in {1..10} -do - $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%' SYNC" >/dev/null - - # After this moment, the server can still run another query. - # For example, the 'timeout' command killed all threads of thread1, - # and the 'timeout' itself has finished, and we have successfully 'wait'-ed for it, - # but just before that, one of the threads successfully sent a query to the server, - # but the server didn't start to run this query yet, - # and even when the KILL QUERY was run, the query from the thread didn't start, - # but only started after the KILL QUERY has been already processed. - - # That's why we have to run this in a loop. - - $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' && break - - sleep 1 -done +$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' \ No newline at end of file From 9b951e965bd324635435ecb38f14971c6f1944a4 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Tue, 18 Jul 2023 19:48:43 +0000 Subject: [PATCH 0798/2047] Restore define --- src/Interpreters/SystemLog.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index efeb22ba370..3193baa551f 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -101,6 +101,7 @@ namespace namespace { +constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500; constexpr size_t DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS = 1000; /// Creates a system log with MergeTree engine using parameters from config From b2164ab97eb008e88974e0e06674eeb796b447dd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 22:13:28 +0200 Subject: [PATCH 0799/2047] Fix integration tests --- .../test_concurrent_ttl_merges/configs/users.xml | 7 +++++++ tests/integration/test_concurrent_ttl_merges/test.py | 4 ++-- .../test_replicated_mutations/configs/users.xml | 7 +++++++ tests/integration/test_replicated_mutations/test.py | 4 ++++ .../configs/users.xml | 7 +++++++ .../test_shutdown_wait_unfinished_queries/test.py | 4 ++-- tests/integration/test_storage_kafka/configs/users.xml | 1 + 7 files changed, 30 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_concurrent_ttl_merges/configs/users.xml create mode 100644 tests/integration/test_replicated_mutations/configs/users.xml create mode 100644 tests/integration/test_shutdown_wait_unfinished_queries/configs/users.xml diff --git a/tests/integration/test_concurrent_ttl_merges/configs/users.xml b/tests/integration/test_concurrent_ttl_merges/configs/users.xml new file mode 100644 index 00000000000..b0990ca3a60 --- /dev/null +++ b/tests/integration/test_concurrent_ttl_merges/configs/users.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/tests/integration/test_concurrent_ttl_merges/test.py b/tests/integration/test_concurrent_ttl_merges/test.py index 07e91dcbc9f..f6ba3834c92 100644 --- a/tests/integration/test_concurrent_ttl_merges/test.py +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -7,10 +7,10 @@ from helpers.test_tools import assert_eq_with_retry, TSV cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/fast_background_pool.xml"], with_zookeeper=True + "node1", main_configs=["configs/fast_background_pool.xml"], user_configs=["configs/users.xml"], with_zookeeper=True ) node2 = cluster.add_instance( - "node2", main_configs=["configs/fast_background_pool.xml"], with_zookeeper=True + "node2", main_configs=["configs/fast_background_pool.xml"], user_configs=["configs/users.xml"], with_zookeeper=True ) diff --git a/tests/integration/test_replicated_mutations/configs/users.xml b/tests/integration/test_replicated_mutations/configs/users.xml new file mode 100644 index 00000000000..b0990ca3a60 --- /dev/null +++ b/tests/integration/test_replicated_mutations/configs/users.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/tests/integration/test_replicated_mutations/test.py b/tests/integration/test_replicated_mutations/test.py index 7479f082b06..e20bcf367e3 100644 --- a/tests/integration/test_replicated_mutations/test.py +++ b/tests/integration/test_replicated_mutations/test.py @@ -15,6 +15,7 @@ node2 = cluster.add_instance( "node2", macros={"cluster": "test1"}, main_configs=["configs/merge_tree.xml"], + user_configs=["configs/users.xml"], with_zookeeper=True, ) @@ -22,12 +23,14 @@ node3 = cluster.add_instance( "node3", macros={"cluster": "test2"}, main_configs=["configs/merge_tree_max_parts.xml"], + user_configs=["configs/users.xml"], with_zookeeper=True, ) node4 = cluster.add_instance( "node4", macros={"cluster": "test2"}, main_configs=["configs/merge_tree_max_parts.xml"], + user_configs=["configs/users.xml"], with_zookeeper=True, ) @@ -35,6 +38,7 @@ node5 = cluster.add_instance( "node5", macros={"cluster": "test3"}, main_configs=["configs/merge_tree_max_parts.xml"], + user_configs=["configs/users.xml"], ) all_nodes = [node1, node2, node3, node4, node5] diff --git a/tests/integration/test_shutdown_wait_unfinished_queries/configs/users.xml b/tests/integration/test_shutdown_wait_unfinished_queries/configs/users.xml new file mode 100644 index 00000000000..b0990ca3a60 --- /dev/null +++ b/tests/integration/test_shutdown_wait_unfinished_queries/configs/users.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/tests/integration/test_shutdown_wait_unfinished_queries/test.py b/tests/integration/test_shutdown_wait_unfinished_queries/test.py index ae0710149de..71f8b9a759d 100644 --- a/tests/integration/test_shutdown_wait_unfinished_queries/test.py +++ b/tests/integration/test_shutdown_wait_unfinished_queries/test.py @@ -6,10 +6,10 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node_wait_queries = cluster.add_instance( - "node_wait_queries", main_configs=["configs/config_wait.xml"], stay_alive=True + "node_wait_queries", main_configs=["configs/config_wait.xml"], user_configs=["configs/users.xml"], stay_alive=True ) node_kill_queries = cluster.add_instance( - "node_kill_queries", main_configs=["configs/config_kill.xml"], stay_alive=True + "node_kill_queries", main_configs=["configs/config_kill.xml"], user_configs=["configs/users.xml"], stay_alive=True ) global result diff --git a/tests/integration/test_storage_kafka/configs/users.xml b/tests/integration/test_storage_kafka/configs/users.xml index 3168de649f8..d13651d5f92 100644 --- a/tests/integration/test_storage_kafka/configs/users.xml +++ b/tests/integration/test_storage_kafka/configs/users.xml @@ -4,6 +4,7 @@ 1 0 + 0 From 6fd27b6cd882b31f73ecd27ca7ae0bb2f0d25854 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Jul 2023 22:19:35 +0200 Subject: [PATCH 0800/2047] Fix build --- src/Storages/StorageMergeTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 085d532b09c..32e100edc4d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -45,6 +45,7 @@ #include #include + namespace DB { @@ -940,7 +941,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( SelectPartsDecision select_decision = SelectPartsDecision::CANNOT_SELECT; - auto is_background_memory_usage_ok = [](String * disable_reason) -> bool + auto is_background_memory_usage_ok = [](String & disable_reason) -> bool { if (canEnqueueBackgroundTask()) return true; From ff6e5ff1c547494ed7c6320c5d62bf789d433ae2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 18 Jul 2023 20:23:55 +0000 Subject: [PATCH 0801/2047] Automatic style fix --- tests/integration/test_concurrent_ttl_merges/test.py | 10 ++++++++-- .../test_shutdown_wait_unfinished_queries/test.py | 10 ++++++++-- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_concurrent_ttl_merges/test.py b/tests/integration/test_concurrent_ttl_merges/test.py index f6ba3834c92..96264e53522 100644 --- a/tests/integration/test_concurrent_ttl_merges/test.py +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -7,10 +7,16 @@ from helpers.test_tools import assert_eq_with_retry, TSV cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/fast_background_pool.xml"], user_configs=["configs/users.xml"], with_zookeeper=True + "node1", + main_configs=["configs/fast_background_pool.xml"], + user_configs=["configs/users.xml"], + with_zookeeper=True, ) node2 = cluster.add_instance( - "node2", main_configs=["configs/fast_background_pool.xml"], user_configs=["configs/users.xml"], with_zookeeper=True + "node2", + main_configs=["configs/fast_background_pool.xml"], + user_configs=["configs/users.xml"], + with_zookeeper=True, ) diff --git a/tests/integration/test_shutdown_wait_unfinished_queries/test.py b/tests/integration/test_shutdown_wait_unfinished_queries/test.py index 71f8b9a759d..074667fc92f 100644 --- a/tests/integration/test_shutdown_wait_unfinished_queries/test.py +++ b/tests/integration/test_shutdown_wait_unfinished_queries/test.py @@ -6,10 +6,16 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node_wait_queries = cluster.add_instance( - "node_wait_queries", main_configs=["configs/config_wait.xml"], user_configs=["configs/users.xml"], stay_alive=True + "node_wait_queries", + main_configs=["configs/config_wait.xml"], + user_configs=["configs/users.xml"], + stay_alive=True, ) node_kill_queries = cluster.add_instance( - "node_kill_queries", main_configs=["configs/config_kill.xml"], user_configs=["configs/users.xml"], stay_alive=True + "node_kill_queries", + main_configs=["configs/config_kill.xml"], + user_configs=["configs/users.xml"], + stay_alive=True, ) global result From 3715c7f461dc9a0c48ea3cfac52ef52c47a53c64 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 01:08:14 +0200 Subject: [PATCH 0802/2047] Fix error in a test --- tests/queries/0_stateless/02293_selected_rows_and_merges.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02293_selected_rows_and_merges.sh b/tests/queries/0_stateless/02293_selected_rows_and_merges.sh index 76c562c9744..2f281d27814 100755 --- a/tests/queries/0_stateless/02293_selected_rows_and_merges.sh +++ b/tests/queries/0_stateless/02293_selected_rows_and_merges.sh @@ -24,4 +24,4 @@ ${CLICKHOUSE_CLIENT} -q "system flush logs" # Here for mutation all values are 0, cause mutation is executed async. # It's pretty hard to write a test with total counter. -${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'] > 10, ProfileEvents['SelectedBytes'], ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'alter%' and current_database = currentDatabase()" +${CLICKHOUSE_CLIENT} -q "select ProfileEvents['SelectedRows'] > 10, ProfileEvents['SelectedBytes'] > 1000, ProfileEvents['MergedRows'], ProfileEvents['MergedUncompressedBytes'] from system.query_log where query_id = '$query_id' and type = 'QueryFinish' and query like 'alter%' and current_database = currentDatabase()" From c724816cb8403c07d2d4c4601e0c4c9dcfc16e5f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 01:15:16 +0200 Subject: [PATCH 0803/2047] Fix test --- .../configs/config.d/merge_tree.xml | 5 +++++ .../configs/config.d/users.xml | 5 ----- .../configs/config.xml | 22 ------------------- .../test_merge_tree_s3_failover/test.py | 1 + 4 files changed, 6 insertions(+), 27 deletions(-) create mode 100644 tests/integration/test_merge_tree_s3_failover/configs/config.d/merge_tree.xml delete mode 100644 tests/integration/test_merge_tree_s3_failover/configs/config.d/users.xml delete mode 100644 tests/integration/test_merge_tree_s3_failover/configs/config.xml diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/merge_tree.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/merge_tree.xml new file mode 100644 index 00000000000..c58c957b596 --- /dev/null +++ b/tests/integration/test_merge_tree_s3_failover/configs/config.d/merge_tree.xml @@ -0,0 +1,5 @@ + + + 1.0 + + diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.d/users.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.d/users.xml deleted file mode 100644 index 0011583a68c..00000000000 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.d/users.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - - - diff --git a/tests/integration/test_merge_tree_s3_failover/configs/config.xml b/tests/integration/test_merge_tree_s3_failover/configs/config.xml deleted file mode 100644 index 743d75d9a21..00000000000 --- a/tests/integration/test_merge_tree_s3_failover/configs/config.xml +++ /dev/null @@ -1,22 +0,0 @@ - - 9000 - 127.0.0.1 - - - - true - none - - AcceptCertificateHandler - - - - - 500 - ./clickhouse/ - users.xml - - - 1.0 - - diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 90dda631924..57ca5ed5ffd 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -67,6 +67,7 @@ def cluster(): "configs/config.d/storage_conf.xml", "configs/config.d/instant_moves.xml", "configs/config.d/part_log.xml", + "configs/config.d/merge_tree.xml" ], with_minio=True, ) From 3c8141529f0f8d4d7c48c077e91af77ee9885ad8 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 18 Jul 2023 23:25:21 +0000 Subject: [PATCH 0804/2047] Automatic style fix --- tests/integration/test_merge_tree_s3_failover/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 57ca5ed5ffd..b47d741e78e 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -67,7 +67,7 @@ def cluster(): "configs/config.d/storage_conf.xml", "configs/config.d/instant_moves.xml", "configs/config.d/part_log.xml", - "configs/config.d/merge_tree.xml" + "configs/config.d/merge_tree.xml", ], with_minio=True, ) From a19a1001f063ce6d992ffc08d6d05d3ef7342b66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 01:28:22 +0200 Subject: [PATCH 0805/2047] Fix 01111_create_drop_replicated_db_stress --- .../01111_create_drop_replicated_db_stress.sh | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh index 4d341e5b8a3..cc63af3676b 100755 --- a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh +++ b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh @@ -8,7 +8,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function create_db() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do SHARD=$(($RANDOM % 2)) REPLICA=$(($RANDOM % 2)) SUFFIX=$(($RANDOM % 16)) @@ -24,7 +25,8 @@ function create_db() function drop_db() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [[ "$database" == "$CLICKHOUSE_DATABASE" ]]; then continue; fi if [ -z "$database" ]; then continue; fi @@ -36,7 +38,8 @@ function drop_db() function sync_db() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$database" ]; then continue; fi $CLICKHOUSE_CLIENT --receive_timeout=1 -q \ @@ -47,7 +50,8 @@ function sync_db() function create_table() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do database=$($CLICKHOUSE_CLIENT -q "select name from system.databases where name like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$database" ]; then continue; fi $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \ @@ -59,7 +63,8 @@ function create_table() function alter_table() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do table=$($CLICKHOUSE_CLIENT -q "select database || '.' || name from system.tables where database like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$table" ]; then continue; fi $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \ @@ -71,7 +76,8 @@ function alter_table() function insert() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do table=$($CLICKHOUSE_CLIENT -q "select database || '.' || name from system.tables where database like '${CLICKHOUSE_DATABASE}%' order by rand() limit 1") if [ -z "$table" ]; then continue; fi $CLICKHOUSE_CLIENT -q \ @@ -81,23 +87,16 @@ function insert() -export -f create_db -export -f drop_db -export -f sync_db -export -f create_table -export -f alter_table -export -f insert - TIMEOUT=30 -timeout $TIMEOUT bash -c create_db & -timeout $TIMEOUT bash -c sync_db & -timeout $TIMEOUT bash -c create_table & -timeout $TIMEOUT bash -c alter_table & -timeout $TIMEOUT bash -c insert & +create_db $TIMEOUT & +sync_db $TIMEOUT & +create_table $TIMEOUT & +alter_table $TIMEOUT & +insert $TIMEOUT & sleep 1 # give other queries a head start -timeout $TIMEOUT bash -c drop_db & +drop_db $TIMEOUT & wait From 6d915042a23ed0fd320b98118d9527e312d49ffe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 01:44:20 +0200 Subject: [PATCH 0806/2047] Fix ugly code --- src/Processors/Formats/Impl/ArrowFieldIndexUtil.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h index 4beffbcf869..909133dfa4a 100644 --- a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h +++ b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h @@ -75,7 +75,7 @@ public: { if (!allow_missing_columns) throw Exception( - ErrorCodes::THERE_IS_NO_COLUMN, "Not found field({}) in arrow schema:{}.", named_col.name, schema.ToString()); + ErrorCodes::THERE_IS_NO_COLUMN, "Not found field ({}) in the Arrow schema: {}.", named_col.name, schema.ToString()); else continue; } @@ -168,4 +168,3 @@ private: }; } #endif - From 0789f388c3f6acbfdb42f44ee6463b3d646ddc27 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 02:45:56 +0300 Subject: [PATCH 0807/2047] Update ArrowFieldIndexUtil.h --- src/Processors/Formats/Impl/ArrowFieldIndexUtil.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h index 909133dfa4a..b7adaa35335 100644 --- a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h +++ b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h @@ -75,7 +75,7 @@ public: { if (!allow_missing_columns) throw Exception( - ErrorCodes::THERE_IS_NO_COLUMN, "Not found field ({}) in the Arrow schema: {}.", named_col.name, schema.ToString()); + ErrorCodes::THERE_IS_NO_COLUMN, "Not found field ({}) in the following Arrow schema:\n{}\n", named_col.name, schema.ToString()); else continue; } From 8e3bd222f00324667a1d41fb62f50adc91045b6c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:42:32 +0200 Subject: [PATCH 0808/2047] Add SQLTest --- .github/workflows/pull_request.yml | 36 +++++++ docker/images.json | 7 +- docker/test/sqltest/Dockerfile | 29 ++++++ docker/test/sqltest/run.sh | 41 ++++++++ docker/test/sqltest/test.py | 118 ++++++++++++++++++++++ tests/ci/ast_fuzzer_check.py | 2 +- tests/ci/sqltest.py | 157 +++++++++++++++++++++++++++++ tests/ci/tests/docker_images.json | 4 + 8 files changed, 391 insertions(+), 3 deletions(-) create mode 100644 docker/test/sqltest/Dockerfile create mode 100755 docker/test/sqltest/run.sh create mode 100644 docker/test/sqltest/test.py create mode 100644 tests/ci/sqltest.py diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index dd834959578..b3283c3afbe 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5182,3 +5182,39 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" +############################################################################################## +##################################### SQL TEST ############################################### +############################################################################################## + SQLTest: + needs: [BuilderDebRelease] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/sqltest + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME="SQLTest" + REPO_COPY=${{runner.temp}}/sqltest/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: SQLTest + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 sqltest.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" diff --git a/docker/images.json b/docker/images.json index e8fc329a640..663fa21a6c9 100644 --- a/docker/images.json +++ b/docker/images.json @@ -155,11 +155,14 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", - "dependent": [ - ] + "dependent": [] }, "docker/test/sqllogic": { "name": "clickhouse/sqllogic-test", "dependent": [] + }, + "docker/test/sqltest": { + "name": "clickhouse/sqltest", + "dependent": [] } } diff --git a/docker/test/sqltest/Dockerfile b/docker/test/sqltest/Dockerfile new file mode 100644 index 00000000000..c58907a98a8 --- /dev/null +++ b/docker/test/sqltest/Dockerfile @@ -0,0 +1,29 @@ +# docker build -t clickhouse/sqltest . +ARG FROM_TAG=latest +FROM clickhouse/test-base:$FROM_TAG + +RUN apt-get update --yes \ + && env DEBIAN_FRONTEND=noninteractive \ + apt-get install --yes --no-install-recommends \ + wget \ + git \ + python3 \ + python3-dev \ + python3-pip \ + sudo \ + && apt-get clean + +RUN pip3 install \ + pyyaml \ + clickhouse-driver + +ARG sqltest_repo="https://github.com/elliotchance/sqltest/" + +RUN git clone ${sqltest_repo} + +ENV TZ=UTC +ENV MAX_RUN_TIME=900 +RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + +COPY run.sh / +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh new file mode 100755 index 00000000000..ef9ecaec33b --- /dev/null +++ b/docker/test/sqltest/run.sh @@ -0,0 +1,41 @@ +#!/bin/bash + +set -x +set -e +set -u +set -o pipefail + +BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-16_debug_none_unsplitted_disable_False_binary"} +BINARY_URL_TO_DOWNLOAD=${BINARY_URL_TO_DOWNLOAD:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/$BINARY_TO_DOWNLOAD/clickhouse"} + +function wget_with_retry +{ + for _ in 1 2 3 4; do + if wget -nv -nd -c "$1";then + return 0 + else + sleep 0.5 + fi + done + return 1 +} + +wget_with_retry "$BINARY_URL_TO_DOWNLOAD" +chmod +x clickhouse +./clickhouse install --noninteractive +clickhouse start + +# Wait for start +for _ in {1..100} +do + clickhouse-client --query "SELECT 1" && break ||: + sleep 1 +done + +# Run the test +pushd sqltest/standards/2016/ +./test.py +popd + +zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log +zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.err.log diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py new file mode 100644 index 00000000000..7b38fdc52ad --- /dev/null +++ b/docker/test/sqltest/test.py @@ -0,0 +1,118 @@ +#!/usr/bin/env python3 + +import os +import yaml +import random +import string +from clickhouse_driver import Client + + +client = Client(host = 'localhost', port = 9000) +settings = {'default_table_engine': 'Memory', 'union_default_mode': 'DISTINCT', 'calculate_text_stack_trace': 0} + +database_name = 'sqltest_' + ''.join(random.choice(string.ascii_lowercase) for _ in range(10)) + +client.execute(f"DROP DATABASE IF EXISTS {database_name}", settings = settings) +client.execute(f"CREATE DATABASE {database_name}", settings = settings) + +client = Client(host = 'localhost', port = 9000, database = database_name) + +summary = {'success': 0, 'total': 0, 'results': {}} + +log_file = open("test.log", "w") +report_html_file = open("report.html", "w") + +with open('features.yml', 'r') as file: + yaml_content = yaml.safe_load(file) + + for category in yaml_content: + log_file.write(category.capitalize() + " features:") + summary['results'][category] = {'success': 0, 'total': 0, 'results': {}} + + for test in yaml_content[category]: + log_file.write(test + ": " + yaml_content[category][test]) + summary['results'][category]['results'][test] = {'success': 0, 'total': 0, 'description': yaml_content[category][test]} + + test_path = test[0] + "/" + test + ".tests.yml" + if os.path.exists(test_path): + with open(test_path, 'r') as test_file: + test_yaml_content = yaml.load_all(test_file, Loader = yaml.FullLoader) + + for test_case in test_yaml_content: + + queries = test_case['sql']; + if not isinstance(queries, list): + queries = [queries] + + for query in queries: + # Example: E011-01 + test_group = '' + if '-' in test: + test_group = test.split("-", 1)[0] + summary['results'][category]['results'][test_group]['total'] += 1 + summary['results'][category]['results'][test]['total'] += 1 + summary['results'][category]['total'] += 1 + summary['total'] += 1 + + log_file.write(query) + + try: + result = client.execute(query, settings = settings) + log_file.write(result) + + if test_group: + summary['results'][category]['results'][test_group]['success'] += 1 + summary['results'][category]['results'][test]['success'] += 1 + summary['results'][category]['success'] += 1 + summary['success'] += 1 + + except Exception as e: + log_file.write(f"Error occurred: {str(e)}") + +client.execute(f"DROP DATABASE {database_name}", settings = settings) + +def enable_color(ratio): + if ratio == 0: + return "" + elif ratio < 0.5: + return "" + elif ratio < 1: + return "" + else: + return "" + +reset_color = "" + +def print_ratio(indent, name, success, total, description): + report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
".format( + ' ' * indent, + name, + enable_color(success / total), + success, + total, + success / total, + reset_color, + f" - {description}" if description else '')) + + +report_html_file.write("

") + +print_ratio(0, 'Total', summary['success'], summary['total'], '') + +for category in summary['results']: + cat_summary = summary['results'][category] + + if cat_summary['total'] == 0: + continue + + print_ratio(2, category, cat_summary['success'], cat_summary['total'], '') + + for test in summary['results'][category]['results']: + test_summary = summary['results'][category]['results'][test] + + if test_summary['total'] == 0: + continue + + print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description']) + +report_html_file.write("

") diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 514aaf7e2ac..74b875c23be 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -79,7 +79,7 @@ def main(): build_url = url break else: - raise Exception("Cannot binary clickhouse among build results") + raise Exception("Cannot find the clickhouse binary among build results") logging.info("Got build url %s", build_url) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py new file mode 100644 index 00000000000..5c20cc4849b --- /dev/null +++ b/tests/ci/sqltest.py @@ -0,0 +1,157 @@ +#!/usr/bin/env python3 + +import logging +import subprocess +import os +import sys + +from github import Github + +from build_download_helper import get_build_name_for_check, read_build_urls +from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse +from commit_status_helper import ( + RerunHelper, + format_description, + get_commit, + post_commit_status, +) +from docker_pull_helper import get_image_with_version +from env_helper import ( + GITHUB_RUN_URL, + REPORTS_PATH, + TEMP_PATH, +) +from get_robot_token import get_best_robot_token +from pr_info import PRInfo +from report import TestResult +from s3_helper import S3Helper +from stopwatch import Stopwatch + +IMAGE_NAME = "clickhouse/sqltest" + + +def get_run_command(pr_number, sha, download_url, workspace_path, image): + return ( + f"docker run " + # For sysctl + "--privileged " + "--network=host " + f"--volume={workspace_path}:/workspace " + "--cap-add syslog --cap-add sys_admin --cap-add=SYS_PTRACE " + f'-e PR_TO_TEST={pr_number} -e SHA_TO_TEST={sha} -e BINARY_URL_TO_DOWNLOAD="{download_url}" ' + f"{image}" + ) + + +def main(): + logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + + temp_path = TEMP_PATH + reports_path = REPORTS_PATH + + check_name = sys.argv[1] + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + pr_info = PRInfo() + + gh = Github(get_best_robot_token(), per_page=100) + commit = get_commit(gh, pr_info.sha) + + rerun_helper = RerunHelper(commit, check_name) + if rerun_helper.is_already_finished_by_status(): + logging.info("Check is already finished according to github status, exiting") + sys.exit(0) + + docker_image = get_image_with_version(reports_path, IMAGE_NAME) + + build_name = get_build_name_for_check(check_name) + print(build_name) + urls = read_build_urls(build_name, reports_path) + if not urls: + raise Exception("No build URLs found") + + for url in urls: + if url.endswith("/clickhouse"): + build_url = url + break + else: + raise Exception("Cannot find the clickhouse binary among build results") + + logging.info("Got build url %s", build_url) + + workspace_path = os.path.join(temp_path, "workspace") + if not os.path.exists(workspace_path): + os.makedirs(workspace_path) + + run_command = get_run_command( + pr_info.number, pr_info.sha, build_url, workspace_path, docker_image + ) + logging.info("Going to run %s", run_command) + + run_log_path = os.path.join(temp_path, "run.log") + with open(run_log_path, "w", encoding="utf-8") as log: + with subprocess.Popen( + run_command, shell=True, stderr=log, stdout=log + ) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") + + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + + check_name_lower = ( + check_name.lower().replace("(", "").replace(")", "").replace(" ", "") + ) + s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" + paths = { + "run.log": run_log_path, + "main.log": os.path.join(workspace_path, "main.log"), + "server.log.zst": os.path.join(workspace_path, "server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"), + "report.html": os.path.join(workspace_path, "report.html"), + "test.log": os.path.join(workspace_path, "test.log"), + } + + s3_helper = S3Helper() + for f in paths: + try: + paths[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + f) + except Exception as ex: + logging.info("Exception uploading file %s text %s", f, ex) + paths[f] = "" + + report_url = GITHUB_RUN_URL + if paths["report.html"]: + report_url = paths["report.html"] + + status = "success" + description = "See the report" + test_result = TestResult(description, "OK") + + ch_helper = ClickHouseHelper() + + prepared_events = prepare_tests_results_for_clickhouse( + pr_info, + [test_result], + status, + stopwatch.duration_seconds, + stopwatch.start_time_str, + report_url, + check_name, + ) + + ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) + + logging.info("Result: '%s', '%s', '%s'", status, description, report_url) + print(f"::notice ::Report url: {report_url}") + post_commit_status(commit, status, report_url, description, check_name, pr_info) + + +if __name__ == "__main__": + main() diff --git a/tests/ci/tests/docker_images.json b/tests/ci/tests/docker_images.json index 0d40d43c33f..0637058e184 100644 --- a/tests/ci/tests/docker_images.json +++ b/tests/ci/tests/docker_images.json @@ -153,5 +153,9 @@ "docker/test/sqllogic": { "name": "clickhouse/sqllogic-test", "dependent": [] + }, + "docker/test/sqltest": { + "name": "clickhouse/sqltest", + "dependent": [] } } From df5abcf685a141d163e4aefc213521fc64b613ca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:45:43 +0200 Subject: [PATCH 0809/2047] Update Docker --- docker/test/sqltest/Dockerfile | 1 + docker/test/sqltest/test.py | 0 2 files changed, 1 insertion(+) mode change 100644 => 100755 docker/test/sqltest/test.py diff --git a/docker/test/sqltest/Dockerfile b/docker/test/sqltest/Dockerfile index c58907a98a8..437677f4fd1 100644 --- a/docker/test/sqltest/Dockerfile +++ b/docker/test/sqltest/Dockerfile @@ -26,4 +26,5 @@ ENV MAX_RUN_TIME=900 RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone COPY run.sh / +COPY test.py / CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py old mode 100644 new mode 100755 From 174166b23382488f1a6fc12cbf121b1a1da7f49b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:47:41 +0200 Subject: [PATCH 0810/2047] Fix error --- docker/test/sqltest/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index ef9ecaec33b..b26d4d89f8b 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -34,7 +34,7 @@ done # Run the test pushd sqltest/standards/2016/ -./test.py +/test.py popd zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log From fcf4a6ea887f957b8d911b1611932ecc59260b7a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:48:48 +0200 Subject: [PATCH 0811/2047] Fix error --- docker/test/sqltest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index b26d4d89f8b..24032874403 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -35,6 +35,7 @@ done # Run the test pushd sqltest/standards/2016/ /test.py +mv report.html test.log / popd zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log From 2255307a477ab86dc7740537a31e4791f3e769a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:49:47 +0200 Subject: [PATCH 0812/2047] Improvement --- docker/test/sqltest/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index 7b38fdc52ad..945446358c3 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -95,7 +95,7 @@ def print_ratio(indent, name, success, total, description): f" - {description}" if description else '')) -report_html_file.write("

") +report_html_file.write("

")
 
 print_ratio(0, 'Total', summary['success'], summary['total'], '')
 
@@ -115,4 +115,4 @@ for category in summary['results']:
 
         print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description'])
 
-report_html_file.write("

") +report_html_file.write("
") From 50dac21483b9003ad546da6d87340124bf108d69 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:54:09 +0200 Subject: [PATCH 0813/2047] Improvement --- docker/test/sqltest/test.py | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index 945446358c3..ae2d6406c78 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -2,6 +2,7 @@ import os import yaml +import html import random import string from clickhouse_driver import Client @@ -26,11 +27,11 @@ with open('features.yml', 'r') as file: yaml_content = yaml.safe_load(file) for category in yaml_content: - log_file.write(category.capitalize() + " features:") + log_file.write(category.capitalize() + " features:\n") summary['results'][category] = {'success': 0, 'total': 0, 'results': {}} for test in yaml_content[category]: - log_file.write(test + ": " + yaml_content[category][test]) + log_file.write(test + ": " + yaml_content[category][test] + "\n") summary['results'][category]['results'][test] = {'success': 0, 'total': 0, 'description': yaml_content[category][test]} test_path = test[0] + "/" + test + ".tests.yml" @@ -54,11 +55,11 @@ with open('features.yml', 'r') as file: summary['results'][category]['total'] += 1 summary['total'] += 1 - log_file.write(query) + log_file.write(query + "\n") try: result = client.execute(query, settings = settings) - log_file.write(result) + log_file.write(result + "\n") if test_group: summary['results'][category]['results'][test_group]['success'] += 1 @@ -67,7 +68,7 @@ with open('features.yml', 'r') as file: summary['success'] += 1 except Exception as e: - log_file.write(f"Error occurred: {str(e)}") + log_file.write(f"Error occurred: {str(e)}\n") client.execute(f"DROP DATABASE {database_name}", settings = settings) @@ -84,7 +85,7 @@ def enable_color(ratio): reset_color = "
" def print_ratio(indent, name, success, total, description): - report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
".format( + report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
\n".format( ' ' * indent, name, enable_color(success / total), @@ -92,10 +93,10 @@ def print_ratio(indent, name, success, total, description): total, success / total, reset_color, - f" - {description}" if description else '')) + f" - " + html.escape(description) if description else '')) -report_html_file.write("
")
+report_html_file.write("
\n")
 
 print_ratio(0, 'Total', summary['success'], summary['total'], '')
 
@@ -115,4 +116,4 @@ for category in summary['results']:
 
         print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description'])
 
-report_html_file.write("
") +report_html_file.write("
\n") From 76677e009b8cf9a501fde1f7273fa5d546f4ffa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:00:50 +0200 Subject: [PATCH 0814/2047] Improvement --- docker/test/sqltest/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index ae2d6406c78..cf2f976e05a 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -59,7 +59,7 @@ with open('features.yml', 'r') as file: try: result = client.execute(query, settings = settings) - log_file.write(result + "\n") + log_file.write(str(result) + "\n") if test_group: summary['results'][category]['results'][test_group]['success'] += 1 From fe5972953703edb8a7e13c6066e5af369359b82c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:01:00 +0200 Subject: [PATCH 0815/2047] Improvement --- docker/test/sqltest/run.sh | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 24032874403..03678971f60 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -23,6 +23,12 @@ function wget_with_retry wget_with_retry "$BINARY_URL_TO_DOWNLOAD" chmod +x clickhouse ./clickhouse install --noninteractive + +echo " +users: + default: + access_management: 1" > /etc/clickhouse-server/users.d/access_management.yaml + clickhouse start # Wait for start From 9b10e0a5bc2b7e3f84614d1aa3fa84c60de76929 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:09:19 +0200 Subject: [PATCH 0816/2047] Improvement --- docker/test/sqltest/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index cf2f976e05a..ff321d3e5df 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -85,9 +85,9 @@ def enable_color(ratio): reset_color = "
" def print_ratio(indent, name, success, total, description): - report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
\n".format( + report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}\n".format( ' ' * indent, - name, + name.capitalize(), enable_color(success / total), success, total, @@ -96,7 +96,7 @@ def print_ratio(indent, name, success, total, description): f" - " + html.escape(description) if description else '')) -report_html_file.write("
\n")
+report_html_file.write("
\n")
 
 print_ratio(0, 'Total', summary['success'], summary['total'], '')
 
@@ -114,6 +114,6 @@ for category in summary['results']:
         if test_summary['total'] == 0:
             continue
 
-        print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description'])
+        print_ratio(6 if '-' in test else 4, test, test_summary['success'], test_summary['total'], test_summary['description'])
 
 report_html_file.write("
\n") From 891c01e6ab30c2d4377a118cb4c449134a56e17a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 19 Jul 2023 02:37:02 +0000 Subject: [PATCH 0817/2047] Automatic style fix --- docker/test/sqltest/test.py | 119 ++++++++++++++++++++++-------------- 1 file changed, 74 insertions(+), 45 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index ff321d3e5df..5807ca79b02 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -8,69 +8,85 @@ import string from clickhouse_driver import Client -client = Client(host = 'localhost', port = 9000) -settings = {'default_table_engine': 'Memory', 'union_default_mode': 'DISTINCT', 'calculate_text_stack_trace': 0} +client = Client(host="localhost", port=9000) +settings = { + "default_table_engine": "Memory", + "union_default_mode": "DISTINCT", + "calculate_text_stack_trace": 0, +} -database_name = 'sqltest_' + ''.join(random.choice(string.ascii_lowercase) for _ in range(10)) +database_name = "sqltest_" + "".join( + random.choice(string.ascii_lowercase) for _ in range(10) +) -client.execute(f"DROP DATABASE IF EXISTS {database_name}", settings = settings) -client.execute(f"CREATE DATABASE {database_name}", settings = settings) +client.execute(f"DROP DATABASE IF EXISTS {database_name}", settings=settings) +client.execute(f"CREATE DATABASE {database_name}", settings=settings) -client = Client(host = 'localhost', port = 9000, database = database_name) +client = Client(host="localhost", port=9000, database=database_name) -summary = {'success': 0, 'total': 0, 'results': {}} +summary = {"success": 0, "total": 0, "results": {}} log_file = open("test.log", "w") report_html_file = open("report.html", "w") -with open('features.yml', 'r') as file: +with open("features.yml", "r") as file: yaml_content = yaml.safe_load(file) for category in yaml_content: log_file.write(category.capitalize() + " features:\n") - summary['results'][category] = {'success': 0, 'total': 0, 'results': {}} + summary["results"][category] = {"success": 0, "total": 0, "results": {}} for test in yaml_content[category]: log_file.write(test + ": " + yaml_content[category][test] + "\n") - summary['results'][category]['results'][test] = {'success': 0, 'total': 0, 'description': yaml_content[category][test]} + summary["results"][category]["results"][test] = { + "success": 0, + "total": 0, + "description": yaml_content[category][test], + } test_path = test[0] + "/" + test + ".tests.yml" if os.path.exists(test_path): - with open(test_path, 'r') as test_file: - test_yaml_content = yaml.load_all(test_file, Loader = yaml.FullLoader) + with open(test_path, "r") as test_file: + test_yaml_content = yaml.load_all(test_file, Loader=yaml.FullLoader) for test_case in test_yaml_content: - - queries = test_case['sql']; + queries = test_case["sql"] if not isinstance(queries, list): queries = [queries] for query in queries: # Example: E011-01 - test_group = '' - if '-' in test: + test_group = "" + if "-" in test: test_group = test.split("-", 1)[0] - summary['results'][category]['results'][test_group]['total'] += 1 - summary['results'][category]['results'][test]['total'] += 1 - summary['results'][category]['total'] += 1 - summary['total'] += 1 + summary["results"][category]["results"][test_group][ + "total" + ] += 1 + summary["results"][category]["results"][test]["total"] += 1 + summary["results"][category]["total"] += 1 + summary["total"] += 1 log_file.write(query + "\n") try: - result = client.execute(query, settings = settings) + result = client.execute(query, settings=settings) log_file.write(str(result) + "\n") if test_group: - summary['results'][category]['results'][test_group]['success'] += 1 - summary['results'][category]['results'][test]['success'] += 1 - summary['results'][category]['success'] += 1 - summary['success'] += 1 + summary["results"][category]["results"][test_group][ + "success" + ] += 1 + summary["results"][category]["results"][test][ + "success" + ] += 1 + summary["results"][category]["success"] += 1 + summary["success"] += 1 except Exception as e: log_file.write(f"Error occurred: {str(e)}\n") -client.execute(f"DROP DATABASE {database_name}", settings = settings) +client.execute(f"DROP DATABASE {database_name}", settings=settings) + def enable_color(ratio): if ratio == 0: @@ -82,38 +98,51 @@ def enable_color(ratio): else: return "" + reset_color = "" + def print_ratio(indent, name, success, total, description): - report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}\n".format( - ' ' * indent, - name.capitalize(), - enable_color(success / total), - success, - total, - success / total, - reset_color, - f" - " + html.escape(description) if description else '')) + report_html_file.write( + "{}{}: {}{} / {} ({:.1%}){}{}\n".format( + " " * indent, + name.capitalize(), + enable_color(success / total), + success, + total, + success / total, + reset_color, + f" - " + html.escape(description) if description else "", + ) + ) -report_html_file.write("
\n")
+report_html_file.write(
+    "
\n"
+)
 
-print_ratio(0, 'Total', summary['success'], summary['total'], '')
+print_ratio(0, "Total", summary["success"], summary["total"], "")
 
-for category in summary['results']:
-    cat_summary = summary['results'][category]
+for category in summary["results"]:
+    cat_summary = summary["results"][category]
 
-    if cat_summary['total'] == 0:
+    if cat_summary["total"] == 0:
         continue
 
-    print_ratio(2, category, cat_summary['success'], cat_summary['total'], '')
+    print_ratio(2, category, cat_summary["success"], cat_summary["total"], "")
 
-    for test in summary['results'][category]['results']:
-        test_summary = summary['results'][category]['results'][test]
+    for test in summary["results"][category]["results"]:
+        test_summary = summary["results"][category]["results"][test]
 
-        if test_summary['total'] == 0:
+        if test_summary["total"] == 0:
             continue
 
-        print_ratio(6 if '-' in test else 4, test, test_summary['success'], test_summary['total'], test_summary['description'])
+        print_ratio(
+            6 if "-" in test else 4,
+            test,
+            test_summary["success"],
+            test_summary["total"],
+            test_summary["description"],
+        )
 
 report_html_file.write("
\n") From d666272b7666967cf1d1bed3804673e3beb1ca64 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 05:29:12 +0200 Subject: [PATCH 0818/2047] Enable `allow_vertical_merges_from_compact_to_wide_parts` by default --- src/Storages/MergeTree/MergeTreeSettings.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index dc24327712c..783fde088dc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -160,7 +160,7 @@ struct Settings; M(UInt64, min_marks_to_honor_max_concurrent_queries, 0, "Minimal number of marks to honor the MergeTree-level's max_concurrent_queries (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ M(UInt64, min_bytes_to_rebalance_partition_over_jbod, 0, "Minimal amount of bytes to enable part rebalance over JBOD array (0 - disabled).", 0) \ M(Bool, check_sample_column_is_correct, true, "Check columns or columns by hash for sampling are unsigned integer.", 0) \ - M(Bool, allow_vertical_merges_from_compact_to_wide_parts, false, "Allows vertical merges from compact to wide parts. This settings must have the same value on all replicas", 0) \ + M(Bool, allow_vertical_merges_from_compact_to_wide_parts, true, "Allows vertical merges from compact to wide parts. This settings must have the same value on all replicas", 0) \ M(Bool, enable_the_endpoint_id_with_zookeeper_name_prefix, false, "Enable the endpoint id with zookeeper name prefix for the replicated merge tree table", 0) \ M(UInt64, zero_copy_merge_mutation_min_parts_size_sleep_before_lock, 1ULL * 1024 * 1024 * 1024, "If zero copy replication is enabled sleep random amount of time before trying to lock depending on parts size for merge or mutation", 0) \ \ @@ -169,8 +169,9 @@ struct Settings; M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ M(Bool, use_metadata_cache, false, "Experimental feature to speed up parts loading process by using MergeTree metadata cache", 0) \ M(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", 0) \ - M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for Zero-copy table-independet info.", 0) \ + M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for zero-copy table-independent info.", 0) \ M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ + \ /** Compress marks and primary key. */ \ M(Bool, compress_marks, true, "Marks support compression, reduce mark file size and speed up network transmission.", 0) \ M(Bool, compress_primary_key, true, "Primary key support compression, reduce primary key file size and speed up network transmission.", 0) \ From c3b8978023fae8adaa98a111f6253be50ee72a35 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 11:53:03 +0800 Subject: [PATCH 0819/2047] Don't use minmax_count projections when counting nullable columns --- .../optimizeUseAggregateProjection.cpp | 32 ++++--------------- ..._count_projection_count_nullable.reference | 1 + ...minmax_count_projection_count_nullable.sql | 9 ++++++ 3 files changed, 17 insertions(+), 25 deletions(-) create mode 100644 tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.reference create mode 100644 tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.sql diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index f183bdca7a9..4f25118958f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -92,18 +92,6 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( return info; } -static bool hasNullableOrMissingColumn(const DAGIndex & index, const Names & names) -{ - for (const auto & query_name : names) - { - auto jt = index.find(query_name); - if (jt == index.end() || jt->second->result_type->isNullable()) - return true; - } - - return false; -} - struct AggregateFunctionMatch { const AggregateDescription * description = nullptr; @@ -170,20 +158,14 @@ std::optional matchAggregateFunctions( } /// This is a special case for the function count(). - /// We can assume that 'count(expr) == count()' if expr is not nullable. - if (typeid_cast(candidate.function.get())) + /// We can assume that 'count(expr) == count()' if expr is not nullable, + /// which can be verified by simply casting to `AggregateFunctionCount *`. + if (typeid_cast(aggregate.function.get())) { - bool has_nullable_or_missing_arg = false; - has_nullable_or_missing_arg |= hasNullableOrMissingColumn(query_index, aggregate.argument_names); - has_nullable_or_missing_arg |= hasNullableOrMissingColumn(proj_index, candidate.argument_names); - - if (!has_nullable_or_missing_arg) - { - /// we can ignore arguments for count() - found_match = true; - res.push_back({&candidate, DataTypes()}); - break; - } + /// we can ignore arguments for count() + found_match = true; + res.push_back({&candidate, DataTypes()}); + break; } /// Now, function names and types matched. diff --git a/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.reference b/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.sql b/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.sql new file mode 100644 index 00000000000..048d725e0a0 --- /dev/null +++ b/tests/queries/0_stateless/01710_minmax_count_projection_count_nullable.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (`val` LowCardinality(Nullable(String))) ENGINE = MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192; + +insert into test select number == 3 ? 'some value' : null from numbers(5); + +SELECT count(val) FROM test SETTINGS optimize_use_implicit_projections = 1; + +DROP TABLE test; From 65de310137a4e192499119128aa069375eb007c8 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 06:15:57 +0000 Subject: [PATCH 0820/2047] Return back SystemLogBase --- src/Common/SystemLogBase.cpp | 40 +++++++++++++++++++++++++++++++++- src/Common/SystemLogBase.h | 33 ++++++++++++++++++++++++++++ src/Interpreters/SystemLog.cpp | 29 ++---------------------- src/Interpreters/SystemLog.h | 16 +++----------- 4 files changed, 77 insertions(+), 41 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 1d0673e30dd..baee7021c35 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -202,7 +202,45 @@ void SystemLogQueue::shutdown() flush_event.notify_all(); } -#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogQueue; +template +SystemLogBase::SystemLogBase( + const String& name, + size_t flush_interval_milliseconds_, + std::shared_ptr> queue_) + : queue(queue_ ? queue_ : std::make_shared>(name, flush_interval_milliseconds_)) +{ +} + +template +void SystemLogBase::startup() +{ + std::lock_guard lock(queue->mutex); + saving_thread = std::make_unique([this] { savingThreadFunction(); }); +} + +template +void SystemLogBase::add(const LogElement & element) +{ + queue->push(element); +} + +template +void SystemLogBase::flush(bool force) +{ + uint64_t this_thread_requested_offset = queue->notifyFlush(force); + if (this_thread_requested_offset == uint64_t(-1)) + return; + + queue->waitFlush(this_thread_requested_offset); +} + +template +void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } + +#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE) +#define INSTANTIATE_SYSTEM_LOG_QUEUE(ELEMENT) template class SystemLogQueue; +SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_QUEUE) + } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 9436137d4a8..5718182e115 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -121,4 +121,37 @@ private: const size_t flush_interval_milliseconds; }; + + +template +class SystemLogBase : public ISystemLog +{ +public: + using Self = SystemLogBase; + + SystemLogBase( + const String& name, + size_t flush_interval_milliseconds_, + std::shared_ptr> queue_ = nullptr); + + void startup() override; + + /** Append a record into log. + * Writing to table will be done asynchronously and in case of failure, record could be lost. + */ + void add(const LogElement & element); + + /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. + void flush(bool force) override; + + /// Non-blocking flush data in the buffer to disk. + void notifyFlush(bool force); + + String getName() const override { return LogElement::name(); } + + static const char * getDefaultOrderBy() { return "event_date, event_time"; } + +protected: + std::shared_ptr> queue; +}; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 3193baa551f..674210cbaad 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -334,23 +334,16 @@ SystemLog::SystemLog( const String & storage_def_, size_t flush_interval_milliseconds_, std::shared_ptr> queue_) - : WithContext(context_) + : Base(database_name_ + "." + table_name_, flush_interval_milliseconds_, queue_) + , WithContext(context_) , log(&Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")")) , table_id(database_name_, table_name_) , storage_def(storage_def_) , create_query(serializeAST(*getCreateTableQuery())) - , queue(queue_ ? queue_ : std::make_shared>(database_name_ + "." + table_name_, flush_interval_milliseconds_)) { assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); } -template -void SystemLog::startup() -{ - std::lock_guard lock(queue->mutex); - saving_thread = std::make_unique([this] { savingThreadFunction(); }); -} - template void SystemLog::shutdown() { @@ -618,24 +611,6 @@ ASTPtr SystemLog::getCreateTableQuery() return create; } -template -void SystemLog::add(const LogElement & element) -{ - queue->push(element); -} - -template -void SystemLog::flush(bool force) -{ - uint64_t this_thread_requested_offset = queue->notifyFlush(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; - - queue->waitFlush(this_thread_requested_offset); -} - -template -void SystemLog::notifyFlush(bool force) { queue->notifyFlush(force); } #define INSTANTIATE_SYSTEM_LOG(ELEMENT) template class SystemLog; SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 6f61e075b49..91fb7f49221 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -89,10 +89,11 @@ struct SystemLogs template -class SystemLog : public ISystemLog, private boost::noncopyable, WithContext +class SystemLog : public SystemLogBase, private boost::noncopyable, WithContext { public: using Self = SystemLog; + using Base = SystemLogBase; /** Parameter: table name where to write log. * If table is not exists, then it get created with specified engine. @@ -110,23 +111,12 @@ public: size_t flush_interval_milliseconds_, std::shared_ptr> queue_ = nullptr); - void startup() override; /** Append a record into log. * Writing to table will be done asynchronously and in case of failure, record could be lost. */ - void add(const LogElement & element); void shutdown() override; - String getName() const override { return LogElement::name(); } - static const char * getDefaultOrderBy() { return "event_date, event_time"; } - - /// Flush data in the buffer to disk. Block the thread until the data is stored on disk. - void flush(bool force) override; - - /// Non-blocking flush data in the buffer to disk. - void notifyFlush(bool force); - void stopFlushThread() override; protected: @@ -134,6 +124,7 @@ protected: using ISystemLog::is_shutdown; using ISystemLog::saving_thread; + using Base::queue; private: @@ -144,7 +135,6 @@ private: String create_query; String old_create_query; bool is_prepared = false; - std::shared_ptr> queue; /** Creates new table if it does not exist. * Renames old table if its structure is not suitable. From ee0453ed00ab5ecb232557e29d4e1f6365d83cd0 Mon Sep 17 00:00:00 2001 From: Chen768959 <67011523+Chen768959@users.noreply.github.com> Date: Wed, 19 Jul 2023 14:18:50 +0800 Subject: [PATCH 0821/2047] fix issue#50582 tests Reproduced issue #50582, which occurs when sorting column contains constants and triggers the FinishSortingTransform. --- .../02815_fix_not_found_constants_col_in_block.reference | 2 ++ .../02815_fix_not_found_constants_col_in_block.sql | 5 +++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference create mode 100644 tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql diff --git a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference new file mode 100644 index 00000000000..f2d4d23d9e3 --- /dev/null +++ b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.reference @@ -0,0 +1,2 @@ +\N 1 19000 +\N 1 19000 diff --git a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql new file mode 100644 index 00000000000..c56d59c72d6 --- /dev/null +++ b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (vkey UInt32, c0 Float32, primary key(c0)) engine = AggregatingMergeTree; +insert into t0 values (19000, 1); +select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc, c_2_2 asc; +select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc; From d601d86fad94250ca3b749baa4478679cd6e1973 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 07:22:25 +0000 Subject: [PATCH 0822/2047] Remove empty line --- src/Common/SystemLogBase.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 5718182e115..fa9f9b6f72e 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -122,7 +122,6 @@ private: }; - template class SystemLogBase : public ISystemLog { From 629e0e0269dc96f88f781eb8a0a711667d50c92b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 16:06:21 +0800 Subject: [PATCH 0823/2047] Fix projection analysis with primary key analysis --- .../QueryPlan/Optimizations/optimizeTree.cpp | 8 +++++--- .../QueryPlan/Optimizations/projectionsCommon.cpp | 3 ++- .../QueryPlan/Optimizations/projectionsCommon.h | 1 - ..._projection_with_query_plan_optimization.reference | 1 + ...normal_projection_with_query_plan_optimization.sql | 11 +++++++++++ 5 files changed, 19 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference create mode 100644 tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 6cb76d540f7..01d192bb1f3 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -146,8 +146,13 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } + /// NOTE: optimizePrewhere can modify the stack. + optimizePrewhere(stack, nodes); + optimizePrimaryKeyCondition(stack); + if (optimization_settings.optimize_projection) { + /// Normal projection optimization relies on PK optimization if (optimizeUseNormalProjections(stack, nodes)) { ++num_applied_projection; @@ -164,9 +169,6 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } - /// NOTE: optimizePrewhere can modify the stack. - optimizePrewhere(stack, nodes); - optimizePrimaryKeyCondition(stack); enableMemoryBoundMerging(*stack.back().node, nodes); stack.pop_back(); diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index cb76ffa84ba..7ddda29cad4 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -131,7 +131,8 @@ bool QueryDAG::buildImpl(QueryPlan::Node & node, ActionsDAG::NodeRawConstPtrs & if (prewhere_info->prewhere_actions) { appendExpression(prewhere_info->prewhere_actions); - if (const auto * filter_expression = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) + if (const auto * filter_expression + = findInOutputs(*dag, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)) filter_nodes.push_back(filter_expression); else return false; diff --git a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h index 1e9ab67c8fe..35daccad115 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.h +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.h @@ -38,7 +38,6 @@ std::shared_ptr getMaxAddedBlocks(ReadFromMergeTree * rea /// This is a common DAG which is a merge of DAGs from Filter and Expression steps chain. /// Additionally, for all the Filter steps, we collect filter conditions into filter_nodes. -/// Flag remove_last_filter_node is set in case if the last step is a Filter step and it should remove filter column. struct QueryDAG { ActionsDAGPtr dag; diff --git a/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql new file mode 100644 index 00000000000..30306ec5637 --- /dev/null +++ b/tests/queries/0_stateless/01710_normal_projection_with_query_plan_optimization.sql @@ -0,0 +1,11 @@ +drop table if exists t; + +CREATE TABLE t (id UInt64, id2 UInt64, id3 UInt64, PROJECTION t_reverse (SELECT id, id2, id3 ORDER BY id2, id, id3)) ENGINE = MergeTree ORDER BY (id) settings index_granularity = 4; + +insert into t SELECT number, -number, number FROM numbers(10000); + +set max_rows_to_read = 4; + +select count() from t where id = 3; + +drop table t; From 549026f0ae8041ba40f4557922c480f2f07715bf Mon Sep 17 00:00:00 2001 From: Chen768959 <67011523+Chen768959@users.noreply.github.com> Date: Wed, 19 Jul 2023 16:11:14 +0800 Subject: [PATCH 0824/2047] fix style error fix Trailing whitespaces --- src/Processors/Transforms/FinishSortingTransform.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index dd61472bc37..066928446f2 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -50,7 +50,6 @@ FinishSortingTransform::FinishSortingTransform( description_sorted_without_constants.push_back(column_description); } } - /// The target description is modified in SortingTransform constructor. /// To avoid doing the same actions with description_sorted just copy it from prefix of target description. for (const auto & column_sort_desc : description_sorted_without_constants) From a86baab88b4444d5bf34e529bb737817daa20096 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 19 Jul 2023 07:49:30 +0000 Subject: [PATCH 0825/2047] Fix test_replicated_database 'node doesn't exist' flakiness --- .../test_replicated_database/test.py | 148 +++++++++--------- 1 file changed, 75 insertions(+), 73 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index b3ba8d4737f..17dd2adcde4 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -132,14 +132,15 @@ def test_create_replicated_table(started_cluster): @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_simple_alter_table(started_cluster, engine): + database = f"test_simple_alter_table_{engine}" main_node.query( - "CREATE DATABASE test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) # test_simple_alter_table - name = "test_simple_alter_table.alter_test_{}".format(engine) + name = f"{database}.alter_test" main_node.query( "CREATE TABLE {} " "(CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32) " @@ -187,10 +188,9 @@ def test_simple_alter_table(started_cluster, engine): # test_create_replica_after_delay competing_node.query( - "CREATE DATABASE IF NOT EXISTS test_simple_alter_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica3');" + f"CREATE DATABASE IF NOT EXISTS {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica3');" ) - name = "test_simple_alter_table.alter_test_{}".format(engine) main_node.query("ALTER TABLE {} ADD COLUMN Added3 UInt32;".format(name)) main_node.query("ALTER TABLE {} DROP COLUMN AddedNested1;".format(name)) main_node.query("ALTER TABLE {} RENAME COLUMN Added1 TO AddedNested1;".format(name)) @@ -210,21 +210,23 @@ def test_simple_alter_table(started_cluster, engine): ) assert_create_query([main_node, dummy_node, competing_node], name, expected) - main_node.query("DROP DATABASE test_simple_alter_table SYNC") - dummy_node.query("DROP DATABASE test_simple_alter_table SYNC") - competing_node.query("DROP DATABASE test_simple_alter_table SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") + competing_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_delete_from_table(started_cluster, engine): + database = f"delete_from_table_{engine}" + main_node.query( - "CREATE DATABASE delete_from_table ENGINE = Replicated('/test/simple_alter_table', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE delete_from_table ENGINE = Replicated('/test/simple_alter_table', 'shard2', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard2', 'replica1');" ) - name = "delete_from_table.delete_test_{}".format(engine) + name = f"{database}.delete_test" main_node.query( "CREATE TABLE {} " "(id UInt64, value String) " @@ -241,7 +243,7 @@ def test_delete_from_table(started_cluster, engine): table_for_select = name if not "Replicated" in engine: - table_for_select = "cluster('delete_from_table', {})".format(name) + table_for_select = f"cluster('{database}', {name})" for node in [main_node, dummy_node]: assert_eq_with_retry( node, @@ -249,8 +251,8 @@ def test_delete_from_table(started_cluster, engine): expected, ) - main_node.query("DROP DATABASE delete_from_table SYNC") - dummy_node.query("DROP DATABASE delete_from_table SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") def get_table_uuid(database, name): @@ -278,18 +280,18 @@ def fixture_attachable_part(started_cluster): @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_attach(started_cluster, attachable_part, engine): + database = f"alter_attach_{engine}" main_node.query( - "CREATE DATABASE alter_attach ENGINE = Replicated('/test/alter_attach', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_attach ENGINE = Replicated('/test/alter_attach', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - name = "alter_attach_test_{}".format(engine) main_node.query( - f"CREATE TABLE alter_attach.{name} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_attach_test (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - table_uuid = get_table_uuid("alter_attach", name) + table_uuid = get_table_uuid(database, "alter_attach_test") # Provide and attach a part to the main node shutil.copytree( attachable_part, @@ -298,146 +300,146 @@ def test_alter_attach(started_cluster, attachable_part, engine): f"database/store/{table_uuid[:3]}/{table_uuid}/detached/all_1_1_0", ), ) - main_node.query(f"ALTER TABLE alter_attach.{name} ATTACH PART 'all_1_1_0'") + main_node.query(f"ALTER TABLE {database}.alter_attach_test ATTACH PART 'all_1_1_0'") # On the main node, data is attached - assert main_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "123\n" + assert main_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" # On the other node, data is replicated only if using a Replicated table engine if engine == "ReplicatedMergeTree": - assert dummy_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "123\n" + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" else: - assert dummy_node.query(f"SELECT CounterID FROM alter_attach.{name}") == "" - main_node.query("DROP DATABASE alter_attach SYNC") - dummy_node.query("DROP DATABASE alter_attach SYNC") + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "" + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_part(started_cluster, engine): + database = f"alter_drop_part_{engine}" main_node.query( - "CREATE DATABASE alter_drop_part ENGINE = Replicated('/test/alter_drop_part', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_drop_part ENGINE = Replicated('/test/alter_drop_part', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - table = f"alter_drop_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE alter_drop_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_drop_part (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO alter_drop_part.{table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_drop_part VALUES (123)") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO alter_drop_part.{table} VALUES (456)") - main_node.query(f"ALTER TABLE alter_drop_part.{table} DROP PART '{part_name}'") - assert main_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") == "" + dummy_node.query(f"INSERT INTO {database}.alter_drop_part VALUES (456)") + main_node.query(f"ALTER TABLE {database}.alter_drop_part DROP PART '{part_name}'") + assert main_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" if engine == "ReplicatedMergeTree": # The DROP operation is still replicated at the table engine level - assert dummy_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") == "" + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" else: assert ( - dummy_node.query(f"SELECT CounterID FROM alter_drop_part.{table}") + dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "456\n" ) - main_node.query("DROP DATABASE alter_drop_part SYNC") - dummy_node.query("DROP DATABASE alter_drop_part SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_detach_part(started_cluster, engine): + database = f"alter_detach_part_{engine}" main_node.query( - "CREATE DATABASE alter_detach_part ENGINE = Replicated('/test/alter_detach_part', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_detach_part ENGINE = Replicated('/test/alter_detach_part', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - table = f"alter_detach_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE alter_detach_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_detach (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO alter_detach_part.{table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_detach VALUES (123)") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO alter_detach_part.{table} VALUES (456)") - main_node.query(f"ALTER TABLE alter_detach_part.{table} DETACH PART '{part_name}'") - detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='alter_detach_part' AND table='{table}'" + dummy_node.query(f"INSERT INTO {database}.alter_detach VALUES (456)") + main_node.query(f"ALTER TABLE {database}.alter_detach DETACH PART '{part_name}'") + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='{database}' AND table='alter_detach'" assert main_node.query(detached_parts_query) == f"{part_name}\n" if engine == "ReplicatedMergeTree": # The detach operation is still replicated at the table engine level assert dummy_node.query(detached_parts_query) == f"{part_name}\n" else: assert dummy_node.query(detached_parts_query) == "" - main_node.query("DROP DATABASE alter_detach_part SYNC") - dummy_node.query("DROP DATABASE alter_detach_part SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_detached_part(started_cluster, engine): + database = f"alter_drop_detached_part_{engine}" main_node.query( - "CREATE DATABASE alter_drop_detached_part ENGINE = Replicated('/test/alter_drop_detached_part', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_drop_detached_part ENGINE = Replicated('/test/alter_drop_detached_part', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) - table = f"alter_drop_detached_{engine}" part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" main_node.query( - f"CREATE TABLE alter_drop_detached_part.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_drop_detached (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO alter_drop_detached_part.{table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_drop_detached VALUES (123)") main_node.query( - f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'" + f"ALTER TABLE {database}.alter_drop_detached DETACH PART '{part_name}'" ) if engine == "MergeTree": - dummy_node.query(f"INSERT INTO alter_drop_detached_part.{table} VALUES (456)") + dummy_node.query(f"INSERT INTO {database}.alter_drop_detached VALUES (456)") dummy_node.query( - f"ALTER TABLE alter_drop_detached_part.{table} DETACH PART '{part_name}'" + f"ALTER TABLE {database}.alter_drop_detached DETACH PART '{part_name}'" ) main_node.query( - f"ALTER TABLE alter_drop_detached_part.{table} DROP DETACHED PART '{part_name}'" + f"ALTER TABLE {database}.alter_drop_detached DROP DETACHED PART '{part_name}'" ) - detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='alter_drop_detached_part' AND table='{table}'" + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='{database}' AND table='alter_drop_detached'" assert main_node.query(detached_parts_query) == "" assert dummy_node.query(detached_parts_query) == f"{part_name}\n" - main_node.query("DROP DATABASE alter_drop_detached_part SYNC") - dummy_node.query("DROP DATABASE alter_drop_detached_part SYNC") + main_node.query(f"DROP DATABASE {database} SYNC") + dummy_node.query(f"DROP DATABASE {database} SYNC") @pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) def test_alter_drop_partition(started_cluster, engine): + database = f"alter_drop_partition_{engine}" main_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard1', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica1');" ) dummy_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard1', 'replica2');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard1', 'replica2');" ) snapshotting_node.query( - "CREATE DATABASE alter_drop_partition ENGINE = Replicated('/test/alter_drop_partition', 'shard2', 'replica1');" + f"CREATE DATABASE {database} ENGINE = Replicated('/test/{database}', 'shard2', 'replica1');" ) - table = f"alter_drop_partition.alter_drop_{engine}" main_node.query( - f"CREATE TABLE {table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" + f"CREATE TABLE {database}.alter_drop (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)" ) - main_node.query(f"INSERT INTO {table} VALUES (123)") + main_node.query(f"INSERT INTO {database}.alter_drop VALUES (123)") if engine == "MergeTree": - dummy_node.query(f"INSERT INTO {table} VALUES (456)") - snapshotting_node.query(f"INSERT INTO {table} VALUES (789)") + dummy_node.query(f"INSERT INTO {database}.alter_drop VALUES (456)") + snapshotting_node.query(f"INSERT INTO {database}.alter_drop VALUES (789)") main_node.query( - f"ALTER TABLE {table} ON CLUSTER alter_drop_partition DROP PARTITION ID 'all'", + f"ALTER TABLE {database}.alter_drop ON CLUSTER {database} DROP PARTITION ID 'all'", settings={"replication_alter_partitions_sync": 2}, ) assert ( main_node.query( - f"SELECT CounterID FROM clusterAllReplicas('alter_drop_partition', {table})" + f"SELECT CounterID FROM clusterAllReplicas('{database}', {database}.alter_drop)" ) == "" ) - assert dummy_node.query(f"SELECT CounterID FROM {table}") == "" - main_node.query("DROP DATABASE alter_drop_partition") - dummy_node.query("DROP DATABASE alter_drop_partition") - snapshotting_node.query("DROP DATABASE alter_drop_partition") + assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop") == "" + main_node.query(f"DROP DATABASE {database}") + dummy_node.query(f"DROP DATABASE {database}") + snapshotting_node.query(f"DROP DATABASE {database}") def test_alter_fetch(started_cluster): From 8b0fc8283460f5678e733cef9803937ff9913177 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 29 Apr 2023 21:23:55 +0000 Subject: [PATCH 0826/2047] test_for_basic_auth_registry - mock up --- .../runner/compose/docker_compose_kafka.yml | 10 +++++++++ tests/integration/helpers/cluster.py | 22 ++++++++++++------- .../secrets/password | 1 + .../secrets/schema_registry_jaas.conf | 5 +++++ .../test_format_avro_confluent/test.py | 18 ++++++++++----- 5 files changed, 43 insertions(+), 13 deletions(-) create mode 100644 tests/integration/test_format_avro_confluent/secrets/password create mode 100644 tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 7e34f4c114d..39247f1dd37 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -39,7 +39,17 @@ services: environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: BASIC SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC + # SCHEMA_REGISTRY_BASIC_AUTH_CREDENTIALS_SOURCE: USER_INFO + # SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein + # SCHEMA_REGISTRY_SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein + SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser + SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar + SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" + volumes: + - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: - kafka_zookeeper - kafka1 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0614cbf0e0d..9b5b33b1968 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1159,10 +1159,13 @@ class ClickHouseCluster: ] return self.base_kerberized_hdfs_cmd - def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): + def setup_kafka_cmd( + self, instance, env_variables, docker_compose_yml_dir + ): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) + env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) env_variables["SCHEMA_REGISTRY_INTERNAL_PORT"] = "8081" self.base_cmd.extend( @@ -1498,6 +1501,7 @@ class ClickHouseCluster: with_kafka=False, with_kerberized_kafka=False, with_kerberos_kdc=False, + with_secrets=False, with_rabbitmq=False, with_nats=False, clickhouse_path_dir=None, @@ -1604,6 +1608,7 @@ class ClickHouseCluster: with_nats=with_nats, with_nginx=with_nginx, with_kerberized_hdfs=with_kerberized_hdfs, + with_secrets=with_secrets or with_kerberized_hdfs or with_kerberos_kdc or with_kerberized_kafka, with_mongo=with_mongo or with_mongo_secure, with_meili=with_meili, with_redis=with_redis, @@ -3135,6 +3140,7 @@ class ClickHouseInstance: with_nats, with_nginx, with_kerberized_hdfs, + with_secrets, with_mongo, with_meili, with_redis, @@ -3197,7 +3203,7 @@ class ClickHouseInstance: if clickhouse_path_dir else None ) - self.kerberos_secrets_dir = p.abspath(p.join(base_path, "secrets")) + self.secrets_dir = p.abspath(p.join(base_path, "secrets")) self.macros = macros if macros is not None else {} self.with_zookeeper = with_zookeeper self.zookeeper_config_path = zookeeper_config_path @@ -3220,6 +3226,7 @@ class ClickHouseInstance: self.with_nats = with_nats self.with_nginx = with_nginx self.with_kerberized_hdfs = with_kerberized_hdfs + self.with_secrets = with_secrets self.with_mongo = with_mongo self.with_meili = with_meili self.with_redis = with_redis @@ -4217,17 +4224,16 @@ class ClickHouseInstance: if self.with_zookeeper: shutil.copy(self.zookeeper_config_path, conf_d_dir) - if ( - self.with_kerberized_kafka - or self.with_kerberized_hdfs - or self.with_kerberos_kdc - ): + if self.with_secrets: if self.with_kerberos_kdc: base_secrets_dir = self.cluster.instances_dir else: base_secrets_dir = self.path + from_dir=self.secrets_dir + to_dir=p.abspath(p.join(base_secrets_dir, "secrets")) + logging.debug(f"Copy secret from {from_dir} to {to_dir}") shutil.copytree( - self.kerberos_secrets_dir, + self.secrets_dir, p.abspath(p.join(base_secrets_dir, "secrets")), dirs_exist_ok=True, ) diff --git a/tests/integration/test_format_avro_confluent/secrets/password b/tests/integration/test_format_avro_confluent/secrets/password new file mode 100644 index 00000000000..8903cf6edd6 --- /dev/null +++ b/tests/integration/test_format_avro_confluent/secrets/password @@ -0,0 +1 @@ +schemauser: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user diff --git a/tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf b/tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf new file mode 100644 index 00000000000..7d0e6e2bf35 --- /dev/null +++ b/tests/integration/test_format_avro_confluent/secrets/schema_registry_jaas.conf @@ -0,0 +1,5 @@ +RealmFooBar { + org.eclipse.jetty.jaas.spi.PropertyFileLoginModule required + file="/etc/schema-registry/secrets/password" + debug="true"; +}; \ No newline at end of file diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 42b7ddce193..921fbdf5ef4 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -14,7 +14,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance def started_cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("dummy", with_kafka=True) + cluster.add_instance("dummy", with_kafka=True, with_secrets=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -39,10 +39,13 @@ def run_query(instance, query, data=None, settings=None): def test_select(started_cluster): # type: (ClickHouseCluster) -> None + input("Cluster created, press any key to destroy...") - schema_registry_client = CachedSchemaRegistryClient( - "http://localhost:{}".format(started_cluster.schema_registry_port) - ) + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_port) + arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + + schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object( @@ -62,7 +65,12 @@ def test_select(started_cluster): data = buf.getvalue() instance = started_cluster.instances["dummy"] # type: ClickHouseInstance - schema_registry_url = "http://{}:{}".format( + # schema_registry_url = "http://{}:{}@{}:{}".format( + # 'schemauser', 'letmein', + # started_cluster.schema_registry_host, 8081 + # ) + schema_registry_url = "http://{}:{}@{}:{}".format( + 'schemauser', 'letmein', started_cluster.schema_registry_host, 8081 ) From 1564eace38072417bf2c188d7c0a0c0e55321626 Mon Sep 17 00:00:00 2001 From: dheerajathrey Date: Thu, 7 Jul 2022 13:48:25 +0530 Subject: [PATCH 0827/2047] enable url-encoded basic auth to fetch avro schema in kafka --- .../Formats/Impl/AvroRowInputFormat.cpp | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 4cd73cb23b5..fe795608970 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -52,6 +52,8 @@ #include #include #include +#include +#include #include #include #include @@ -934,6 +936,29 @@ private: Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, url.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(url.getHost()); + if (!url.getUserInfo().empty()) { + Poco::Net::HTTPCredentials http_credentials; + Poco::Net::HTTPBasicCredentials http_basic_credentials; + std::string decoded_username; + std::string decoded_password; + + http_credentials.fromUserInfo(url.getUserInfo()); + + if (!http_credentials.getPassword().empty()) { + Poco::URI::decode(http_credentials.getUsername(), decoded_username); + Poco::URI::decode(http_credentials.getPassword(), decoded_password); + + http_basic_credentials.setUsername(decoded_username); + http_basic_credentials.setPassword(decoded_password); + } + else { + Poco::URI::decode(http_credentials.getUsername(), decoded_username); + http_basic_credentials.setUsername(decoded_username); + } + + http_basic_credentials.authenticate(request); + } + auto session = makePooledHTTPSession(url, timeouts, 1); session->sendRequest(request); From 8e1de7897a0f950a44b9c67b5d7d97b47d380f25 Mon Sep 17 00:00:00 2001 From: dheerajathrey Date: Wed, 24 Aug 2022 19:19:09 +0530 Subject: [PATCH 0828/2047] indentation fix --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index fe795608970..318ba3cb443 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -936,7 +936,8 @@ private: Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_GET, url.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(url.getHost()); - if (!url.getUserInfo().empty()) { + if (!url.getUserInfo().empty()) + { Poco::Net::HTTPCredentials http_credentials; Poco::Net::HTTPBasicCredentials http_basic_credentials; std::string decoded_username; @@ -944,14 +945,16 @@ private: http_credentials.fromUserInfo(url.getUserInfo()); - if (!http_credentials.getPassword().empty()) { + if (!http_credentials.getPassword().empty()) + { Poco::URI::decode(http_credentials.getUsername(), decoded_username); Poco::URI::decode(http_credentials.getPassword(), decoded_password); http_basic_credentials.setUsername(decoded_username); http_basic_credentials.setPassword(decoded_password); } - else { + else + { Poco::URI::decode(http_credentials.getUsername(), decoded_username); http_basic_credentials.setUsername(decoded_username); } From e3523cb1a463931513cb7f3edc9937d64ae82331 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 2 May 2023 12:31:00 +0000 Subject: [PATCH 0829/2047] test_for_basic_auth_registry - attempt to have two kafka instances --- .../runner/compose/docker_compose_kafka.yml | 44 +++++++++++-- tests/integration/helpers/cluster.py | 58 +++++++++++++++-- .../test_format_avro_confluent/test.py | 65 +++++++++++++++---- 3 files changed, 145 insertions(+), 22 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 39247f1dd37..fc476c09378 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -31,6 +31,28 @@ services: security_opt: - label:disable + kafka2: + image: confluentinc/cp-kafka:5.2.0 + hostname: kafka2 + ports: + - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} + environment: + # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} + # KAFKA_HOST: ${KAFKA2_HOST} + KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 + KAFKA_ADVERTISED_HOST_NAME: kafka2 + KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE + KAFKA_BROKER_ID: 2 + KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" + KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + depends_on: + - kafka_zookeeper + security_opt: + - label:disable + schema-registry: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry @@ -39,12 +61,24 @@ services: environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT - # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: BASIC SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + depends_on: + - kafka_zookeeper + - kafka1 + security_opt: + - label:disable + + schema-registry-auth: + image: confluentinc/cp-schema-registry:5.2.0 + hostname: schema-registry-auth + ports: + - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT:-12313}:${SCHEMA_REGISTRY_INTERNAL_PORT:-12313} + environment: + SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} + SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth + SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka2:19093 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC - # SCHEMA_REGISTRY_BASIC_AUTH_CREDENTIALS_SOURCE: USER_INFO - # SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein - # SCHEMA_REGISTRY_SCHEMA_REGISTRY_BASIC_AUTH_USER_INFO: fred:letmein SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" @@ -52,6 +86,6 @@ services: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: - kafka_zookeeper - - kafka1 + - kafka2 security_opt: - label:disable diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9b5b33b1968..e0286f6e5c9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -481,12 +481,18 @@ class ClickHouseCluster: # available when with_kafka == True self.kafka_host = "kafka1" + self.kafka2_host = "kafka2" self.kafka_dir = os.path.join(self.instances_dir, "kafka") self._kafka_port = 0 + self._kafka2_port = 0 self.kafka_docker_id = None + self.kafka2_docker_id = None self.schema_registry_host = "schema-registry" self._schema_registry_port = 0 + self.schema_registry_auth_host = "schema-registry-auth" + self._schema_registry_auth_port = 0 self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) + self.kafka2_docker_id = self.get_instance_docker_id(self.kafka2_host) self.coredns_host = "coredns" @@ -650,6 +656,13 @@ class ClickHouseCluster: self._kafka_port = get_free_port() return self._kafka_port + @property + def kafka2_port(self): + if self._kafka2_port: + return self._kafka2_port + self._kafka2_port = get_free_port() + return self._kafka2_port + @property def schema_registry_port(self): if self._schema_registry_port: @@ -657,6 +670,13 @@ class ClickHouseCluster: self._schema_registry_port = get_free_port() return self._schema_registry_port + @property + def schema_registry_auth_port(self): + if self._schema_registry_auth_port: + return self._schema_registry_auth_port + self._schema_registry_auth_port = get_free_port() + return self._schema_registry_auth_port + @property def kerberized_kafka_port(self): if self._kerberized_kafka_port: @@ -1164,10 +1184,13 @@ class ClickHouseCluster: ): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host + env_variables["KAFKA2_HOST"] = self.kafka2_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) + env_variables["KAFKA2_EXTERNAL_PORT"] = str(self.kafka2_port) env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) env_variables["SCHEMA_REGISTRY_INTERNAL_PORT"] = "8081" + env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str(self.schema_registry_auth_port) self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml")] ) @@ -2498,20 +2521,44 @@ class ClickHouseCluster: raise Exception("Can't wait Azurite to start") def wait_schema_registry_to_start(self, timeout=180): - sr_client = CachedSchemaRegistryClient( - {"url": "http://localhost:{}".format(self.schema_registry_port)} - ) + reg_url="http://localhost:{}".format(self.schema_registry_port) + arg={'url':reg_url} + sr_client = CachedSchemaRegistryClient(arg) + start = time.time() + sr_started = False + sr_auth_started = False while time.time() - start < timeout: try: sr_client._send_request(sr_client.url) logging.debug("Connected to SchemaRegistry") - return sr_client + sr_started = True + break except Exception as ex: logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) time.sleep(1) - raise Exception("Can't wait Schema Registry to start") + if not sr_started: + raise Exception("Can't wait Schema Registry to start") + + + auth_reg_url="http://localhost:{}".format(self.schema_registry_auth_port) + auth_arg={'url':auth_reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + + + sr_auth_client = CachedSchemaRegistryClient(auth_arg) + while time.time() - start < timeout: + try: + sr_auth_client._send_request(sr_auth_client.url) + logging.debug("Connected to SchemaRegistry with auth") + sr_auth_started = True + break + except Exception as ex: + logging.debug(("Can't connect to SchemaRegistry with auth: %s", str(ex))) + time.sleep(1) + + if not sr_auth_started: + raise Exception("Can't wait Schema Registry with auth to start") def wait_cassandra_to_start(self, timeout=180): self.cassandra_ip = self.get_instance_ip(self.cassandra_host) @@ -2718,6 +2765,7 @@ class ClickHouseCluster: ) self.up_called = True self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) + self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) self.wait_schema_registry_to_start() if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 921fbdf5ef4..7261ce1b97d 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -37,15 +37,18 @@ def run_query(instance, query, data=None, settings=None): return result + + # reg_url="http://localhost:{}".format(started_cluster.schema_registry_port) + # arg={'url':reg_url} + # schema_registry_client = CachedSchemaRegistryClient(arg) + + def test_select(started_cluster): # type: (ClickHouseCluster) -> None - input("Cluster created, press any key to destroy...") - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_port) - arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} - - schema_registry_client = CachedSchemaRegistryClient(arg) + schema_registry_client = CachedSchemaRegistryClient( + "http://localhost:{}".format(started_cluster.schema_registry_port) + ) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object( @@ -65,12 +68,7 @@ def test_select(started_cluster): data = buf.getvalue() instance = started_cluster.instances["dummy"] # type: ClickHouseInstance - # schema_registry_url = "http://{}:{}@{}:{}".format( - # 'schemauser', 'letmein', - # started_cluster.schema_registry_host, 8081 - # ) - schema_registry_url = "http://{}:{}@{}:{}".format( - 'schemauser', 'letmein', + schema_registry_url = "http://{}:{}".format( started_cluster.schema_registry_host, 8081 ) @@ -83,3 +81,46 @@ def test_select(started_cluster): ["1"], ["2"], ] + + +# def test_select_auth(started_cluster): +# # type: (ClickHouseCluster) -> None + +# reg_url="http://localhost:{}".format( +# started_cluster.schema_registry_auth_port) +# arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + +# schema_registry_client = CachedSchemaRegistryClient(arg) +# serializer = MessageSerializer(schema_registry_client) + +# schema = avro.schema.make_avsc_object( +# { +# "name": "test_record", +# "type": "record", +# "fields": [{"name": "value", "type": "long"}], +# } +# ) + +# buf = io.BytesIO() +# for x in range(0, 3): +# message = serializer.encode_record_with_schema( +# "test_subject", schema, {"value": x} +# ) +# buf.write(message) +# data = buf.getvalue() + +# instance = started_cluster.instances["dummy"] # type: ClickHouseInstance +# schema_registry_url = "http://{}:{}@{}:{}".format( +# 'schemauser', 'letmein', +# started_cluster.schema_registry_auth_host, 8081 +# ) + +# run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") +# settings = {"format_avro_schema_registry_url": schema_registry_url} +# run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) +# stdout = run_query(instance, "select * from avro_data_auth") +# assert list(map(str.split, stdout.splitlines())) == [ +# ["0"], +# ["1"], +# ["2"], +# ] From fb3a860d7f02ddf321875eefefeeaeb46b265bf9 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 5 May 2023 10:56:35 +0000 Subject: [PATCH 0830/2047] test_for_basic_auth_registry - one kafka instance again --- .../runner/compose/docker_compose_kafka.yml | 49 ++++++++++--------- tests/integration/helpers/cluster.py | 2 +- 2 files changed, 27 insertions(+), 24 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index fc476c09378..e0b58fee73d 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -31,27 +31,27 @@ services: security_opt: - label:disable - kafka2: - image: confluentinc/cp-kafka:5.2.0 - hostname: kafka2 - ports: - - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} - environment: - # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} - # KAFKA_HOST: ${KAFKA2_HOST} - KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 - KAFKA_ADVERTISED_HOST_NAME: kafka2 - KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT - KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE - KAFKA_BROKER_ID: 2 - KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" - KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" - KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 - depends_on: - - kafka_zookeeper - security_opt: - - label:disable + # kafka2: + # image: confluentinc/cp-kafka:5.2.0 + # hostname: kafka2 + # ports: + # - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} + # environment: + # # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} + # # KAFKA_HOST: ${KAFKA2_HOST} + # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 + # KAFKA_ADVERTISED_HOST_NAME: kafka2 + # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 + # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + # KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE + # KAFKA_BROKER_ID: 2 + # KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" + # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + # KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + # depends_on: + # - kafka_zookeeper + # security_opt: + # - label:disable schema-registry: image: confluentinc/cp-schema-registry:5.2.0 @@ -77,15 +77,18 @@ services: SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT - SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka2:19093 + SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" + SCHEMA_REGISTRY_GROUP_ID: auth + SCHEMA_REGISTRY_ZK_NAMESPACE: auth + SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemaauth volumes: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: - kafka_zookeeper - - kafka2 + - kafka1 security_opt: - label:disable diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e0286f6e5c9..e261364ab05 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2765,7 +2765,7 @@ class ClickHouseCluster: ) self.up_called = True self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) - self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) + # self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) self.wait_schema_registry_to_start() if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: From db8e96147a9deb92364c8276577dedf68b7653a5 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 5 May 2023 17:52:15 +0000 Subject: [PATCH 0831/2047] test_for_basic_auth_registry - started, but only auth test works --- .../runner/compose/docker_compose_kafka.yml | 94 +++++++++---------- .../test_format_avro_confluent/test.py | 27 ++++-- 2 files changed, 60 insertions(+), 61 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index e0b58fee73d..47e41812cf5 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -1,90 +1,80 @@ version: '2.3' services: + # kafka_zookeeper: + # image: zookeeper:3.4.9 + # hostname: kafka_zookeeper + # environment: + # ZOO_MY_ID: 1 + # ZOO_PORT: 2181 + # ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 + # security_opt: + # - label:disable kafka_zookeeper: - image: zookeeper:3.4.9 - hostname: kafka_zookeeper + image: confluentinc/cp-zookeeper + ports: + - 2181:2181 environment: - ZOO_MY_ID: 1 - ZOO_PORT: 2181 - ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 - security_opt: - - label:disable - + ZOOKEEPER_CLIENT_PORT: 2181 + # security_opt: + # - label:disable kafka1: - image: confluentinc/cp-kafka:5.2.0 + image: confluentinc/cp-kafka hostname: kafka1 ports: - - ${KAFKA_EXTERNAL_PORT:-8081}:${KAFKA_EXTERNAL_PORT:-8081} + - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: - KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA_EXTERNAL_PORT},OUTSIDE://kafka1:19092 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT, PLAINTEXT_HOST:PLAINTEXT + # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka1:19092, PLAINTEXT_HOST://localhost:${KAFKA_EXTERNAL_PORT} KAFKA_ADVERTISED_HOST_NAME: kafka1 - KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT - KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE - KAFKA_BROKER_ID: 1 - KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" - KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 + KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT + # KAFKA_BROKER_ID: 1 + KAFKA_ZOOKEEPER_CONNECT: kafka_zookeeper:2181 + # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 depends_on: - kafka_zookeeper - security_opt: - - label:disable - - # kafka2: - # image: confluentinc/cp-kafka:5.2.0 - # hostname: kafka2 - # ports: - # - ${KAFKA2_EXTERNAL_PORT:-8082}:${KAFKA2_EXTERNAL_PORT:-8082} - # environment: - # # KAFKA_EXTERNAL_PORT: ${KAFKA2_EXTERNAL_PORT} - # # KAFKA_HOST: ${KAFKA2_HOST} - # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA2_EXTERNAL_PORT},OUTSIDE://kafka2:19093 - # KAFKA_ADVERTISED_HOST_NAME: kafka2 - # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19093 - # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT - # KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE - # KAFKA_BROKER_ID: 2 - # KAFKA_ZOOKEEPER_CONNECT: "kafka_zookeeper:2181" - # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" - # KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 - # depends_on: - # - kafka_zookeeper - # security_opt: - # - label:disable + # security_opt: + # - label:disable schema-registry: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry ports: - - ${SCHEMA_REGISTRY_EXTERNAL_PORT:-12313}:${SCHEMA_REGISTRY_INTERNAL_PORT:-12313} + - ${SCHEMA_REGISTRY_EXTERNAL_PORT}:${SCHEMA_REGISTRY_EXTERNAL_PORT} environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry - SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 + SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_EXTERNAL_PORT:-12313} depends_on: - kafka_zookeeper - kafka1 - security_opt: - - label:disable + # security_opt: + # - label:disable schema-registry-auth: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry-auth ports: - - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT:-12313}:${SCHEMA_REGISTRY_INTERNAL_PORT:-12313} + - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT}:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} environment: - SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} + # SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth - SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT + SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} + # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 + SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC - SCHEMA_REGISTRY_AUTHENTICATION_ROLES: schemaadmin, schemauser + SCHEMA_REGISTRY_AUTHENTICATION_ROLES: user SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" - SCHEMA_REGISTRY_GROUP_ID: auth - SCHEMA_REGISTRY_ZK_NAMESPACE: auth - SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemaauth + # SCHEMA_REGISTRY_GROUP_ID: auth + SCHEMA_REGISTRY_ZK_NAMESPACE: schema_registry_auth + SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemas2 volumes: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 7261ce1b97d..cd0906bedee 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -1,5 +1,6 @@ import io import logging +import time import avro.schema import pytest @@ -46,14 +47,21 @@ def run_query(instance, query, data=None, settings=None): def test_select(started_cluster): # type: (ClickHouseCluster) -> None - schema_registry_client = CachedSchemaRegistryClient( - "http://localhost:{}".format(started_cluster.schema_registry_port) - ) + time.sleep(3) + + # schema_registry_client = CachedSchemaRegistryClient( + # "http://localhost:{}".format(started_cluster.schema_registry_port) + # ) + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_port) + arg={'url':reg_url} + + schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object( { - "name": "test_record", + "name": "test_record1", "type": "record", "fields": [{"name": "value", "type": "long"}], } @@ -62,14 +70,14 @@ def test_select(started_cluster): buf = io.BytesIO() for x in range(0, 3): message = serializer.encode_record_with_schema( - "test_subject", schema, {"value": x} + "test_subject1", schema, {"value": x} ) buf.write(message) data = buf.getvalue() instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}".format( - started_cluster.schema_registry_host, 8081 + started_cluster.schema_registry_host, started_cluster.schema_registry_port ) run_query(instance, "create table avro_data(value Int64) engine = Memory()") @@ -85,6 +93,7 @@ def test_select(started_cluster): # def test_select_auth(started_cluster): # # type: (ClickHouseCluster) -> None +# time.sleep(5) # reg_url="http://localhost:{}".format( # started_cluster.schema_registry_auth_port) @@ -95,7 +104,7 @@ def test_select(started_cluster): # schema = avro.schema.make_avsc_object( # { -# "name": "test_record", +# "name": "test_record_auth", # "type": "record", # "fields": [{"name": "value", "type": "long"}], # } @@ -104,7 +113,7 @@ def test_select(started_cluster): # buf = io.BytesIO() # for x in range(0, 3): # message = serializer.encode_record_with_schema( -# "test_subject", schema, {"value": x} +# "test_subject_auth", schema, {"value": x} # ) # buf.write(message) # data = buf.getvalue() @@ -112,7 +121,7 @@ def test_select(started_cluster): # instance = started_cluster.instances["dummy"] # type: ClickHouseInstance # schema_registry_url = "http://{}:{}@{}:{}".format( # 'schemauser', 'letmein', -# started_cluster.schema_registry_auth_host, 8081 +# started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port # ) # run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") From 66581d091af3eda08591e12af551e83a88a95520 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 07:08:04 +0000 Subject: [PATCH 0832/2047] test_for_basic_auth_registry - both tests works, simplifications --- .../runner/compose/docker_compose_kafka.yml | 46 +++---- tests/integration/helpers/cluster.py | 70 +++------- .../secrets/password | 1 + .../test_format_avro_confluent/test.py | 122 ++++++++++++------ 4 files changed, 116 insertions(+), 123 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 47e41812cf5..5e2e9d87c39 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -1,43 +1,33 @@ version: '2.3' services: - # kafka_zookeeper: - # image: zookeeper:3.4.9 - # hostname: kafka_zookeeper - # environment: - # ZOO_MY_ID: 1 - # ZOO_PORT: 2181 - # ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 - # security_opt: - # - label:disable kafka_zookeeper: - image: confluentinc/cp-zookeeper + image: zookeeper:3.4.9 + hostname: kafka_zookeeper ports: - 2181:2181 environment: - ZOOKEEPER_CLIENT_PORT: 2181 - # security_opt: - # - label:disable + ZOOKEEPER_CLIENT_PORT: 2181 + security_opt: + - label:disable + kafka1: - image: confluentinc/cp-kafka + image: confluentinc/cp-kafka:5.2.0 hostname: kafka1 ports: - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT, PLAINTEXT_HOST:PLAINTEXT - # KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka1:19092, PLAINTEXT_HOST://localhost:${KAFKA_EXTERNAL_PORT} KAFKA_ADVERTISED_HOST_NAME: kafka1 - # KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT - # KAFKA_BROKER_ID: 1 KAFKA_ZOOKEEPER_CONNECT: kafka_zookeeper:2181 - # KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 depends_on: - kafka_zookeeper - # security_opt: - # - label:disable + security_opt: + - label:disable schema-registry: image: confluentinc/cp-schema-registry:5.2.0 @@ -46,15 +36,14 @@ services: - ${SCHEMA_REGISTRY_EXTERNAL_PORT}:${SCHEMA_REGISTRY_EXTERNAL_PORT} environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry - # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 - SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 - SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_EXTERNAL_PORT:-12313} + SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_EXTERNAL_PORT} + SCHEMA_REGISTRY_SCHEMA_REGISTRY_GROUP_ID: noauth depends_on: - kafka_zookeeper - kafka1 - # security_opt: - # - label:disable + security_opt: + - label:disable schema-registry-auth: image: confluentinc/cp-schema-registry:5.2.0 @@ -62,19 +51,14 @@ services: ports: - ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT}:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} environment: - # SCHEMA_REGISTRY_EXTERNAL_PORT: ${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} SCHEMA_REGISTRY_HOST_NAME: schema-registry-auth SCHEMA_REGISTRY_LISTENERS: http://0.0.0.0:${SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT} - # SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: PLAINTEXT://kafka1:19092 - SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: kafka_zookeeper:2181 SCHEMA_REGISTRY_AUTHENTICATION_METHOD: BASIC SCHEMA_REGISTRY_AUTHENTICATION_ROLES: user SCHEMA_REGISTRY_AUTHENTICATION_REALM: RealmFooBar SCHEMA_REGISTRY_OPTS: "-Djava.security.auth.login.config=/etc/schema-registry/secrets/schema_registry_jaas.conf" - # SCHEMA_REGISTRY_GROUP_ID: auth - SCHEMA_REGISTRY_ZK_NAMESPACE: schema_registry_auth - SCHEMA_REGISTRY_KAFKASTORE_TOPIC: _schemas2 + SCHEMA_REGISTRY_SCHEMA_REGISTRY_GROUP_ID: auth volumes: - ${SCHEMA_REGISTRY_DIR:-}/secrets:/etc/schema-registry/secrets depends_on: diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e261364ab05..c51c97ee6c4 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -481,18 +481,14 @@ class ClickHouseCluster: # available when with_kafka == True self.kafka_host = "kafka1" - self.kafka2_host = "kafka2" self.kafka_dir = os.path.join(self.instances_dir, "kafka") self._kafka_port = 0 - self._kafka2_port = 0 self.kafka_docker_id = None - self.kafka2_docker_id = None self.schema_registry_host = "schema-registry" self._schema_registry_port = 0 self.schema_registry_auth_host = "schema-registry-auth" self._schema_registry_auth_port = 0 self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) - self.kafka2_docker_id = self.get_instance_docker_id(self.kafka2_host) self.coredns_host = "coredns" @@ -656,13 +652,6 @@ class ClickHouseCluster: self._kafka_port = get_free_port() return self._kafka_port - @property - def kafka2_port(self): - if self._kafka2_port: - return self._kafka2_port - self._kafka2_port = get_free_port() - return self._kafka2_port - @property def schema_registry_port(self): if self._schema_registry_port: @@ -1184,12 +1173,9 @@ class ClickHouseCluster: ): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host - env_variables["KAFKA2_HOST"] = self.kafka2_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) - env_variables["KAFKA2_EXTERNAL_PORT"] = str(self.kafka2_port) env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) - env_variables["SCHEMA_REGISTRY_INTERNAL_PORT"] = "8081" env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str(self.schema_registry_auth_port) self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml")] @@ -2521,44 +2507,27 @@ class ClickHouseCluster: raise Exception("Can't wait Azurite to start") def wait_schema_registry_to_start(self, timeout=180): - reg_url="http://localhost:{}".format(self.schema_registry_port) - arg={'url':reg_url} - sr_client = CachedSchemaRegistryClient(arg) + for port in self.schema_registry_port, self.schema_registry_auth_port: + reg_url="http://localhost:{}".format(port) + arg={'url':reg_url} + sr_client = CachedSchemaRegistryClient(arg) - start = time.time() - sr_started = False - sr_auth_started = False - while time.time() - start < timeout: - try: - sr_client._send_request(sr_client.url) - logging.debug("Connected to SchemaRegistry") - sr_started = True - break - except Exception as ex: - logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) - time.sleep(1) + start = time.time() + sr_started = False + sr_auth_started = False + while time.time() - start < timeout: + try: + sr_client._send_request(sr_client.url) + logging.debug("Connected to SchemaRegistry") + # don't care about possible auth errors + sr_started = True + break + except Exception as ex: + logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) + time.sleep(1) - if not sr_started: - raise Exception("Can't wait Schema Registry to start") - - - auth_reg_url="http://localhost:{}".format(self.schema_registry_auth_port) - auth_arg={'url':auth_reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} - - - sr_auth_client = CachedSchemaRegistryClient(auth_arg) - while time.time() - start < timeout: - try: - sr_auth_client._send_request(sr_auth_client.url) - logging.debug("Connected to SchemaRegistry with auth") - sr_auth_started = True - break - except Exception as ex: - logging.debug(("Can't connect to SchemaRegistry with auth: %s", str(ex))) - time.sleep(1) - - if not sr_auth_started: - raise Exception("Can't wait Schema Registry with auth to start") + if not sr_started: + raise Exception("Can't wait Schema Registry to start") def wait_cassandra_to_start(self, timeout=180): self.cassandra_ip = self.get_instance_ip(self.cassandra_host) @@ -2765,7 +2734,6 @@ class ClickHouseCluster: ) self.up_called = True self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) - # self.wait_kafka_is_available(self.kafka2_docker_id, self.kafka2_port) self.wait_schema_registry_to_start() if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: diff --git a/tests/integration/test_format_avro_confluent/secrets/password b/tests/integration/test_format_avro_confluent/secrets/password index 8903cf6edd6..7fde510bf5a 100644 --- a/tests/integration/test_format_avro_confluent/secrets/password +++ b/tests/integration/test_format_avro_confluent/secrets/password @@ -1 +1,2 @@ schemauser: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user +schemauser/slash: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index cd0906bedee..d58f6d972d1 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -9,7 +9,7 @@ from confluent_kafka.avro.cached_schema_registry_client import ( ) from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance - +from urllib import parse @pytest.fixture(scope="module") def started_cluster(): @@ -47,11 +47,8 @@ def run_query(instance, query, data=None, settings=None): def test_select(started_cluster): # type: (ClickHouseCluster) -> None - time.sleep(3) + # input("Top of test_select, press any key") - # schema_registry_client = CachedSchemaRegistryClient( - # "http://localhost:{}".format(started_cluster.schema_registry_port) - # ) reg_url="http://localhost:{}".format( started_cluster.schema_registry_port) arg={'url':reg_url} @@ -91,45 +88,88 @@ def test_select(started_cluster): ] -# def test_select_auth(started_cluster): -# # type: (ClickHouseCluster) -> None -# time.sleep(5) +def test_select_auth(started_cluster): + # type: (ClickHouseCluster) -> None + time.sleep(5) -# reg_url="http://localhost:{}".format( -# started_cluster.schema_registry_auth_port) -# arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_auth_port) + arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} -# schema_registry_client = CachedSchemaRegistryClient(arg) -# serializer = MessageSerializer(schema_registry_client) + schema_registry_client = CachedSchemaRegistryClient(arg) + serializer = MessageSerializer(schema_registry_client) -# schema = avro.schema.make_avsc_object( -# { -# "name": "test_record_auth", -# "type": "record", -# "fields": [{"name": "value", "type": "long"}], -# } -# ) + schema = avro.schema.make_avsc_object( + { + "name": "test_record_auth", + "type": "record", + "fields": [{"name": "value", "type": "long"}], + } + ) -# buf = io.BytesIO() -# for x in range(0, 3): -# message = serializer.encode_record_with_schema( -# "test_subject_auth", schema, {"value": x} -# ) -# buf.write(message) -# data = buf.getvalue() + buf = io.BytesIO() + for x in range(0, 3): + message = serializer.encode_record_with_schema( + "test_subject_auth", schema, {"value": x} + ) + buf.write(message) + data = buf.getvalue() -# instance = started_cluster.instances["dummy"] # type: ClickHouseInstance -# schema_registry_url = "http://{}:{}@{}:{}".format( -# 'schemauser', 'letmein', -# started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port -# ) + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + schema_registry_url = "http://{}:{}@{}:{}".format( + 'schemauser', 'letmein', + started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + ) -# run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") -# settings = {"format_avro_schema_registry_url": schema_registry_url} -# run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) -# stdout = run_query(instance, "select * from avro_data_auth") -# assert list(map(str.split, stdout.splitlines())) == [ -# ["0"], -# ["1"], -# ["2"], -# ] + run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") + settings = {"format_avro_schema_registry_url": schema_registry_url} + run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) + stdout = run_query(instance, "select * from avro_data_auth") + assert list(map(str.split, stdout.splitlines())) == [ + ["0"], + ["1"], + ["2"], + ] + +def test_select_auth_encoded(started_cluster): + # type: (ClickHouseCluster) -> None + time.sleep(5) + + reg_url="http://localhost:{}".format( + started_cluster.schema_registry_auth_port) + arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + + schema_registry_client = CachedSchemaRegistryClient(arg) + serializer = MessageSerializer(schema_registry_client) + + schema = avro.schema.make_avsc_object( + { + "name": "test_record_auth_encoded", + "type": "record", + "fields": [{"name": "value", "type": "long"}], + } + ) + + buf = io.BytesIO() + for x in range(0, 3): + message = serializer.encode_record_with_schema( + "test_subject_auth_encoded", schema, {"value": x} + ) + buf.write(message) + data = buf.getvalue() + + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + schema_registry_url = "http://{}:{}@{}:{}".format( + parse.quote_plus('schemauser/slash'), parse.quote_plus('letmein'), + started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + ) + + run_query(instance, "create table avro_data_auth_encoded(value Int64) engine = Memory()") + settings = {"format_avro_schema_registry_url": schema_registry_url} + run_query(instance, "insert into avro_data_auth_encoded format AvroConfluent", data, settings) + stdout = run_query(instance, "select * from avro_data_auth_encoded") + assert list(map(str.split, stdout.splitlines())) == [ + ["0"], + ["1"], + ["2"], + ] From 83569688cba16b80ed959c054fd2f36187c520d4 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 12:59:08 +0000 Subject: [PATCH 0833/2047] test_for_basic_auth_registry - UnknownTopicOrPartitionException --- docker/test/integration/runner/compose/docker_compose_kafka.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 5e2e9d87c39..d701af1d425 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -42,6 +42,7 @@ services: depends_on: - kafka_zookeeper - kafka1 + restart: always security_opt: - label:disable @@ -64,5 +65,6 @@ services: depends_on: - kafka_zookeeper - kafka1 + restart: always security_opt: - label:disable From c1c5ffa309c20899f81548bd3314233d84eb03e1 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 13:29:19 +0000 Subject: [PATCH 0834/2047] test_for_basic_auth_registry - cpp code small improvement --- .../Formats/Impl/AvroRowInputFormat.cpp | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 318ba3cb443..a7efc823fbb 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -940,24 +940,19 @@ private: { Poco::Net::HTTPCredentials http_credentials; Poco::Net::HTTPBasicCredentials http_basic_credentials; - std::string decoded_username; - std::string decoded_password; http_credentials.fromUserInfo(url.getUserInfo()); + std::string decoded_username; + Poco::URI::decode(http_credentials.getUsername(), decoded_username); + http_basic_credentials.setUsername(decoded_username); + if (!http_credentials.getPassword().empty()) { - Poco::URI::decode(http_credentials.getUsername(), decoded_username); + std::string decoded_password; Poco::URI::decode(http_credentials.getPassword(), decoded_password); - - http_basic_credentials.setUsername(decoded_username); http_basic_credentials.setPassword(decoded_password); } - else - { - Poco::URI::decode(http_credentials.getUsername(), decoded_username); - http_basic_credentials.setUsername(decoded_username); - } http_basic_credentials.authenticate(request); } From c550a532e649bfde1382bc7f56cac6a38dee0dee Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 8 May 2023 20:49:43 +0000 Subject: [PATCH 0835/2047] test_for_basic_auth_registry - black formatter happy + some doc --- .../operations/settings/settings-formats.md | 11 ++++ tests/integration/helpers/cluster.py | 21 ++++---- .../test_format_avro_confluent/test.py | 54 ++++++++++++------- 3 files changed, 59 insertions(+), 27 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 0915c51806a..637ade17296 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1325,6 +1325,17 @@ Default value: 0. Sets [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html) URL to use with [AvroConfluent](../../interfaces/formats.md/#data-format-avro-confluent) format. +Format: +``` text +http://[user:password@]machine[:port]" +``` + +Examples: +``` text +http://registry.example.com:8081 +http://admin:secret@registry.example.com:8081 +``` + Default value: `Empty`. ### output_format_avro_codec {#output_format_avro_codec} diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c51c97ee6c4..c52442ecb9c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1168,15 +1168,15 @@ class ClickHouseCluster: ] return self.base_kerberized_hdfs_cmd - def setup_kafka_cmd( - self, instance, env_variables, docker_compose_yml_dir - ): + def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_kafka = True env_variables["KAFKA_HOST"] = self.kafka_host env_variables["KAFKA_EXTERNAL_PORT"] = str(self.kafka_port) env_variables["SCHEMA_REGISTRY_DIR"] = instance.path + "/" env_variables["SCHEMA_REGISTRY_EXTERNAL_PORT"] = str(self.schema_registry_port) - env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str(self.schema_registry_auth_port) + env_variables["SCHEMA_REGISTRY_AUTH_EXTERNAL_PORT"] = str( + self.schema_registry_auth_port + ) self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml")] ) @@ -1617,7 +1617,10 @@ class ClickHouseCluster: with_nats=with_nats, with_nginx=with_nginx, with_kerberized_hdfs=with_kerberized_hdfs, - with_secrets=with_secrets or with_kerberized_hdfs or with_kerberos_kdc or with_kerberized_kafka, + with_secrets=with_secrets + or with_kerberized_hdfs + or with_kerberos_kdc + or with_kerberized_kafka, with_mongo=with_mongo or with_mongo_secure, with_meili=with_meili, with_redis=with_redis, @@ -2508,8 +2511,8 @@ class ClickHouseCluster: def wait_schema_registry_to_start(self, timeout=180): for port in self.schema_registry_port, self.schema_registry_auth_port: - reg_url="http://localhost:{}".format(port) - arg={'url':reg_url} + reg_url = "http://localhost:{}".format(port) + arg = {"url": reg_url} sr_client = CachedSchemaRegistryClient(arg) start = time.time() @@ -4245,8 +4248,8 @@ class ClickHouseInstance: base_secrets_dir = self.cluster.instances_dir else: base_secrets_dir = self.path - from_dir=self.secrets_dir - to_dir=p.abspath(p.join(base_secrets_dir, "secrets")) + from_dir = self.secrets_dir + to_dir = p.abspath(p.join(base_secrets_dir, "secrets")) logging.debug(f"Copy secret from {from_dir} to {to_dir}") shutil.copytree( self.secrets_dir, diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index d58f6d972d1..61d839ee63e 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -11,6 +11,7 @@ from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance from urllib import parse + @pytest.fixture(scope="module") def started_cluster(): try: @@ -37,8 +38,6 @@ def run_query(instance, query, data=None, settings=None): return result - - # reg_url="http://localhost:{}".format(started_cluster.schema_registry_port) # arg={'url':reg_url} # schema_registry_client = CachedSchemaRegistryClient(arg) @@ -49,9 +48,8 @@ def test_select(started_cluster): # input("Top of test_select, press any key") - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_port) - arg={'url':reg_url} + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_port) + arg = {"url": reg_url} schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) @@ -92,9 +90,12 @@ def test_select_auth(started_cluster): # type: (ClickHouseCluster) -> None time.sleep(5) - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_auth_port) - arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) + arg = { + "url": reg_url, + "basic.auth.credentials.source": "USER_INFO", + "basic.auth.user.info": "schemauser:letmein", + } schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) @@ -117,13 +118,17 @@ def test_select_auth(started_cluster): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}@{}:{}".format( - 'schemauser', 'letmein', - started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + "schemauser", + "letmein", + started_cluster.schema_registry_auth_host, + started_cluster.schema_registry_auth_port, ) run_query(instance, "create table avro_data_auth(value Int64) engine = Memory()") settings = {"format_avro_schema_registry_url": schema_registry_url} - run_query(instance, "insert into avro_data_auth format AvroConfluent", data, settings) + run_query( + instance, "insert into avro_data_auth format AvroConfluent", data, settings + ) stdout = run_query(instance, "select * from avro_data_auth") assert list(map(str.split, stdout.splitlines())) == [ ["0"], @@ -131,13 +136,17 @@ def test_select_auth(started_cluster): ["2"], ] + def test_select_auth_encoded(started_cluster): # type: (ClickHouseCluster) -> None time.sleep(5) - reg_url="http://localhost:{}".format( - started_cluster.schema_registry_auth_port) - arg={'url':reg_url,'basic.auth.credentials.source':'USER_INFO','basic.auth.user.info':'schemauser:letmein'} + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) + arg = { + "url": reg_url, + "basic.auth.credentials.source": "USER_INFO", + "basic.auth.user.info": "schemauser:letmein", + } schema_registry_client = CachedSchemaRegistryClient(arg) serializer = MessageSerializer(schema_registry_client) @@ -160,13 +169,22 @@ def test_select_auth_encoded(started_cluster): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}@{}:{}".format( - parse.quote_plus('schemauser/slash'), parse.quote_plus('letmein'), - started_cluster.schema_registry_auth_host, started_cluster.schema_registry_auth_port + parse.quote_plus("schemauser/slash"), + parse.quote_plus("letmein"), + started_cluster.schema_registry_auth_host, + started_cluster.schema_registry_auth_port, ) - run_query(instance, "create table avro_data_auth_encoded(value Int64) engine = Memory()") + run_query( + instance, "create table avro_data_auth_encoded(value Int64) engine = Memory()" + ) settings = {"format_avro_schema_registry_url": schema_registry_url} - run_query(instance, "insert into avro_data_auth_encoded format AvroConfluent", data, settings) + run_query( + instance, + "insert into avro_data_auth_encoded format AvroConfluent", + data, + settings, + ) stdout = run_query(instance, "select * from avro_data_auth_encoded") assert list(map(str.split, stdout.splitlines())) == [ ["0"], From aa2b7e248d719ed94386e5cf066ed03fe71fef12 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 9 May 2023 08:37:46 +0000 Subject: [PATCH 0836/2047] test_for_basic_auth_registry - fix port in test_kafka_formats --- .../runner/compose/docker_compose_kafka.yml | 7 ++++--- tests/integration/test_storage_kafka/test.py | 14 +++++++------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index d701af1d425..c0185afb7df 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -17,10 +17,11 @@ services: ports: - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: PLAINTEXT:PLAINTEXT, PLAINTEXT_HOST:PLAINTEXT - KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka1:19092, PLAINTEXT_HOST://localhost:${KAFKA_EXTERNAL_PORT} + KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA_EXTERNAL_PORT},OUTSIDE://kafka1:19092 KAFKA_ADVERTISED_HOST_NAME: kafka1 - KAFKA_INTER_BROKER_LISTENER_NAME: PLAINTEXT + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE + KAFKA_BROKER_ID: 1 KAFKA_ZOOKEEPER_CONNECT: kafka_zookeeper:2181 KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 9a6d3e0513c..d0686c7c36f 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -762,7 +762,7 @@ def test_kafka_formats(kafka_cluster): ), ], "extra_settings": ", format_avro_schema_registry_url='http://{}:{}'".format( - kafka_cluster.schema_registry_host, 8081 + kafka_cluster.schema_registry_host, kafka_cluster.schema_registry_port ), "supports_empty_value": True, }, @@ -4339,7 +4339,7 @@ def test_row_based_formats(kafka_cluster): f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.kafka; - + CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', @@ -4347,10 +4347,10 @@ def test_row_based_formats(kafka_cluster): kafka_group_name = '{format_name}', kafka_format = '{format_name}', kafka_max_rows_per_message = 5; - + CREATE MATERIALIZED VIEW test.view Engine=Log AS SELECT key, value FROM test.kafka; - + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}); """ ) @@ -4459,17 +4459,17 @@ def test_block_based_formats_2(kafka_cluster): f""" DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.kafka; - + CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = '{format_name}', kafka_group_name = '{format_name}', kafka_format = '{format_name}'; - + CREATE MATERIALIZED VIEW test.view Engine=Log AS SELECT key, value FROM test.kafka; - + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0; """ ) From 4259176f24b223decafd0d07bef430a30844e850 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 9 May 2023 09:23:28 +0000 Subject: [PATCH 0837/2047] test_for_basic_auth_registry - original zk configuration restored --- .../test/integration/runner/compose/docker_compose_kafka.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index c0185afb7df..30d1b0bed3f 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -7,7 +7,9 @@ services: ports: - 2181:2181 environment: - ZOOKEEPER_CLIENT_PORT: 2181 + ZOO_MY_ID: 1 + ZOO_PORT: 2181 + ZOO_SERVERS: server.1=kafka_zookeeper:2888:3888 security_opt: - label:disable From f1ce1da00744f17c42d94f1736417474eba478fe Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 18 May 2023 23:21:29 +0000 Subject: [PATCH 0838/2047] test_for_basic_auth_registry - new test and cleanup per code review --- .../secrets/password | 1 + .../test_format_avro_confluent/test.py | 56 ++++++++++++++++++- 2 files changed, 55 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_format_avro_confluent/secrets/password b/tests/integration/test_format_avro_confluent/secrets/password index 7fde510bf5a..a367925c806 100644 --- a/tests/integration/test_format_avro_confluent/secrets/password +++ b/tests/integration/test_format_avro_confluent/secrets/password @@ -1,2 +1,3 @@ schemauser: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user schemauser/slash: MD5:0d107d09f5bbe40cade3de5c71e9e9b7,user +complexschemauser: MD5:fcaeda86837fcd37755044e7258edc5d,user diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 61d839ee63e..2d78668f000 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -88,7 +88,6 @@ def test_select(started_cluster): def test_select_auth(started_cluster): # type: (ClickHouseCluster) -> None - time.sleep(5) reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) arg = { @@ -139,7 +138,6 @@ def test_select_auth(started_cluster): def test_select_auth_encoded(started_cluster): # type: (ClickHouseCluster) -> None - time.sleep(5) reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) arg = { @@ -191,3 +189,57 @@ def test_select_auth_encoded(started_cluster): ["1"], ["2"], ] + +def test_select_auth_encoded_complex(started_cluster): + # type: (ClickHouseCluster) -> None + + reg_url = "http://localhost:{}".format(started_cluster.schema_registry_auth_port) + arg = { + "url": reg_url, + "basic.auth.credentials.source": "USER_INFO", + "basic.auth.user.info": "schemauser:letmein", + } + + schema_registry_client = CachedSchemaRegistryClient(arg) + serializer = MessageSerializer(schema_registry_client) + + schema = avro.schema.make_avsc_object( + { + "name": "test_record_auth_encoded_complex", + "type": "record", + "fields": [{"name": "value", "type": "long"}], + } + ) + + buf = io.BytesIO() + for x in range(0, 3): + message = serializer.encode_record_with_schema( + "test_subject_auth_encoded_complex", schema, {"value": x} + ) + buf.write(message) + data = buf.getvalue() + + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + schema_registry_url = "http://{}:{}@{}:{}".format( + parse.quote_plus("complexschemauser"), + parse.quote_plus("letmein%@:/"), + started_cluster.schema_registry_auth_host, + started_cluster.schema_registry_auth_port, + ) + + run_query( + instance, "create table avro_data_auth_encoded_complex(value Int64) engine = Memory()" + ) + settings = {"format_avro_schema_registry_url": schema_registry_url} + run_query( + instance, + "insert into avro_data_auth_encoded_complex format AvroConfluent", + data, + settings, + ) + stdout = run_query(instance, "select * from avro_data_auth_encoded_complex") + assert list(map(str.split, stdout.splitlines())) == [ + ["0"], + ["1"], + ["2"], + ] From 9f6ab5e816378dce815957e396cf4389986256e4 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 19 May 2023 09:31:24 +0000 Subject: [PATCH 0839/2047] test_for_basic_auth_registry - made black formatter happy --- tests/integration/test_format_avro_confluent/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 2d78668f000..b27642c921b 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -190,6 +190,7 @@ def test_select_auth_encoded(started_cluster): ["2"], ] + def test_select_auth_encoded_complex(started_cluster): # type: (ClickHouseCluster) -> None @@ -228,7 +229,8 @@ def test_select_auth_encoded_complex(started_cluster): ) run_query( - instance, "create table avro_data_auth_encoded_complex(value Int64) engine = Memory()" + instance, + "create table avro_data_auth_encoded_complex(value Int64) engine = Memory()", ) settings = {"format_avro_schema_registry_url": schema_registry_url} run_query( From c8347bd31300bec4cdd3277680f398808d37533c Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 14 Jun 2023 15:55:44 +0000 Subject: [PATCH 0840/2047] test_for_basic_auth_registry: some comments removed per code review --- tests/integration/test_format_avro_confluent/test.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index b27642c921b..540f90ae05e 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -38,16 +38,10 @@ def run_query(instance, query, data=None, settings=None): return result - # reg_url="http://localhost:{}".format(started_cluster.schema_registry_port) - # arg={'url':reg_url} - # schema_registry_client = CachedSchemaRegistryClient(arg) - def test_select(started_cluster): # type: (ClickHouseCluster) -> None - # input("Top of test_select, press any key") - reg_url = "http://localhost:{}".format(started_cluster.schema_registry_port) arg = {"url": reg_url} From 2d46052d62bf8b7efd16aeb769e278b9df54971b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 08:35:46 +0000 Subject: [PATCH 0841/2047] Update description of events "QueryCacheHits/Misses" --- src/Common/ProfileEvents.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0838e0366df..75d1e493873 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -57,8 +57,8 @@ M(TableFunctionExecute, "Number of table function calls.") \ M(MarkCacheHits, "Number of times an entry has been found in the mark cache, so we didn't have to load a mark file.") \ M(MarkCacheMisses, "Number of times an entry has not been found in the mark cache, so we had to load a mark file in memory, which is a costly operation, adding to query latency.") \ - M(QueryCacheHits, "Number of times a query result has been found in the query cache (and query computation was avoided).") \ - M(QueryCacheMisses, "Number of times a query result has not been found in the query cache (and required query computation).") \ + M(QueryCacheHits, "Number of times a query result has been found in the query cache (and query computation was avoided). Only updated for SELECT queries with SETTING use_query_cache = 1.") \ + M(QueryCacheMisses, "Number of times a query result has not been found in the query cache (and required query computation). Only updated for SELECT queries with SETTING use_query_cache = 1.") \ M(CreatedReadBufferOrdinary, "Number of times ordinary read buffer was created for reading data (while choosing among other read methods).") \ M(CreatedReadBufferDirectIO, "Number of times a read buffer with O_DIRECT was created for reading data (while choosing among other read methods).") \ M(CreatedReadBufferDirectIOFailed, "Number of times a read buffer with O_DIRECT was attempted to be created for reading data (while choosing among other read methods), but the OS did not allow it (due to lack of filesystem support or other reasons) and we fallen back to the ordinary reading method.") \ From aa888ad64a95ef801977844b2b253bb8162cfc1a Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 08:46:57 +0000 Subject: [PATCH 0842/2047] Separate thread mutex, add test --- src/Common/SystemLogBase.cpp | 2 +- src/Common/SystemLogBase.h | 3 ++- src/Interpreters/SystemLog.cpp | 2 +- src/Interpreters/SystemLog.h | 1 + tests/queries/0_stateless/02813_starting_in_text_log.reference | 1 + tests/queries/0_stateless/02813_starting_in_text_log.sql | 2 ++ 6 files changed, 8 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02813_starting_in_text_log.reference create mode 100755 tests/queries/0_stateless/02813_starting_in_text_log.sql diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index baee7021c35..bed6d661db7 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -214,7 +214,7 @@ SystemLogBase::SystemLogBase( template void SystemLogBase::startup() { - std::lock_guard lock(queue->mutex); + std::lock_guard lock(thread_mutex); saving_thread = std::make_unique([this] { savingThreadFunction(); }); } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index fa9f9b6f72e..0ac376769ad 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -67,6 +67,7 @@ public: virtual void savingThreadFunction() = 0; protected: + std::mutex thread_mutex; std::unique_ptr saving_thread; bool is_shutdown = false; @@ -93,10 +94,10 @@ public: Index pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread); void confirm(Index to_flush_end); +private: /// Data shared between callers of add()/flush()/shutdown(), and the saving thread std::mutex mutex; -private: Poco::Logger * log; // Queue is bounded. But its size is quite large to not block in all normal cases. diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 674210cbaad..0b89b1dec26 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -358,7 +358,7 @@ template void SystemLog::stopFlushThread() { { - std::lock_guard lock(queue->mutex); + std::lock_guard lock(thread_mutex); if (!saving_thread || !saving_thread->joinable()) return; diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 91fb7f49221..5d8bb30150d 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -124,6 +124,7 @@ protected: using ISystemLog::is_shutdown; using ISystemLog::saving_thread; + using ISystemLog::thread_mutex; using Base::queue; private: diff --git a/tests/queries/0_stateless/02813_starting_in_text_log.reference b/tests/queries/0_stateless/02813_starting_in_text_log.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02813_starting_in_text_log.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02813_starting_in_text_log.sql b/tests/queries/0_stateless/02813_starting_in_text_log.sql new file mode 100755 index 00000000000..8ef78945a72 --- /dev/null +++ b/tests/queries/0_stateless/02813_starting_in_text_log.sql @@ -0,0 +1,2 @@ +SYSTEM FLUSH LOGS; +SELECT count() > 0 FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%Application: Starting ClickHouse%'; From 70543e8ef9fe8523c5604d62fac3376da91c6d2c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 19 Jul 2023 08:47:53 +0000 Subject: [PATCH 0843/2047] Automatic style fix --- .../test_replicated_database/test.py | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 17dd2adcde4..ed034a326da 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -302,12 +302,21 @@ def test_alter_attach(started_cluster, attachable_part, engine): ) main_node.query(f"ALTER TABLE {database}.alter_attach_test ATTACH PART 'all_1_1_0'") # On the main node, data is attached - assert main_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" + assert ( + main_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") + == "123\n" + ) # On the other node, data is replicated only if using a Replicated table engine if engine == "ReplicatedMergeTree": - assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "123\n" + assert ( + dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") + == "123\n" + ) else: - assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") == "" + assert ( + dummy_node.query(f"SELECT CounterID FROM {database}.alter_attach_test") + == "" + ) main_node.query(f"DROP DATABASE {database} SYNC") dummy_node.query(f"DROP DATABASE {database} SYNC") @@ -333,7 +342,9 @@ def test_alter_drop_part(started_cluster, engine): assert main_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" if engine == "ReplicatedMergeTree": # The DROP operation is still replicated at the table engine level - assert dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" + assert ( + dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") == "" + ) else: assert ( dummy_node.query(f"SELECT CounterID FROM {database}.alter_drop_part") From 96f048f7f8895507c6827f373699244f345730ec Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Wed, 19 Jul 2023 10:46:02 +0200 Subject: [PATCH 0844/2047] Convert output UInt128 to FixedString even if input is empty --- src/Functions/FunctionsHashing.h | 48 +++++++++---------- .../0_stateless/02534_keyed_siphash.reference | 2 + .../0_stateless/02534_keyed_siphash.sql | 3 ++ 3 files changed, 29 insertions(+), 24 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 4965d1f7b49..82944630b10 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1535,33 +1535,33 @@ public: { auto col_to = ColumnVector::create(input_rows_count); - if (input_rows_count == 0) - return col_to; - - typename ColumnVector::Container & vec_to = col_to->getData(); - - /// If using a "keyed" algorithm, the first argument is the key and - /// the data starts from the second argument. - /// Otherwise there is no key and all arguments are interpreted as data. - constexpr size_t first_data_argument = Keyed; - - if (arguments.size() <= first_data_argument) + if (input_rows_count != 0) { - /// Return a fixed random-looking magic number when input is empty - vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); - } + typename ColumnVector::Container & vec_to = col_to->getData(); - KeyColumnsType key_cols{}; - if constexpr (Keyed) - if (!arguments.empty()) - key_cols = Impl::parseKeyColumns(arguments[0]); + /// If using a "keyed" algorithm, the first argument is the key and + /// the data starts from the second argument. + /// Otherwise there is no key and all arguments are interpreted as data. + constexpr size_t first_data_argument = Keyed; - /// The function supports arbitrary number of arguments of arbitrary types. - bool is_first_argument = true; - for (size_t i = first_data_argument; i < arguments.size(); ++i) - { - const auto & col = arguments[i]; - executeForArgument(key_cols, col.type.get(), col.column.get(), vec_to, is_first_argument); + if (arguments.size() <= first_data_argument) + { + /// Return a fixed random-looking magic number when input is empty + vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); + } + + KeyColumnsType key_cols{}; + if constexpr (Keyed) + if (!arguments.empty()) + key_cols = Impl::parseKeyColumns(arguments[0]); + + /// The function supports arbitrary number of arguments of arbitrary types. + bool is_first_argument = true; + for (size_t i = first_data_argument; i < arguments.size(); ++i) + { + const auto & col = arguments[i]; + executeForArgument(key_cols, col.type.get(), col.column.get(), vec_to, is_first_argument); + } } if constexpr (std::is_same_v) /// backward-compatible diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index de783d7dddf..a9f724365a8 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -232,3 +232,5 @@ Check multiple keys as separate ints from a table with constant data 9357996107237883963 86AE90BB6A238D3F6221457630142C9B F6D93D8FEA6D7DECCDD95A7A0A2AA36D +Check asan bug +0 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 14b422ac713..4f3ae7d62bd 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -331,3 +331,6 @@ INSERT INTO sipHashKeyed_keys VALUES (4, 4); SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; SELECT hex(sipHash128Keyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0; DROP TABLE sipHashKeyed_keys; + +SELECT 'Check asan bug'; +SELECT sipHash128((toUInt64(9223372036854775806), 1)) = sipHash128(1) GROUP BY sipHash128(1::UInt8), toUInt64(9223372036854775806); From f22452c78c36c2d2529444a137d4853749c04945 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 08:54:31 +0000 Subject: [PATCH 0845/2047] Beautify pretty-printing of the query string in SYSTEM.QUERY_CACHE. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Output of SYSTEM.QUERY_CACHE - before this PR: SELECT * FROM system.query_cache Query id: 4989008b-b84c-4e57-bfe5-7fb551814812 Row 1: ────── query: SELECT 1 SETTINGS [...] - after this PR: SELECT * FROM system.query_cache Query id: 4989008b-b84c-4e57-bfe5-7fb551814812 Row 1: ────── query: SELECT 1 SETTINGS use_query_cache = 1 [...] --- src/Interpreters/Cache/QueryCache.cpp | 35 ++++++++++--------- src/Interpreters/Cache/QueryCache.h | 8 +++-- .../System/StorageSystemQueryCache.cpp | 2 +- .../02494_query_cache_secrets.reference | 2 +- 4 files changed, 26 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index f46a10ca51d..1d1543844a2 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -115,6 +115,15 @@ ASTPtr removeQueryCacheSettings(ASTPtr ast) return transformed_ast; } +String queryStringFromAst(ASTPtr ast) +{ + WriteBufferFromOwnString buf; + IAST::FormatSettings format_settings(buf, /*one_line*/ true); + format_settings.show_secrets = false; + ast->format(format_settings); + return buf.str(); +} + } QueryCache::Key::Key( @@ -129,6 +138,7 @@ QueryCache::Key::Key( , is_shared(is_shared_) , expires_at(expires_at_) , is_compressed(is_compressed_) + , query_string(queryStringFromAst(ast_)) { } @@ -142,15 +152,6 @@ bool QueryCache::Key::operator==(const Key & other) const return ast->getTreeHash() == other.ast->getTreeHash(); } -String QueryCache::Key::queryStringFromAst() const -{ - WriteBufferFromOwnString buf; - IAST::FormatSettings format_settings(buf, /*one_line*/ true); - format_settings.show_secrets = false; - ast->format(format_settings); - return buf.str(); -} - size_t QueryCache::KeyHasher::operator()(const Key & key) const { SipHash hash; @@ -191,7 +192,7 @@ QueryCache::Writer::Writer( if (auto entry = cache.getWithKey(key); entry.has_value() && !IsStale()(entry->key)) { skip_insert = true; /// Key already contained in cache and did not expire yet --> don't replace it - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.query_string); } } @@ -263,14 +264,14 @@ void QueryCache::Writer::finalizeWrite() if (std::chrono::duration_cast(std::chrono::system_clock::now() - query_start_time) < min_query_runtime) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query not expensive enough), query: {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query not expensive enough), query: {}", key.query_string); return; } if (auto entry = cache.getWithKey(key); entry.has_value() && !IsStale()(entry->key)) { /// Same check as in ctor because a parallel Writer could have inserted the current key in the meantime - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (non-stale entry found), query: {}", key.query_string); return; } @@ -353,7 +354,7 @@ void QueryCache::Writer::finalizeWrite() if ((new_entry_size_in_bytes > max_entry_size_in_bytes) || (new_entry_size_in_rows > max_entry_size_in_rows)) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query result too big), new_entry_size_in_bytes: {} ({}), new_entry_size_in_rows: {} ({}), query: {}", new_entry_size_in_bytes, max_entry_size_in_bytes, new_entry_size_in_rows, max_entry_size_in_rows, key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Skipped insert (query result too big), new_entry_size_in_bytes: {} ({}), new_entry_size_in_rows: {} ({}), query: {}", new_entry_size_in_bytes, max_entry_size_in_bytes, new_entry_size_in_rows, max_entry_size_in_rows, key.query_string); return; } @@ -388,7 +389,7 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar if (!entry.has_value()) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "No entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "No entry found for query {}", key.query_string); return; } @@ -397,13 +398,13 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar if (!entry_key.is_shared && entry_key.user_name != key.user_name) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.query_string); return; } if (IsStale()(entry_key)) { - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found for query {}", key.query_string); return; } @@ -441,7 +442,7 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar buildSourceFromChunks(entry_key.header, std::move(decompressed_chunks), entry_mapped->totals, entry_mapped->extremes); } - LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.queryStringFromAst()); + LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.query_string); } bool QueryCache::Reader::hasCacheEntryForKey() const diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 6ef7cc60918..a67adcc86c9 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -30,7 +30,7 @@ public: /// ---------------------------------------------------- /// The actual key (data which gets hashed): - /// Unlike the query string, the AST is agnostic to lower/upper case (SELECT vs. select) + /// Unlike the query string, the AST is agnostic to lower/upper case (SELECT vs. select). const ASTPtr ast; /// Note: For a transactionally consistent cache, we would need to include the system settings in the cache key or invalidate the @@ -58,6 +58,11 @@ public: /// (we could theoretically apply compression also to the totals and extremes but it's an obscure use case) const bool is_compressed; + /// The SELECT query as plain string, displayed in SYSTEM.QUERY_CACHE. Stored explicitly, i.e. not constructed from the AST, for the + /// sole reason that QueryCache-related SETTINGS are pruned from the AST (see removeQueryCacheSettings()) which will look ugly in + /// the SYSTEM.QUERY_CACHE. + const String query_string; + /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, Block header_, @@ -69,7 +74,6 @@ public: Key(ASTPtr ast_, const String & user_name_); bool operator==(const Key & other) const; - String queryStringFromAst() const; }; struct Entry diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 117fb4e8a5c..288e4fd52a0 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -44,7 +44,7 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr if (!key.is_shared && key.user_name != user_name) continue; - res_columns[0]->insert(key.queryStringFromAst()); /// approximates the original query string + res_columns[0]->insert(key.query_string); /// approximates the original query string res_columns[1]->insert(QueryCache::QueryCacheEntryWeight()(*query_result)); res_columns[2]->insert(key.expires_at < std::chrono::system_clock::now()); res_columns[3]->insert(key.is_shared); diff --git a/tests/queries/0_stateless/02494_query_cache_secrets.reference b/tests/queries/0_stateless/02494_query_cache_secrets.reference index dd6341262bc..306374eed4b 100644 --- a/tests/queries/0_stateless/02494_query_cache_secrets.reference +++ b/tests/queries/0_stateless/02494_query_cache_secrets.reference @@ -1,2 +1,2 @@ A2193552DCF8A9F99AC35F86BC4D2FFD -SELECT hex(encrypt(\'aes-128-ecb\', \'[HIDDEN]\')) SETTINGS +SELECT hex(encrypt(\'aes-128-ecb\', \'[HIDDEN]\')) SETTINGS use_query_cache = 1 From d0e886070c6acbb57c35a1d3df287e2a07a9f115 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 19 Jul 2023 17:39:00 +0800 Subject: [PATCH 0846/2047] system.events and system.metrics tables add column name as an alias to event and metric --- src/Storages/System/StorageSystemEvents.cpp | 7 +++++++ src/Storages/System/StorageSystemEvents.h | 1 + src/Storages/System/StorageSystemMetrics.cpp | 7 +++++++ src/Storages/System/StorageSystemMetrics.h | 1 + 4 files changed, 16 insertions(+) diff --git a/src/Storages/System/StorageSystemEvents.cpp b/src/Storages/System/StorageSystemEvents.cpp index b9b07cfe0ac..a914c60abf4 100644 --- a/src/Storages/System/StorageSystemEvents.cpp +++ b/src/Storages/System/StorageSystemEvents.cpp @@ -16,6 +16,13 @@ NamesAndTypesList StorageSystemEvents::getNamesAndTypes() }; } +NamesAndAliases StorageSystemEvents::getNamesAndAliases() +{ + return { + {"name", std::make_shared(), "event"} + }; +} + void StorageSystemEvents::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) diff --git a/src/Storages/System/StorageSystemEvents.h b/src/Storages/System/StorageSystemEvents.h index ea0322c13b8..ed80a441529 100644 --- a/src/Storages/System/StorageSystemEvents.h +++ b/src/Storages/System/StorageSystemEvents.h @@ -17,6 +17,7 @@ public: static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases(); protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemMetrics.cpp b/src/Storages/System/StorageSystemMetrics.cpp index 6007c8a7c71..ced363fed17 100644 --- a/src/Storages/System/StorageSystemMetrics.cpp +++ b/src/Storages/System/StorageSystemMetrics.cpp @@ -17,6 +17,13 @@ NamesAndTypesList StorageSystemMetrics::getNamesAndTypes() }; } +NamesAndAliases StorageSystemMetrics::getNamesAndAliases() +{ + return { + {"name", std::make_shared(), "metric"} + }; +} + void StorageSystemMetrics::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const { for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) diff --git a/src/Storages/System/StorageSystemMetrics.h b/src/Storages/System/StorageSystemMetrics.h index def95e0a934..bffc7e6f1fc 100644 --- a/src/Storages/System/StorageSystemMetrics.h +++ b/src/Storages/System/StorageSystemMetrics.h @@ -18,6 +18,7 @@ public: static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases(); protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; From d3663c356f4a8cab8b77f36fcc9b8251e6ddd02e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 19 Jul 2023 12:11:57 +0200 Subject: [PATCH 0847/2047] Wait fo KILL MUTATION to finish --- .../00834_kill_mutation_replicated_zookeeper.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh index 2e917f67fe8..16ad08deeb2 100755 --- a/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh @@ -57,6 +57,14 @@ $CLICKHOUSE_CLIENT --query="SELECT count() FROM system.mutations WHERE database ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = '$CLICKHOUSE_DATABASE' AND table = 'kill_mutation_r1' AND mutation_id = '0000000001'" +# Wait for the 1st mutation to be actually killed and the 2nd to finish +query_result=$($CLICKHOUSE_CLIENT --query="$check_query1" 2>&1) +while [ "$query_result" != "0" ] +do + query_result=$($CLICKHOUSE_CLIENT --query="$check_query1" 2>&1) + sleep 0.5 +done + ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA kill_mutation_r1" ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA kill_mutation_r2" From 95424177d5de5bd7973823ffdaaacafce442e8ba Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 18:26:54 +0800 Subject: [PATCH 0848/2047] review fix --- docs/en/interfaces/formats.md | 3 +- .../operations/settings/settings-formats.md | 12 +-- src/Core/Settings.h | 3 +- src/Formats/FormatFactory.cpp | 3 +- src/Formats/FormatSettings.h | 3 +- .../Formats/Impl/CSVRowInputFormat.cpp | 78 ++++++++++--------- .../Formats/Impl/CSVRowInputFormat.h | 2 + ...11_csv_input_field_type_mismatch.reference | 8 +- .../02811_csv_input_field_type_mismatch.sh | 4 +- .../data_csv/csv_with_bad_field_values.csv | 5 ++ .../data_csv/csv_with_diff_field_types.csv | 3 - 11 files changed, 62 insertions(+), 62 deletions(-) create mode 100644 tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv delete mode 100644 tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index f45c55a9734..c20f304c346 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,8 +472,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_allow_check_field_deserialization](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_check_field_deserialization) - Allow to check whether the csv input field can be successful deserialized. Default value - `false`. -- [input_format_csv_set_default_if_deserialization_failed](/docs/en/operations/settings/settings-formats.md/#input_format_csv_set_default_if_deserialization_failed) - Set default value to column if the csv input field deserialization failed. Default value - `false`. +- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialize failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 409ac4bd58a..5fac8df02d7 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -969,15 +969,9 @@ Result a b ``` -### input_format_csv_allow_check_field_deserialization {#input_format_csv_allow_check_field_deserialization} +### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} -Allow to use whitespace or tab as field delimiter in CSV strings. - -Default value: `false`. - -### input_format_csv_set_default_if_deserialization_failed {#input_format_csv_set_default_if_deserialization_failed} - -Allow to set default value to column if the csv input field's deserialization failed +Allow to set default value to column when CSV field deserialize failed on bad value Default value: `false`. @@ -988,7 +982,7 @@ Query ```bash echo 'a,b,c' > 1.txt ./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_field_deserialization=true, input_format_csv_set_default_if_deserialization_failed=true FORMAT CSV" +cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" ./clickhouse local -q "select * from test_tbl" ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9d93ba9ad2c..311813fb38f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,8 +872,7 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_allow_check_field_deserialization, false, "Allow to check the csv input field deserialization whether success or not.", 0) \ - M(Bool, input_format_csv_set_default_if_deserialization_failed, false, "All to set column default value if the input field's deserialization failed.", 0) \ + M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialize failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index b3b9609f9fe..3df2ca7d2e0 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,8 +73,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; - format_settings.csv.allow_check_field_deserialization = settings.input_format_csv_allow_check_field_deserialization; - format_settings.csv.set_default_if_deserialization_failed = settings.input_format_csv_set_default_if_deserialization_failed; + format_settings.csv.use_default_on_bad_values = settings.input_format_csv_use_default_on_bad_values; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 4d4eb926992..4e49d338e43 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -141,8 +141,7 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; - bool allow_check_field_deserialization=false; - bool set_default_if_deserialization_failed=false; + bool use_default_on_bad_values = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index e1be6b21610..34d5b589591 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -316,49 +317,52 @@ bool CSVFormatReader::readField( return false; } - BufferBase::Position pos_start = buf->position(); + if (format_settings.csv.use_default_on_bad_values) + return readFieldOrDefault(column, type, serialization); + return readFieldImpl(*buf, column, type, serialization); +} + +bool CSVFormatReader::readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ + if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) + { + /// If value is null but type is not nullable then use default value instead. + return SerializationNullable::deserializeTextCSVImpl(column, istr, format_settings, serialization); + } + + /// Read the column normally. + serialization->deserializeTextCSV(column, istr, format_settings); + return true; +} + +bool CSVFormatReader::readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ + String field; + readCSVField(field, *buf, format_settings.csv); + ReadBufferFromString tmp_buf(field); + bool is_bad_value = false; + bool res = false; + size_t col_size = column.size(); try { - if (format_settings.csv.allow_check_field_deserialization) - { - std::string field; - readCSVField(field, *buf, format_settings.csv); - ReadBufferFromMemory tmp(field); - if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) - SerializationNullable::deserializeTextCSVImpl(column, tmp, format_settings, serialization); - else - serialization->deserializeTextCSV(column, tmp, format_settings); - if (column.size() == col_size + 1 && field.size() > 0 && !tmp.eof()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Text CSV deserialize field bytes logical error."); - } - else - { - if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) - { - /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); - } - /// Read the column normally. - serialization->deserializeTextCSV(column, *buf, format_settings); - } + res = readFieldImpl(tmp_buf, column, type, serialization); + /// Check if we parsed the whole field successfully. + if (!field.empty() && !tmp_buf.eof()) + is_bad_value = true; } - catch (Exception & e) + catch (const Exception &) { - LOG_DEBUG(&Poco::Logger::get("CSVRowInputFormat"), "Failed to deserialize CSV column, exception message:{}", e.what()); - if (format_settings.csv.set_default_if_deserialization_failed) - { - // Reset the column and buffer position, then skip the field and set column default value. - if (column.size() == col_size + 1) - column.popBack(1); - buf->position() = pos_start; - skipField(); - column.insertDefault(); - } - else - throw; + is_bad_value = true; } - return true; + + if (!is_bad_value) + return res; + + if (column.size() == col_size + 1) + column.popBack(1); + column.insertDefault(); + return false; } void CSVFormatReader::skipPrefixBeforeHeader() diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 8ccf04feed3..7b1a1fc433d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -89,6 +89,8 @@ public: void setReadBuffer(ReadBuffer & in_) override; FormatSettings::EscapingRule getEscapingRule() const override { return FormatSettings::EscapingRule::CSV; } + bool readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); + bool readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); protected: PeekableReadBuffer * buf; diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference index c5ee611a230..19c7956ba84 100644 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference @@ -1,3 +1,5 @@ -a 1 2023-03-14 -a 0 1970-01-01 -c 1 1970-01-01 +0 111 1970-01-01 2023-03-24 00:00:00 false +1 abc 2023-03-14 2023-03-14 11:22:33 true +2 c 1970-01-01 1970-01-01 08:00:00 false +4 888 2023-03-14 1970-06-03 14:43:53 false +5 bks 1970-01-01 2023-07-19 18:17:59 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh index df736ea6792..3961664b9b3 100644 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" -$CLICKHOUSE_CLIENT -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat $CURDIR/data_csv/csv_with_diff_field_types.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_allow_check_deserialize=true, input_format_csv_set_default_if_deserialize_failed=true FORMAT CSV" +$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, d DateTime, e Boolean) engine=MergeTree order by a" +cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" $CLICKHOUSE_CLIENT -q "select * from test_tbl" $CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv new file mode 100644 index 00000000000..faedd9b6705 --- /dev/null +++ b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv @@ -0,0 +1,5 @@ +1,abc,2023-03-14,2023-03-14 11:22:33,true +2,c,ab,2023,false +bc,111,ab,2023-03-24,ban +4,888,2023-03-14,13243433,false +5,bks,2023-03,1689761879,abdd \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv b/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv deleted file mode 100644 index 464172c515c..00000000000 --- a/tests/queries/0_stateless/data_csv/csv_with_diff_field_types.csv +++ /dev/null @@ -1,3 +0,0 @@ -a,1,2023-03-14 -a,b,c -c,1,a \ No newline at end of file From 380b4ffe2be4107ae3965cba19c5b697e7108128 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 19 Jul 2023 12:29:39 +0200 Subject: [PATCH 0849/2047] Reduce dependencies for skim by avoid using default features By default skim requires cli -> clap -> termcolor -> winapi-util Signed-off-by: Azat Khuzhin --- rust/skim/Cargo.lock | 204 ++++++++++++------------------------------- rust/skim/Cargo.toml | 2 +- 2 files changed, 58 insertions(+), 148 deletions(-) diff --git a/rust/skim/Cargo.lock b/rust/skim/Cargo.lock index 9f948ee1c38..f55ea8a84b0 100644 --- a/rust/skim/Cargo.lock +++ b/rust/skim/Cargo.lock @@ -42,17 +42,6 @@ version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" -[[package]] -name = "atty" -version = "0.2.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" -dependencies = [ - "hermit-abi 0.1.19", - "libc", - "winapi", -] - [[package]] name = "autocfg" version = "1.1.0" @@ -104,31 +93,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "clap" -version = "3.2.25" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ea181bf566f71cb9a5d17a59e1871af638180a18fb0035c92ae62b705207123" -dependencies = [ - "atty", - "bitflags", - "clap_lex", - "indexmap", - "once_cell", - "strsim", - "termcolor", - "textwrap", -] - -[[package]] -name = "clap_lex" -version = "0.2.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2850f2f5a82cbf437dd5af4d49848fbdfc27c157c3d010345776f952765261c5" -dependencies = [ - "os_str_bytes", -] - [[package]] name = "codespan-reporting" version = "0.11.1" @@ -214,9 +178,9 @@ dependencies = [ [[package]] name = "cxx" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e88abab2f5abbe4c56e8f1fb431b784d710b709888f35755a160e62e33fe38e8" +checksum = "5032837c1384de3708043de9d4e97bb91290faca6c16529a28aa340592a78166" dependencies = [ "cc", "cxxbridge-flags", @@ -226,9 +190,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c0c11acd0e63bae27dcd2afced407063312771212b7a823b4fd72d633be30fb" +checksum = "51368b3d0dbf356e10fcbfd455a038503a105ee556f7ee79b6bb8c53a7247456" dependencies = [ "cc", "codespan-reporting", @@ -236,24 +200,24 @@ dependencies = [ "proc-macro2", "quote", "scratch", - "syn 2.0.23", + "syn 2.0.26", ] [[package]] name = "cxxbridge-flags" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d3816ed957c008ccd4728485511e3d9aaf7db419aa321e3d2c5a2f3411e36c8" +checksum = "0d9062157072e4aafc8e56ceaf8325ce850c5ae37578c852a0d4de2cecdded13" [[package]] name = "cxxbridge-macro" -version = "1.0.97" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a26acccf6f445af85ea056362561a24ef56cdc15fcc685f03aec50b9c702cb6d" +checksum = "cf01e8a540f5a4e0f284595834f81cf88572f244b768f051724537afa99a2545" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", ] [[package]] @@ -359,19 +323,6 @@ version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" -[[package]] -name = "env_logger" -version = "0.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a12e6657c4c97ebab115a42dcee77225f7f482cdd841cf7088c657a42e9e00e7" -dependencies = [ - "atty", - "humantime", - "log", - "regex", - "termcolor", -] - [[package]] name = "fnv" version = "1.0.7" @@ -398,32 +349,11 @@ dependencies = [ "wasi 0.11.0+wasi-snapshot-preview1", ] -[[package]] -name = "hashbrown" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" - [[package]] name = "hermit-abi" -version = "0.1.19" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62b467343b94ba476dcb2500d242dadbb39557df889310ac77c5d99100aaac33" -dependencies = [ - "libc", -] - -[[package]] -name = "hermit-abi" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fed44880c466736ef9a5c5b5facefb5ed0785676d0c02d612db14e54f0d84286" - -[[package]] -name = "humantime" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" +checksum = "443144c8cdadd93ebf52ddb4056d257f5b52c04d3c804e657d19eb73fc33668b" [[package]] name = "iana-time-zone" @@ -454,16 +384,6 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" -[[package]] -name = "indexmap" -version = "1.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" -dependencies = [ - "autocfg", - "hashbrown", -] - [[package]] name = "js-sys" version = "0.3.64" @@ -487,9 +407,9 @@ checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" [[package]] name = "link-cplusplus" -version = "1.0.8" +version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecd207c9c713c34f95a097a5b029ac2ce6010530c7b49d7fea24d977dede04f5" +checksum = "9d240c6f7e1ba3a28b0249f774e6a9dd0175054b52dfbb61b16eb8505c3785c9" dependencies = [ "cc", ] @@ -564,7 +484,7 @@ version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" dependencies = [ - "hermit-abi 0.3.1", + "hermit-abi", "libc", ] @@ -574,12 +494,6 @@ version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" -[[package]] -name = "os_str_bytes" -version = "6.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d5d9eb14b174ee9aa2ef96dc2b94637a2d4b6e7cb873c7e171f0c20c6cf3eac" - [[package]] name = "pin-utils" version = "0.1.0" @@ -588,18 +502,18 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "proc-macro2" -version = "1.0.63" +version = "1.0.66" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b368fba921b0dce7e60f5e04ec15e565b3303972b42bcfde1d0713b881959eb" +checksum = "18fb31db3f9bddb2ea821cde30a9f70117e3f119938b5ee630b7403aa6e2ead9" dependencies = [ "unicode-ident", ] [[package]] name = "quote" -version = "1.0.29" +version = "1.0.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "573015e8ab27661678357f27dc26460738fd2b6c86e46f386fde94cb5d913105" +checksum = "5fe8a65d69dd0808184ebb5f836ab526bb259db23c657efa38711b1072ee47f0" dependencies = [ "proc-macro2", ] @@ -648,9 +562,21 @@ dependencies = [ [[package]] name = "regex" -version = "1.8.4" +version = "1.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0ab3ca65655bb1e41f2a8c8cd662eb4fb035e67c3f78da1d61dffe89d07300f" +checksum = "b2eae68fc220f7cf2532e4494aded17545fce192d59cd996e0fe7887f4ceb575" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39354c10dd07468c2e73926b23bb9c2caca74c5501e38a35da70406f1d923310" dependencies = [ "aho-corasick", "memchr", @@ -659,39 +585,33 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.7.2" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "436b050e76ed2903236f032a59761c1eb99e1b0aead2c257922771dab1fc8c78" +checksum = "e5ea92a5b6195c6ef2a0295ea818b312502c6fc94dde986c5553242e18fd4ce2" [[package]] name = "rustversion" -version = "1.0.12" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f3208ce4d8448b3f3e7d168a73f5e0c43a61e32930de3bceeccedb388b6bf06" +checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" [[package]] name = "scopeguard" -version = "1.1.0" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "scratch" -version = "1.0.5" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1792db035ce95be60c3f8853017b3999209281c24e2ba5bc8e59bf97a0c590c1" +checksum = "a3cf7c11c38cb994f3d40e8a8cde3bbd1f72a435e4c49e85d6553d8312306152" [[package]] name = "serde" -version = "1.0.164" +version = "1.0.171" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e8c8cf938e98f769bc164923b06dce91cea1751522f46f8466461af04c9027d" - -[[package]] -name = "shlex" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43b2853a4d09f215c24cc5489c992ce46052d359b5109343cbafbf26bc62f8a3" +checksum = "30e27d1e4fd7659406c492fd6cfaf2066ba8773de45ca75e855590f856dc34a9" [[package]] name = "skim" @@ -699,23 +619,19 @@ version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5d28de0a6cb2cdd83a076f1de9d965b973ae08b244df1aa70b432946dda0f32" dependencies = [ - "atty", "beef", "bitflags", "chrono", - "clap", "crossbeam", "defer-drop", "derive_builder", - "env_logger", "fuzzy-matcher", "lazy_static", "log", "nix 0.25.1", "rayon", "regex", - "shlex", - "time 0.3.22", + "time 0.3.23", "timer", "tuikit", "unicode-width", @@ -741,9 +657,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.23" +version = "2.0.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59fb7d6d8281a51045d62b8eb3a7d1ce347b76f312af50cd3dc0af39c87c1737" +checksum = "45c3457aacde3c65315de5031ec191ce46604304d2446e803d71ade03308d970" dependencies = [ "proc-macro2", "quote", @@ -770,30 +686,24 @@ dependencies = [ "winapi-util", ] -[[package]] -name = "textwrap" -version = "0.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" - [[package]] name = "thiserror" -version = "1.0.40" +version = "1.0.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "978c9a314bd8dc99be594bc3c175faaa9794be04a5a5e153caba6915336cebac" +checksum = "a35fc5b8971143ca348fa6df4f024d4d55264f3468c71ad1c2f365b0a4d58c42" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.40" +version = "1.0.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9456a42c5b0d803c8cd86e73dd7cc9edd429499f37a3550d286d5e86720569f" +checksum = "463fe12d7993d3b327787537ce8dd4dfa058de32fc2b195ef3cde03dc4771e8f" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", ] [[package]] @@ -819,9 +729,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.22" +version = "0.3.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea9e1b3cf1243ae005d9e74085d4d542f3125458f3a81af210d901dcd7411efd" +checksum = "59e399c068f43a5d116fedaf73b203fa4f9c519f17e2b34f63221d3792f81446" dependencies = [ "serde", "time-core", @@ -858,9 +768,9 @@ dependencies = [ [[package]] name = "unicode-ident" -version = "1.0.9" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b15811caf2415fb889178633e7724bad2509101cde276048e013b9def5e51fa0" +checksum = "301abaae475aa91687eb82514b328ab47a211a533026cb25fc3e519b86adfc3c" [[package]] name = "unicode-width" @@ -928,7 +838,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", "wasm-bindgen-shared", ] @@ -950,7 +860,7 @@ checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.23", + "syn 2.0.26", "wasm-bindgen-backend", "wasm-bindgen-shared", ] diff --git a/rust/skim/Cargo.toml b/rust/skim/Cargo.toml index e5801a26f77..0381ad81619 100644 --- a/rust/skim/Cargo.toml +++ b/rust/skim/Cargo.toml @@ -6,7 +6,7 @@ edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -skim = "0.10.2" +skim = { version = "0.10.2", default-features = false } cxx = "1.0.83" term = "0.7.0" From af6361e2a0c78f45500a37bc67f563bd74412076 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 19 Jul 2023 12:35:52 +0200 Subject: [PATCH 0850/2047] Fix 02725_memory-for-merges --- tests/queries/0_stateless/02725_memory-for-merges.sql | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02725_memory-for-merges.sql b/tests/queries/0_stateless/02725_memory-for-merges.sql index 347c8b2a8d3..1a8402dff4b 100644 --- a/tests/queries/0_stateless/02725_memory-for-merges.sql +++ b/tests/queries/0_stateless/02725_memory-for-merges.sql @@ -1,4 +1,4 @@ --- Tags: no-s3-storage +-- Tags: no-s3-storage, no-random-merge-tree-settings -- We allocate a lot of memory for buffers when reading or writing to S3 DROP TABLE IF EXISTS 02725_memory_for_merges SYNC; @@ -21,7 +21,6 @@ OPTIMIZE TABLE 02725_memory_for_merges FINAL; SYSTEM FLUSH LOGS; -WITH (SELECT uuid FROM system.tables WHERE table='02725_memory_for_merges' and database=currentDatabase()) as uuid -SELECT (sum(peak_memory_usage) < 1024 * 1024 * 200 AS x) ? x : sum(peak_memory_usage) from system.part_log where table_uuid=uuid and event_type='MergeParts'; +SELECT (sum(peak_memory_usage) < 1024 * 1024 * 200 AS x) ? x : sum(peak_memory_usage) from system.part_log where database=currentDatabase() and table='02725_memory_for_merges' and event_type='MergeParts'; DROP TABLE IF EXISTS 02725_memory_for_merges SYNC; From 08409059cc198873ffbf11060bfdabaa0c74f07f Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Wed, 19 Jul 2023 18:46:20 +0800 Subject: [PATCH 0851/2047] support alias for new analyzer --- src/Analyzer/Passes/UniqToCountPass.cpp | 108 +++++++++++++----- .../test_rewrite_uniq_to_count/test.py | 16 +-- 2 files changed, 90 insertions(+), 34 deletions(-) diff --git a/src/Analyzer/Passes/UniqToCountPass.cpp b/src/Analyzer/Passes/UniqToCountPass.cpp index ae7952051e7..7533a99107b 100644 --- a/src/Analyzer/Passes/UniqToCountPass.cpp +++ b/src/Analyzer/Passes/UniqToCountPass.cpp @@ -21,36 +21,82 @@ bool matchFnUniq(String func_name) || name == "uniqCombined64"; } -bool nodeEquals(const QueryTreeNodePtr & lhs, const QueryTreeNodePtr & rhs) +/// Extract the corresponding projection columns for group by node list. +/// For example: +/// SELECT a as aa, any(b) FROM table group by a; -> aa(ColumnNode) +NamesAndTypes extractProjectionColumnsForGroupBy(const QueryNode * query_node) { - auto * lhs_node = lhs->as(); - auto * rhs_node = rhs->as(); + if (!query_node->hasGroupBy()) + return {}; - if (lhs_node && rhs_node && lhs_node->getColumn() == rhs_node->getColumn()) - return true; - return false; + NamesAndTypes result; + for (const auto & group_by_ele : query_node->getGroupByNode()->getChildren()) + { + const auto & projection_columns = query_node->getProjectionColumns(); + const auto & projection_nodes = query_node->getProjection().getNodes(); + + assert(projection_columns.size() == projection_nodes.size()); + + for (size_t i = 0; i < projection_columns.size(); i++) + { + if (projection_nodes[i]->isEqual(*group_by_ele)) + result.push_back(projection_columns[i]); + } + } + return result; } -bool nodeListEquals(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs) +/// Whether query_columns equals subquery_columns. +/// query_columns: query columns from query +/// subquery_columns: projection columns from subquery +bool nodeListEquals(const QueryTreeNodes & query_columns, const NamesAndTypes & subquery_columns) { - if (lhs.size() != rhs.size()) + if (query_columns.size() != subquery_columns.size()) return false; - for (size_t i = 0; i < lhs.size(); i++) + + for (const auto & query_column : query_columns) { - if (!nodeEquals(lhs[i], rhs[i])) + auto find = std::find_if( + subquery_columns.begin(), + subquery_columns.end(), + [&](const auto & subquery_column) -> bool + { + if (auto * column_node = query_column->as()) + { + return subquery_column == column_node->getColumn(); + } + return false; + }); + + if (find == subquery_columns.end()) return false; } return true; } -bool nodeListContainsAll(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs) +/// Whether subquery_columns contains all columns in subquery_columns. +/// query_columns: query columns from query +/// subquery_columns: projection columns from subquery +bool nodeListContainsAll(const QueryTreeNodes & query_columns, const NamesAndTypes & subquery_columns) { - if (lhs.size() < rhs.size()) + if (query_columns.size() > subquery_columns.size()) return false; - for (const auto & re : rhs) + + for (const auto & query_column : query_columns) { - auto predicate = [&](const QueryTreeNodePtr & le) { return nodeEquals(le, re); }; - if (std::find_if(lhs.begin(), lhs.end(), predicate) == lhs.end()) + auto find = std::find_if( + subquery_columns.begin(), + subquery_columns.end(), + [&](const auto & subquery_column) -> bool + { + if (auto * column_node = query_column->as()) + { + return subquery_column == column_node->getColumn(); + } + return false; + }); + + if (find == subquery_columns.end()) return false; } return true; @@ -58,17 +104,14 @@ bool nodeListContainsAll(const QueryTreeNodes & lhs, const QueryTreeNodes & rhs) } -class UniqToCountVisitor : public InDepthQueryTreeVisitorWithContext +class UniqToCountVisitor : public InDepthQueryTreeVisitor { public: - using Base = InDepthQueryTreeVisitorWithContext; + using Base = InDepthQueryTreeVisitor; using Base::Base; void visitImpl(QueryTreeNodePtr & node) { - if (!getSettings().optimize_uniq_to_count) - return; - auto * query_node = node->as(); if (!query_node) return; @@ -100,9 +143,11 @@ public: { if (!subquery_node->isDistinct()) return false; - /// uniq expression list == subquery group by expression list - if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjection().getNodes())) + + /// uniq expression list == subquery projection columns + if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjectionColumns())) return false; + return true; }; @@ -111,12 +156,17 @@ public: { if (!subquery_node->hasGroupBy()) return false; + /// uniq argument node list == subquery group by node list - if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getGroupByNode()->getChildren())) + auto group_by_columns = extractProjectionColumnsForGroupBy(subquery_node); + + if (!nodeListEquals(uniq_arguments_nodes, group_by_columns)) return false; - /// subquery select node list must contain all columns in uniq argument node list - if (!nodeListContainsAll(subquery_node->getProjection().getNodes(), uniq_arguments_nodes)) + + /// subquery projection columns must contain all columns in uniq argument node list + if (!nodeListContainsAll(uniq_arguments_nodes, subquery_node->getProjectionColumns())) return false; + return true; }; @@ -125,8 +175,11 @@ public: { AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get("count", {}, {}, properties); + function_node->resolveAsAggregateFunction(std::move(aggregate_function)); function_node->getArguments().getNodes().clear(); + + /// Update projection columns query_node->resolveProjectionColumns({{"count()", function_node->getResultType()}}); } } @@ -135,7 +188,10 @@ public: void UniqToCountPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { - UniqToCountVisitor visitor(std::move(context)); + if (!context->getSettings().optimize_uniq_to_count) + return; + + UniqToCountVisitor visitor; visitor.visit(query_tree_node); } diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py index d7fa9f39441..e38e57f5cee 100644 --- a/tests/integration/test_rewrite_uniq_to_count/test.py +++ b/tests/integration/test_rewrite_uniq_to_count/test.py @@ -83,13 +83,13 @@ def test_rewrite_distinct(started_cluster): ) # test select expression alias - check_by_old_analyzer( - "SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", + check( + "SELECT uniq(alias_of_a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", 3, ) # test select expression alias - check_by_old_analyzer( + check( "SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t", 3, ) @@ -109,19 +109,19 @@ def test_rewrite_group_by(started_cluster): ) # test select expression alias - check_by_old_analyzer( + check( "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", 3, ) # test select expression alias - check_by_old_analyzer( - "SELECT uniq(t.a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + check( + "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", 3, ) # test select expression alias - check_by_old_analyzer( - "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", + check( + "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", 3, ) From 2ebbbf0000ce7f5767d754b0aee777a4255ab7b3 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 19:13:45 +0800 Subject: [PATCH 0852/2047] Also need to fix aggregate projections --- .../QueryPlan/Optimizations/optimizeTree.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 01d192bb1f3..b13dda9a8f0 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -114,6 +114,10 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s while (!stack.empty()) { + /// NOTE: optimizePrewhere can modify the stack. + optimizePrewhere(stack, nodes); + optimizePrimaryKeyCondition(stack); + { /// NOTE: frame cannot be safely used after stack was modified. auto & frame = stack.back(); @@ -125,6 +129,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.read_in_order) optimizeReadInOrder(*frame.node, nodes); + /// Projection optimization relies on PK optimization if (optimization_settings.optimize_projection) num_applied_projection += optimizeUseAggregateProjections(*frame.node, nodes, optimization_settings.optimize_use_implicit_projections); @@ -146,13 +151,9 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } - /// NOTE: optimizePrewhere can modify the stack. - optimizePrewhere(stack, nodes); - optimizePrimaryKeyCondition(stack); - if (optimization_settings.optimize_projection) { - /// Normal projection optimization relies on PK optimization + /// Projection optimization relies on PK optimization if (optimizeUseNormalProjections(stack, nodes)) { ++num_applied_projection; From 94796f28adcd5b304b9fbc8a715462f4cfb1c1fd Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 19:24:16 +0800 Subject: [PATCH 0853/2047] ci fix --- docs/en/interfaces/formats.md | 2 +- docs/en/operations/settings/settings-formats.md | 2 +- src/Core/Settings.h | 2 +- .../queries/0_stateless/02811_csv_input_field_type_mismatch.sh | 0 4 files changed, 3 insertions(+), 3 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index c20f304c346..ddf4ab3f78e 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,7 +472,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialize failed on bad value. Default value - `false`. +- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 5fac8df02d7..fb04ac23d3a 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -971,7 +971,7 @@ a b ### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} -Allow to set default value to column when CSV field deserialize failed on bad value +Allow to set default value to column when CSV field deserialization failed on bad value Default value: `false`. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 311813fb38f..309dfe0d2ec 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -872,7 +872,7 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialize failed on bad value", 0) \ + M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialization failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh old mode 100644 new mode 100755 From 7837559dbfdc194f28681dda808bc06b6609dd8b Mon Sep 17 00:00:00 2001 From: Song Liyong Date: Wed, 12 Jul 2023 17:13:04 +0200 Subject: [PATCH 0854/2047] MaterializedMySQL: Support CREATE TABLE AS SELECT --- src/Core/MySQL/MySQLReplication.cpp | 11 +++++++ .../materialized_with_ddl.py | 29 +++++++++++++++++++ .../test_materialized_mysql_database/test.py | 6 ++++ 3 files changed, 46 insertions(+) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 1ee027b7185..ab4a37d2466 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -121,6 +121,17 @@ namespace MySQLReplication { typ = QUERY_SAVEPOINT; } + + // https://dev.mysql.com/worklog/task/?id=13355 + // When doing query "CREATE TABLE xx AS SELECT", the binlog will be + // "CREATE TABLE ... START TRANSACTION", the DDL will be failed + // so, just ignore the "START TRANSACTION" suffix + if (query.ends_with("START TRANSACTION")) + { + auto pos = query.rfind("START TRANSACTION"); + if (pos > 0) + query.resize(pos); + } } void QueryEvent::dump(WriteBuffer & out) const diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 8cf9e67bf63..60326e422c9 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -2336,3 +2336,32 @@ def named_collections(clickhouse_node, mysql_node, service_name): ) clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + + +def create_table_as_select(clickhouse_node, mysql_node, service_name): + db = "create_table_as_select" + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"CREATE DATABASE {db}") + clickhouse_node.query( + f"CREATE DATABASE {db} ENGINE = MaterializeMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')" + ) + mysql_node.query( + f"CREATE TABLE {db}.t1(a INT NOT NULL PRIMARY KEY) ENGINE = InnoDB" + ) + mysql_node.query(f"INSERT INTO {db}.t1 VALUES (1)") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t1\n", + ) + + mysql_node.query(f"CREATE TABLE {db}.t2(PRIMARY KEY(a)) AS SELECT * FROM {db}.t1") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t1\nt2\n", + ) + + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 21316d1a474..f227c19e6b8 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -529,3 +529,9 @@ def test_named_collections(started_cluster, started_mysql_8_0, clickhouse_node): materialized_with_ddl.named_collections( clickhouse_node, started_mysql_8_0, "mysql80" ) + + +def test_create_table_as_select(started_cluster, started_mysql_8_0, clickhouse_node): + materialized_with_ddl.create_table_as_select( + clickhouse_node, started_mysql_8_0, "mysql80" + ) From dcf7ba25348f88bda0ef144ce068cc9005cb3ada Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 19:36:19 +0800 Subject: [PATCH 0855/2047] remove unuseful code --- docs/en/operations/settings/settings-formats.md | 3 +-- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index fb04ac23d3a..b3bc3afafd3 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -980,9 +980,8 @@ Default value: `false`. Query ```bash -echo 'a,b,c' > 1.txt ./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -cat 1.txt | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" +echo 'a,b,c' | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" ./clickhouse local -q "select * from test_tbl" ``` diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 34d5b589591..244b906549e 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -12,7 +12,6 @@ #include #include #include -#include namespace DB From cc9da46efa2af4dfd4f8dfdfa84327f5f14a8630 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 19 Jul 2023 20:11:03 +0800 Subject: [PATCH 0856/2047] ci fix --- .../02811_csv_input_field_type_mismatch.reference | 10 +++++----- .../0_stateless/02811_csv_input_field_type_mismatch.sh | 2 +- .../0_stateless/data_csv/csv_with_bad_field_values.csv | 10 +++++----- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference index 19c7956ba84..6abcc56bacc 100644 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference @@ -1,5 +1,5 @@ -0 111 1970-01-01 2023-03-24 00:00:00 false -1 abc 2023-03-14 2023-03-14 11:22:33 true -2 c 1970-01-01 1970-01-01 08:00:00 false -4 888 2023-03-14 1970-06-03 14:43:53 false -5 bks 1970-01-01 2023-07-19 18:17:59 false +0 111 1970-01-01 false +1 abc 2023-03-14 true +2 c 1970-01-01 false +4 888 2023-03-14 false +5 bks 1970-01-01 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh index 3961664b9b3..30223329eca 100755 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" -$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, d DateTime, e Boolean) engine=MergeTree order by a" +$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, e Boolean) engine=MergeTree order by a" cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" $CLICKHOUSE_CLIENT -q "select * from test_tbl" $CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv index faedd9b6705..e829cc0106a 100644 --- a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv +++ b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv @@ -1,5 +1,5 @@ -1,abc,2023-03-14,2023-03-14 11:22:33,true -2,c,ab,2023,false -bc,111,ab,2023-03-24,ban -4,888,2023-03-14,13243433,false -5,bks,2023-03,1689761879,abdd \ No newline at end of file +1,abc,2023-03-14,true +2,c,ab,false +bc,111,ab,ban +4,888,2023-03-14,false +5,bks,2023-03,abdd \ No newline at end of file From b9b3222c6ec9efc61b07a68bd812188894b6d1e1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 19 Jul 2023 12:23:39 +0000 Subject: [PATCH 0857/2047] Add support for CREATE UNIQUE INDEX systax in parser. No UNIQUE index created. --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterCreateIndexQuery.cpp | 9 +++++++++ src/Parsers/ASTCreateIndexQuery.cpp | 2 +- src/Parsers/ASTCreateIndexQuery.h | 1 + src/Parsers/ParserCreateIndexQuery.cpp | 6 ++++++ src/Parsers/ParserCreateIndexQuery.h | 2 +- .../0_stateless/02814_create_index_uniq_noop.reference | 0 .../queries/0_stateless/02814_create_index_uniq_noop.sql | 3 +++ 8 files changed, 22 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02814_create_index_uniq_noop.reference create mode 100644 tests/queries/0_stateless/02814_create_index_uniq_noop.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8f61c3973e3..427b625b5e4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -775,6 +775,7 @@ class IColumn; M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0)\ + M(Bool, create_index_ignore_unique, false, "Ignore UNIQUE keyword in CREATE UNIQUE INDEX. Made for SQL compatibility tests.", 0)\ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp index 752bc6200ce..ef7a7af5303 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.cpp +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -24,6 +24,15 @@ BlockIO InterpreterCreateIndexQuery::execute() auto current_context = getContext(); const auto & create_index = query_ptr->as(); + if (create_index.unique) + { + if (!current_context->getSettingsRef().create_index_ignore_unique) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "CREATE UNIQUE INDEX is not supported." + " SET create_index_ignore_unique=1 to ignore this UNIQUE keyword."); + } + + } // Noop if allow_create_index_without_type = true. throw otherwise if (!create_index.index_decl->as()->type) { diff --git a/src/Parsers/ASTCreateIndexQuery.cpp b/src/Parsers/ASTCreateIndexQuery.cpp index 0d580d5bb21..17d4b9a9d58 100644 --- a/src/Parsers/ASTCreateIndexQuery.cpp +++ b/src/Parsers/ASTCreateIndexQuery.cpp @@ -38,7 +38,7 @@ void ASTCreateIndexQuery::formatQueryImpl(const FormatSettings & settings, Forma settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str; - settings.ostr << "CREATE INDEX " << (if_not_exists ? "IF NOT EXISTS " : ""); + settings.ostr << "CREATE " << (unique ? "UNIQUE " : "") << "INDEX " << (if_not_exists ? "IF NOT EXISTS " : ""); index_name->formatImpl(settings, state, frame); settings.ostr << " ON "; diff --git a/src/Parsers/ASTCreateIndexQuery.h b/src/Parsers/ASTCreateIndexQuery.h index 424a0e493d9..b7577f2634e 100644 --- a/src/Parsers/ASTCreateIndexQuery.h +++ b/src/Parsers/ASTCreateIndexQuery.h @@ -20,6 +20,7 @@ public: ASTPtr index_decl; bool if_not_exists{false}; + bool unique{false}; String getID(char delim) const override; diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index d2ae7f972b7..67051d84999 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -80,6 +80,7 @@ bool ParserCreateIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect node = query; ParserKeyword s_create("CREATE"); + ParserKeyword s_unique("UNIQUE"); ParserKeyword s_index("INDEX"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); ParserKeyword s_on("ON"); @@ -91,10 +92,14 @@ bool ParserCreateIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect String cluster_str; bool if_not_exists = false; + bool unique = false; if (!s_create.ignore(pos, expected)) return false; + if (s_unique.ignore(pos, expected)) + unique = true; + if (!s_index.ignore(pos, expected)) return false; @@ -131,6 +136,7 @@ bool ParserCreateIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect query->children.push_back(index_decl); query->if_not_exists = if_not_exists; + query->unique = unique; query->cluster = cluster_str; if (query->database) diff --git a/src/Parsers/ParserCreateIndexQuery.h b/src/Parsers/ParserCreateIndexQuery.h index 3cb91cd03c6..701586d6e11 100644 --- a/src/Parsers/ParserCreateIndexQuery.h +++ b/src/Parsers/ParserCreateIndexQuery.h @@ -6,7 +6,7 @@ namespace DB { /** Query like this: - * CREATE INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value + * CREATE [UNIQUE] INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value */ class ParserCreateIndexQuery : public IParserBase diff --git a/tests/queries/0_stateless/02814_create_index_uniq_noop.reference b/tests/queries/0_stateless/02814_create_index_uniq_noop.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02814_create_index_uniq_noop.sql b/tests/queries/0_stateless/02814_create_index_uniq_noop.sql new file mode 100644 index 00000000000..127b3cbdeb6 --- /dev/null +++ b/tests/queries/0_stateless/02814_create_index_uniq_noop.sql @@ -0,0 +1,3 @@ +SET allow_create_index_without_type=1; +SET create_index_ignore_unique=1; +CREATE UNIQUE INDEX idx_tab2_0 ON tab2 (col1); From bb4924ab63bff774077dda41df4c11b40f2613e0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 19 Jul 2023 12:24:31 +0000 Subject: [PATCH 0858/2047] Multiple sqllogictest improvements --- docker/test/sqllogic/run.sh | 1 - tests/sqllogic/connection.py | 3 +- tests/sqllogic/runner.py | 70 ++++++++++++++++++-- tests/sqllogic/self-test/canonic_report.json | 2 +- tests/sqllogic/self-test/test.test | 9 +++ tests/sqllogic/test_parser.py | 8 ++- tests/sqllogic/test_runner.py | 5 +- 7 files changed, 87 insertions(+), 11 deletions(-) diff --git a/docker/test/sqllogic/run.sh b/docker/test/sqllogic/run.sh index 444252837a3..4ef42ed377d 100755 --- a/docker/test/sqllogic/run.sh +++ b/docker/test/sqllogic/run.sh @@ -96,5 +96,4 @@ rg -Fa "Fatal" /var/log/clickhouse-server/clickhouse-server.log ||: zstd < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.zst & # Compressed (FIXME: remove once only github actions will be left) -rm /var/log/clickhouse-server/clickhouse-server.log mv /var/log/clickhouse-server/stderr.log /test_output/ ||: diff --git a/tests/sqllogic/connection.py b/tests/sqllogic/connection.py index 0033c29c41c..a49e8f5c62f 100644 --- a/tests/sqllogic/connection.py +++ b/tests/sqllogic/connection.py @@ -62,7 +62,8 @@ def default_clickhouse_odbc_conn_str(): return str( OdbcConnectingArgs.create_from_kw( dsn="ClickHouse DSN (ANSI)", - Url="http://localhost:8123/query?default_format=ODBCDriver2&default_table_engine=MergeTree&union_default_mode=DISTINCT&group_by_use_nulls=1&join_use_nulls=1&allow_create_index_without_type=1", + Timeout="300", + Url="http://localhost:8123/query?default_format=ODBCDriver2&default_table_engine=MergeTree&union_default_mode=DISTINCT&group_by_use_nulls=1&join_use_nulls=1&allow_create_index_without_type=1&create_index_ignore_unique=1", ) ) diff --git a/tests/sqllogic/runner.py b/tests/sqllogic/runner.py index 1cf4c19c649..650cf18cbbe 100755 --- a/tests/sqllogic/runner.py +++ b/tests/sqllogic/runner.py @@ -186,10 +186,10 @@ def mode_check_statements(parser): out_stages_dir = os.path.join(out_dir, f"{args.mode}-stages") - complete_sqlite_dir = os.path.join(out_stages_dir, "complete-sqlite") + complete_sqlite_dir = os.path.join(out_stages_dir, "statements-sqlite") os.makedirs(complete_sqlite_dir, exist_ok=True) - reports["complete-sqlite"] = run_all_tests_in_parallel( + reports["statements-sqlite"] = run_all_tests_in_parallel( setup_kwargs=as_kwargs( engine=Engines.SQLITE, ), @@ -223,6 +223,62 @@ def mode_check_statements(parser): parser.set_defaults(func=calle) +def mode_check_complete(parser): + parser.add_argument("--input-dir", metavar="DIR", required=True) + parser.add_argument("--out-dir", metavar="DIR", required=True) + + def calle(args): + input_dir = os.path.realpath(args.input_dir) + out_dir = os.path.realpath(args.out_dir) + + if not os.path.exists(input_dir): + raise FileNotFoundError( + input_dir, f"check statements: no such file or directory {input_dir}" + ) + + if not os.path.isdir(input_dir): + raise NotADirectoryError( + input_dir, f"check statements:: not a dir {input_dir}" + ) + + reports = dict() + + out_stages_dir = os.path.join(out_dir, f"{args.mode}-stages") + + complete_sqlite_dir = os.path.join(out_stages_dir, "complete-sqlite") + os.makedirs(complete_sqlite_dir, exist_ok=True) + + reports["complete-sqlite"] = run_all_tests_in_parallel( + setup_kwargs=as_kwargs( + engine=Engines.SQLITE, + ), + runner_kwargs=as_kwargs( + verify_mode=False, + stop_at_statement_error=True, + ), + input_dir=input_dir, + output_dir=complete_sqlite_dir, + ) + + verify_clickhouse_dir = os.path.join(out_stages_dir, "complete-clickhouse") + os.makedirs(verify_clickhouse_dir, exist_ok=True) + + reports["complete-clickhouse"] = run_all_tests_in_parallel( + setup_kwargs=as_kwargs( + engine=Engines.ODBC, + conn_str=default_clickhouse_odbc_conn_str(), + ), + runner_kwargs=as_kwargs( + verify_mode=True, + stop_at_statement_error=True, + ), + input_dir=complete_sqlite_dir, + output_dir=verify_clickhouse_dir, + ) + + statements_report(reports, out_dir, args.mode) + + parser.set_defaults(func=calle) def make_actual_report(reports): return {stage: report.get_map() for stage, report in reports.items()} @@ -399,16 +455,22 @@ def parse_args(): ) subparsers = parser.add_subparsers(dest="mode") + mode_check_complete( + subparsers.add_parser( + "complete-test", + help="Run all tests. Check that all statements and queries are passed", + ) + ) mode_check_statements( subparsers.add_parser( "statements-test", - help="Run all test. Check that all statements are passed", + help="Run all tests. Check that all statements are passed", ) ) mode_self_test( subparsers.add_parser( "self-test", - help="Run all test. Check that all statements are passed", + help="Run all tests. Check that all statements are passed", ) ) args = parser.parse_args() diff --git a/tests/sqllogic/self-test/canonic_report.json b/tests/sqllogic/self-test/canonic_report.json index 0cd1aa4b43b..09adc0e1c1d 100644 --- a/tests/sqllogic/self-test/canonic_report.json +++ b/tests/sqllogic/self-test/canonic_report.json @@ -1 +1 @@ -{"sqlite-complete": {"dbms_name": "sqlite", "stats": {"statements": {"success": 4, "fail": 0}, "queries": {"success": 14, "fail": 4}, "total": {"success": 18, "fail": 4}}, "input_dir": "/clickhouse-tests/sqllogic/self-test", "output_dir": "/test_output/self-test/self-test-stages/sqlite-complete", "tests": {"test.test": {"test_name": "test.test", "test_file": "/clickhouse-tests/sqllogic/self-test/test.test", "stats": {"statements": {"success": 4, "fail": 0}, "queries": {"success": 14, "fail": 4}, "total": {"success": 18, "fail": 4}}, "requests": {"5": {"status": "success", "position": 5, "request_type": "statement", "request": "CREATE TABLE t1(a INTEGER, b INTEGER)", "reason": "success"}, "9": {"status": "success", "position": 9, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(1,2)", "reason": "success"}, "12": {"status": "success", "position": 12, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(3,4)", "reason": "success"}, "15": {"status": "success", "position": 15, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(5,6)", "reason": "success"}, "18": {"status": "success", "position": 18, "request_type": "query", "request": "SELECT a, b FROM t1 ORDER BY 2,1", "reason": "success"}, "25": {"status": "error", "position": 25, "request_type": "query", "request": "SELECT a, c FROM t1 ORDER BY 2,1", "reason": "query execution failed with an exception, exception: no such column: c"}, "38": {"status": "success", "position": 38, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "success"}, "44": {"status": "error", "position": 44, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "query is expected to fail with different error, details: expected error: expect to fail in a different way, exception: no such column: c"}, "49": {"status": "success", "position": 49, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "55": {"status": "success", "position": 55, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "60": {"status": "success", "position": 60, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "64": {"status": "success", "position": 64, "request_type": "query", "request": "SELECT -1.0", "reason": "success"}, "70": {"status": "success", "position": 70, "request_type": "query", "request": "SELECT -1", "reason": "success"}, "76": {"status": "success", "position": 76, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "success"}, "82": {"status": "error", "position": 82, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "canonic and actual columns count differ, details: expected columns 1, actual columns 2"}, "88": {"status": "error", "position": 88, "request_type": "query", "request": "SELECT 1.0", "reason": "canonic and actual columns count differ, details: expected columns 2, actual columns 1"}, "94": {"status": "success", "position": 94, "request_type": "query", "request": "select a, b from t1 where a = b", "reason": "success"}, "99": {"status": "success", "position": 99, "request_type": "query", "request": "SELECT 1.0013", "reason": "success"}, "105": {"status": "success", "position": 105, "request_type": "query", "request": "SELECT NULL as a", "reason": "success"}, "116": {"status": "success", "position": 116, "request_type": "query", "request": "SELECT NULL", "reason": "success"}, "121": {"status": "success", "position": 121, "request_type": "query", "request": "SELECT 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15", "reason": "success"}, "126": {"status": "success", "position": 126, "request_type": "query", "request": "WITH RECURSIVE cnt(x) AS ( SELECT 1 UNION ALL SELECT x+1 FROM cnt LIMIT 20 ) SELECT x FROM cnt;", "reason": "success"}}}}}, "sqlite-vs-sqlite": {"dbms_name": "sqlite", "stats": {"statements": {"success": 4, "fail": 0}, "queries": {"success": 15, "fail": 3}, "total": {"success": 19, "fail": 3}}, "input_dir": "/test_output/self-test/self-test-stages/sqlite-complete", "output_dir": "/test_output/self-test/self-test-stages/sqlite-vs-sqlite", "tests": {"test.test": {"test_name": "test.test", "test_file": "/test_output/self-test/self-test-stages/sqlite-complete/test.test", "stats": {"statements": {"success": 4, "fail": 0}, "queries": {"success": 15, "fail": 3}, "total": {"success": 19, "fail": 3}}, "requests": {"5": {"status": "success", "position": 5, "request_type": "statement", "request": "CREATE TABLE t1(a INTEGER, b INTEGER)", "reason": "success"}, "9": {"status": "success", "position": 9, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(1,2)", "reason": "success"}, "12": {"status": "success", "position": 12, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(3,4)", "reason": "success"}, "15": {"status": "success", "position": 15, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(5,6)", "reason": "success"}, "18": {"status": "success", "position": 18, "request_type": "query", "request": "SELECT a, b FROM t1 ORDER BY 2,1", "reason": "success"}, "28": {"status": "success", "position": 28, "request_type": "query", "request": "SELECT a, c FROM t1 ORDER BY 2,1", "reason": "success"}, "42": {"status": "success", "position": 42, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "success"}, "48": {"status": "error", "position": 48, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "query is expected to fail with different error, details: expected error: expect to fail in a different way, exception: no such column: c"}, "54": {"status": "success", "position": 54, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "60": {"status": "success", "position": 60, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "66": {"status": "success", "position": 66, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "72": {"status": "success", "position": 72, "request_type": "query", "request": "SELECT -1.0", "reason": "success"}, "78": {"status": "success", "position": 78, "request_type": "query", "request": "SELECT -1", "reason": "success"}, "84": {"status": "success", "position": 84, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "success"}, "90": {"status": "error", "position": 90, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "canonic and actual columns count differ, details: expected columns 1, actual columns 2"}, "96": {"status": "error", "position": 96, "request_type": "query", "request": "SELECT 1.0", "reason": "canonic and actual columns count differ, details: expected columns 2, actual columns 1"}, "102": {"status": "success", "position": 102, "request_type": "query", "request": "select a, b from t1 where a = b", "reason": "success"}, "107": {"status": "success", "position": 107, "request_type": "query", "request": "SELECT 1.0013", "reason": "success"}, "113": {"status": "success", "position": 113, "request_type": "query", "request": "SELECT NULL as a", "reason": "success"}, "124": {"status": "success", "position": 124, "request_type": "query", "request": "SELECT NULL", "reason": "success"}, "129": {"status": "success", "position": 129, "request_type": "query", "request": "SELECT 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15", "reason": "success"}, "135": {"status": "success", "position": 135, "request_type": "query", "request": "WITH RECURSIVE cnt(x) AS ( SELECT 1 UNION ALL SELECT x+1 FROM cnt LIMIT 20 ) SELECT x FROM cnt;", "reason": "success"}}}}}, "clickhouse-complete": {"dbms_name": "ClickHouse", "stats": {"statements": {"success": 4, "fail": 0}, "queries": {"success": 15, "fail": 4}, "total": {"success": 19, "fail": 4}}, "input_dir": "/clickhouse-tests/sqllogic/self-test", "output_dir": "/test_output/self-test/self-test-stages/clickhouse-complete", "tests": {"test.test": {"test_name": "test.test", "test_file": "/clickhouse-tests/sqllogic/self-test/test.test", "stats": {"statements": {"success": 4, "fail": 0}, "queries": {"success": 15, "fail": 4}, "total": {"success": 19, "fail": 4}}, "requests": {"1": {"status": "success", "position": 1, "request_type": "statement", "request": "CREATE TABLE t1(a INTEGER, b INTEGER) ENGINE = MergeTree() PRIMARY KEY tuple()", "reason": "success"}, "9": {"status": "success", "position": 9, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(1,2)", "reason": "success"}, "12": {"status": "success", "position": 12, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(3,4)", "reason": "success"}, "15": {"status": "success", "position": 15, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(5,6)", "reason": "success"}, "18": {"status": "success", "position": 18, "request_type": "query", "request": "SELECT a, b FROM t1 ORDER BY 2,1", "reason": "success"}, "25": {"status": "error", "position": 25, "request_type": "query", "request": "SELECT a, c FROM t1 ORDER BY 2,1", "reason": "query execution failed with an exception, exception: ('HY000', \"[HY000] HTTP status code: 404\nReceived error:\nCode: 47. DB::Exception: Missing columns: 'c' while processing query: 'SELECT a, c FROM t1 ORDER BY c ASC, a ASC', required columns: 'a' 'c', maybe you meant: 'a'. (UNKNOWN_IDENTIFIER) (version 23.3.1.1654 (official build))\n\n (1) (SQLExecDirectW)\")"}, "32": {"status": "success", "position": 32, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "success"}, "44": {"status": "error", "position": 44, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "query is expected to fail with different error, details: expected error: expect to fail in a different way, exception: ('HY000', \"[HY000] HTTP status code: 404\nReceived error:\nCode: 47. DB::Exception: Missing columns: 'c' while processing query: 'SELECT a, c FROM t1', required columns: 'a' 'c', maybe you meant: 'a'. (UNKNOWN_IDENTIFIER) (version 23.3.1.1654 (official build))\n\n (1) (SQLExecDirectW)\")"}, "49": {"status": "success", "position": 49, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "55": {"status": "success", "position": 55, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "60": {"status": "success", "position": 60, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "64": {"status": "success", "position": 64, "request_type": "query", "request": "SELECT -1.0", "reason": "success"}, "70": {"status": "success", "position": 70, "request_type": "query", "request": "SELECT -1", "reason": "success"}, "76": {"status": "success", "position": 76, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "success"}, "82": {"status": "error", "position": 82, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "canonic and actual columns count differ, details: expected columns 1, actual columns 2"}, "88": {"status": "error", "position": 88, "request_type": "query", "request": "SELECT 1.0", "reason": "canonic and actual columns count differ, details: expected columns 2, actual columns 1"}, "94": {"status": "success", "position": 94, "request_type": "query", "request": "select a, b from t1 where a = b", "reason": "success"}, "99": {"status": "success", "position": 99, "request_type": "query", "request": "SELECT 1.0013", "reason": "success"}, "105": {"status": "success", "position": 105, "request_type": "query", "request": "SELECT NULL as a", "reason": "success"}, "110": {"status": "success", "position": 110, "request_type": "query", "request": "SELECT CAST(NULL AS Nullable(INTEGER))", "reason": "success"}, "116": {"status": "success", "position": 116, "request_type": "query", "request": "SELECT NULL", "reason": "success"}, "121": {"status": "success", "position": 121, "request_type": "query", "request": "SELECT 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15", "reason": "success"}, "139": {"status": "success", "position": 139, "request_type": "query", "request": "SELECT number+1 from system.numbers LIMIT 20", "reason": "success"}}}}}, "clickhouse-vs-clickhouse": {"dbms_name": "ClickHouse", "stats": {"statements": {"success": 4, "fail": 0}, "queries": {"success": 16, "fail": 3}, "total": {"success": 20, "fail": 3}}, "input_dir": "/test_output/self-test/self-test-stages/clickhouse-complete", "output_dir": "/test_output/self-test/self-test-stages/clickhouse-vs-clickhouse", "tests": {"test.test": {"test_name": "test.test", "test_file": "/test_output/self-test/self-test-stages/clickhouse-complete/test.test", "stats": {"statements": {"success": 4, "fail": 0}, "queries": {"success": 16, "fail": 3}, "total": {"success": 20, "fail": 3}}, "requests": {"1": {"status": "success", "position": 1, "request_type": "statement", "request": "CREATE TABLE t1(a INTEGER, b INTEGER) ENGINE = MergeTree() PRIMARY KEY tuple()", "reason": "success"}, "9": {"status": "success", "position": 9, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(1,2)", "reason": "success"}, "12": {"status": "success", "position": 12, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(3,4)", "reason": "success"}, "15": {"status": "success", "position": 15, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(5,6)", "reason": "success"}, "18": {"status": "success", "position": 18, "request_type": "query", "request": "SELECT a, b FROM t1 ORDER BY 2,1", "reason": "success"}, "28": {"status": "success", "position": 28, "request_type": "query", "request": "SELECT a, c FROM t1 ORDER BY 2,1", "reason": "success"}, "36": {"status": "success", "position": 36, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "success"}, "48": {"status": "error", "position": 48, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "query is expected to fail with different error, details: expected error: expect to fail in a different way, exception: ('HY000', \"[HY000] HTTP status code: 404\nReceived error:\nCode: 47. DB::Exception: Missing columns: 'c' while processing query: 'SELECT a, c FROM t1', required columns: 'a' 'c', maybe you meant: 'a'. (UNKNOWN_IDENTIFIER) (version 23.3.1.1654 (official build))\n\n (1) (SQLExecDirectW)\")"}, "54": {"status": "success", "position": 54, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "60": {"status": "success", "position": 60, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "66": {"status": "success", "position": 66, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "72": {"status": "success", "position": 72, "request_type": "query", "request": "SELECT -1.0", "reason": "success"}, "78": {"status": "success", "position": 78, "request_type": "query", "request": "SELECT -1", "reason": "success"}, "84": {"status": "success", "position": 84, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "success"}, "90": {"status": "error", "position": 90, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "canonic and actual columns count differ, details: expected columns 1, actual columns 2"}, "96": {"status": "error", "position": 96, "request_type": "query", "request": "SELECT 1.0", "reason": "canonic and actual columns count differ, details: expected columns 2, actual columns 1"}, "102": {"status": "success", "position": 102, "request_type": "query", "request": "select a, b from t1 where a = b", "reason": "success"}, "107": {"status": "success", "position": 107, "request_type": "query", "request": "SELECT 1.0013", "reason": "success"}, "113": {"status": "success", "position": 113, "request_type": "query", "request": "SELECT NULL as a", "reason": "success"}, "118": {"status": "success", "position": 118, "request_type": "query", "request": "SELECT CAST(NULL AS Nullable(INTEGER))", "reason": "success"}, "124": {"status": "success", "position": 124, "request_type": "query", "request": "SELECT NULL", "reason": "success"}, "129": {"status": "success", "position": 129, "request_type": "query", "request": "SELECT 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15", "reason": "success"}, "148": {"status": "success", "position": 148, "request_type": "query", "request": "SELECT number+1 from system.numbers LIMIT 20", "reason": "success"}}}}}, "sqlite-vs-clickhouse": {"dbms_name": "ClickHouse", "stats": {"statements": {"success": 4, "fail": 0}, "queries": {"success": 13, "fail": 6}, "total": {"success": 17, "fail": 6}}, "input_dir": "/test_output/self-test/self-test-stages/sqlite-complete", "output_dir": "/test_output/self-test/self-test-stages/sqlite-vs-clickhouse", "tests": {"test.test": {"test_name": "test.test", "test_file": "/test_output/self-test/self-test-stages/sqlite-complete/test.test", "stats": {"statements": {"success": 4, "fail": 0}, "queries": {"success": 13, "fail": 6}, "total": {"success": 17, "fail": 6}}, "requests": {"1": {"status": "success", "position": 1, "request_type": "statement", "request": "CREATE TABLE t1(a INTEGER, b INTEGER) ENGINE = MergeTree() PRIMARY KEY tuple()", "reason": "success"}, "9": {"status": "success", "position": 9, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(1,2)", "reason": "success"}, "12": {"status": "success", "position": 12, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(3,4)", "reason": "success"}, "15": {"status": "success", "position": 15, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(5,6)", "reason": "success"}, "18": {"status": "success", "position": 18, "request_type": "query", "request": "SELECT a, b FROM t1 ORDER BY 2,1", "reason": "success"}, "28": {"status": "error", "position": 28, "request_type": "query", "request": "SELECT a, c FROM t1 ORDER BY 2,1", "reason": "canonic and actual results have different exceptions, details: canonic: query execution failed with an exception, original is: no such column: c, actual: query execution failed with an exception, original is: ('HY000', \"[HY000] HTTP status code: 404\nReceived error:\nCode: 47. DB::Exception: Missing columns: 'c' while processing query: 'SELECT a, c FROM t1 ORDER BY c ASC, a ASC', required columns: 'a' 'c', maybe you meant: 'a'. (UNKNOWN_IDENTIFIER) (version 23.3.1.1654 (official build))\n\n (1) (SQLExecDirectW)\")"}, "36": {"status": "success", "position": 36, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "success"}, "48": {"status": "error", "position": 48, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "query is expected to fail with different error, details: expected error: expect to fail in a different way, exception: ('HY000', \"[HY000] HTTP status code: 404\nReceived error:\nCode: 47. DB::Exception: Missing columns: 'c' while processing query: 'SELECT a, c FROM t1', required columns: 'a' 'c', maybe you meant: 'a'. (UNKNOWN_IDENTIFIER) (version 23.3.1.1654 (official build))\n\n (1) (SQLExecDirectW)\")"}, "54": {"status": "success", "position": 54, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "60": {"status": "success", "position": 60, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "66": {"status": "success", "position": 66, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "72": {"status": "success", "position": 72, "request_type": "query", "request": "SELECT -1.0", "reason": "success"}, "78": {"status": "success", "position": 78, "request_type": "query", "request": "SELECT -1", "reason": "success"}, "84": {"status": "success", "position": 84, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "success"}, "90": {"status": "error", "position": 90, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "canonic and actual columns count differ, details: expected columns 1, actual columns 2"}, "96": {"status": "error", "position": 96, "request_type": "query", "request": "SELECT 1.0", "reason": "canonic and actual columns count differ, details: expected columns 2, actual columns 1"}, "102": {"status": "success", "position": 102, "request_type": "query", "request": "select a, b from t1 where a = b", "reason": "success"}, "107": {"status": "success", "position": 107, "request_type": "query", "request": "SELECT 1.0013", "reason": "success"}, "113": {"status": "error", "position": 113, "request_type": "query", "request": "SELECT NULL as a", "reason": "actual result has exception and canonic result doesn't, details: actual: query execution failed with an exception, original is: ('ODBC SQL type 0 is not yet supported. column-index=0 type=0', 'HY106')"}, "118": {"status": "success", "position": 118, "request_type": "query", "request": "SELECT CAST(NULL AS Nullable(INTEGER))", "reason": "success"}, "124": {"status": "error", "position": 124, "request_type": "query", "request": "SELECT NULL", "reason": "actual result has exception and canonic result doesn't, details: actual: query execution failed with an exception, original is: ('ODBC SQL type 0 is not yet supported. column-index=0 type=0', 'HY106')"}, "129": {"status": "success", "position": 129, "request_type": "query", "request": "SELECT 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15", "reason": "success"}, "148": {"status": "success", "position": 148, "request_type": "query", "request": "SELECT number+1 from system.numbers LIMIT 20", "reason": "success"}}}}}} +{"sqlite-complete": {"dbms_name": "sqlite", "stats": {"statements": {"success": 6, "fail": 0}, "queries": {"success": 14, "fail": 5}, "total": {"success": 20, "fail": 5}}, "input_dir": "/clickhouse-tests/sqllogic/self-test", "output_dir": "/test_output/self-test/self-test-stages/sqlite-complete", "tests": {"test.test": {"test_name": "test.test", "test_file": "/clickhouse-tests/sqllogic/self-test/test.test", "stats": {"statements": {"success": 6, "fail": 0}, "queries": {"success": 14, "fail": 5}, "total": {"success": 20, "fail": 5}}, "requests": {"5": {"status": "success", "position": 5, "request_type": "statement", "request": "CREATE TABLE t1(a INTEGER, b INTEGER)", "reason": "success"}, "9": {"status": "success", "position": 9, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(1,2)", "reason": "success"}, "12": {"status": "success", "position": 12, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(3,4)", "reason": "success"}, "15": {"status": "success", "position": 15, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(5,6)", "reason": "success"}, "18": {"status": "success", "position": 18, "request_type": "query", "request": "SELECT a, b FROM t1 ORDER BY 2,1", "reason": "success"}, "25": {"status": "error", "position": 25, "request_type": "query", "request": "SELECT a, c FROM t1 ORDER BY 2,1", "reason": "query execution failed with an exception, exception: no such column: c"}, "38": {"status": "success", "position": 38, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "success"}, "44": {"status": "error", "position": 44, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "query is expected to fail with different error, details: expected error: expect to fail in a different way, exception: no such column: c"}, "49": {"status": "success", "position": 49, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "55": {"status": "success", "position": 55, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "60": {"status": "success", "position": 60, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "64": {"status": "success", "position": 64, "request_type": "query", "request": "SELECT -1.0", "reason": "success"}, "70": {"status": "success", "position": 70, "request_type": "query", "request": "SELECT -1", "reason": "success"}, "76": {"status": "success", "position": 76, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "success"}, "82": {"status": "error", "position": 82, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "canonic and actual columns count differ, details: expected columns 1, actual columns 2"}, "88": {"status": "error", "position": 88, "request_type": "query", "request": "SELECT 1.0", "reason": "canonic and actual columns count differ, details: expected columns 2, actual columns 1"}, "94": {"status": "success", "position": 94, "request_type": "query", "request": "select a, b from t1 where a = b", "reason": "success"}, "99": {"status": "success", "position": 99, "request_type": "query", "request": "SELECT 1.0013", "reason": "success"}, "105": {"status": "success", "position": 105, "request_type": "query", "request": "SELECT NULL as a", "reason": "success"}, "116": {"status": "success", "position": 116, "request_type": "query", "request": "SELECT NULL", "reason": "success"}, "121": {"status": "success", "position": 121, "request_type": "query", "request": "SELECT 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15", "reason": "success"}, "126": {"status": "success", "position": 126, "request_type": "query", "request": "WITH RECURSIVE cnt(x) AS ( SELECT 1 UNION ALL SELECT x+1 FROM cnt LIMIT 20 ) SELECT x FROM cnt;", "reason": "success"}, "145": {"status": "success", "position": 145, "request_type": "statement", "request": "CREATE TABLE tab0(pk INTEGER PRIMARY KEY, col0 INTEGER, col1 FLOAT, col2 TEXT, col3 INTEGER, col4 FLOAT, col5 TEXT)", "reason": "success"}, "149": {"status": "success", "position": 149, "request_type": "statement", "request": "INSERT INTO tab0 VALUES(0,535,860.48,'uxbns',253,640.58,'jvqkl')", "reason": "success"}, "152": {"status": "error", "position": 152, "request_type": "query", "request": "SELECT + col2 AS col5 FROM tab0 WHERE NOT ( col0 ) * - - col4 IS NULL", "reason": "Got non-integer result 'uxbns' for I type."}}}}}, "sqlite-vs-sqlite": {"dbms_name": "sqlite", "stats": {"statements": {"success": 6, "fail": 0}, "queries": {"success": 15, "fail": 4}, "total": {"success": 21, "fail": 4}}, "input_dir": "/test_output/self-test/self-test-stages/sqlite-complete", "output_dir": "/test_output/self-test/self-test-stages/sqlite-vs-sqlite", "tests": {"test.test": {"test_name": "test.test", "test_file": "/test_output/self-test/self-test-stages/sqlite-complete/test.test", "stats": {"statements": {"success": 6, "fail": 0}, "queries": {"success": 15, "fail": 4}, "total": {"success": 21, "fail": 4}}, "requests": {"5": {"status": "success", "position": 5, "request_type": "statement", "request": "CREATE TABLE t1(a INTEGER, b INTEGER)", "reason": "success"}, "9": {"status": "success", "position": 9, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(1,2)", "reason": "success"}, "12": {"status": "success", "position": 12, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(3,4)", "reason": "success"}, "15": {"status": "success", "position": 15, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(5,6)", "reason": "success"}, "18": {"status": "success", "position": 18, "request_type": "query", "request": "SELECT a, b FROM t1 ORDER BY 2,1", "reason": "success"}, "28": {"status": "success", "position": 28, "request_type": "query", "request": "SELECT a, c FROM t1 ORDER BY 2,1", "reason": "success"}, "42": {"status": "success", "position": 42, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "success"}, "48": {"status": "error", "position": 48, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "query is expected to fail with different error, details: expected error: expect to fail in a different way, exception: no such column: c"}, "54": {"status": "success", "position": 54, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "60": {"status": "success", "position": 60, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "66": {"status": "success", "position": 66, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "72": {"status": "success", "position": 72, "request_type": "query", "request": "SELECT -1.0", "reason": "success"}, "78": {"status": "success", "position": 78, "request_type": "query", "request": "SELECT -1", "reason": "success"}, "84": {"status": "success", "position": 84, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "success"}, "90": {"status": "error", "position": 90, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "canonic and actual columns count differ, details: expected columns 1, actual columns 2"}, "96": {"status": "error", "position": 96, "request_type": "query", "request": "SELECT 1.0", "reason": "canonic and actual columns count differ, details: expected columns 2, actual columns 1"}, "102": {"status": "success", "position": 102, "request_type": "query", "request": "select a, b from t1 where a = b", "reason": "success"}, "107": {"status": "success", "position": 107, "request_type": "query", "request": "SELECT 1.0013", "reason": "success"}, "113": {"status": "success", "position": 113, "request_type": "query", "request": "SELECT NULL as a", "reason": "success"}, "124": {"status": "success", "position": 124, "request_type": "query", "request": "SELECT NULL", "reason": "success"}, "129": {"status": "success", "position": 129, "request_type": "query", "request": "SELECT 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15", "reason": "success"}, "135": {"status": "success", "position": 135, "request_type": "query", "request": "WITH RECURSIVE cnt(x) AS ( SELECT 1 UNION ALL SELECT x+1 FROM cnt LIMIT 20 ) SELECT x FROM cnt;", "reason": "success"}, "154": {"status": "success", "position": 154, "request_type": "statement", "request": "CREATE TABLE tab0(pk INTEGER PRIMARY KEY, col0 INTEGER, col1 FLOAT, col2 TEXT, col3 INTEGER, col4 FLOAT, col5 TEXT)", "reason": "success"}, "158": {"status": "success", "position": 158, "request_type": "statement", "request": "INSERT INTO tab0 VALUES(0,535,860.48,'uxbns',253,640.58,'jvqkl')", "reason": "success"}, "161": {"status": "error", "position": 161, "request_type": "query", "request": "SELECT + col2 AS col5 FROM tab0 WHERE NOT ( col0 ) * - - col4 IS NULL", "reason": "Got non-integer result 'uxbns' for I type."}}}}}, "clickhouse-complete": {"dbms_name": "ClickHouse", "stats": {"statements": {"success": 6, "fail": 0}, "queries": {"success": 15, "fail": 4}, "total": {"success": 21, "fail": 4}}, "input_dir": "/clickhouse-tests/sqllogic/self-test", "output_dir": "/test_output/self-test/self-test-stages/clickhouse-complete", "tests": {"test.test": {"test_name": "test.test", "test_file": "/clickhouse-tests/sqllogic/self-test/test.test", "stats": {"statements": {"success": 6, "fail": 0}, "queries": {"success": 15, "fail": 4}, "total": {"success": 21, "fail": 4}}, "requests": {"1": {"status": "success", "position": 1, "request_type": "statement", "request": "CREATE TABLE t1(a INTEGER, b INTEGER) ENGINE = MergeTree() PRIMARY KEY tuple()", "reason": "success"}, "9": {"status": "success", "position": 9, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(1,2)", "reason": "success"}, "12": {"status": "success", "position": 12, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(3,4)", "reason": "success"}, "15": {"status": "success", "position": 15, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(5,6)", "reason": "success"}, "18": {"status": "success", "position": 18, "request_type": "query", "request": "SELECT a, b FROM t1 ORDER BY 2,1", "reason": "success"}, "25": {"status": "error", "position": 25, "request_type": "query", "request": "SELECT a, c FROM t1 ORDER BY 2,1", "reason": "query execution failed with an exception, exception: ('HY000', \"[HY000] HTTP status code: 404\\nReceived error:\\nCode: 47. DB::Exception: Missing columns: 'c' while processing query: 'SELECT a, c FROM t1 ORDER BY c ASC, a ASC', required columns: 'a' 'c', maybe you meant: 'a'. (UNKNOWN_IDENTIFIER) (version 23.7.1.1)\\n\\n (1) (SQLExecDirectW)\")"}, "32": {"status": "success", "position": 32, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "success"}, "44": {"status": "error", "position": 44, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "query is expected to fail with different error, details: expected error: expect to fail in a different way, exception: ('HY000', \"[HY000] HTTP status code: 404\\nReceived error:\\nCode: 47. DB::Exception: Missing columns: 'c' while processing query: 'SELECT a, c FROM t1', required columns: 'a' 'c', maybe you meant: 'a'. (UNKNOWN_IDENTIFIER) (version 23.7.1.1)\\n\\n (1) (SQLExecDirectW)\")"}, "49": {"status": "success", "position": 49, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "55": {"status": "success", "position": 55, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "60": {"status": "success", "position": 60, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "64": {"status": "success", "position": 64, "request_type": "query", "request": "SELECT -1.0", "reason": "success"}, "70": {"status": "success", "position": 70, "request_type": "query", "request": "SELECT -1", "reason": "success"}, "76": {"status": "success", "position": 76, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "success"}, "82": {"status": "error", "position": 82, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "canonic and actual columns count differ, details: expected columns 1, actual columns 2"}, "88": {"status": "error", "position": 88, "request_type": "query", "request": "SELECT 1.0", "reason": "canonic and actual columns count differ, details: expected columns 2, actual columns 1"}, "94": {"status": "success", "position": 94, "request_type": "query", "request": "select a, b from t1 where a = b", "reason": "success"}, "99": {"status": "success", "position": 99, "request_type": "query", "request": "SELECT 1.0013", "reason": "success"}, "105": {"status": "success", "position": 105, "request_type": "query", "request": "SELECT NULL as a", "reason": "success"}, "110": {"status": "success", "position": 110, "request_type": "query", "request": "SELECT CAST(NULL AS Nullable(INTEGER))", "reason": "success"}, "116": {"status": "success", "position": 116, "request_type": "query", "request": "SELECT NULL", "reason": "success"}, "121": {"status": "success", "position": 121, "request_type": "query", "request": "SELECT 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15", "reason": "success"}, "139": {"status": "success", "position": 139, "request_type": "query", "request": "SELECT number+1 from system.numbers LIMIT 20", "reason": "success"}, "145": {"status": "success", "position": 145, "request_type": "statement", "request": "CREATE TABLE tab0(pk INTEGER PRIMARY KEY, col0 INTEGER, col1 FLOAT, col2 TEXT, col3 INTEGER, col4 FLOAT, col5 TEXT)", "reason": "success"}, "149": {"status": "success", "position": 149, "request_type": "statement", "request": "INSERT INTO tab0 VALUES(0,535,860.48,'uxbns',253,640.58,'jvqkl')", "reason": "success"}}}}}, "clickhouse-vs-clickhouse": {"dbms_name": "ClickHouse", "stats": {"statements": {"success": 6, "fail": 0}, "queries": {"success": 16, "fail": 3}, "total": {"success": 22, "fail": 3}}, "input_dir": "/test_output/self-test/self-test-stages/clickhouse-complete", "output_dir": "/test_output/self-test/self-test-stages/clickhouse-vs-clickhouse", "tests": {"test.test": {"test_name": "test.test", "test_file": "/test_output/self-test/self-test-stages/clickhouse-complete/test.test", "stats": {"statements": {"success": 6, "fail": 0}, "queries": {"success": 16, "fail": 3}, "total": {"success": 22, "fail": 3}}, "requests": {"1": {"status": "success", "position": 1, "request_type": "statement", "request": "CREATE TABLE t1(a INTEGER, b INTEGER) ENGINE = MergeTree() PRIMARY KEY tuple()", "reason": "success"}, "9": {"status": "success", "position": 9, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(1,2)", "reason": "success"}, "12": {"status": "success", "position": 12, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(3,4)", "reason": "success"}, "15": {"status": "success", "position": 15, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(5,6)", "reason": "success"}, "18": {"status": "success", "position": 18, "request_type": "query", "request": "SELECT a, b FROM t1 ORDER BY 2,1", "reason": "success"}, "28": {"status": "success", "position": 28, "request_type": "query", "request": "SELECT a, c FROM t1 ORDER BY 2,1", "reason": "success"}, "36": {"status": "success", "position": 36, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "success"}, "48": {"status": "error", "position": 48, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "query is expected to fail with different error, details: expected error: expect to fail in a different way, exception: ('HY000', \"[HY000] HTTP status code: 404\\nReceived error:\\nCode: 47. DB::Exception: Missing columns: 'c' while processing query: 'SELECT a, c FROM t1', required columns: 'a' 'c', maybe you meant: 'a'. (UNKNOWN_IDENTIFIER) (version 23.7.1.1)\\n\\n (1) (SQLExecDirectW)\")"}, "54": {"status": "success", "position": 54, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "60": {"status": "success", "position": 60, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "66": {"status": "success", "position": 66, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "72": {"status": "success", "position": 72, "request_type": "query", "request": "SELECT -1.0", "reason": "success"}, "78": {"status": "success", "position": 78, "request_type": "query", "request": "SELECT -1", "reason": "success"}, "84": {"status": "success", "position": 84, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "success"}, "90": {"status": "error", "position": 90, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "canonic and actual columns count differ, details: expected columns 1, actual columns 2"}, "96": {"status": "error", "position": 96, "request_type": "query", "request": "SELECT 1.0", "reason": "canonic and actual columns count differ, details: expected columns 2, actual columns 1"}, "102": {"status": "success", "position": 102, "request_type": "query", "request": "select a, b from t1 where a = b", "reason": "success"}, "107": {"status": "success", "position": 107, "request_type": "query", "request": "SELECT 1.0013", "reason": "success"}, "113": {"status": "success", "position": 113, "request_type": "query", "request": "SELECT NULL as a", "reason": "success"}, "118": {"status": "success", "position": 118, "request_type": "query", "request": "SELECT CAST(NULL AS Nullable(INTEGER))", "reason": "success"}, "124": {"status": "success", "position": 124, "request_type": "query", "request": "SELECT NULL", "reason": "success"}, "129": {"status": "success", "position": 129, "request_type": "query", "request": "SELECT 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15", "reason": "success"}, "148": {"status": "success", "position": 148, "request_type": "query", "request": "SELECT number+1 from system.numbers LIMIT 20", "reason": "success"}, "154": {"status": "success", "position": 154, "request_type": "statement", "request": "CREATE TABLE tab0(pk INTEGER PRIMARY KEY, col0 INTEGER, col1 FLOAT, col2 TEXT, col3 INTEGER, col4 FLOAT, col5 TEXT)", "reason": "success"}, "158": {"status": "success", "position": 158, "request_type": "statement", "request": "INSERT INTO tab0 VALUES(0,535,860.48,'uxbns',253,640.58,'jvqkl')", "reason": "success"}}}}}, "sqlite-vs-clickhouse": {"dbms_name": "ClickHouse", "stats": {"statements": {"success": 6, "fail": 0}, "queries": {"success": 13, "fail": 6}, "total": {"success": 19, "fail": 6}}, "input_dir": "/test_output/self-test/self-test-stages/sqlite-complete", "output_dir": "/test_output/self-test/self-test-stages/sqlite-vs-clickhouse", "tests": {"test.test": {"test_name": "test.test", "test_file": "/test_output/self-test/self-test-stages/sqlite-complete/test.test", "stats": {"statements": {"success": 6, "fail": 0}, "queries": {"success": 13, "fail": 6}, "total": {"success": 19, "fail": 6}}, "requests": {"1": {"status": "success", "position": 1, "request_type": "statement", "request": "CREATE TABLE t1(a INTEGER, b INTEGER) ENGINE = MergeTree() PRIMARY KEY tuple()", "reason": "success"}, "9": {"status": "success", "position": 9, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(1,2)", "reason": "success"}, "12": {"status": "success", "position": 12, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(3,4)", "reason": "success"}, "15": {"status": "success", "position": 15, "request_type": "statement", "request": "INSERT INTO t1(a,b) VALUES(5,6)", "reason": "success"}, "18": {"status": "success", "position": 18, "request_type": "query", "request": "SELECT a, b FROM t1 ORDER BY 2,1", "reason": "success"}, "28": {"status": "error", "position": 28, "request_type": "query", "request": "SELECT a, c FROM t1 ORDER BY 2,1", "reason": "canonic and actual results have different exceptions, details: canonic: query execution failed with an exception, original is: no such column: c, actual: query execution failed with an exception, original is: ('HY000', \"[HY000] HTTP status code: 404\\nReceived error:\\nCode: 47. DB::Exception: Missing columns: 'c' while processing query: 'SELECT a, c FROM t1 ORDER BY c ASC, a ASC', required columns: 'a' 'c', maybe you meant: 'a'. (UNKNOWN_IDENTIFIER) (version 23.7.1.1)\\n\\n (1) (SQLExecDirectW)\")"}, "36": {"status": "success", "position": 36, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "success"}, "48": {"status": "error", "position": 48, "request_type": "query", "request": "SELECT a, c FROM t1", "reason": "query is expected to fail with different error, details: expected error: expect to fail in a different way, exception: ('HY000', \"[HY000] HTTP status code: 404\\nReceived error:\\nCode: 47. DB::Exception: Missing columns: 'c' while processing query: 'SELECT a, c FROM t1', required columns: 'a' 'c', maybe you meant: 'a'. (UNKNOWN_IDENTIFIER) (version 23.7.1.1)\\n\\n (1) (SQLExecDirectW)\")"}, "54": {"status": "success", "position": 54, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "60": {"status": "success", "position": 60, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "66": {"status": "success", "position": 66, "request_type": "query", "request": "SELECT ''", "reason": "success"}, "72": {"status": "success", "position": 72, "request_type": "query", "request": "SELECT -1.0", "reason": "success"}, "78": {"status": "success", "position": 78, "request_type": "query", "request": "SELECT -1", "reason": "success"}, "84": {"status": "success", "position": 84, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "success"}, "90": {"status": "error", "position": 90, "request_type": "query", "request": "SELECT 1.0, 1", "reason": "canonic and actual columns count differ, details: expected columns 1, actual columns 2"}, "96": {"status": "error", "position": 96, "request_type": "query", "request": "SELECT 1.0", "reason": "canonic and actual columns count differ, details: expected columns 2, actual columns 1"}, "102": {"status": "success", "position": 102, "request_type": "query", "request": "select a, b from t1 where a = b", "reason": "success"}, "107": {"status": "success", "position": 107, "request_type": "query", "request": "SELECT 1.0013", "reason": "success"}, "113": {"status": "error", "position": 113, "request_type": "query", "request": "SELECT NULL as a", "reason": "actual result has exception and canonic result doesn't, details: actual: query execution failed with an exception, original is: ('ODBC SQL type 0 is not yet supported. column-index=0 type=0', 'HY106')"}, "118": {"status": "success", "position": 118, "request_type": "query", "request": "SELECT CAST(NULL AS Nullable(INTEGER))", "reason": "success"}, "124": {"status": "error", "position": 124, "request_type": "query", "request": "SELECT NULL", "reason": "actual result has exception and canonic result doesn't, details: actual: query execution failed with an exception, original is: ('ODBC SQL type 0 is not yet supported. column-index=0 type=0', 'HY106')"}, "129": {"status": "success", "position": 129, "request_type": "query", "request": "SELECT 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15", "reason": "success"}, "148": {"status": "success", "position": 148, "request_type": "query", "request": "SELECT number+1 from system.numbers LIMIT 20", "reason": "success"}, "154": {"status": "success", "position": 154, "request_type": "statement", "request": "CREATE TABLE tab0(pk INTEGER PRIMARY KEY, col0 INTEGER, col1 FLOAT, col2 TEXT, col3 INTEGER, col4 FLOAT, col5 TEXT)", "reason": "success"}, "158": {"status": "success", "position": 158, "request_type": "statement", "request": "INSERT INTO tab0 VALUES(0,535,860.48,'uxbns',253,640.58,'jvqkl')", "reason": "success"}}}}}} \ No newline at end of file diff --git a/tests/sqllogic/self-test/test.test b/tests/sqllogic/self-test/test.test index 85b27ed7d60..503153acef8 100644 --- a/tests/sqllogic/self-test/test.test +++ b/tests/sqllogic/self-test/test.test @@ -142,4 +142,13 @@ SELECT number+1 from system.numbers LIMIT 20 ---- 20 values hashing to 52c46dff81346ead02fcf6245c762b1a +# Debug how incorrect result type parses +statement ok +CREATE TABLE tab0(pk INTEGER PRIMARY KEY, col0 INTEGER, col1 FLOAT, col2 TEXT, col3 INTEGER, col4 FLOAT, col5 TEXT) +statement ok +INSERT INTO tab0 VALUES(0,535,860.48,'uxbns',253,640.58,'jvqkl') + +skipif ClickHouse +query I rowsort label-20 +SELECT + col2 AS col5 FROM tab0 WHERE NOT ( col0 ) * - - col4 IS NULL diff --git a/tests/sqllogic/test_parser.py b/tests/sqllogic/test_parser.py index 42adb83809f..d417af8e44f 100755 --- a/tests/sqllogic/test_parser.py +++ b/tests/sqllogic/test_parser.py @@ -9,7 +9,7 @@ from enum import Enum from hashlib import md5 from functools import reduce -from exceptions import Error, ProgramError, ErrorWithParent, DataResultDiffer +from exceptions import Error, ProgramError, ErrorWithParent, DataResultDiffer, QueryExecutionError logger = logging.getLogger("parser") @@ -480,6 +480,7 @@ class QueryResult: for row in rows: res_row = [] for c, t in zip(row, types): + logger.debug(f"Builging row. c:{c} t:{t}") if c is None: res_row.append("NULL") continue @@ -490,7 +491,10 @@ class QueryResult: else: res_row.append(str(c)) elif t == "I": - res_row.append(str(int(c))) + try: + res_row.append(str(int(c))) + except ValueError as ex: + raise QueryExecutionError(f"Got non-integer result '{c}' for I type.") elif t == "R": res_row.append(f"{c:.3f}") diff --git a/tests/sqllogic/test_runner.py b/tests/sqllogic/test_runner.py index 3df38e7fce5..f9ed23566b4 100644 --- a/tests/sqllogic/test_runner.py +++ b/tests/sqllogic/test_runner.py @@ -361,7 +361,7 @@ class TestRunner: continue if block.get_block_type() == test_parser.BlockType.control: - clogger.debug("Skip control block", name_pos) + clogger.debug("Skip control block %s", name_pos) block.dump_to(out_stream) continue @@ -374,13 +374,14 @@ class TestRunner: continue request = block.get_request() - exec_res = execute_request(request, self.connection) if block.get_block_type() in self.skip_request_types: clogger.debug("Runtime skip block for %s", self.dbms_name) block.dump_to(out_stream) continue + exec_res = execute_request(request, self.connection) + if block.get_block_type() == test_parser.BlockType.statement: try: clogger.debug("this is statement") From 0c86df519ffa8921b6c546b304705366838dfa21 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 19 Jul 2023 12:41:25 +0000 Subject: [PATCH 0859/2047] Fix unspported disks in Keeper --- src/Coordination/KeeperContext.cpp | 31 ++++++++++++++++++- src/Disks/DiskSelector.cpp | 5 ++- src/Disks/DiskSelector.h | 3 +- .../configs/enable_keeper.xml | 4 +++ tests/integration/test_keeper_disks/test.py | 12 ++++++- 5 files changed, 51 insertions(+), 4 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 408344ee67f..32f8b98a7ed 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -41,9 +41,38 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config) initializeDisks(config); } +namespace +{ + +bool diskValidator(const Poco::Util::AbstractConfiguration & config, const std::string & disk_config_prefix) +{ + const auto disk_type = config.getString(disk_config_prefix + ".type", "local"); + + using namespace std::literals; + static constexpr std::array supported_disk_types + { + "s3"sv, + "s3_plain"sv, + "local"sv + }; + + if (std::all_of( + supported_disk_types.begin(), + supported_disk_types.end(), + [&](const auto supported_type) { return disk_type != supported_type; })) + { + LOG_INFO(&Poco::Logger::get("KeeperContext"), "Disk type '{}' is not supported for Keeper", disk_type); + return false; + } + + return true; +} + +} + void KeeperContext::initializeDisks(const Poco::Util::AbstractConfiguration & config) { - disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance()); + disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance(), diskValidator); log_storage = getLogsPathFromConfig(config); diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index e51f79867b5..415e10a55fc 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -27,7 +27,7 @@ void DiskSelector::assertInitialized() const } -void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) +void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, DiskValidator disk_validator) { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); @@ -46,6 +46,9 @@ void DiskSelector::initialize(const Poco::Util::AbstractConfiguration & config, auto disk_config_prefix = config_prefix + "." + disk_name; + if (disk_validator && !disk_validator(config, disk_config_prefix)) + continue; + disks.emplace(disk_name, factory.create(disk_name, config, disk_config_prefix, context, disks)); } if (!has_default_disk) diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index 58adeb953db..c91c3acb3bd 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -23,7 +23,8 @@ public: DiskSelector() = default; DiskSelector(const DiskSelector & from) = default; - void initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); + using DiskValidator = std::function; + void initialize(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, DiskValidator disk_validator = {}); DiskSelectorPtr updateFromConfig( const Poco::Util::AbstractConfiguration & config, diff --git a/tests/integration/test_keeper_disks/configs/enable_keeper.xml b/tests/integration/test_keeper_disks/configs/enable_keeper.xml index 5814979229c..50d0329637a 100644 --- a/tests/integration/test_keeper_disks/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_disks/configs/enable_keeper.xml @@ -1,6 +1,10 @@ + + hdfs + hdfs://hdfs1:9000/ + local /var/lib/clickhouse/coordination/logs/ diff --git a/tests/integration/test_keeper_disks/test.py b/tests/integration/test_keeper_disks/test.py index 11bb215be54..86682bcde01 100644 --- a/tests/integration/test_keeper_disks/test.py +++ b/tests/integration/test_keeper_disks/test.py @@ -9,7 +9,11 @@ import os CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", main_configs=["configs/enable_keeper.xml"], stay_alive=True, with_minio=True + "node", + main_configs=["configs/enable_keeper.xml"], + stay_alive=True, + with_minio=True, + with_hdfs=True, ) from kazoo.client import KazooClient, KazooState @@ -117,6 +121,12 @@ def get_local_snapshots(): return get_local_files("/var/lib/clickhouse/coordination/snapshots") +def test_supported_disk_types(started_cluster): + node.stop_clickhouse() + node.start_clickhouse() + node.contains_in_log("Disk type 'hdfs' is not supported for Keeper") + + def test_logs_with_disks(started_cluster): setup_local_storage(started_cluster) From 7b3564f96aa44bde8aa33914930ca3bbf1c5f52e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Jul 2023 14:44:59 +0200 Subject: [PATCH 0860/2047] Revert "Improve CSVInputFormat to check and set default value to column if deserialize failed" --- docs/en/interfaces/formats.md | 1 - .../operations/settings/settings-formats.md | 22 ---------- src/Core/Settings.h | 1 - src/Formats/FormatFactory.cpp | 1 - src/Formats/FormatSettings.h | 1 - .../Formats/Impl/CSVRowInputFormat.cpp | 42 +------------------ .../Formats/Impl/CSVRowInputFormat.h | 2 - ...11_csv_input_field_type_mismatch.reference | 5 --- .../02811_csv_input_field_type_mismatch.sh | 13 ------ .../data_csv/csv_with_bad_field_values.csv | 5 --- 10 files changed, 2 insertions(+), 91 deletions(-) delete mode 100644 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference delete mode 100755 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh delete mode 100644 tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ddf4ab3f78e..ed2f010a632 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,7 +472,6 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. -- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index c8adc83d3ad..0915c51806a 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -989,28 +989,6 @@ Result a b ``` -### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} - -Allow to set default value to column when CSV field deserialization failed on bad value - -Default value: `false`. - -**Examples** - -Query - -```bash -./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" -echo 'a,b,c' | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" -./clickhouse local -q "select * from test_tbl" -``` - -Result - -```text -a 0 1971-01-01 -``` - ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a4cb0c2dbd9..730b6ab80ed 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -874,7 +874,6 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ - M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialization failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 6e3e086859b..8eacc7acc97 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,7 +73,6 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; - format_settings.csv.use_default_on_bad_values = settings.input_format_csv_use_default_on_bad_values; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index e321e5264ca..af90e4462dd 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -152,7 +152,6 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; - bool use_default_on_bad_values = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 244b906549e..79ce2549b4d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -316,54 +315,17 @@ bool CSVFormatReader::readField( return false; } - if (format_settings.csv.use_default_on_bad_values) - return readFieldOrDefault(column, type, serialization); - return readFieldImpl(*buf, column, type, serialization); -} - -bool CSVFormatReader::readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) -{ if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) { /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, istr, format_settings, serialization); + return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); } /// Read the column normally. - serialization->deserializeTextCSV(column, istr, format_settings); + serialization->deserializeTextCSV(column, *buf, format_settings); return true; } -bool CSVFormatReader::readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) -{ - String field; - readCSVField(field, *buf, format_settings.csv); - ReadBufferFromString tmp_buf(field); - bool is_bad_value = false; - bool res = false; - - size_t col_size = column.size(); - try - { - res = readFieldImpl(tmp_buf, column, type, serialization); - /// Check if we parsed the whole field successfully. - if (!field.empty() && !tmp_buf.eof()) - is_bad_value = true; - } - catch (const Exception &) - { - is_bad_value = true; - } - - if (!is_bad_value) - return res; - - if (column.size() == col_size + 1) - column.popBack(1); - column.insertDefault(); - return false; -} - void CSVFormatReader::skipPrefixBeforeHeader() { for (size_t i = 0; i != format_settings.csv.skip_first_lines; ++i) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 7b1a1fc433d..8ccf04feed3 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -89,8 +89,6 @@ public: void setReadBuffer(ReadBuffer & in_) override; FormatSettings::EscapingRule getEscapingRule() const override { return FormatSettings::EscapingRule::CSV; } - bool readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); - bool readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); protected: PeekableReadBuffer * buf; diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference deleted file mode 100644 index 6abcc56bacc..00000000000 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference +++ /dev/null @@ -1,5 +0,0 @@ -0 111 1970-01-01 false -1 abc 2023-03-14 true -2 c 1970-01-01 false -4 888 2023-03-14 false -5 bks 1970-01-01 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh deleted file mode 100755 index 30223329eca..00000000000 --- a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/usr/bin/env bash - -# NOTE: this sh wrapper is required because of shell_config - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" -$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, e Boolean) engine=MergeTree order by a" -cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" -$CLICKHOUSE_CLIENT -q "select * from test_tbl" -$CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv deleted file mode 100644 index e829cc0106a..00000000000 --- a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv +++ /dev/null @@ -1,5 +0,0 @@ -1,abc,2023-03-14,true -2,c,ab,false -bc,111,ab,ban -4,888,2023-03-14,false -5,bks,2023-03,abdd \ No newline at end of file From bab212f25e91cea8c39b2d025735283b7adcaa71 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 19 Jul 2023 12:47:29 +0000 Subject: [PATCH 0861/2047] Automatic style fix --- tests/sqllogic/runner.py | 2 ++ tests/sqllogic/test_parser.py | 12 ++++++++++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/tests/sqllogic/runner.py b/tests/sqllogic/runner.py index 650cf18cbbe..5f4baf8e59b 100755 --- a/tests/sqllogic/runner.py +++ b/tests/sqllogic/runner.py @@ -223,6 +223,7 @@ def mode_check_statements(parser): parser.set_defaults(func=calle) + def mode_check_complete(parser): parser.add_argument("--input-dir", metavar="DIR", required=True) parser.add_argument("--out-dir", metavar="DIR", required=True) @@ -280,6 +281,7 @@ def mode_check_complete(parser): parser.set_defaults(func=calle) + def make_actual_report(reports): return {stage: report.get_map() for stage, report in reports.items()} diff --git a/tests/sqllogic/test_parser.py b/tests/sqllogic/test_parser.py index d417af8e44f..f6ad955e7b0 100755 --- a/tests/sqllogic/test_parser.py +++ b/tests/sqllogic/test_parser.py @@ -9,7 +9,13 @@ from enum import Enum from hashlib import md5 from functools import reduce -from exceptions import Error, ProgramError, ErrorWithParent, DataResultDiffer, QueryExecutionError +from exceptions import ( + Error, + ProgramError, + ErrorWithParent, + DataResultDiffer, + QueryExecutionError, +) logger = logging.getLogger("parser") @@ -494,7 +500,9 @@ class QueryResult: try: res_row.append(str(int(c))) except ValueError as ex: - raise QueryExecutionError(f"Got non-integer result '{c}' for I type.") + raise QueryExecutionError( + f"Got non-integer result '{c}' for I type." + ) elif t == "R": res_row.append(f"{c:.3f}") From f0026af1893772a7a14f21d11cc88307ba07500a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Jul 2023 14:51:11 +0200 Subject: [PATCH 0862/2047] Revert "Revert "Improve CSVInputFormat to check and set default value to column if deserialize failed"" --- docs/en/interfaces/formats.md | 1 + .../operations/settings/settings-formats.md | 22 ++++++++++ src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/CSVRowInputFormat.cpp | 42 ++++++++++++++++++- .../Formats/Impl/CSVRowInputFormat.h | 2 + ...11_csv_input_field_type_mismatch.reference | 5 +++ .../02811_csv_input_field_type_mismatch.sh | 13 ++++++ .../data_csv/csv_with_bad_field_values.csv | 5 +++ 10 files changed, 91 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference create mode 100755 tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh create mode 100644 tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ed2f010a632..ddf4ab3f78e 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -472,6 +472,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. - [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. +- [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 0915c51806a..c8adc83d3ad 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -989,6 +989,28 @@ Result a b ``` +### input_format_csv_use_default_on_bad_values {#input_format_csv_use_default_on_bad_values} + +Allow to set default value to column when CSV field deserialization failed on bad value + +Default value: `false`. + +**Examples** + +Query + +```bash +./clickhouse local -q "create table test_tbl (x String, y UInt32, z Date) engine=MergeTree order by x" +echo 'a,b,c' | ./clickhouse local -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" +./clickhouse local -q "select * from test_tbl" +``` + +Result + +```text +a 0 1971-01-01 +``` + ## Values format settings {#values-format-settings} ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 730b6ab80ed..a4cb0c2dbd9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -874,6 +874,7 @@ class IColumn; M(Bool, input_format_csv_detect_header, true, "Automatically detect header with names and types in CSV format", 0) \ M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ + M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialization failed on bad value", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 8eacc7acc97..6e3e086859b 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -73,6 +73,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.trim_whitespaces = settings.input_format_csv_trim_whitespaces; format_settings.csv.allow_whitespace_or_tab_as_delimiter = settings.input_format_csv_allow_whitespace_or_tab_as_delimiter; format_settings.csv.allow_variable_number_of_columns = settings.input_format_csv_allow_variable_number_of_columns; + format_settings.csv.use_default_on_bad_values = settings.input_format_csv_use_default_on_bad_values; format_settings.hive_text.fields_delimiter = settings.input_format_hive_text_fields_delimiter; format_settings.hive_text.collection_items_delimiter = settings.input_format_hive_text_collection_items_delimiter; format_settings.hive_text.map_keys_delimiter = settings.input_format_hive_text_map_keys_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index af90e4462dd..e321e5264ca 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -152,6 +152,7 @@ struct FormatSettings bool trim_whitespaces = true; bool allow_whitespace_or_tab_as_delimiter = false; bool allow_variable_number_of_columns = false; + bool use_default_on_bad_values = false; } csv; struct HiveText diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 79ce2549b4d..244b906549e 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -315,17 +316,54 @@ bool CSVFormatReader::readField( return false; } + if (format_settings.csv.use_default_on_bad_values) + return readFieldOrDefault(column, type, serialization); + return readFieldImpl(*buf, column, type, serialization); +} + +bool CSVFormatReader::readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ if (format_settings.null_as_default && !isNullableOrLowCardinalityNullable(type)) { /// If value is null but type is not nullable then use default value instead. - return SerializationNullable::deserializeTextCSVImpl(column, *buf, format_settings, serialization); + return SerializationNullable::deserializeTextCSVImpl(column, istr, format_settings, serialization); } /// Read the column normally. - serialization->deserializeTextCSV(column, *buf, format_settings); + serialization->deserializeTextCSV(column, istr, format_settings); return true; } +bool CSVFormatReader::readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization) +{ + String field; + readCSVField(field, *buf, format_settings.csv); + ReadBufferFromString tmp_buf(field); + bool is_bad_value = false; + bool res = false; + + size_t col_size = column.size(); + try + { + res = readFieldImpl(tmp_buf, column, type, serialization); + /// Check if we parsed the whole field successfully. + if (!field.empty() && !tmp_buf.eof()) + is_bad_value = true; + } + catch (const Exception &) + { + is_bad_value = true; + } + + if (!is_bad_value) + return res; + + if (column.size() == col_size + 1) + column.popBack(1); + column.insertDefault(); + return false; +} + void CSVFormatReader::skipPrefixBeforeHeader() { for (size_t i = 0; i != format_settings.csv.skip_first_lines; ++i) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 8ccf04feed3..7b1a1fc433d 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -89,6 +89,8 @@ public: void setReadBuffer(ReadBuffer & in_) override; FormatSettings::EscapingRule getEscapingRule() const override { return FormatSettings::EscapingRule::CSV; } + bool readFieldImpl(ReadBuffer & istr, DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); + bool readFieldOrDefault(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization); protected: PeekableReadBuffer * buf; diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference new file mode 100644 index 00000000000..6abcc56bacc --- /dev/null +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.reference @@ -0,0 +1,5 @@ +0 111 1970-01-01 false +1 abc 2023-03-14 true +2 c 1970-01-01 false +4 888 2023-03-14 false +5 bks 1970-01-01 false diff --git a/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh new file mode 100755 index 00000000000..30223329eca --- /dev/null +++ b/tests/queries/0_stateless/02811_csv_input_field_type_mismatch.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +# NOTE: this sh wrapper is required because of shell_config + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test_tbl" +$CLICKHOUSE_CLIENT -q "create table test_tbl (a Int32, b String, c Date, e Boolean) engine=MergeTree order by a" +cat $CURDIR/data_csv/csv_with_bad_field_values.csv | ${CLICKHOUSE_CLIENT} -q "INSERT INTO test_tbl SETTINGS input_format_csv_use_default_on_bad_values=true FORMAT CSV" +$CLICKHOUSE_CLIENT -q "select * from test_tbl" +$CLICKHOUSE_CLIENT -q "drop table test_tbl" \ No newline at end of file diff --git a/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv new file mode 100644 index 00000000000..e829cc0106a --- /dev/null +++ b/tests/queries/0_stateless/data_csv/csv_with_bad_field_values.csv @@ -0,0 +1,5 @@ +1,abc,2023-03-14,true +2,c,ab,false +bc,111,ab,ban +4,888,2023-03-14,false +5,bks,2023-03,abdd \ No newline at end of file From bdaf82922a599ab5abb123b4bbd1f41249543ec7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 12:54:25 +0000 Subject: [PATCH 0863/2047] Use formatAST() --- src/Interpreters/Cache/QueryCache.cpp | 9 ++++----- src/Parsers/formatAST.cpp | 4 ++-- src/Parsers/formatAST.h | 2 +- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 1d1543844a2..a6c509e8bb1 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -115,12 +116,10 @@ ASTPtr removeQueryCacheSettings(ASTPtr ast) return transformed_ast; } -String queryStringFromAst(ASTPtr ast) +String queryStringFromAST(ASTPtr ast) { WriteBufferFromOwnString buf; - IAST::FormatSettings format_settings(buf, /*one_line*/ true); - format_settings.show_secrets = false; - ast->format(format_settings); + formatAST(*ast, buf, /*hilite*/ false, /*one_line*/ true, /*show_secrets*/ false); return buf.str(); } @@ -138,7 +137,7 @@ QueryCache::Key::Key( , is_shared(is_shared_) , expires_at(expires_at_) , is_compressed(is_compressed_) - , query_string(queryStringFromAst(ast_)) + , query_string(queryStringFromAST(ast_)) { } diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index fca8ea0aa35..aa1afe17c75 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -4,9 +4,9 @@ namespace DB { -void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line) +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool show_secrets) { - IAST::FormatSettings settings(buf, one_line); + IAST::FormatSettings settings(buf, one_line, show_secrets); settings.hilite = hilite; ast.format(settings); diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index 28af2400a4c..ebd284fc18a 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -11,7 +11,7 @@ class WriteBuffer; /** Takes a syntax tree and turns it back into text. * In case of INSERT query, the data will be missing. */ -void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false); +void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false, bool show_secrets = true); String serializeAST(const IAST & ast, bool one_line = true); From 9c4005b33fc74ab4ec3b68ebc877fdda499e8932 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 19 Jul 2023 15:12:47 +0200 Subject: [PATCH 0864/2047] Add logging about all found workflows for merge_pr.py --- tests/ci/merge_pr.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 14844ed9b25..35b0614b01f 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -246,6 +246,12 @@ def main(): if args.check_running_workflows: workflows = get_workflows_for_head(repo, pr.head.sha) + logging.info( + "The PR #%s has following workflows:\n%s", + pr.number, + "\n".join(f"{wf.html_url}: status is {wf.status}" for wf in workflows), + ) + workflows_in_progress = [wf for wf in workflows if wf.status != "completed"] # At most one workflow in progress is fine. We check that there no # cases like, e.g. PullRequestCI and DocksCheck in progress at once From 53818dde8cef7dd573217fa049d01b233a076ac2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 19 Jul 2023 15:22:25 +0200 Subject: [PATCH 0865/2047] MergeTree/ReplicatedMergeTree should use server timezone for log entries Otherwise session_timezone/use_client_time_zone will break things Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeMutationEntry.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index cac26c5ac23..4dbccb91620 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -61,7 +61,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP { auto out = disk->writeFile(std::filesystem::path(path_prefix) / file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings); *out << "format version: 1\n" - << "create time: " << LocalDateTime(create_time) << "\n"; + << "create time: " << LocalDateTime(create_time, DateLUT::serverTimezoneInstance()) << "\n"; *out << "commands: "; commands.writeText(*out, /* with_pure_metadata_commands = */ false); *out << "\n"; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index ac956433eab..9eb8b6ce24c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -48,7 +48,7 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const format_version = std::max(format_version, FORMAT_WITH_LOG_ENTRY_ID); out << "format version: " << format_version << "\n" - << "create_time: " << LocalDateTime(create_time ? create_time : time(nullptr)) << "\n" + << "create_time: " << LocalDateTime(create_time ? create_time : time(nullptr), DateLUT::serverTimezoneInstance()) << "\n" << "source replica: " << source_replica << '\n' << "block_id: " << escape << block_id << '\n'; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp index 1bbb246338c..e2c23ecfe85 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp @@ -12,7 +12,7 @@ namespace DB void ReplicatedMergeTreeMutationEntry::writeText(WriteBuffer & out) const { out << "format version: 1\n" - << "create time: " << LocalDateTime(create_time ? create_time : time(nullptr)) << "\n" + << "create time: " << LocalDateTime(create_time ? create_time : time(nullptr), DateLUT::serverTimezoneInstance()) << "\n" << "source replica: " << source_replica << "\n" << "block numbers count: " << block_numbers.size() << "\n"; From 7ad399cc6d97aaa356cbbe50f697b563d0a2f995 Mon Sep 17 00:00:00 2001 From: Rory Crispin Date: Wed, 19 Jul 2023 14:25:36 +0100 Subject: [PATCH 0866/2047] Rephrase ALL supported note I interpreted the note as 'ALL' is only supported before 23.4 I think this reordering makes it clearer --- docs/en/operations/backup.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index c3ddee07d0b..61c8a73673f 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -30,7 +30,7 @@ description: In order to effectively mitigate possible human errors, you should ``` :::note ALL -`ALL` is only applicable to the `RESTORE` command prior to version 23.4 of Clickhouse. +Prior to version 23.4 of Clickhouse, `ALL` was only applicable to the `RESTORE` command. ::: ## Background From da6a31bb628de3f038fbf786b2102cbb3a7fee2d Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 19 Jul 2023 13:26:09 +0000 Subject: [PATCH 0867/2047] Fix tests and style --- src/Formats/StructureToCapnProtoSchema.cpp | 2 +- src/Formats/StructureToProtobufSchema.cpp | 2 +- src/Functions/structureToFormatSchema.cpp | 20 +++++++++++++------ .../0_stateless/02817_structure_to_schema.sh | 1 + 4 files changed, 17 insertions(+), 8 deletions(-) diff --git a/src/Formats/StructureToCapnProtoSchema.cpp b/src/Formats/StructureToCapnProtoSchema.cpp index 56e2155e5bd..9f4d96b7c8a 100644 --- a/src/Formats/StructureToCapnProtoSchema.cpp +++ b/src/Formats/StructureToCapnProtoSchema.cpp @@ -188,7 +188,7 @@ String prepareEnumAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & String prepareAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent) { TypeIndex type_id = data_type->getTypeId(); - + switch (data_type->getTypeId()) { case TypeIndex::Nullable: diff --git a/src/Formats/StructureToProtobufSchema.cpp b/src/Formats/StructureToProtobufSchema.cpp index 086a6ff6864..4a704e8d428 100644 --- a/src/Formats/StructureToProtobufSchema.cpp +++ b/src/Formats/StructureToProtobufSchema.cpp @@ -168,7 +168,7 @@ String prepareEnumAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & String prepareAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent) { TypeIndex type_id = data_type->getTypeId(); - + switch (data_type->getTypeId()) { case TypeIndex::Nullable: diff --git a/src/Functions/structureToFormatSchema.cpp b/src/Functions/structureToFormatSchema.cpp index 8c561595504..f2bd78e4fb7 100644 --- a/src/Functions/structureToFormatSchema.cpp +++ b/src/Functions/structureToFormatSchema.cpp @@ -19,8 +19,6 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; } template @@ -108,10 +106,14 @@ REGISTER_FUNCTION(StructureToCapnProtoSchema) factory.registerFunction>(FunctionDocumentation { .description=R"( - +Function that converts ClickHouse table structure to CapnProto format schema )", .examples{ - {"random", "SELECT structureToCapnProtoSchema('s String, x UInt32', 'MessageName')", ""}, + {"random", "SELECT structureToCapnProtoSchema('s String, x UInt32', 'MessageName') format TSVRaw", "struct MessageName\n" +"{\n" +" s @0 : Data;\n" +" x @1 : UInt32;\n" +"}"}, }, .categories{"Other"} }, @@ -124,10 +126,16 @@ REGISTER_FUNCTION(StructureToProtobufSchema) factory.registerFunction>(FunctionDocumentation { .description=R"( - +Function that converts ClickHouse table structure to Protobuf format schema )", .examples{ - {"random", "SELECT structureToCapnProtoSchema()", ""}, + {"random", "SELECT structureToCapnProtoSchema('s String, x UInt32', 'MessageName') format TSVRaw", "syntax = \"proto3\";\n" +"\n" +"message MessageName\n" +"{\n" +" bytes s = 1;\n" +" uint32 x = 2;\n" +"}"}, }, .categories{"Other"} }, diff --git a/tests/queries/0_stateless/02817_structure_to_schema.sh b/tests/queries/0_stateless/02817_structure_to_schema.sh index 8aaf35acb33..76c5b1a1d85 100755 --- a/tests/queries/0_stateless/02817_structure_to_schema.sh +++ b/tests/queries/0_stateless/02817_structure_to_schema.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From ac18ca117807d3191c7dac505103e2d6191f3220 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 19 Jul 2023 21:26:37 +0800 Subject: [PATCH 0868/2047] add test --- .../02813_system_events_and_metrics_add_alias.reference | 4 ++++ .../0_stateless/02813_system_events_and_metrics_add_alias.sql | 4 ++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference create mode 100644 tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql diff --git a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference new file mode 100644 index 00000000000..93be2764a57 --- /dev/null +++ b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference @@ -0,0 +1,4 @@ +CREATE TABLE system.metrics\n(\n `metric` String,\n `value` Int64,\n `description` String,\n `name` String\n)\nENGINE = SystemMetrics\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +1 +CREATE TABLE system.events\n(\n `event` String,\n `value` UInt64,\n `description` String,\n `name` String\n)\nENGINE = SystemEvents\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +1 diff --git a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql new file mode 100644 index 00000000000..a2250608f8f --- /dev/null +++ b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql @@ -0,0 +1,4 @@ +show create table system.metrics; +select equals((select count() from system.metrics where name=metric) as r1, (select count() from system.metrics) as r2); +show create table system.events; +select equals((select count() from system.events where name=event) as r1, (select count() from system.events) as r2); \ No newline at end of file From c47f19303afc3e4f18ceddbfe7b8f7e7cc4622b3 Mon Sep 17 00:00:00 2001 From: Rory Crispin Date: Wed, 19 Jul 2023 14:27:21 +0100 Subject: [PATCH 0869/2047] Correct CH capitalisation --- docs/en/operations/backup.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 61c8a73673f..62f931a76b4 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -30,7 +30,7 @@ description: In order to effectively mitigate possible human errors, you should ``` :::note ALL -Prior to version 23.4 of Clickhouse, `ALL` was only applicable to the `RESTORE` command. +Prior to version 23.4 of ClickHouse, `ALL` was only applicable to the `RESTORE` command. ::: ## Background From 688b55b6ff80ee333ab9ef318d42937d5b5d3064 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 13:29:07 +0000 Subject: [PATCH 0870/2047] Try to fix test, rename arg --- src/Common/SystemLogBase.cpp | 9 +++++---- src/Common/SystemLogBase.h | 4 ++-- src/Loggers/Loggers.cpp | 6 +++--- tests/queries/0_stateless/02813_starting_in_text_log.sql | 2 +- 4 files changed, 11 insertions(+), 10 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02813_starting_in_text_log.sql diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index bed6d661db7..8cf8103e1c7 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -41,9 +41,9 @@ ISystemLog::~ISystemLog() = default; template SystemLogQueue::SystemLogQueue( - const String & name_, + const String & table_name_, size_t flush_interval_milliseconds_) - : log(&Poco::Logger::get(name_)) + : log(&Poco::Logger::get("SystemLogQueue (" + table_name_ + ")")) , flush_interval_milliseconds(flush_interval_milliseconds_) {} @@ -120,6 +120,7 @@ void SystemLogQueue::push(const LogElement & element) template uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) { + uint64_t this_thread_requested_offset; { @@ -204,10 +205,10 @@ void SystemLogQueue::shutdown() template SystemLogBase::SystemLogBase( - const String& name, + const String& table_name_, size_t flush_interval_milliseconds_, std::shared_ptr> queue_) - : queue(queue_ ? queue_ : std::make_shared>(name, flush_interval_milliseconds_)) + : queue(queue_ ? queue_ : std::make_shared>(table_name_, flush_interval_milliseconds_)) { } diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 0ac376769ad..3716584be24 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -80,7 +80,7 @@ class SystemLogQueue public: SystemLogQueue( - const String & name_, + const String & table_name_, size_t flush_interval_milliseconds_); void shutdown(); @@ -130,7 +130,7 @@ public: using Self = SystemLogBase; SystemLogBase( - const String& name, + const String& table_name_, size_t flush_interval_milliseconds_, std::shared_ptr> queue_ = nullptr); diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 4cc74902ee1..85a8152602f 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -255,10 +255,10 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log #ifndef WITHOUT_TEXT_LOG if (config.has("text_log")) { - String text_log_level_str = config.getString("text_log.level", ""); - int text_log_level = text_log_level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(text_log_level_str); + String text_log_level_str = config.getString("text_log.level", "trace"); + int text_log_level = Poco::Logger::parseLevel(text_log_level_str); size_t flush_interval_milliseconds = config.getUInt64("text_log.flush_interval_milliseconds", - DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); + DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); split->addTextLog(DB::TextLog::getLogQueue(flush_interval_milliseconds), text_log_level); } #endif diff --git a/tests/queries/0_stateless/02813_starting_in_text_log.sql b/tests/queries/0_stateless/02813_starting_in_text_log.sql old mode 100755 new mode 100644 index 8ef78945a72..e007f58189e --- a/tests/queries/0_stateless/02813_starting_in_text_log.sql +++ b/tests/queries/0_stateless/02813_starting_in_text_log.sql @@ -1,2 +1,2 @@ SYSTEM FLUSH LOGS; -SELECT count() > 0 FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%Application: Starting ClickHouse%'; +SELECT count() > 0 FROM system.text_log WHERE event_date >= yesterday() AND message LIKE '%Starting ClickHouse%'; From 3225c30f730939ee688013d776b7f8651982c4a2 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 19 Jul 2023 21:29:07 +0800 Subject: [PATCH 0871/2047] fix test --- .../0_stateless/02117_show_create_table_system.reference | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 3c221d6a473..4285908fa53 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -262,7 +262,8 @@ CREATE TABLE system.events ( `event` String, `value` UInt64, - `description` String + `description` String, + `name` String ) ENGINE = SystemEvents COMMENT 'SYSTEM TABLE is built on the fly.' @@ -382,7 +383,8 @@ CREATE TABLE system.metrics ( `metric` String, `value` Int64, - `description` String + `description` String, + `name` String ) ENGINE = SystemMetrics COMMENT 'SYSTEM TABLE is built on the fly.' From 9f7e40e8e57cc5e8c997dff16b5c6645283ffcb3 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Wed, 19 Jul 2023 13:43:22 +0000 Subject: [PATCH 0872/2047] Remove empty line --- src/Common/SystemLogBase.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 8cf8103e1c7..294ba09e375 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -120,7 +120,6 @@ void SystemLogQueue::push(const LogElement & element) template uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) { - uint64_t this_thread_requested_offset; { From 777026e42e00311c4a751e8899fac407d6c8b874 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 19 Jul 2023 13:43:40 +0000 Subject: [PATCH 0873/2047] Fix test --- src/Functions/s2RectAdd.cpp | 8 ++++---- src/Functions/s2RectContains.cpp | 8 ++++---- src/Functions/s2RectIntersection.cpp | 4 ++-- src/Functions/s2RectUnion.cpp | 6 +++--- tests/queries/0_stateless/01849_geoToS2.reference | 1 - tests/queries/0_stateless/01849_geoToS2.sql | 2 +- .../0_stateless/02224_s2_test_const_columns.reference | 2 +- tests/queries/0_stateless/02224_s2_test_const_columns.sql | 2 +- 8 files changed, 16 insertions(+), 17 deletions(-) diff --git a/src/Functions/s2RectAdd.cpp b/src/Functions/s2RectAdd.cpp index 9266f4ae1a7..0be304234cb 100644 --- a/src/Functions/s2RectAdd.cpp +++ b/src/Functions/s2RectAdd.cpp @@ -118,14 +118,14 @@ public: if (!point.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive " "and the longitude is between -180 and 180 degrees inclusive."); if (!rect.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." - "Also, if either the latitude or longitude bound is empty then both must be."); + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " + "Also, if either the latitude or longitude bound is empty then both must be. "); rect.AddPoint(point.ToPoint()); diff --git a/src/Functions/s2RectContains.cpp b/src/Functions/s2RectContains.cpp index aed123ce8ee..898e12a6466 100644 --- a/src/Functions/s2RectContains.cpp +++ b/src/Functions/s2RectContains.cpp @@ -111,14 +111,14 @@ public: if (!point.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive" + "Point is invalid. For valid point the latitude is between -90 and 90 degrees inclusive " "and the longitude is between -180 and 180 degrees inclusive."); if (!rect.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." - "Also, if either the latitude or longitude bound is empty then both must be."); + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " + "Also, if either the latitude or longitude bound is empty then both must be. "); dst_data.emplace_back(rect.Contains(point.ToLatLng())); } diff --git a/src/Functions/s2RectIntersection.cpp b/src/Functions/s2RectIntersection.cpp index ffe26d171d0..f0cc02de9d9 100644 --- a/src/Functions/s2RectIntersection.cpp +++ b/src/Functions/s2RectIntersection.cpp @@ -133,8 +133,8 @@ public: if (!rect1.is_valid() || !rect2.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " "Also, if either the latitude or longitude bound is empty then both must be."); S2LatLngRect rect_intersection = rect1.Intersection(rect2); diff --git a/src/Functions/s2RectUnion.cpp b/src/Functions/s2RectUnion.cpp index 472b30c2d55..a5cedd35812 100644 --- a/src/Functions/s2RectUnion.cpp +++ b/src/Functions/s2RectUnion.cpp @@ -131,9 +131,9 @@ public: if (!rect1.is_valid() || !rect2.is_valid()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed" - "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value." - "Also, if either the latitude or longitude bound is empty then both must be."); + "Rectangle is invalid. For valid rectangles the latitude bounds do not exceed " + "Pi/2 in absolute value and the longitude bounds do not exceed Pi in absolute value. " + "Also, if either the latitude or longitude bound is empty then both must be. "); S2LatLngRect rect_union = rect1.Union(rect2); diff --git a/tests/queries/0_stateless/01849_geoToS2.reference b/tests/queries/0_stateless/01849_geoToS2.reference index 08d76978791..a8196994361 100644 --- a/tests/queries/0_stateless/01849_geoToS2.reference +++ b/tests/queries/0_stateless/01849_geoToS2.reference @@ -39,4 +39,3 @@ Checking s2 index generation. (74.0061,-68.32124) (74.0061,-68.32124) ok (10.61077,-64.1841) (10.61077,-64.1841) ok (-89.81096,-57.01398) (-89.81096,-57.01398) ok -4864204703484167331 diff --git a/tests/queries/0_stateless/01849_geoToS2.sql b/tests/queries/0_stateless/01849_geoToS2.sql index abd084a2b19..e997fec14e5 100644 --- a/tests/queries/0_stateless/01849_geoToS2.sql +++ b/tests/queries/0_stateless/01849_geoToS2.sql @@ -44,7 +44,7 @@ SELECT first, second, result FROM ( SELECT s2ToGeo(toUInt64(-1)); -- { serverError 36 } SELECT s2ToGeo(nan); -- { serverError 43 } -SELECT geoToS2(toFloat64(toUInt64(-1)), toFloat64(toUInt64(-1))); +SELECT geoToS2(toFloat64(toUInt64(-1)), toFloat64(toUInt64(-1))); -- { serverError BAD_ARGUMENTS } SELECT geoToS2(nan, nan); -- { serverError 43 } SELECT geoToS2(-inf, 1.1754943508222875e-38); -- { serverError 43 } diff --git a/tests/queries/0_stateless/02224_s2_test_const_columns.reference b/tests/queries/0_stateless/02224_s2_test_const_columns.reference index 9982596f097..20f32ec6be7 100644 --- a/tests/queries/0_stateless/02224_s2_test_const_columns.reference +++ b/tests/queries/0_stateless/02224_s2_test_const_columns.reference @@ -16,4 +16,4 @@ (5179062030687166815,5177056748191934217) (5179062030687166815,5177057445452335297) (5178914411069187297,5177056748191934217) -(6304347505408739331,8070450532247928833) +(5178914411069187297,5177912432982045463) diff --git a/tests/queries/0_stateless/02224_s2_test_const_columns.sql b/tests/queries/0_stateless/02224_s2_test_const_columns.sql index f33a7f2b696..1d3e51065b5 100644 --- a/tests/queries/0_stateless/02224_s2_test_const_columns.sql +++ b/tests/queries/0_stateless/02224_s2_test_const_columns.sql @@ -9,4 +9,4 @@ SELECT s2CapUnion(3814912406305146967, toFloat64(1), 1157347770437378819, toFloa SELECT s2RectAdd(5178914411069187297, 5177056748191934217, arrayJoin([5179056748191934217,5177914411069187297])); SELECT s2RectContains(5179062030687166815, 5177056748191934217, arrayJoin([5177914411069187297, 5177914411069187297])); SELECT s2RectUnion(5178914411069187297, 5177056748191934217, 5179062030687166815, arrayJoin([5177056748191934217, 5177914411069187297])); -SELECT s2RectIntersection(5178914411069187297, 5177056748191934217, 5179062030687166815, arrayJoin([5177056748191934217,1157347770437378819])); +SELECT s2RectIntersection(5178914411069187297, 5177056748191934217, 5179062030687166815, arrayJoin([5177056748191934217,5177914411069187297])); From 10cdaac32f64c6346625ab2ee7e8af89fb2e0b22 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Jul 2023 16:32:09 +0200 Subject: [PATCH 0874/2047] update Settings.h --- src/Core/Settings.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f1a314fed37..97e297c6feb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1011,7 +1011,8 @@ class IColumn; M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ - M(Bool, input_format_csv_allow_variable_number_of_columns, false, "Ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values", 0) \ + M(Bool, input_format_csv_allow_variable_number_of_columns, false, "Ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values", 0) \ + M(Bool, precise_float_parsing, false, "Prefer more precise (but slower) float parsing algorithm", 0) \ // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. From 53500be941bc1d63ef85c3b5afb6bcc01103fb85 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 19 Jul 2023 17:03:00 +0200 Subject: [PATCH 0875/2047] Updated fix of multiple usage in parameterized view to support cte by not adding column which is previously added --- src/Interpreters/ActionsVisitor.cpp | 22 +++++++------------ ...zed_view_with_cte_multiple_usage.reference | 2 ++ ...meterized_view_with_cte_multiple_usage.sql | 16 ++++++++++++++ 3 files changed, 26 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.reference create mode 100755 tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index efab11003f5..8b10df516dc 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1202,22 +1202,16 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & else if (data.is_create_parameterized_view && query_parameter) { const auto data_type = DataTypeFactory::instance().get(query_parameter->type); - /// Use getUniqueName() to allow multiple use of query parameter in the query: - /// - /// CREATE VIEW view AS - /// SELECT * - /// FROM system.one - /// WHERE dummy = {k1:Int}+1 OR dummy = {k1:Int}+2 - /// ^^ ^^ - /// - /// NOTE: query in the VIEW will not be modified this is needed - /// only during analysis for CREATE VIEW to avoid duplicated - /// column names. - ColumnWithTypeAndName column(data_type, data.getUniqueName("__" + query_parameter->getColumnName())); - data.addColumn(column); + /// During analysis for CREATE VIEW of a parameterized view, if parameter is + /// used multiple times, column is only added once + if (!data.hasColumn(query_parameter->name)) + { + ColumnWithTypeAndName column(data_type, query_parameter->name); + data.addColumn(column); + } argument_types.push_back(data_type); - argument_names.push_back(column.name); + argument_names.push_back(query_parameter->name); } else { diff --git a/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.reference b/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.reference new file mode 100644 index 00000000000..004d27bacad --- /dev/null +++ b/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.reference @@ -0,0 +1,2 @@ +3 2 +3 2 3 diff --git a/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql b/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql new file mode 100755 index 00000000000..d56d9c4e181 --- /dev/null +++ b/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql @@ -0,0 +1,16 @@ +create view test_param_view as +with {param_test_val:UInt8} as param_test_val +select param_test_val, + arrayCount((a)->(a < param_test_val), t.arr) as cnt1 +from (select [1,2,3,4,5] as arr) t; + +select * from test_param_view(param_test_val = 3); + +create view test_param_view2 as +with {param_test_val:UInt8} as param_test_val +select param_test_val, + arrayCount((a)->(a < param_test_val), t.arr) as cnt1, + arrayCount((a)->(a < param_test_val+1), t.arr) as cnt2 +from (select [1,2,3,4,5] as arr) t; + +select * from test_param_view2(param_test_val = 3); \ No newline at end of file From 6ad8c450ad39d26b3347a1acfb0b3decdce48dfc Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 19 Jul 2023 15:03:23 +0000 Subject: [PATCH 0876/2047] style --- src/Interpreters/InterpreterCreateIndexQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterCreateIndexQuery.cpp b/src/Interpreters/InterpreterCreateIndexQuery.cpp index ef7a7af5303..3b47a002e50 100644 --- a/src/Interpreters/InterpreterCreateIndexQuery.cpp +++ b/src/Interpreters/InterpreterCreateIndexQuery.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes { extern const int TABLE_IS_READ_ONLY; extern const int INCORRECT_QUERY; + extern const int NOT_IMPLEMENTED; } From b34655e74310dba07c25b06cac817168b6012907 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 19 Jul 2023 17:05:03 +0200 Subject: [PATCH 0877/2047] Update src/Storages/StorageReplicatedMergeTree.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 53fac578fca..04799a08e37 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4114,7 +4114,7 @@ std::set StorageReplicatedMergeTree::findReplicaUniqueParts(c } if (!our_parts.empty() && our_unique_parts.empty()) - LOG_TRACE(log_, "All parts found on replica"); + LOG_TRACE(log_, "All parts found on replicas"); return our_unique_parts; } From 544081163d751a62dcdfc21e5841c9cb53877cb0 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 19 Jul 2023 17:21:01 +0200 Subject: [PATCH 0878/2047] Remove redundant deactivate --- src/Storages/StorageReplicatedMergeTree.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 04799a08e37..06f5330f6d9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4903,7 +4903,6 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() fetcher.blocker.cancelForever(); merger_mutator.merges_blocker.cancelForever(); parts_mover.moves_blocker.cancelForever(); - mutations_finalizing_task->deactivate(); stopBeingLeader(); if (attach_thread) From dd411d8f547bf647e0854f8345a5d5c1597d724e Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 19 Jul 2023 09:14:10 -0700 Subject: [PATCH 0879/2047] Implement endianness-indepedent support for MergeTree checksums --- .../Serializations/SerializationNumber.cpp | 31 ++++++++++++++----- .../MergeTree/MergeTreeDataPartChecksum.cpp | 28 ++++++++--------- .../MergeTreeDataPartWriterCompact.cpp | 3 +- 3 files changed, 40 insertions(+), 22 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 8cabaec753d..ca2616f4582 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -10,6 +10,8 @@ #include #include +#include + namespace DB { @@ -135,13 +137,25 @@ template void SerializationNumber::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const { const typename ColumnVector::Container & x = typeid_cast &>(column).getData(); - - size_t size = x.size(); - - if (limit == 0 || offset + limit > size) + if (const size_t size = x.size(); limit == 0 || offset + limit > size) limit = size - offset; - if (limit) + if (limit == 0) + return; + + if constexpr (std::endian::native == std::endian::big && sizeof(T) >= 2) + { + static constexpr auto to_little_endian = [](auto i) + { + transformEndianness(i); + return i; + }; + + std::ranges::for_each( + x | std::views::drop(offset) | std::views::take(limit) | std::views::transform(to_little_endian), + [&ostr](const auto & i) { ostr.write(reinterpret_cast(&i), sizeof(typename ColumnVector::ValueType)); }); + } + else ostr.write(reinterpret_cast(&x[offset]), sizeof(typename ColumnVector::ValueType) * limit); } @@ -149,10 +163,13 @@ template void SerializationNumber::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const { typename ColumnVector::Container & x = typeid_cast &>(column).getData(); - size_t initial_size = x.size(); + const size_t initial_size = x.size(); x.resize(initial_size + limit); - size_t size = istr.readBig(reinterpret_cast(&x[initial_size]), sizeof(typename ColumnVector::ValueType) * limit); + const size_t size = istr.readBig(reinterpret_cast(&x[initial_size]), sizeof(typename ColumnVector::ValueType) * limit); x.resize(initial_size + size / sizeof(typename ColumnVector::ValueType)); + + if constexpr (std::endian::native == std::endian::big && sizeof(T) >= 2) + std::ranges::for_each(x | std::views::drop(initial_size), [](auto & i) { transformEndianness(i); }); } template class SerializationNumber; diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 6628cd68eaf..5a7b2dfbca8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -187,15 +187,15 @@ bool MergeTreeDataPartChecksums::readV3(ReadBuffer & in) String name; Checksum sum; - readBinary(name, in); + readStringBinary(name, in); readVarUInt(sum.file_size, in); - readPODBinary(sum.file_hash, in); - readBinary(sum.is_compressed, in); + readBinaryLittleEndian(sum.file_hash, in); + readBinaryLittleEndian(sum.is_compressed, in); if (sum.is_compressed) { readVarUInt(sum.uncompressed_size, in); - readPODBinary(sum.uncompressed_hash, in); + readBinaryLittleEndian(sum.uncompressed_hash, in); } files.emplace(std::move(name), sum); @@ -223,15 +223,15 @@ void MergeTreeDataPartChecksums::write(WriteBuffer & to) const const String & name = it.first; const Checksum & sum = it.second; - writeBinary(name, out); + writeStringBinary(name, out); writeVarUInt(sum.file_size, out); - writePODBinary(sum.file_hash, out); - writeBinary(sum.is_compressed, out); + writeBinaryLittleEndian(sum.file_hash, out); + writeBinaryLittleEndian(sum.is_compressed, out); if (sum.is_compressed) { writeVarUInt(sum.uncompressed_size, out); - writePODBinary(sum.uncompressed_hash, out); + writeBinaryLittleEndian(sum.uncompressed_hash, out); } } } @@ -339,9 +339,9 @@ void MinimalisticDataPartChecksums::serializeWithoutHeader(WriteBuffer & to) con writeVarUInt(num_compressed_files, to); writeVarUInt(num_uncompressed_files, to); - writePODBinary(hash_of_all_files, to); - writePODBinary(hash_of_uncompressed_files, to); - writePODBinary(uncompressed_hash_of_compressed_files, to); + writeBinaryLittleEndian(hash_of_all_files, to); + writeBinaryLittleEndian(hash_of_uncompressed_files, to); + writeBinaryLittleEndian(uncompressed_hash_of_compressed_files, to); } String MinimalisticDataPartChecksums::getSerializedString() const @@ -382,9 +382,9 @@ void MinimalisticDataPartChecksums::deserializeWithoutHeader(ReadBuffer & in) readVarUInt(num_compressed_files, in); readVarUInt(num_uncompressed_files, in); - readPODBinary(hash_of_all_files, in); - readPODBinary(hash_of_uncompressed_files, in); - readPODBinary(uncompressed_hash_of_compressed_files, in); + readBinaryLittleEndian(hash_of_all_files, in); + readBinaryLittleEndian(hash_of_uncompressed_files, in); + readBinaryLittleEndian(uncompressed_hash_of_compressed_files, in); } void MinimalisticDataPartChecksums::computeTotalChecksums(const MergeTreeDataPartChecksums & full_checksums_) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 5e1da21da5b..75e6aca0793 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -365,8 +365,9 @@ void MergeTreeDataPartWriterCompact::addToChecksums(MergeTreeDataPartChecksums & { uncompressed_size += stream->hashing_buf.count(); auto stream_hash = stream->hashing_buf.getHash(); + transformEndianness(stream_hash); uncompressed_hash = CityHash_v1_0_2::CityHash128WithSeed( - reinterpret_cast(&stream_hash), sizeof(stream_hash), uncompressed_hash); + reinterpret_cast(&stream_hash), sizeof(stream_hash), uncompressed_hash); } checksums.files[data_file_name].is_compressed = true; From 13d1e21da820dd97ddb624eb7671ca2fee86d530 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 19 Jul 2023 18:26:26 +0200 Subject: [PATCH 0880/2047] Fixed test file permissions --- .../02818_parameterized_view_with_cte_multiple_usage.sql | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql diff --git a/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql b/tests/queries/0_stateless/02818_parameterized_view_with_cte_multiple_usage.sql old mode 100755 new mode 100644 From 51e2c58a539a31a6268f2d4f50fd91701caaa915 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 21 Jun 2023 20:15:12 -0700 Subject: [PATCH 0881/2047] Implement endianness-independent SipHash and MergeTree checksum serialization --- programs/obfuscator/Obfuscator.cpp | 13 +- .../AggregateFunctionUniq.h | 3 +- src/AggregateFunctions/UniqVariadicHash.h | 8 +- src/Analyzer/IQueryTreeNode.cpp | 5 +- src/Client/QueryFuzzer.cpp | 3 +- src/Columns/ColumnUnique.h | 3 +- src/Common/SipHash.h | 112 +++++++++--------- src/Common/TransformEndianness.hpp | 3 +- src/Common/examples/hashes_test.cpp | 3 +- src/Common/getHashOfLoadedBinary.cpp | 3 +- src/Common/randomSeed.cpp | 2 +- src/DataTypes/Serializations/PathInData.cpp | 4 +- src/Functions/PolygonUtils.h | 4 +- src/Functions/array/arrayDistinct.cpp | 3 +- src/Functions/array/arrayEnumerateRanked.h | 6 +- src/IO/MMappedFileCache.h | 5 +- src/IO/UncompressedCache.h | 5 +- src/Interpreters/AggregationCommon.h | 6 +- src/Interpreters/AsynchronousInsertQueue.cpp | 4 +- src/Parsers/IAST.cpp | 4 +- .../Impl/ConstantExpressionTemplate.cpp | 3 +- .../Transforms/LimitByTransform.cpp | 5 +- src/Storages/LiveView/LiveViewSink.h | 7 +- src/Storages/LiveView/StorageLiveView.cpp | 3 +- src/Storages/MarkCache.h | 6 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 10 +- src/Storages/MergeTree/MarkRange.cpp | 12 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 15 +-- .../MergeTree/MergeTreeDataPartCompact.cpp | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 2 +- .../MergeTree/MergeTreeDataPartWide.cpp | 6 +- .../MergeTreeDataPartWriterCompact.cpp | 12 +- .../MergeTreeDataPartWriterOnDisk.cpp | 6 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 12 +- .../MergeTree/MergeTreeMarksLoader.cpp | 12 +- src/Storages/MergeTree/MergeTreePartition.cpp | 10 +- .../ReplicatedMergeTreePartHeader.cpp | 4 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 9 +- 38 files changed, 138 insertions(+), 197 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 3042ae2bb57..43b31843afe 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -365,17 +365,14 @@ static void transformFixedString(const UInt8 * src, UInt8 * dst, size_t size, UI hash.update(seed); hash.update(i); + const auto checksum = getSipHash128AsArray(hash); if (size >= 16) { - char * hash_dst = reinterpret_cast(std::min(pos, end - 16)); - hash.get128(hash_dst); + auto * hash_dst = std::min(pos, end - 16); + memcpy(hash_dst, checksum.data(), checksum.size()); } else - { - char value[16]; - hash.get128(value); - memcpy(dst, value, end - dst); - } + memcpy(dst, checksum.data(), end - dst); pos += 16; ++i; @@ -401,7 +398,7 @@ static void transformUUID(const UUID & src_uuid, UUID & dst_uuid, UInt64 seed) hash.update(reinterpret_cast(&src), sizeof(UUID)); /// Saving version and variant from an old UUID - hash.get128(reinterpret_cast(&dst)); + // hash.get128Impl(reinterpret_cast(&dst)); dst.items[1] = (dst.items[1] & 0x1fffffffffffffffull) | (src.items[1] & 0xe000000000000000ull); dst.items[0] = (dst.items[0] & 0xffffffffffff0fffull) | (src.items[0] & 0x000000000000f000ull); diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index de68e9076a0..8310f02767c 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -298,10 +298,9 @@ struct Adder { StringRef value = column.getDataAt(row_num); - UInt128 key; SipHash hash; hash.update(value.data, value.size); - hash.get128(key); + const auto key = hash.get128(); data.set.template insert(key); } diff --git a/src/AggregateFunctions/UniqVariadicHash.h b/src/AggregateFunctions/UniqVariadicHash.h index 94f54a7a059..840380e7f0f 100644 --- a/src/AggregateFunctions/UniqVariadicHash.h +++ b/src/AggregateFunctions/UniqVariadicHash.h @@ -107,9 +107,7 @@ struct UniqVariadicHash ++column; } - UInt128 key; - hash.get128(key); - return key; + return hash.get128(); } }; @@ -131,9 +129,7 @@ struct UniqVariadicHash ++column; } - UInt128 key; - hash.get128(key); - return key; + return hash.get128(); } }; diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index f1056975f7f..7e5d778f1c1 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -229,10 +229,7 @@ IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const } } - Hash result; - hash_state.get128(result); - - return result; + return getSipHash128AsLoHi(hash_state); } QueryTreeNodePtr IQueryTreeNode::clone() const diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 5ce95c82528..f20c869e119 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -521,8 +521,7 @@ void QueryFuzzer::fuzzCreateQuery(ASTCreateQuery & create) if (create.storage) create.storage->updateTreeHash(sip_hash); - IAST::Hash hash; - sip_hash.get128(hash); + const auto hash = getSipHash128AsLoHi(sip_hash); /// Save only tables with unique definition. if (created_tables_hashes.insert(hash).second) diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 377255d80c7..fd0144cba12 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -670,8 +670,9 @@ UInt128 ColumnUnique::IncrementalHash::getHash(const ColumnType & co for (size_t i = 0; i < column_size; ++i) column.updateHashWithValue(i, sip_hash); + hash = sip_hash.get128(); + std::lock_guard lock(mutex); - sip_hash.get128(hash); cur_hash = hash; num_added_rows.store(column_size); } diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index cdec00d4bcc..e9d45c7201f 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -13,6 +13,8 @@ * (~ 700 MB/sec, 15 million strings per second) */ +#include "TransformEndianness.hpp" + #include #include #include @@ -22,14 +24,10 @@ #include #include - -namespace DB -{ -namespace ErrorCodes +namespace DB::ErrorCodes { extern const int LOGICAL_ERROR; } -} #define SIPROUND \ do \ @@ -90,6 +88,20 @@ private: SIPROUND; } + /// @brief Retrieves the result in some form with the endianness of the platform taken into account. + /// @warning This can only be done once! + void get128Impl(char * out) + { + finalize(); +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + unalignedStore(out + 8, v0 ^ v1); + unalignedStore(out, v2 ^ v3); +#else + unalignedStore(out, v0 ^ v1); + unalignedStore(out + 8, v2 ^ v3); +#endif + } + public: /// Arguments - seed. SipHash(UInt64 key0 = 0, UInt64 key1 = 0, bool is_reference_128_ = false) /// NOLINT @@ -161,60 +173,26 @@ public: } } - template + template ALWAYS_INLINE void update(const T & x) { if constexpr (std::endian::native == std::endian::big) { - T rev_x = x; - char *start = reinterpret_cast(&rev_x); - char *end = start + sizeof(T); - std::reverse(start, end); - update(reinterpret_cast(&rev_x), sizeof(rev_x)); /// NOLINT + auto transformed_x = x; + if constexpr (!std::is_same_v) + transformed_x = Transform()(x); + else + DB::transformEndianness(transformed_x); + + update(reinterpret_cast(&transformed_x), sizeof(transformed_x)); /// NOLINT } else update(reinterpret_cast(&x), sizeof(x)); /// NOLINT } - ALWAYS_INLINE void update(const std::string & x) - { - update(x.data(), x.length()); - } - - ALWAYS_INLINE void update(const std::string_view x) - { - update(x.data(), x.size()); - } - - /// Get the result in some form. This can only be done once! - - ALWAYS_INLINE void get128(char * out) - { - finalize(); -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - unalignedStore(out + 8, v0 ^ v1); - unalignedStore(out, v2 ^ v3); -#else - unalignedStore(out, v0 ^ v1); - unalignedStore(out + 8, v2 ^ v3); -#endif - } - - template - ALWAYS_INLINE void get128(T & lo, T & hi) - { - static_assert(sizeof(T) == 8); - finalize(); - lo = v0 ^ v1; - hi = v2 ^ v3; - } - - template - ALWAYS_INLINE void get128(T & dst) - { - static_assert(sizeof(T) == 16); - get128(reinterpret_cast(&dst)); - } + ALWAYS_INLINE void update(const std::string & x) { update(x.data(), x.length()); } + ALWAYS_INLINE void update(const std::string_view x) { update(x.data(), x.size()); } + ALWAYS_INLINE void update(const char * s) { update(std::string_view(s)); } UInt64 get64() { @@ -222,10 +200,23 @@ public: return v0 ^ v1 ^ v2 ^ v3; } + template + requires (sizeof(T) == 8) + ALWAYS_INLINE void get128(T & lo, T & hi) + { + finalize(); + lo = v0 ^ v1; + hi = v2 ^ v3; + } + UInt128 get128() { UInt128 res; - get128(res); +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + get128(res.items[1], res.items[0]); +#else + get128(res.items[0], res.items[1]); +#endif return res; } @@ -247,9 +238,7 @@ public: { lo = std::byteswap(lo); hi = std::byteswap(hi); - auto tmp = hi; - hi = lo; - lo = tmp; + std::swap(lo, hi); } UInt128 res = hi; @@ -265,11 +254,18 @@ public: #include -inline void sipHash128(const char * data, const size_t size, char * out) +inline std::array getSipHash128AsArray(SipHash & sip_hash) { - SipHash hash; - hash.update(data, size); - hash.get128(out); + std::array arr; + *reinterpret_cast(arr.data()) = sip_hash.get128(); + return arr; +} + +inline std::pair getSipHash128AsLoHi(SipHash & sip_hash) +{ + std::pair lo_hi; + sip_hash.get128(lo_hi.first, lo_hi.second); + return lo_hi; } inline UInt128 sipHash128Keyed(UInt64 key0, UInt64 key1, const char * data, const size_t size) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 0a9055dde15..ccfec78ab64 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -2,6 +2,7 @@ #include #include +#include #include @@ -46,7 +47,7 @@ inline void transformEndianness(T & value) } template -requires std::is_scoped_enum_v +requires std::is_enum_v || std::is_scoped_enum_v inline void transformEndianness(T & x) { using UnderlyingType = std::underlying_type_t; diff --git a/src/Common/examples/hashes_test.cpp b/src/Common/examples/hashes_test.cpp index eccf7c9b2e6..99479e79302 100644 --- a/src/Common/examples/hashes_test.cpp +++ b/src/Common/examples/hashes_test.cpp @@ -94,7 +94,8 @@ int main(int, char **) { SipHash hash; hash.update(strings[i].data(), strings[i].size()); - hash.get128(&hashes[i * 16]); + const auto hashed_value = getSipHash128AsArray(hash); + memcpy(&hashes[i * 16], hashed_value.data(), hashed_value.size()); } watch.stop(); diff --git a/src/Common/getHashOfLoadedBinary.cpp b/src/Common/getHashOfLoadedBinary.cpp index b81300b8536..6487bcd4f1c 100644 --- a/src/Common/getHashOfLoadedBinary.cpp +++ b/src/Common/getHashOfLoadedBinary.cpp @@ -37,8 +37,7 @@ SipHash getHashOfLoadedBinary() std::string getHashOfLoadedBinaryHex() { SipHash hash = getHashOfLoadedBinary(); - UInt128 checksum; - hash.get128(checksum); + const auto checksum = hash.get128(); return getHexUIntUppercase(checksum); } diff --git a/src/Common/randomSeed.cpp b/src/Common/randomSeed.cpp index 9f0ffd8a6c7..e1aa56fa811 100644 --- a/src/Common/randomSeed.cpp +++ b/src/Common/randomSeed.cpp @@ -39,7 +39,7 @@ DB::UInt64 randomSeed() #if defined(__linux__) struct utsname sysinfo; if (uname(&sysinfo) == 0) - hash.update(sysinfo); + hash.update(sysinfo); #endif return hash.get64(); diff --git a/src/DataTypes/Serializations/PathInData.cpp b/src/DataTypes/Serializations/PathInData.cpp index 2a5f508650f..cf78d7cbb14 100644 --- a/src/DataTypes/Serializations/PathInData.cpp +++ b/src/DataTypes/Serializations/PathInData.cpp @@ -65,9 +65,7 @@ UInt128 PathInData::getPartsHash(const Parts::const_iterator & begin, const Part hash.update(part_it->anonymous_array_level); } - UInt128 res; - hash.get128(res); - return res; + return hash.get128(); } void PathInData::buildPath(const Parts & other_parts) diff --git a/src/Functions/PolygonUtils.h b/src/Functions/PolygonUtils.h index 9aae42cce41..9c28e349413 100644 --- a/src/Functions/PolygonUtils.h +++ b/src/Functions/PolygonUtils.h @@ -635,9 +635,7 @@ UInt128 sipHash128(Polygon && polygon) for (auto & inner : inners) hash_ring(inner); - UInt128 res; - hash.get128(res); - return res; + return hash.get128(); } } diff --git a/src/Functions/array/arrayDistinct.cpp b/src/Functions/array/arrayDistinct.cpp index 527624794ea..ea331d6bdad 100644 --- a/src/Functions/array/arrayDistinct.cpp +++ b/src/Functions/array/arrayDistinct.cpp @@ -268,10 +268,9 @@ void FunctionArrayDistinct::executeHashed( if (nullable_col && (*src_null_map)[j]) continue; - UInt128 hash; SipHash hash_function; src_data.updateHashWithValue(j, hash_function); - hash_function.get128(hash); + const auto hash = hash_function.get128(); if (!set.find(hash)) { diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index 0733f1e2d43..3fc31e46f5e 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -134,18 +134,14 @@ private: /// Hash a set of keys into a UInt128 value. static inline UInt128 ALWAYS_INLINE hash128depths(const std::vector & indices, const ColumnRawPtrs & key_columns) { - UInt128 key; SipHash hash; - for (size_t j = 0, keys_size = key_columns.size(); j < keys_size; ++j) { // Debug: const auto & field = (*key_columns[j])[indices[j]]; DUMP(j, indices[j], field); key_columns[j]->updateHashWithValue(indices[j], hash); } - hash.get128(key); - - return key; + return hash.get128(); } diff --git a/src/IO/MMappedFileCache.h b/src/IO/MMappedFileCache.h index 0a8a80d15d0..bb30829ed69 100644 --- a/src/IO/MMappedFileCache.h +++ b/src/IO/MMappedFileCache.h @@ -33,15 +33,12 @@ public: /// Calculate key from path to file and offset. static UInt128 hash(const String & path_to_file, size_t offset, ssize_t length = -1) { - UInt128 key; - SipHash hash; hash.update(path_to_file.data(), path_to_file.size() + 1); hash.update(offset); hash.update(length); - hash.get128(key); - return key; + return hash.get128(); } template diff --git a/src/IO/UncompressedCache.h b/src/IO/UncompressedCache.h index 2e654b27ed7..b115a18014f 100644 --- a/src/IO/UncompressedCache.h +++ b/src/IO/UncompressedCache.h @@ -51,14 +51,11 @@ public: /// Calculate key from path to file and offset. static UInt128 hash(const String & path_to_file, size_t offset) { - UInt128 key; - SipHash hash; hash.update(path_to_file.data(), path_to_file.size() + 1); hash.update(offset); - hash.get128(key); - return key; + return hash.get128(); } template diff --git a/src/Interpreters/AggregationCommon.h b/src/Interpreters/AggregationCommon.h index 7ba9011f18b..77ca2c49b31 100644 --- a/src/Interpreters/AggregationCommon.h +++ b/src/Interpreters/AggregationCommon.h @@ -253,15 +253,11 @@ static inline T ALWAYS_INLINE packFixed( static inline UInt128 ALWAYS_INLINE hash128( /// NOLINT size_t i, size_t keys_size, const ColumnRawPtrs & key_columns) { - UInt128 key; SipHash hash; - for (size_t j = 0; j < keys_size; ++j) key_columns[j]->updateHashWithValue(i, hash); - hash.get128(key); - - return key; + return hash.get128(); } /** Serialize keys into a continuous chunk of memory. diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 00347663fbd..d174c626dd6 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -105,9 +105,7 @@ UInt128 AsynchronousInsertQueue::InsertQuery::calculateHash() const applyVisitor(FieldVisitorHash(siphash), setting.getValue()); } - UInt128 res; - siphash.get128(res); - return res; + return siphash.get128(); } bool AsynchronousInsertQueue::InsertQuery::operator==(const InsertQuery & other) const diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 0138372ce89..7c85b63ceff 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -118,9 +118,7 @@ IAST::Hash IAST::getTreeHash() const { SipHash hash_state; updateTreeHash(hash_state); - IAST::Hash res; - hash_state.get128(res); - return res; + return getSipHash128AsLoHi(hash_state); } diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 5d438d47de6..2d5264fa4e3 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -369,8 +369,7 @@ size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTP /// Allows distinguish expression in the last column in Values format hash_state.update(salt); - IAST::Hash res128; - hash_state.get128(res128); + const auto res128 = getSipHash128AsLoHi(hash_state); size_t res = 0; boost::hash_combine(res, res128.first); boost::hash_combine(res, res128.second); diff --git a/src/Processors/Transforms/LimitByTransform.cpp b/src/Processors/Transforms/LimitByTransform.cpp index cb2804007bd..5e6d7dc816a 100644 --- a/src/Processors/Transforms/LimitByTransform.cpp +++ b/src/Processors/Transforms/LimitByTransform.cpp @@ -33,14 +33,11 @@ void LimitByTransform::transform(Chunk & chunk) for (UInt64 row = 0; row < num_rows; ++row) { - UInt128 key{}; SipHash hash; - for (auto position : key_positions) columns[position]->updateHashWithValue(row, hash); - hash.get128(key); - + const auto key = hash.get128(); auto count = keys_counts[key]++; if (count >= group_offset && (group_length > std::numeric_limits::max() - group_offset || count < group_length + group_offset)) diff --git a/src/Storages/LiveView/LiveViewSink.h b/src/Storages/LiveView/LiveViewSink.h index e163400f2af..792133ced64 100644 --- a/src/Storages/LiveView/LiveViewSink.h +++ b/src/Storages/LiveView/LiveViewSink.h @@ -32,11 +32,8 @@ public: void onFinish() override { - UInt128 key; - String key_str; - - new_hash->get128(key); - key_str = getHexUIntLowercase(key); + const auto key = new_hash->get128(); + const auto key_str = getHexUIntLowercase(key); std::lock_guard lock(storage.mutex); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 173bb128c4a..5719529533e 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -681,7 +681,6 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) bool StorageLiveView::getNewBlocks(const std::lock_guard & lock) { SipHash hash; - UInt128 key; BlocksPtr new_blocks = std::make_shared(); BlocksMetadataPtr new_blocks_metadata = std::make_shared(); @@ -713,7 +712,7 @@ bool StorageLiveView::getNewBlocks(const std::lock_guard & lock) new_blocks->push_back(block); } - hash.get128(key); + const auto key = hash.get128(); /// Update blocks only if hash keys do not match /// NOTE: hash could be different for the same result diff --git a/src/Storages/MarkCache.h b/src/Storages/MarkCache.h index 0f4af57fc8e..495cebb080f 100644 --- a/src/Storages/MarkCache.h +++ b/src/Storages/MarkCache.h @@ -50,13 +50,9 @@ public: /// Calculate key from path to file and offset. static UInt128 hash(const String & path_to_file) { - UInt128 key; - SipHash hash; hash.update(path_to_file.data(), path_to_file.size() + 1); - hash.get128(key); - - return key; + return hash.get128(); } template diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 7050a98a4bc..91069d30c03 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -2052,14 +2052,8 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID(std::string_view token) const hash.update(token.data(), token.size()); } - union - { - char bytes[16]; - UInt64 words[2]; - } hash_value; - hash.get128(hash_value.bytes); - - return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]); + const auto hash_value = hash.get128(); + return info.partition_id + "_" + toString(hash_value.items[0]) + "_" + toString(hash_value.items[1]); } IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_name) const diff --git a/src/Storages/MergeTree/MarkRange.cpp b/src/Storages/MergeTree/MarkRange.cpp index c6c197919f4..bd8546f04cc 100644 --- a/src/Storages/MergeTree/MarkRange.cpp +++ b/src/Storages/MergeTree/MarkRange.cpp @@ -83,12 +83,12 @@ size_t MarkRanges::getNumberOfMarks() const void MarkRanges::serialize(WriteBuffer & out) const { - writeIntBinary(this->size(), out); + writeBinaryLittleEndian(this->size(), out); for (const auto & [begin, end] : *this) { - writeIntBinary(begin, out); - writeIntBinary(end, out); + writeBinaryLittleEndian(begin, out); + writeBinaryLittleEndian(end, out); } } @@ -100,13 +100,13 @@ String MarkRanges::describe() const void MarkRanges::deserialize(ReadBuffer & in) { size_t size = 0; - readIntBinary(size, in); + readBinaryLittleEndian(size, in); this->resize(size); for (size_t i = 0; i < size; ++i) { - readIntBinary((*this)[i].begin, in); - readIntBinary((*this)[i].end, in); + readBinaryLittleEndian((*this)[i].begin, in); + readBinaryLittleEndian((*this)[i].end, in); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 6628cd68eaf..d09cae7870f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -323,9 +323,7 @@ MergeTreeDataPartChecksums::Checksum::uint128 MergeTreeDataPartChecksums::getTot hash_of_all_files.update(checksum.file_hash); } - MergeTreeDataPartChecksums::Checksum::uint128 ret; - hash_of_all_files.get128(reinterpret_cast(&ret)); - return ret; + return getSipHash128AsLoHi(hash_of_all_files); } void MinimalisticDataPartChecksums::serialize(WriteBuffer & to) const @@ -415,14 +413,9 @@ void MinimalisticDataPartChecksums::computeTotalChecksums(const MergeTreeDataPar } } - auto get_hash = [] (SipHash & hash, uint128 & data) - { - hash.get128(data); - }; - - get_hash(hash_of_all_files_state, hash_of_all_files); - get_hash(hash_of_uncompressed_files_state, hash_of_uncompressed_files); - get_hash(uncompressed_hash_of_compressed_files_state, uncompressed_hash_of_compressed_files); + hash_of_all_files = getSipHash128AsLoHi(hash_of_all_files_state); + hash_of_uncompressed_files = getSipHash128AsLoHi(hash_of_uncompressed_files_state); + uncompressed_hash_of_compressed_files = getSipHash128AsLoHi(uncompressed_hash_of_compressed_files_state); } String MinimalisticDataPartChecksums::getSerializedString(const MergeTreeDataPartChecksums & full_checksums, bool minimalistic) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 9c47608e364..70e5ee05d2e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -115,7 +115,7 @@ void MergeTreeDataPartCompact::loadIndexGranularityImpl( { marks_reader->ignore(columns_count * sizeof(MarkInCompressedFile)); size_t granularity; - readIntBinary(granularity, *marks_reader); + readBinaryLittleEndian(granularity, *marks_reader); index_granularity_.appendMark(granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 468747a6c36..a8d38a1bff8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -167,7 +167,7 @@ IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() column.column->updateHashFast(hash); checksum.uncompressed_size = block.bytes(); - hash.get128(checksum.uncompressed_hash); + checksum.uncompressed_hash = getSipHash128AsLoHi(hash); return checksum; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 2d886e2058b..846442fd3e0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -130,13 +130,13 @@ void MergeTreeDataPartWide::loadIndexGranularityImpl( MarkInCompressedFile mark; size_t granularity; - readBinary(mark.offset_in_compressed_file, *marks_reader); - readBinary(mark.offset_in_decompressed_block, *marks_reader); + readBinaryLittleEndian(mark.offset_in_compressed_file, *marks_reader); + readBinaryLittleEndian(mark.offset_in_decompressed_block, *marks_reader); ++marks_count; if (index_granularity_info_.mark_type.adaptive) { - readIntBinary(granularity, *marks_reader); + readBinaryLittleEndian(granularity, *marks_reader); index_granularity_.appendMark(granularity); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 5e1da21da5b..9cd297c4e80 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -228,8 +228,8 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G }; - writeIntBinary(plain_hashing.count(), marks_out); - writeIntBinary(static_cast(0), marks_out); + writeBinaryLittleEndian(plain_hashing.count(), marks_out); + writeBinaryLittleEndian(static_cast(0), marks_out); writeColumnSingleGranule( block.getByName(name_and_type->name), data_part->getSerialization(name_and_type->name), @@ -239,7 +239,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G prev_stream->hashing_buf.next(); } - writeIntBinary(granule.rows_to_write, marks_out); + writeBinaryLittleEndian(granule.rows_to_write, marks_out); } } @@ -270,10 +270,10 @@ void MergeTreeDataPartWriterCompact::fillDataChecksums(IMergeTreeDataPart::Check { for (size_t i = 0; i < columns_list.size(); ++i) { - writeIntBinary(plain_hashing.count(), marks_out); - writeIntBinary(static_cast(0), marks_out); + writeBinaryLittleEndian(plain_hashing.count(), marks_out); + writeBinaryLittleEndian(static_cast(0), marks_out); } - writeIntBinary(static_cast(0), marks_out); + writeBinaryLittleEndian(static_cast(0), marks_out); } for (const auto & [_, stream] : streams_by_codec) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index f57ffa5ee14..79b72d4ae39 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -313,13 +313,13 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block if (stream.compressed_hashing.offset() >= settings.min_compress_block_size) stream.compressed_hashing.next(); - writeIntBinary(stream.plain_hashing.count(), marks_out); - writeIntBinary(stream.compressed_hashing.offset(), marks_out); + writeBinaryLittleEndian(stream.plain_hashing.count(), marks_out); + writeBinaryLittleEndian(stream.compressed_hashing.offset(), marks_out); /// Actually this numbers is redundant, but we have to store them /// to be compatible with the normal .mrk2 file format if (settings.can_use_adaptive_granularity) - writeIntBinary(1UL, marks_out); + writeBinaryLittleEndian(1UL, marks_out); } size_t pos = granule.start_row; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index f9fe6f2c8ab..bcf340e0f55 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -275,10 +275,10 @@ void MergeTreeDataPartWriterWide::flushMarkToFile(const StreamNameAndMark & stre Stream & stream = *column_streams[stream_with_mark.stream_name]; WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing; - writeIntBinary(stream_with_mark.mark.offset_in_compressed_file, marks_out); - writeIntBinary(stream_with_mark.mark.offset_in_decompressed_block, marks_out); + writeBinaryLittleEndian(stream_with_mark.mark.offset_in_compressed_file, marks_out); + writeBinaryLittleEndian(stream_with_mark.mark.offset_in_decompressed_block, marks_out); if (settings.can_use_adaptive_granularity) - writeIntBinary(rows_in_mark, marks_out); + writeBinaryLittleEndian(rows_in_mark, marks_out); } StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( @@ -452,10 +452,10 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai "Incorrect number of marks in memory {}, on disk (at least) {}", index_granularity.getMarksCount(), mark_num + 1); - DB::readBinary(offset_in_compressed_file, *mrk_in); - DB::readBinary(offset_in_decompressed_block, *mrk_in); + readBinaryLittleEndian(offset_in_compressed_file, *mrk_in); + readBinaryLittleEndian(offset_in_decompressed_block, *mrk_in); if (settings.can_use_adaptive_granularity) - DB::readBinary(index_granularity_rows, *mrk_in); + readBinaryLittleEndian(index_granularity_rows, *mrk_in); else index_granularity_rows = data_part->index_granularity_info.fixed_index_granularity; diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 5c722eec380..c9b22c8a03e 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -160,7 +160,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() size_t granularity; reader->readStrict( reinterpret_cast(plain_marks.data() + i * columns_in_mark), columns_in_mark * sizeof(MarkInCompressedFile)); - readIntBinary(granularity, *reader); + readBinaryLittleEndian(granularity, *reader); } if (!reader->eof()) @@ -170,6 +170,16 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() mrk_path, marks_count, expected_uncompressed_size); } +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + std::ranges::for_each( + plain_marks, + [](auto & plain_mark) + { + plain_mark.offset_in_compressed_file = std::byteswap(plain_mark.offset_in_compressed_file); + plain_mark.offset_in_decompressed_block = std::byteswap(plain_mark.offset_in_decompressed_block); + }); +#endif + auto res = std::make_shared(plain_marks); ProfileEvents::increment(ProfileEvents::LoadedMarksCount, marks_count * columns_in_mark); diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index bce33438229..63efd137b0b 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -265,12 +265,12 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const for (const Field & field : value) applyVisitor(hashing_visitor, field); - char hash_data[16]; - hash.get128(hash_data); - result.resize(32); - for (size_t i = 0; i < 16; ++i) + const auto hash_data = getSipHash128AsArray(hash); + const auto hash_size = hash_data.size(); + result.resize(hash_size * 2); + for (size_t i = 0; i < hash_size; ++i) #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - writeHexByteLowercase(hash_data[16 - 1 - i], &result[2 * i]); + writeHexByteLowercase(hash_data[hash_size - 1 - i], &result[2 * i]); #else writeHexByteLowercase(hash_data[i], &result[2 * i]); #endif diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp index 88f4a3ec66f..24d907dbad6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp @@ -12,9 +12,7 @@ static std::array getSipHash(const String & str) { SipHash hash; hash.update(str.data(), str.size()); - std::array result; - hash.get128(result.data()); - return result; + return getSipHash128AsArray(hash); } ReplicatedMergeTreePartHeader ReplicatedMergeTreePartHeader::fromColumnsAndChecksumsZNodes( diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 393d418c083..1d0acd782b3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -254,14 +254,9 @@ namespace for (const auto & col : cols) col->updateHashWithValue(j, hash); } - union - { - char bytes[16]; - UInt64 words[2]; - } hash_value; - hash.get128(hash_value.bytes); - block_id_vec.push_back(partition_id + "_" + DB::toString(hash_value.words[0]) + "_" + DB::toString(hash_value.words[1])); + const auto hash_value = hash.get128(); + block_id_vec.push_back(partition_id + "_" + DB::toString(hash_value.items[0]) + "_" + DB::toString(hash_value.items[1])); } else block_id_vec.push_back(partition_id + "_" + std::string(token)); From 967be3e13cb742245c9124fea2c89c3acdeb75da Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 22 Jun 2023 08:41:30 -0700 Subject: [PATCH 0882/2047] Correct a line commented out by mistake --- programs/obfuscator/Obfuscator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 43b31843afe..31288b4aa01 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -398,7 +398,7 @@ static void transformUUID(const UUID & src_uuid, UUID & dst_uuid, UInt64 seed) hash.update(reinterpret_cast(&src), sizeof(UUID)); /// Saving version and variant from an old UUID - // hash.get128Impl(reinterpret_cast(&dst)); + dst = hash.get128(); dst.items[1] = (dst.items[1] & 0x1fffffffffffffffull) | (src.items[1] & 0xe000000000000000ull); dst.items[0] = (dst.items[0] & 0xffffffffffff0fffull) | (src.items[0] & 0x000000000000f000ull); From 6b96a3943de8799cd7531e66f575276099012757 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 23 Jun 2023 15:22:21 +0000 Subject: [PATCH 0883/2047] Update further uses of SipHash --- src/Common/SipHash.h | 16 +--------------- src/Core/SortDescription.cpp | 3 +-- src/Interpreters/Aggregator.cpp | 3 +-- src/Interpreters/JIT/CompileDAG.cpp | 4 +--- 4 files changed, 4 insertions(+), 22 deletions(-) diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index e9d45c7201f..c77fe7f2bcf 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -88,20 +88,6 @@ private: SIPROUND; } - /// @brief Retrieves the result in some form with the endianness of the platform taken into account. - /// @warning This can only be done once! - void get128Impl(char * out) - { - finalize(); -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - unalignedStore(out + 8, v0 ^ v1); - unalignedStore(out, v2 ^ v3); -#else - unalignedStore(out, v0 ^ v1); - unalignedStore(out + 8, v2 ^ v3); -#endif - } - public: /// Arguments - seed. SipHash(UInt64 key0 = 0, UInt64 key1 = 0, bool is_reference_128_ = false) /// NOLINT @@ -305,7 +291,7 @@ inline UInt64 sipHash64(const char * data, const size_t size) } template -UInt64 sipHash64(const T & x) +inline UInt64 sipHash64(const T & x) { SipHash hash; hash.update(x); diff --git a/src/Core/SortDescription.cpp b/src/Core/SortDescription.cpp index ae6aedf202d..9ba7df8ef24 100644 --- a/src/Core/SortDescription.cpp +++ b/src/Core/SortDescription.cpp @@ -133,8 +133,7 @@ void compileSortDescriptionIfNeeded(SortDescription & description, const DataTyp SipHash sort_description_dump_hash; sort_description_dump_hash.update(description_dump); - UInt128 sort_description_hash_key; - sort_description_dump_hash.get128(sort_description_hash_key); + const auto sort_description_hash_key = sort_description_dump_hash.get128(); { std::lock_guard lock(mutex); diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index c7d4b87694b..b02bfc5835c 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -694,8 +694,7 @@ void Aggregator::compileAggregateFunctionsIfNeeded() SipHash aggregate_functions_description_hash; aggregate_functions_description_hash.update(functions_description); - UInt128 aggregate_functions_description_hash_key; - aggregate_functions_description_hash.get128(aggregate_functions_description_hash_key); + const auto aggregate_functions_description_hash_key = aggregate_functions_description_hash.get128(); { std::lock_guard lock(mutex); diff --git a/src/Interpreters/JIT/CompileDAG.cpp b/src/Interpreters/JIT/CompileDAG.cpp index 6da17fb4c67..9c56341eae0 100644 --- a/src/Interpreters/JIT/CompileDAG.cpp +++ b/src/Interpreters/JIT/CompileDAG.cpp @@ -160,9 +160,7 @@ UInt128 CompileDAG::hash() const } } - UInt128 result; - hash.get128(result); - return result; + return hash.get128(); } } From ba4072f049bbf22a9b40f088527fe461ad36d910 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 5 Jul 2023 17:35:44 -0700 Subject: [PATCH 0884/2047] Adapt changes around SipHash --- contrib/cityhash102/include/city.h | 4 ++-- src/Analyzer/HashUtils.h | 4 ++-- src/Analyzer/IQueryTreeNode.h | 2 +- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Common/SipHash.h | 10 ++++++---- src/Common/TransformEndianness.hpp | 9 +++++++++ src/IO/WriteHelpers.h | 9 +++++++++ src/Interpreters/ComparisonGraph.h | 2 +- src/Interpreters/Context.cpp | 8 ++++---- src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Interpreters/MonotonicityCheckVisitor.h | 4 ++-- src/Interpreters/PreparedSets.cpp | 2 +- src/Interpreters/PreparedSets.h | 4 ++-- src/Interpreters/TreeOptimizer.cpp | 8 ++++---- src/Parsers/ASTSubquery.cpp | 6 ++---- src/Parsers/IAST.h | 2 +- src/Planner/PlannerActionsVisitor.cpp | 2 +- src/Planner/PlannerContext.cpp | 4 ++-- .../Formats/Impl/ConstantExpressionTemplate.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- src/Storages/System/StorageSystemQueryCache.cpp | 2 +- src/Storages/buildQueryTreeForShard.cpp | 4 ++-- 23 files changed, 59 insertions(+), 41 deletions(-) diff --git a/contrib/cityhash102/include/city.h b/contrib/cityhash102/include/city.h index 87363d16444..c98eb7e3585 100644 --- a/contrib/cityhash102/include/city.h +++ b/contrib/cityhash102/include/city.h @@ -73,8 +73,8 @@ struct uint128 uint128() = default; uint128(uint64 low64_, uint64 high64_) : low64(low64_), high64(high64_) {} - friend bool operator ==(const uint128 & x, const uint128 & y) { return (x.low64 == y.low64) && (x.high64 == y.high64); } - friend bool operator !=(const uint128 & x, const uint128 & y) { return !(x == y); } + + friend auto operator<=>(const uint128 &, const uint128 &) = default; }; inline uint64 Uint128Low64(const uint128 & x) { return x.low64; } diff --git a/src/Analyzer/HashUtils.h b/src/Analyzer/HashUtils.h index 2203e7d5203..3727ea1ea14 100644 --- a/src/Analyzer/HashUtils.h +++ b/src/Analyzer/HashUtils.h @@ -20,7 +20,7 @@ struct QueryTreeNodeWithHash {} QueryTreeNodePtrType node = nullptr; - std::pair hash; + CityHash_v1_0_2::uint128 hash; }; template @@ -55,6 +55,6 @@ struct std::hash> { size_t operator()(const DB::QueryTreeNodeWithHash & node_with_hash) const { - return node_with_hash.hash.first; + return node_with_hash.hash.low64; } }; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 763963b734a..3f6816696b4 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -106,7 +106,7 @@ public: */ bool isEqual(const IQueryTreeNode & rhs, CompareOptions compare_options = { .compare_aliases = true }) const; - using Hash = std::pair; + using Hash = CityHash_v1_0_2::uint128; using HashState = SipHash; /** Get tree hash identifying current tree diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index abf02547ccd..2d997bb7eff 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2032,7 +2032,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden auto & nearest_query_scope_query_node = nearest_query_scope->scope_node->as(); auto & mutable_context = nearest_query_scope_query_node.getMutableContext(); - auto scalar_query_hash_string = std::to_string(node_with_hash.hash.first) + '_' + std::to_string(node_with_hash.hash.second); + const auto scalar_query_hash_string = DB::toString(node_with_hash.hash); if (mutable_context->hasQueryContext()) mutable_context->getQueryContext()->addScalar(scalar_query_hash_string, scalar_block); diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index c77fe7f2bcf..00fe07648a5 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -24,6 +24,8 @@ #include #include +#include + namespace DB::ErrorCodes { extern const int LOGICAL_ERROR; @@ -247,11 +249,11 @@ inline std::array getSipHash128AsArray(SipHash & sip_hash) return arr; } -inline std::pair getSipHash128AsLoHi(SipHash & sip_hash) +inline CityHash_v1_0_2::uint128 getSipHash128AsLoHi(SipHash & sip_hash) { - std::pair lo_hi; - sip_hash.get128(lo_hi.first, lo_hi.second); - return lo_hi; + CityHash_v1_0_2::uint128 result; + sip_hash.get128(result.low64, result.high64); + return result; } inline UInt128 sipHash128Keyed(UInt64 key0, UInt64 key1, const char * data, const size_t size) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index ccfec78ab64..c7bea2ef15a 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -4,6 +4,8 @@ #include #include +#include + #include namespace DB @@ -66,4 +68,11 @@ inline void transformEndianness(StrongTypedef & x) { transformEndianness(x.toUnderType()); } + +template +inline void transformEndianness(CityHash_v1_0_2::uint128 & x) +{ + transformEndianness(x.low64); + transformEndianness(x.high64); +} } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index aa4c9b17e48..a9d1c108061 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1161,6 +1161,15 @@ inline String toString(const T & x) return buf.str(); } +inline String toString(const CityHash_v1_0_2::uint128 & hash) +{ + WriteBufferFromOwnString buf; + writeText(hash.low64, buf); + writeChar('_', buf); + writeText(hash.high64, buf); + return buf.str(); +} + template inline String toStringWithFinalSeparator(const std::vector & x, const String & final_sep) { diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 70543227b58..4fd90dad371 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -118,7 +118,7 @@ private: { size_t operator() (const IAST::Hash & hash) const { - return hash.first; + return hash.low64; } }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a9055bbb1b9..f179e38b0f8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1505,7 +1505,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const } } auto hash = table_expression->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + auto key = toString(hash); StoragePtr & res = table_function_results[key]; if (!res) { @@ -1656,7 +1656,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const auto new_hash = table_expression->getTreeHash(); if (hash != new_hash) { - key = toString(new_hash.first) + '_' + toString(new_hash.second); + key = toString(new_hash); table_function_results[key] = res; } } @@ -1665,8 +1665,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const TableFunctionPtr & table_function_ptr) { - auto hash = table_expression->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = table_expression->getTreeHash(); + const auto key = toString(hash); StoragePtr & res = table_function_results[key]; if (!res) diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 3d821b60e81..ded99df9f74 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -98,7 +98,7 @@ static auto getQueryInterpreter(const ASTSubquery & subquery, ExecuteScalarSubqu void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr & ast, Data & data) { auto hash = subquery.getTreeHash(); - auto scalar_query_hash_str = toString(hash.first) + "_" + toString(hash.second); + const auto scalar_query_hash_str = toString(hash); std::unique_ptr interpreter = nullptr; bool hit = false; diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 5b633fee9b6..08378c3158b 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -115,7 +115,7 @@ public: if (alias.empty()) { auto hash = subquery_or_table_name->getTreeHash(); - external_table_name = fmt::format("_data_{}_{}", hash.first, hash.second); + external_table_name = fmt::format("_data_{}", toString(hash)); } else external_table_name = alias; diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index c95f5209760..c1838fa105c 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -39,8 +39,8 @@ public: bool canOptimize(const ASTFunction & ast_function) const { /// if GROUP BY contains the same function ORDER BY shouldn't be optimized - auto hash = ast_function.getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = ast_function.getTreeHash(); + const auto key = toString(hash); if (group_by_function_hashes.count(key)) return false; diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 67822ecf440..c2c01b912f9 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -207,7 +207,7 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & types) { WriteBufferFromOwnString buf; - buf << "__set_" << key.first << "_" << key.second; + buf << "__set_" << DB::toString(key); if (!types.empty()) { buf << "("; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index cb240f5260a..b953b8470e1 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -127,10 +127,10 @@ class PreparedSets { public: - using Hash = std::pair; + using Hash = CityHash_v1_0_2::uint128; struct Hashing { - UInt64 operator()(const Hash & key) const { return key.first ^ key.second; } + UInt64 operator()(const Hash & key) const { return key.low64 ^ key.high64; } }; using SetsFromTuple = std::unordered_map>, Hashing>; diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index fd4d2c9d846..1a7b5cd73ec 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -451,8 +451,8 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context { for (auto & elem : set->children) { - auto hash = elem->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = elem->getTreeHash(); + const auto key = toString(hash); group_by_hashes.insert(key); } } @@ -461,8 +461,8 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context { for (auto & elem : group_by->children) { - auto hash = elem->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = elem->getTreeHash(); + const auto key = toString(hash); group_by_hashes.insert(key); } } diff --git a/src/Parsers/ASTSubquery.cpp b/src/Parsers/ASTSubquery.cpp index a3408f12330..92adad666ed 100644 --- a/src/Parsers/ASTSubquery.cpp +++ b/src/Parsers/ASTSubquery.cpp @@ -19,11 +19,9 @@ void ASTSubquery::appendColumnNameImpl(WriteBuffer & ostr) const } else { - Hash hash = getTreeHash(); + const auto hash = getTreeHash(); writeCString("__subquery_", ostr); - writeText(hash.first, ostr); - ostr.write('_'); - writeText(hash.second, ostr); + writeString(toString(hash), ostr); } } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 7a8ab36518d..d0464c7b950 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -78,7 +78,7 @@ public: /** Get hash code, identifying this element and its subtree. */ - using Hash = std::pair; + using Hash = CityHash_v1_0_2::uint128; Hash getTreeHash() const; void updateTreeHash(SipHash & hash_state) const; virtual void updateTreeHashImpl(SipHash & hash_state) const; diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 7575828e64d..b39716c7712 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -165,7 +165,7 @@ public: case QueryTreeNodeType::LAMBDA: { auto lambda_hash = node->getTreeHash(); - result = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); + result = "__lambda_" + toString(lambda_hash); break; } default: diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 3c75d4fbea8..f0e6e44ad8c 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -114,8 +114,8 @@ const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const Que PlannerContext::SetKey PlannerContext::createSetKey(const QueryTreeNodePtr & set_source_node) { - auto set_source_hash = set_source_node->getTreeHash(); - return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); + const auto set_source_hash = set_source_node->getTreeHash(); + return "__set_" + toString(set_source_hash); } } diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 2d5264fa4e3..c388c370848 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -371,8 +371,8 @@ size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTP const auto res128 = getSipHash128AsLoHi(hash_state); size_t res = 0; - boost::hash_combine(res, res128.first); - boost::hash_combine(res, res128.second); + boost::hash_combine(res, res128.low64); + boost::hash_combine(res, res128.high64); return res; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9c5e45aa488..fafa35d34c1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -187,8 +187,8 @@ static void checkSuspiciousIndices(const ASTFunction * index_function) std::unordered_set unique_index_expression_hashes; for (const auto & child : index_function->arguments->children) { - IAST::Hash hash = child->getTreeHash(); - UInt64 first_half_of_hash = hash.first; + const IAST::Hash hash = child->getTreeHash(); + const auto & first_half_of_hash = hash.low64; if (!unique_index_expression_hashes.emplace(first_half_of_hash).second) throw Exception(ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 117fb4e8a5c..b6ab2c54407 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -50,7 +50,7 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr res_columns[3]->insert(key.is_shared); res_columns[4]->insert(key.is_compressed); res_columns[5]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); - res_columns[6]->insert(key.ast->getTreeHash().first); + res_columns[6]->insert(key.ast->getTreeHash().low64); } } diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 1ee7d747fcc..18ec8e48c3e 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -232,8 +232,8 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, ContextMutablePtr & mutable_context, size_t subquery_depth) { - auto subquery_hash = subquery_node->getTreeHash(); - String temporary_table_name = fmt::format("_data_{}_{}", subquery_hash.first, subquery_hash.second); + const auto subquery_hash = subquery_node->getTreeHash(); + const auto temporary_table_name = fmt::format("_data_{}", toString(subquery_hash)); const auto & external_tables = mutable_context->getExternalTables(); auto external_table_it = external_tables.find(temporary_table_name); From b8a959da8927b05395a22052a969a803b1a0d1f4 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 6 Jul 2023 05:41:34 -0700 Subject: [PATCH 0885/2047] Correct a finding from clang-tidy --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 2d997bb7eff..98e5d326cbf 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2032,7 +2032,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden auto & nearest_query_scope_query_node = nearest_query_scope->scope_node->as(); auto & mutable_context = nearest_query_scope_query_node.getMutableContext(); - const auto scalar_query_hash_string = DB::toString(node_with_hash.hash); + auto scalar_query_hash_string = DB::toString(node_with_hash.hash); if (mutable_context->hasQueryContext()) mutable_context->getQueryContext()->addScalar(scalar_query_hash_string, scalar_block); From 1208e59b8e939adedf7ff7c12a91bb04b05471d5 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 14 Jul 2023 14:28:03 +0000 Subject: [PATCH 0886/2047] Inline hash geetter mthods for potential performance gains --- src/Common/SipHash.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 00fe07648a5..22c2c61abdc 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -182,7 +182,7 @@ public: ALWAYS_INLINE void update(const std::string_view x) { update(x.data(), x.size()); } ALWAYS_INLINE void update(const char * s) { update(std::string_view(s)); } - UInt64 get64() + ALWAYS_INLINE UInt64 get64() { finalize(); return v0 ^ v1 ^ v2 ^ v3; @@ -197,7 +197,7 @@ public: hi = v2 ^ v3; } - UInt128 get128() + ALWAYS_INLINE UInt128 get128() { UInt128 res; #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ From 29094a22cf4cc787cc50f5d494b29dcf511cc765 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 20:00:00 +0200 Subject: [PATCH 0887/2047] Fix Docker --- docker/images.json | 1 + tests/ci/tests/docker_images.json | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/images.json b/docker/images.json index 663fa21a6c9..8e19401ba72 100644 --- a/docker/images.json +++ b/docker/images.json @@ -125,6 +125,7 @@ "docker/test/keeper-jepsen", "docker/test/server-jepsen", "docker/test/sqllogic", + "docker/test/sqltest", "docker/test/stateless" ] }, diff --git a/tests/ci/tests/docker_images.json b/tests/ci/tests/docker_images.json index 0637058e184..70db8760561 100644 --- a/tests/ci/tests/docker_images.json +++ b/tests/ci/tests/docker_images.json @@ -119,7 +119,8 @@ "docker/test/stateless", "docker/test/integration/base", "docker/test/fuzzer", - "docker/test/keeper-jepsen" + "docker/test/keeper-jepsen", + "docker/test/sqltest" ] }, "docker/test/integration/kerberized_hadoop": { From a3d0335f536668af3d6dd325e639d67cce624b46 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 19 Jul 2023 11:20:42 -0700 Subject: [PATCH 0888/2047] Fix a build error --- src/Parsers/ASTSetQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index 76ad812e713..1b7b76fe231 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -69,9 +69,9 @@ void ASTSetQuery::appendColumnName(WriteBuffer & ostr) const Hash hash = getTreeHash(); writeCString("__settings_", ostr); - writeText(hash.first, ostr); + writeText(hash.low64, ostr); ostr.write('_'); - writeText(hash.second, ostr); + writeText(hash.high64, ostr); } } From 90be5e6160f2342121dfe2d014f4d247a4efa39c Mon Sep 17 00:00:00 2001 From: Samuel Colvin Date: Wed, 19 Jul 2023 20:04:43 +0100 Subject: [PATCH 0889/2047] Remove reference to `TIMEOUT` in live views documentation Temporary live views were removed in #42173, but the documentation was not fully updated to reflect the change. --- docs/en/sql-reference/statements/create/view.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 10b15638152..11026340a0f 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -97,7 +97,7 @@ This is an experimental feature that may change in backwards-incompatible ways i ::: ```sql -CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH [TIMEOUT [value_in_sec] [AND]] [REFRESH [value_in_sec]]] AS SELECT ... +CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH REFRESH [value_in_sec]] AS SELECT ... ``` Live views store result of the corresponding [SELECT](../../../sql-reference/statements/select/index.md) query and are updated any time the result of the query changes. Query result as well as partial result needed to combine with new data are stored in memory providing increased performance for repeated queries. Live views can provide push notifications when query result changes using the [WATCH](../../../sql-reference/statements/watch.md) query. From 02fe735b768e2d171191091c402f4732ace4669e Mon Sep 17 00:00:00 2001 From: Samuel Colvin Date: Wed, 19 Jul 2023 20:12:35 +0100 Subject: [PATCH 0890/2047] fix in other other languages --- docs/ru/sql-reference/statements/create/view.md | 2 +- docs/zh/sql-reference/statements/create/view.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md index d3846aac289..1a60dc0716c 100644 --- a/docs/ru/sql-reference/statements/create/view.md +++ b/docs/ru/sql-reference/statements/create/view.md @@ -73,7 +73,7 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na Чтобы использовать `LIVE VIEW` и запросы `WATCH`, включите настройку [allow_experimental_live_view](../../../operations/settings/settings.md#allow-experimental-live-view). ::: ```sql -CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH [TIMEOUT [value_in_sec] [AND]] [REFRESH [value_in_sec]]] AS SELECT ... +CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH REFRESH [value_in_sec]] AS SELECT ... ``` `LIVE VIEW` хранит результат запроса [SELECT](../../../sql-reference/statements/select/index.md), указанного при создании, и обновляется сразу же при изменении этого результата. Конечный результат запроса и промежуточные данные, из которых формируется результат, хранятся в оперативной памяти, и это обеспечивает высокую скорость обработки для повторяющихся запросов. LIVE-представления могут отправлять push-уведомления при изменении результата исходного запроса `SELECT`. Для этого используйте запрос [WATCH](../../../sql-reference/statements/watch.md). diff --git a/docs/zh/sql-reference/statements/create/view.md b/docs/zh/sql-reference/statements/create/view.md index 8ce2d20a10c..bce0994ecd2 100644 --- a/docs/zh/sql-reference/statements/create/view.md +++ b/docs/zh/sql-reference/statements/create/view.md @@ -72,7 +72,7 @@ ClickHouse 中的物化视图更像是插入触发器。 如果视图查询中 使用[allow_experimental_live_view](../../../operations/settings/settings.md#allow-experimental-live-view)设置启用实时视图和`WATCH`查询的使用。 输入命令`set allow_experimental_live_view = 1`。 ```sql -CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH [TIMEOUT [value_in_sec] [AND]] [REFRESH [value_in_sec]]] AS SELECT ... +CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH REFRESH [value_in_sec]] AS SELECT ... ``` 实时视图存储相应[SELECT](../../../sql-reference/statements/select/index.md)查询的结果,并在查询结果更改时随时更新。 查询结果以及与新数据结合所需的部分结果存储在内存中,为重复查询提供更高的性能。当使用[WATCH](../../../sql-reference/statements/watch.md)查询更改查询结果时,实时视图可以提供推送通知。 From dbdac5d823d431fb34405649f7125e76c88f1f05 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 19 Jul 2023 19:34:49 +0000 Subject: [PATCH 0891/2047] Add query with UNION --- .../02500_remove_redundant_distinct.reference | 29 +++++++++++++++++++ .../02500_remove_redundant_distinct.sh | 12 ++++++++ ...move_redundant_distinct_analyzer.reference | 29 +++++++++++++++++++ 3 files changed, 70 insertions(+) diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct.reference index 2e049dbc936..763a7cc4286 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct.reference @@ -477,3 +477,32 @@ Expression (Projection) ReadFromStorage (SystemNumbers) -- execute 1 +-- UNION ALL with DISTINCT => do _not_ remove DISTINCT +-- query +SELECT DISTINCT number +FROM +( + SELECT DISTINCT number + FROM numbers(1) + UNION ALL + SELECT DISTINCT number + FROM numbers(2) +) +-- explain +Expression (Projection) + Distinct + Distinct (Preliminary DISTINCT) + Union + Expression ((Before ORDER BY + Projection)) + Distinct + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + ReadFromStorage (SystemNumbers) + Expression (( + Projection)) + Distinct + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + ReadFromStorage (SystemNumbers) +-- execute +0 +1 diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct.sh b/tests/queries/0_stateless/02500_remove_redundant_distinct.sh index 41744cc59f9..f07cdca4b5a 100755 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct.sh +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct.sh @@ -264,3 +264,15 @@ run_query "$query" echo "-- DISTINCT COUNT() with GROUP BY => do _not_ remove DISTINCT" query="select distinct count() from numbers(10) group by number" run_query "$query" + +echo "-- UNION ALL with DISTINCT => do _not_ remove DISTINCT" +query="SELECT DISTINCT number +FROM +( + SELECT DISTINCT number + FROM numbers(1) + UNION ALL + SELECT DISTINCT number + FROM numbers(2) +)" +run_query "$query" diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference index c9301c1f0a3..50ca5981cf1 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference @@ -479,3 +479,32 @@ Expression (Project names) ReadFromStorage (SystemNumbers) -- execute 1 +-- UNION ALL with DISTINCT => do _not_ remove DISTINCT +-- query +SELECT DISTINCT number +FROM +( + SELECT DISTINCT number + FROM numbers(1) + UNION ALL + SELECT DISTINCT number + FROM numbers(2) +) +-- explain +Expression (Project names) + Distinct (DISTINCT) + Distinct (Preliminary DISTINCT) + Union + Expression ((Projection + (Change column names to column identifiers + Project names))) + Distinct (DISTINCT) + Distinct (Preliminary DISTINCT) + Expression ((Projection + Change column names to column identifiers)) + ReadFromStorage (SystemNumbers) + Expression (( + ( + Project names))) + Distinct (DISTINCT) + Distinct (Preliminary DISTINCT) + Expression ((Projection + Change column names to column identifiers)) + ReadFromStorage (SystemNumbers) +-- execute +0 +1 From 2b8e4ebd4c3df56c2d3e445321cedb157c7956f7 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 19 Jul 2023 19:48:39 +0000 Subject: [PATCH 0892/2047] Allow to disable decoding/encoding path in uri in URL engine --- base/poco/Foundation/include/Poco/URI.h | 6 +++- base/poco/Foundation/src/URI.cpp | 39 ++++++++++++++++++------- docs/en/operations/settings/settings.md | 6 ++++ src/Core/Settings.h | 1 + src/IO/ReadWriteBufferFromHTTP.cpp | 6 ++-- src/Storages/StorageURL.cpp | 2 +- 6 files changed, 45 insertions(+), 15 deletions(-) diff --git a/base/poco/Foundation/include/Poco/URI.h b/base/poco/Foundation/include/Poco/URI.h index 1880af4ccd2..5e6e7efd938 100644 --- a/base/poco/Foundation/include/Poco/URI.h +++ b/base/poco/Foundation/include/Poco/URI.h @@ -57,7 +57,7 @@ public: URI(); /// Creates an empty URI. - explicit URI(const std::string & uri); + explicit URI(const std::string & uri, bool decode_and_encode_path = true); /// Parses an URI from the given string. Throws a /// SyntaxException if the uri is not valid. @@ -350,6 +350,8 @@ protected: static const std::string ILLEGAL; private: + void encodePath(std::string & encodedStr) const; + std::string _scheme; std::string _userInfo; std::string _host; @@ -357,6 +359,8 @@ private: std::string _path; std::string _query; std::string _fragment; + + bool _decode_and_encode_path = true; }; diff --git a/base/poco/Foundation/src/URI.cpp b/base/poco/Foundation/src/URI.cpp index 5543e02b279..91a82868dcf 100644 --- a/base/poco/Foundation/src/URI.cpp +++ b/base/poco/Foundation/src/URI.cpp @@ -36,8 +36,8 @@ URI::URI(): } -URI::URI(const std::string& uri): - _port(0) +URI::URI(const std::string& uri, bool decode_and_encode_path): + _port(0), _decode_and_encode_path(decode_and_encode_path) { parse(uri); } @@ -107,7 +107,8 @@ URI::URI(const URI& uri): _port(uri._port), _path(uri._path), _query(uri._query), - _fragment(uri._fragment) + _fragment(uri._fragment), + _decode_and_encode_path(uri._decode_and_encode_path) { } @@ -119,7 +120,8 @@ URI::URI(const URI& baseURI, const std::string& relativeURI): _port(baseURI._port), _path(baseURI._path), _query(baseURI._query), - _fragment(baseURI._fragment) + _fragment(baseURI._fragment), + _decode_and_encode_path(baseURI._decode_and_encode_path) { resolve(relativeURI); } @@ -151,6 +153,7 @@ URI& URI::operator = (const URI& uri) _path = uri._path; _query = uri._query; _fragment = uri._fragment; + _decode_and_encode_path = uri._decode_and_encode_path; } return *this; } @@ -181,6 +184,7 @@ void URI::swap(URI& uri) std::swap(_path, uri._path); std::swap(_query, uri._query); std::swap(_fragment, uri._fragment); + std::swap(_decode_and_encode_path, uri._decode_and_encode_path); } @@ -201,7 +205,7 @@ std::string URI::toString() const std::string uri; if (isRelative()) { - encode(_path, RESERVED_PATH, uri); + encodePath(uri); } else { @@ -217,7 +221,7 @@ std::string URI::toString() const { if (!auth.empty() && _path[0] != '/') uri += '/'; - encode(_path, RESERVED_PATH, uri); + encodePath(uri); } else if (!_query.empty() || !_fragment.empty()) { @@ -313,7 +317,10 @@ void URI::setAuthority(const std::string& authority) void URI::setPath(const std::string& path) { _path.clear(); - decode(path, _path); + if (_decode_and_encode_path) + decode(path, _path); + else + _path = path; } @@ -418,7 +425,7 @@ void URI::setPathEtc(const std::string& pathEtc) std::string URI::getPathEtc() const { std::string pathEtc; - encode(_path, RESERVED_PATH, pathEtc); + encodePath(pathEtc); if (!_query.empty()) { pathEtc += '?'; @@ -436,7 +443,7 @@ std::string URI::getPathEtc() const std::string URI::getPathAndQuery() const { std::string pathAndQuery; - encode(_path, RESERVED_PATH, pathAndQuery); + encodePath(pathAndQuery); if (!_query.empty()) { pathAndQuery += '?'; @@ -626,6 +633,8 @@ void URI::encode(const std::string& str, const std::string& reserved, std::strin for (std::string::const_iterator it = str.begin(); it != str.end(); ++it) { char c = *it; + if (c == '%') + throw std::runtime_error("WTF"); if ((c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') || (c >= '0' && c <= '9') || @@ -681,6 +690,13 @@ void URI::decode(const std::string& str, std::string& decodedStr, bool plusAsSpa } } +void URI::encodePath(std::string & encodedStr) const +{ + if (_decode_and_encode_path) + encode(_path, RESERVED_PATH, encodedStr); + else + encodedStr = _path; +} bool URI::isWellKnownPort() const { @@ -820,7 +836,10 @@ void URI::parsePath(std::string::const_iterator& it, const std::string::const_it { std::string path; while (it != end && *it != '?' && *it != '#') path += *it++; - decode(path, _path); + if (_decode_and_encode_path) + decode(path, _path); + else + _path = path; } diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8b969f87a4d..db5d1a2f5d9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3466,6 +3466,12 @@ Possible values: Default value: `0`. +## decode_and_encode_path_in_url {#decode_and_encode_path_in_url} + +Enables or disables decoding/encoding path in uri in [URL](../../engines/table-engines/special/url.md) engine tables. + +Enabled by default. + ## database_atomic_wait_for_drop_and_detach_synchronously {#database_atomic_wait_for_drop_and_detach_synchronously} Adds a modifier `SYNC` to all `DROP` and `DETACH` queries. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8f304f0aab6..ffa72d841be 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -621,6 +621,7 @@ class IColumn; M(Bool, engine_file_allow_create_multiple_files, false, "Enables or disables creating a new file on each insert in file engine tables if format has suffix.", 0) \ M(Bool, engine_file_skip_empty_files, false, "Allows to skip empty files in file table engine", 0) \ M(Bool, engine_url_skip_empty_files, false, "Allows to skip empty files in url table engine", 0) \ + M(Bool, decode_and_encode_path_in_url, true, "Enables or disables decoding/encoding path in uri in URL table engine", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(Bool, database_replicated_enforce_synchronous_settings, false, "Enforces synchronous waiting for some queries (see also database_atomic_wait_for_drop_and_detach_synchronously, mutation_sync, alter_sync). Not recommended to enable these settings.", 0) \ diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index 6d1c0f7aafa..eea801ce65e 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -305,12 +305,12 @@ void ReadWriteBufferFromHTTPBase::callWithRedirects(Poco::N current_session = session; call(current_session, response, method_, throw_on_all_errors, for_object_info); - Poco::URI prev_uri = uri; + saved_uri_redirect = uri; while (isRedirect(response.getStatus())) { - Poco::URI uri_redirect = getUriAfterRedirect(prev_uri, response); - prev_uri = uri_redirect; + Poco::URI uri_redirect = getUriAfterRedirect(*saved_uri_redirect, response); + saved_uri_redirect = uri_redirect; if (remote_host_filter) remote_host_filter->checkURL(uri_redirect); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index e6953afe68e..4cfefbc5527 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -389,7 +389,7 @@ std::pair> StorageURLSource: for (; option != end; ++option) { bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end); - auto request_uri = Poco::URI(*option); + auto request_uri = Poco::URI(*option, context->getSettingsRef().decode_and_encode_path_in_url); for (const auto & [param, value] : params) request_uri.addQueryParameter(param, value); From 483ddb53ebfa01c02deda76a39bc44cc08df4f00 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 19 Jul 2023 19:51:58 +0000 Subject: [PATCH 0893/2047] Fixes --- base/poco/Foundation/src/URI.cpp | 2 -- docs/en/engines/table-engines/special/url.md | 1 + docs/en/sql-reference/table-functions/url.md | 3 ++- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/base/poco/Foundation/src/URI.cpp b/base/poco/Foundation/src/URI.cpp index 91a82868dcf..9bad1b39a87 100644 --- a/base/poco/Foundation/src/URI.cpp +++ b/base/poco/Foundation/src/URI.cpp @@ -633,8 +633,6 @@ void URI::encode(const std::string& str, const std::string& reserved, std::strin for (std::string::const_iterator it = str.begin(); it != str.end(); ++it) { char c = *it; - if (c == '%') - throw std::runtime_error("WTF"); if ((c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') || (c >= '0' && c <= '9') || diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index 26d4975954f..9f2bf177c96 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -106,3 +106,4 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. +- [decode_and_encode_path_in_url](/docs/en/operations/settings/settings.md#decode_and_encode_path_in_url) - enables or disables decoding/encoding path in uri. Enabled by default. diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 2ab43f1b895..96f36f03949 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -56,7 +56,8 @@ Character `|` inside patterns is used to specify failover addresses. They are it ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. +- [decode_and_encode_path_in_url](/docs/en/operations/settings/settings.md#decode_and_encode_path_in_url) - enables or disables decoding/encoding path in uri. Enabled by default. -**See Also** +- **See Also** - [Virtual columns](/docs/en/engines/table-engines/index.md#table_engines-virtual_columns) From e7b8767585e748d91796e669b871d40546c40bc8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Jul 2023 22:14:05 +0200 Subject: [PATCH 0894/2047] Mark test 02125_many_mutations_2 as no-parallel to avoid flakiness --- tests/queries/0_stateless/02125_many_mutations_2.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02125_many_mutations_2.sh b/tests/queries/0_stateless/02125_many_mutations_2.sh index 5b779c1b276..819ac8c9524 100755 --- a/tests/queries/0_stateless/02125_many_mutations_2.sh +++ b/tests/queries/0_stateless/02125_many_mutations_2.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan +# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From f458108c44f2d9f0c92725f05d3959658c97ee6f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 22:15:23 +0200 Subject: [PATCH 0895/2047] Fix style --- docker/test/sqllogic/run.sh | 2 ++ tests/ci/sqltest.py | 1 - 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/sqllogic/run.sh b/docker/test/sqllogic/run.sh index 444252837a3..3b900c097e2 100755 --- a/docker/test/sqllogic/run.sh +++ b/docker/test/sqllogic/run.sh @@ -1,4 +1,6 @@ #!/bin/bash +# shellcheck disable=SC2015 + set -exu trap "exit" INT TERM diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 5c20cc4849b..b752d4e4aee 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -11,7 +11,6 @@ from build_download_helper import get_build_name_for_check, read_build_urls from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse from commit_status_helper import ( RerunHelper, - format_description, get_commit, post_commit_status, ) From fa44f84377bc097cbe0c25852603de9be536f75e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jul 2023 04:50:12 +0200 Subject: [PATCH 0896/2047] Fix style --- docker/test/sqllogic/run.sh | 1 - docker/test/sqltest/run.sh | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqllogic/run.sh b/docker/test/sqllogic/run.sh index 3b900c097e2..5be44fc148c 100755 --- a/docker/test/sqllogic/run.sh +++ b/docker/test/sqllogic/run.sh @@ -1,5 +1,4 @@ #!/bin/bash -# shellcheck disable=SC2015 set -exu trap "exit" INT TERM diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 03678971f60..42aeef9df15 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -1,4 +1,5 @@ #!/bin/bash +# shellcheck disable=SC2015 set -x set -e From ff235e0f3078f6c27a9a1ab1383a91378313ab77 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 20 Jul 2023 05:41:39 +0000 Subject: [PATCH 0897/2047] Turn off log in queue, fix data race --- src/Common/SystemLogBase.cpp | 9 +++++++-- src/Common/SystemLogBase.h | 3 ++- src/Interpreters/TextLog.h | 6 ++++-- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 294ba09e375..d1845a292b9 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -42,10 +42,14 @@ ISystemLog::~ISystemLog() = default; template SystemLogQueue::SystemLogQueue( const String & table_name_, - size_t flush_interval_milliseconds_) + size_t flush_interval_milliseconds_, + bool turn_off_logger_) : log(&Poco::Logger::get("SystemLogQueue (" + table_name_ + ")")) , flush_interval_milliseconds(flush_interval_milliseconds_) -{} +{ + if (turn_off_logger_) + log->setLevel(0); +} static thread_local bool recursive_push_call = false; @@ -197,6 +201,7 @@ SystemLogQueue::Index SystemLogQueue::pop(std::vector void SystemLogQueue::shutdown() { + std::unique_lock lock(mutex); is_shutdown = true; /// Tell thread to shutdown. flush_event.notify_all(); diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 3716584be24..f6e4a579edf 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -81,7 +81,8 @@ class SystemLogQueue public: SystemLogQueue( const String & table_name_, - size_t flush_interval_milliseconds_); + size_t flush_interval_milliseconds_, + bool turn_off_logger_ = false); void shutdown(); diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 0febce03abc..60ca11632aa 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -40,6 +40,8 @@ struct TextLogElement class TextLog : public SystemLog { public: + using Queue = SystemLogQueue; + TextLog( ContextPtr context_, const String & database_name_, @@ -47,9 +49,9 @@ public: const String & storage_def_, size_t flush_interval_milliseconds_); - static std::shared_ptr> getLogQueue(size_t flush_interval_milliseconds) + static std::shared_ptr getLogQueue(size_t flush_interval_milliseconds) { - static std::shared_ptr> queue = std::make_shared>("text_log", flush_interval_milliseconds); + static std::shared_ptr queue = std::make_shared("text_log", flush_interval_milliseconds, true); return queue; } }; From 27ca367b2cb349391946c45d7e3d22fe6d543c42 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jul 2023 10:01:07 +0200 Subject: [PATCH 0898/2047] ZooKeeperRetriesControl rethrows with original callstack. --- src/Storages/MergeTree/ZooKeeperRetries.h | 44 ++++++++++++++++++----- 1 file changed, 35 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h index e55b04c27b3..512c0800de7 100644 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -72,7 +72,7 @@ public: if (!Coordination::isHardwareError(e.code)) throw; - setKeeperError(e.code, e.message()); + setKeeperError(std::current_exception(), e.code, e.message()); } catch (...) { @@ -91,16 +91,16 @@ public: } catch (const zkutil::KeeperException & e) { - setKeeperError(e.code, e.message()); + setKeeperError(std::current_exception(), e.code, e.message()); } catch (const Exception & e) { - setUserError(e.code(), e.what()); + setUserError(std::current_exception(), e.code(), e.what()); } return false; } - void setUserError(int code, std::string message) + void setUserError(std::exception_ptr exception, int code, std::string message) { if (retries_info.logger) LOG_TRACE( @@ -113,16 +113,28 @@ public: iteration_succeeded = false; user_error.code = code; user_error.message = std::move(message); + user_error.exception = exception; keeper_error = KeeperError{}; } + template + void setUserError(std::exception_ptr exception, int code, fmt::format_string fmt, Args &&... args) + { + setUserError(exception, code, fmt::format(fmt, std::forward(args)...)); + } + + void setUserError(int code, std::string message) + { + setUserError(std::make_exception_ptr(Exception::createDeprecated(message, code)), code, message); + } + template void setUserError(int code, fmt::format_string fmt, Args &&... args) { setUserError(code, fmt::format(fmt, std::forward(args)...)); } - void setKeeperError(Coordination::Error code, std::string message) + void setKeeperError(std::exception_ptr exception, Coordination::Error code, std::string message) { if (retries_info.logger) LOG_TRACE( @@ -135,9 +147,21 @@ public: iteration_succeeded = false; keeper_error.code = code; keeper_error.message = std::move(message); + keeper_error.exception = exception; user_error = UserError{}; } + template + void setKeeperError(std::exception_ptr exception, Coordination::Error code, fmt::format_string fmt, Args &&... args) + { + setKeeperError(exception, code, fmt::format(fmt, std::forward(args)...)); + } + + void setKeeperError(Coordination::Error code, std::string message) + { + setKeeperError(std::make_exception_ptr(zkutil::KeeperException(message, code)), code, message); + } + template void setKeeperError(Coordination::Error code, fmt::format_string fmt, Args &&... args) { @@ -163,12 +187,14 @@ private: using Code = Coordination::Error; Code code = Code::ZOK; std::string message; + std::exception_ptr exception; }; struct UserError { int code = ErrorCodes::OK; std::string message; + std::exception_ptr exception; }; bool canTry() @@ -232,11 +258,11 @@ private: void throwIfError() const { - if (user_error.code != ErrorCodes::OK) - throw Exception::createDeprecated(user_error.message, user_error.code); + if (user_error.exception) + std::rethrow_exception(user_error.exception); - if (keeper_error.code != KeeperError::Code::ZOK) - throw zkutil::KeeperException(keeper_error.message, keeper_error.code); + if (keeper_error.exception) + std::rethrow_exception(keeper_error.exception); } void logLastError(std::string_view header) From 6a21995b2097e747a28a23333e651208c25f0224 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 20 Jul 2023 10:42:19 +0200 Subject: [PATCH 0899/2047] Added test to analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e0f259306aa..9a9412e55db 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -130,3 +130,4 @@ 02581_share_big_sets_between_mutation_tasks_long 02581_share_big_sets_between_multiple_mutations_tasks_long 00992_system_parts_race_condition_zookeeper_long +02818_parameterized_view_with_cte_multiple_usage From c7ab6e908adf2a088ad41e00ea2bfad5ea16526a Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Thu, 20 Jul 2023 08:55:22 +0000 Subject: [PATCH 0900/2047] Move tode to to try to make the diff simpler --- src/Common/SystemLogBase.cpp | 60 ++++++++++++++++++------------------ 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index d1845a292b9..ed5ffd78a7b 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -121,6 +121,36 @@ void SystemLogQueue::push(const LogElement & element) LOG_INFO(log, "Queue is half full for system log '{}'.", demangle(typeid(*this).name())); } +template +void SystemLogBase::flush(bool force) +{ + uint64_t this_thread_requested_offset = queue->notifyFlush(force); + if (this_thread_requested_offset == uint64_t(-1)) + return; + + queue->waitFlush(this_thread_requested_offset); +} + +template +void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) +{ + // Use an arbitrary timeout to avoid endless waiting. 60s proved to be + // too fast for our parallel functional tests, probably because they + // heavily load the disk. + const int timeout_seconds = 180; + std::unique_lock lock(mutex); + bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] + { + return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables; + }); + + if (!result) + { + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Timeout exceeded ({} s) while flushing system log '{}'.", + toString(timeout_seconds), demangle(typeid(*this).name())); + } +} + template uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyway) { @@ -145,26 +175,6 @@ uint64_t SystemLogQueue::notifyFlush(bool should_prepare_tables_anyw return this_thread_requested_offset; } -template -void SystemLogQueue::waitFlush(uint64_t expected_flushed_up_to) -{ - // Use an arbitrary timeout to avoid endless waiting. 60s proved to be - // too fast for our parallel functional tests, probably because they - // heavily load the disk. - const int timeout_seconds = 180; - std::unique_lock lock(mutex); - bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&] - { - return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables; - }); - - if (!result) - { - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Timeout exceeded ({} s) while flushing system log '{}'.", - toString(timeout_seconds), demangle(typeid(*this).name())); - } -} - template void SystemLogQueue::confirm(uint64_t to_flush_end) { @@ -229,16 +239,6 @@ void SystemLogBase::add(const LogElement & element) queue->push(element); } -template -void SystemLogBase::flush(bool force) -{ - uint64_t this_thread_requested_offset = queue->notifyFlush(force); - if (this_thread_requested_offset == uint64_t(-1)) - return; - - queue->waitFlush(this_thread_requested_offset); -} - template void SystemLogBase::notifyFlush(bool force) { queue->notifyFlush(force); } From 16cc00784217574bfa4b434936b25f24c531e542 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 20 Jul 2023 09:21:18 +0000 Subject: [PATCH 0901/2047] Fix table ad variable name --- tests/integration/test_storage_kafka/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 27a315b53bb..7013f0198f3 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -865,7 +865,7 @@ def test_kafka_formats(kafka_cluster): expected_rows_count = raw_expected.count("\n") instance.query_with_retry( - f"SELECT * FROM test.kafka_data_{list(all_formats.keys())[-1]}_mv;", + f"SELECT * FROM test.kafka_{list(all_formats.keys())[-1]}_mv;", retry_count=30, sleep_time=1, check_callback=lambda res: res.count("\n") == expected_rows_count, @@ -3798,7 +3798,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): format_name=format_name ) ) - expected = pre_formatted_expected.format( + expected = raw_expected.format( topic_name=topic_name, offset_0=offsets[0], offset_1=offsets[1], From 067e3caa2c43ed981a7c598c45668f37b0ac32c6 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Thu, 20 Jul 2023 18:13:19 +0800 Subject: [PATCH 0902/2047] Remove constants from description_sorted_. --- src/Processors/Transforms/FinishSortingTransform.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index 066928446f2..744d035d0ee 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -38,15 +38,11 @@ FinishSortingTransform::FinishSortingTransform( /// Remove constants from description_sorted_. SortDescription description_sorted_without_constants; description_sorted_without_constants.reserve(description_sorted_.size()); - size_t num_columns = header.columns(); - ColumnNumbers map(num_columns, num_columns); for (const auto & column_description : description_sorted_) { - auto old_pos = header.getPositionByName(column_description.column_name); - auto new_pos = map[old_pos]; + auto pos = header.getPositionByName(column_description.column_name); - if (new_pos < num_columns) - { + if (!const_columns_to_remove[pos]){ description_sorted_without_constants.push_back(column_description); } } From 0ba97eeea597ad027c375cf292419dd555a9cb73 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Thu, 15 Jun 2023 08:05:47 +0800 Subject: [PATCH 0903/2047] wip: grace hash join support full & right join --- docs/en/operations/settings/settings.md | 2 + src/Interpreters/GraceHashJoin.cpp | 31 +++++++-- src/Interpreters/GraceHashJoin.h | 3 +- .../Transforms/JoiningTransform.cpp | 65 +++++++++++++++++-- src/Processors/Transforms/JoiningTransform.h | 24 ++++++- src/QueryPipeline/QueryPipelineBuilder.cpp | 2 +- ...01721_join_implicit_cast_long.reference.j2 | 40 ------------ .../01721_join_implicit_cast_long.sql.j2 | 1 - .../02273_full_sort_join.reference.j2 | 18 +---- .../0_stateless/02273_full_sort_join.sql.j2 | 4 +- ...274_full_sort_join_nodistinct.reference.j2 | 34 +--------- .../02274_full_sort_join_nodistinct.sql.j2 | 6 +- .../02275_full_sort_join_long.reference | 24 ++++++- .../02275_full_sort_join_long.sql.j2 | 9 +-- 14 files changed, 138 insertions(+), 125 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 22aeecf4335..580b51a984d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -537,6 +537,8 @@ Possible values: The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which don’t belong to the current bucket are flushed and reassigned. + Supports `INNER/LEFT/RIGHT/FULL ALL/ANY JOIN`. + - hash [Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section. diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index edf604bc0b4..f94453293f6 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -301,8 +301,10 @@ void GraceHashJoin::initBuckets() bool GraceHashJoin::isSupported(const std::shared_ptr & table_join) { + bool is_asof = (table_join->strictness() == JoinStrictness::Asof); - return !is_asof && isInnerOrLeft(table_join->kind()) && table_join->oneDisjunct(); + auto kind = table_join->kind(); + return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct(); } GraceHashJoin::~GraceHashJoin() = default; @@ -322,7 +324,6 @@ bool GraceHashJoin::hasMemoryOverflow(size_t total_rows, size_t total_bytes) con /// One row can't be split, avoid loop if (total_rows < 2) return false; - bool has_overflow = !table_join->sizeLimits().softCheck(total_rows, total_bytes); if (has_overflow) @@ -494,17 +495,30 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const return hash_join_is_empty; } -IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block &, const Block &, UInt64) const +/// Each bucket are handled by the following steps +/// 1. build hash_join by the right side blocks. +/// 2. join left side with the hash_join, +/// 3. read right non-joined blocks from hash_join. +/// buckets are handled one by one, each hash_join will not be release before the right non-joined blocks are emitted. +/// +/// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform, +/// only one processor could take the non-joined blocks from right stream, and ensure all rows from +/// left stream have been emitted before this. +IBlocksStreamPtr +GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const { - /// We do no support returning non joined blocks here. - /// TODO: They _should_ be reported by getDelayedBlocks instead - return nullptr; + return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_); } class GraceHashJoin::DelayedBlocks : public IBlocksStream { public: - explicit DelayedBlocks(size_t current_bucket_, Buckets buckets_, InMemoryJoinPtr hash_join_, const Names & left_key_names_, const Names & right_key_names_) + explicit DelayedBlocks( + size_t current_bucket_, + Buckets buckets_, + InMemoryJoinPtr hash_join_, + const Names & left_key_names_, + const Names & right_key_names_) : current_bucket(current_bucket_) , buckets(std::move(buckets_)) , hash_join(std::move(hash_join_)) @@ -522,12 +536,15 @@ public: do { + // One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransform. + // There is a lock inside left_reader.read() . block = left_reader.read(); if (!block) { return {}; } + // block comes from left_reader, need to join with right table to get the result. Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets); block = std::move(blocks[current_idx]); diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index bce04ee6b04..ce519892b0e 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -13,7 +13,6 @@ namespace DB { - class TableJoin; class HashJoin; @@ -79,7 +78,7 @@ public: bool supportTotals() const override { return false; } IBlocksStreamPtr - getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; + getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size) const override; /// Open iterator over joined blocks. /// Must be called after all @joinBlock calls. diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 49b90d04b81..f1ceefbf229 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -189,7 +189,6 @@ void JoiningTransform::transform(Chunk & chunk) } else block = readExecute(chunk); - auto num_rows = block.rows(); chunk.setColumns(block.getColumns(), num_rows); } @@ -311,8 +310,16 @@ void FillingRightJoinSideTransform::work() } -DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform(Block output_header) - : IProcessor(InputPorts{Block()}, OutputPorts{output_header}) +DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform( + Block left_header_, + Block output_header_, + size_t max_block_size_, + JoinPtr join_) + : IProcessor(InputPorts{Block()}, OutputPorts{output_header_}) + , left_header(left_header_) + , output_header(output_header_) + , max_block_size(max_block_size_) + , join(join_) { } @@ -365,6 +372,7 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() if (!data.chunk.hasChunkInfo()) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); + task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); } else @@ -387,11 +395,24 @@ void DelayedJoinedBlocksWorkerTransform::work() if (!task) return; - Block block = task->delayed_blocks->next(); + Block block; + if (!left_delayed_stream_finished) + { + block = task->delayed_blocks->next(); + if (!block) + { + left_delayed_stream_finished = true; + block = nextNonJoinedBlock(); + } + } + else + { + block = nextNonJoinedBlock(); + } if (!block) { - task.reset(); + resetTask(); return; } @@ -400,6 +421,31 @@ void DelayedJoinedBlocksWorkerTransform::work() output_chunk.setColumns(block.getColumns(), rows); } +void DelayedJoinedBlocksWorkerTransform::resetTask() +{ + task.reset(); + left_delayed_stream_finished = false; + non_joined_delayed_stream = nullptr; +} + +Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() +{ + // Before read from non-joined stream, all blocks in left file reader must have been joined. + // For example, in HashJoin, it may return invalid mismatch rows from non-joined stream before + // the all blocks in left file reader have been finished, since the used flags are incomplete. + // To make only one processor could read from non-joined stream seems be a easy way. + if (!non_joined_delayed_stream && task && task->left_delayed_stream_finish_counter->isLast()) + { + non_joined_delayed_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); + } + + if (non_joined_delayed_stream) + { + return non_joined_delayed_stream->next(); + } + return {}; +} + DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, JoinPtr join_) : IProcessor(InputPorts{}, OutputPorts(num_streams, Block())) , join(std::move(join_)) @@ -433,6 +479,9 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (finished) { + // Since have memory limit, cannot handle all buckets parallelly by different + // DelayedJoinedBlocksWorkerTransform. So send the same task to all outputs. + // Wait for all DelayedJoinedBlocksWorkerTransform be idle before getting next bucket. for (auto & output : outputs) { if (output.isFinished()) @@ -448,10 +497,14 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (delayed_blocks) { + // This counter is used to ensure that only the last DelayedJoinedBlocksWorkerTransform + // could read right non-joined blocks from the join. + auto left_delayed_stream_finished_counter = std::make_shared(outputs.size()); for (auto & output : outputs) { Chunk chunk; - chunk.setChunkInfo(std::make_shared(delayed_blocks)); + auto task = std::make_shared(delayed_blocks, left_delayed_stream_finished_counter); + chunk.setChunkInfo(task); output.push(std::move(chunk)); } delayed_blocks = nullptr; diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index e7edff40c56..10b413ed4e5 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -116,9 +116,14 @@ class DelayedBlocksTask : public ChunkInfo public: explicit DelayedBlocksTask() : finished(true) {} - explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_) : delayed_blocks(std::move(delayed_blocks_)) {} + explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) + : delayed_blocks(std::move(delayed_blocks_)) + , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) + { + } IBlocksStreamPtr delayed_blocks = nullptr; + JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter = nullptr; bool finished = false; }; @@ -147,7 +152,11 @@ private: class DelayedJoinedBlocksWorkerTransform : public IProcessor { public: - explicit DelayedJoinedBlocksWorkerTransform(Block output_header); + explicit DelayedJoinedBlocksWorkerTransform( + Block left_header_, + Block output_header_, + size_t max_block_size_, + JoinPtr join_); String getName() const override { return "DelayedJoinedBlocksWorkerTransform"; } @@ -155,10 +164,19 @@ public: void work() override; private: + Block left_header; + Block output_header; + size_t max_block_size; + JoinPtr join; DelayedBlocksTaskPtr task; Chunk output_chunk; - bool finished = false; + /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left + bool left_delayed_stream_finished = false; + IBlocksStreamPtr non_joined_delayed_stream = nullptr; + + void resetTask(); + Block nextNonJoinedBlock(); }; } diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index dedf85e409c..ba98d725532 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -491,7 +491,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe if (delayed_root) { // Process delayed joined blocks when all JoiningTransform are finished. - auto delayed = std::make_shared(joined_header); + auto delayed = std::make_shared(left_header, joined_header, max_block_size, join); if (delayed->getInputs().size() != 1 || delayed->getOutputs().size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform should have one input and one output"); diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 index e9f32087439..ae43aa7195c 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.reference.j2 @@ -1,7 +1,6 @@ {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} === {{ join_algorithm }} === = full = -{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -17,7 +16,6 @@ 8 108 \N 9 109 \N 10 110 \N -{% endif -%} = left = 1 101 201 2 102 202 @@ -30,7 +28,6 @@ 9 109 \N 10 110 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} -4 0 196 -3 0 197 -2 0 198 @@ -41,7 +38,6 @@ 3 103 203 4 104 204 5 105 205 -{% endif -%} = inner = 1 101 201 2 102 202 @@ -49,7 +45,6 @@ 4 104 204 5 105 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -65,7 +60,6 @@ 8 8 0 9 9 0 10 10 0 -{% endif -%} = left = 1 1 1 2 2 2 @@ -78,7 +72,6 @@ 9 9 0 10 10 0 = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 0 0 -3 0 0 -2 @@ -89,7 +82,6 @@ 3 3 3 4 4 4 5 5 5 -{% endif -%} = inner = 1 1 1 2 2 2 @@ -98,7 +90,6 @@ 5 5 5 = join on = = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -114,7 +105,6 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -127,7 +117,6 @@ 9 109 0 \N 10 110 0 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -138,7 +127,6 @@ 3 103 3 203 4 104 4 204 5 105 5 205 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -146,7 +134,6 @@ 4 104 4 204 5 105 5 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -162,7 +149,6 @@ 8 108 0 \N 9 109 0 \N 10 110 0 \N -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -175,7 +161,6 @@ 9 109 0 \N 10 110 0 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 0 0 -4 196 0 0 -3 197 0 0 -2 198 @@ -186,7 +171,6 @@ 3 103 3 203 4 104 4 204 5 105 5 205 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -196,7 +180,6 @@ = agg = 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -205,13 +188,11 @@ 1 55 1055 0 0 -10 0 990 1 55 15 1055 1015 -{% endif -%} = types = 1 1 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -219,11 +200,9 @@ 1 1 1 -{% endif -%} {% if join_algorithm not in ['full_sorting_merge'] -%} === join use nulls === = full = -{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -239,7 +218,6 @@ 8 108 \N 9 109 \N 10 110 \N -{% endif -%} = left = 1 101 201 2 102 202 @@ -252,7 +230,6 @@ 9 109 \N 10 110 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} -4 \N 196 -3 \N 197 -2 \N 198 @@ -263,7 +240,6 @@ 3 103 203 4 104 204 5 105 205 -{% endif -%} = inner = 1 101 201 2 102 202 @@ -271,7 +247,6 @@ 4 104 204 5 105 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -287,7 +262,6 @@ \N \N -2 \N \N -1 \N \N 0 -{% endif -%} = left = 1 1 1 2 2 2 @@ -300,7 +274,6 @@ 9 9 \N 10 10 \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 2 2 2 3 3 3 @@ -311,7 +284,6 @@ \N \N -2 \N \N -1 \N \N 0 -{% endif -%} = inner = 1 1 1 2 2 2 @@ -320,7 +292,6 @@ 5 5 5 = join on = = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -336,7 +307,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -349,7 +319,6 @@ 9 109 \N \N 10 110 \N \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -360,7 +329,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -368,7 +336,6 @@ 4 104 4 204 5 105 5 205 = full = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -384,7 +351,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = left = 1 101 1 201 2 102 2 202 @@ -397,7 +363,6 @@ 9 109 \N \N 10 110 \N \N = right = -{% if join_algorithm not in ['grace_hash'] -%} 1 101 1 201 2 102 2 202 3 103 3 203 @@ -408,7 +373,6 @@ \N \N -2 198 \N \N -1 199 \N \N 0 200 -{% endif -%} = inner = 1 101 1 201 2 102 2 202 @@ -418,7 +382,6 @@ = agg = 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -427,13 +390,11 @@ 1 55 1055 1 55 15 1055 1015 \N \N -10 \N 990 -{% endif -%} = types = 1 1 1 1 -{% if join_algorithm not in ['grace_hash'] -%} 1 1 1 @@ -442,5 +403,4 @@ 1 1 {% endif -%} -{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 index f5321939f28..38f71f4c5ec 100644 --- a/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 +++ b/tests/queries/0_stateless/01721_join_implicit_cast_long.sql.j2 @@ -10,7 +10,6 @@ INSERT INTO t1 SELECT number as a, 100 + number as b FROM system.numbers LIMIT 1 INSERT INTO t2 SELECT number - 5 as a, 200 + number - 5 as b FROM system.numbers LIMIT 1, 10; {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['hash', 'partial_merge', 'auto', 'full_sorting_merge', 'grace_hash'] -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 index 98bfd9d9b2b..0af4158e971 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.reference.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.reference.j2 @@ -1,7 +1,7 @@ {% set table_size = 15 -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} -- {{ join_algorithm }} -- -{% for block_size in range(1, table_size + 1) -%} +{% for block_size in range(1, table_size + 1, 4) -%} ALL INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 @@ -50,7 +50,6 @@ ALL LEFT | bs = {{ block_size }} 14 14 val9 0 14 14 val9 0 ALL RIGHT | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -64,7 +63,6 @@ ALL RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ALL INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -85,7 +83,6 @@ ALL LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | bs = {{ block_size }} | copmosite key -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -99,7 +96,6 @@ ALL RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 -{% endif -%} ANY INNER USING | bs = {{ block_size }} 4 0 0 5 0 0 @@ -137,7 +133,6 @@ ANY LEFT | bs = {{ block_size }} 13 13 val13 0 14 14 val9 0 ANY RIGHT | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -150,7 +145,6 @@ ANY RIGHT | bs = {{ block_size }} 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ANY INNER | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | bs = {{ block_size }} | copmosite key @@ -170,7 +164,6 @@ ANY LEFT | bs = {{ block_size }} | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | bs = {{ block_size }} | copmosite key -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 1 1 1 1 val2 0 \N 0 1 1 1 1 val7 0 \N 0 1 1 2 1 val5 @@ -183,7 +176,6 @@ ANY RIGHT | bs = {{ block_size }} | copmosite key 0 \N 0 2 1 \N 1 val3 0 \N 0 2 2 \N 1 val9 2 2 2 2 2 2 0 val4 -{% endif -%} {% endfor -%} ALL INNER | join_use_nulls = 1 4 4 0 0 @@ -219,7 +211,6 @@ ALL LEFT | join_use_nulls = 1 14 14 val9 0 14 14 val9 0 ALL RIGHT | join_use_nulls = 1 -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -233,7 +224,6 @@ ALL RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ALL INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 2 2 2 2 2 2 0 0 @@ -254,7 +244,6 @@ ALL LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ALL RIGHT | join_use_nulls = 1 | copmosite key -{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 @@ -268,7 +257,6 @@ ALL RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 -{% endif -%} ANY INNER | join_use_nulls = 1 4 4 0 0 5 5 0 0 @@ -296,7 +284,6 @@ ANY LEFT | join_use_nulls = 1 13 13 val13 0 14 14 val9 0 ANY RIGHT | join_use_nulls = 1 -{% if join_algorithm != 'grace_hash' -%} 4 4 0 val10 5 5 0 val6 6 6 0 val8 @@ -309,7 +296,6 @@ ANY RIGHT | join_use_nulls = 1 13 13 0 val9 14 14 0 val3 14 14 0 val7 -{% endif -%} ANY INNER | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 0 0 ANY LEFT | join_use_nulls = 1 | copmosite key @@ -329,7 +315,6 @@ ANY LEFT | join_use_nulls = 1 | copmosite key 2 2 2 2 2 2 val12 0 2 2 2 2 2 2 val9 0 ANY RIGHT | join_use_nulls = 1 | copmosite key -{% if join_algorithm != 'grace_hash' -%} 2 2 2 2 2 2 0 val4 \N \N \N 1 1 1 \N val2 \N \N \N 1 1 1 \N val7 @@ -342,5 +327,4 @@ ANY RIGHT | join_use_nulls = 1 | copmosite key \N \N \N 2 1 2 \N val8 \N \N \N 2 1 \N \N val3 \N \N \N 2 2 \N \N val9 -{% endif -%} {% endfor -%} diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 43f7354017c..6b6aa53836e 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -28,9 +28,7 @@ INSERT INTO t2 'val' || toString(number) as s FROM numbers_mt({{ table_size - 3 }}); - {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['default', 'full_sorting_merge', 'grace_hash'] -%} @@ -40,7 +38,7 @@ SET max_bytes_in_join = '{% if join_algorithm == 'grace_hash' %}10K{% else %}0{% SELECT '-- {{ join_algorithm }} --'; SET join_algorithm = '{{ join_algorithm }}'; -{% for block_size in range(1, table_size + 1) -%} +{% for block_size in range(1, table_size + 1, 4) -%} {% for kind in ['ALL', 'ANY'] -%} SET max_block_size = {{ block_size }}; diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 index 2cc6c6e85d6..df968e86e8d 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.reference.j2 @@ -1,6 +1,6 @@ {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} --- {{ join_algorithm }} --- -{% for block_size in range(1, 11) -%} +{% for block_size in range(1, 11, 4) -%} t1 ALL INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -108,7 +108,6 @@ t1 ALL LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -161,7 +160,6 @@ t1 ALL RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val28 2 2 5 val28 3 3 4 val3 -{% endif -%} t1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 2 2 5 5 @@ -177,7 +175,6 @@ t1 ANY LEFT JOIN t2 | bs = {{ block_size }} 2 2 val27 5 3 3 val3 4 t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 2 2 5 val22 @@ -188,9 +185,7 @@ t1 ANY RIGHT JOIN t2 | bs = {{ block_size }} 2 2 5 val27 2 2 5 val28 3 3 4 val3 -{% endif -%} t1 ALL FULL JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 2 2 5 5 @@ -243,9 +238,7 @@ t1 ALL FULL JOIN t2 | bs = {{ block_size }} 2 2 5 5 2 2 5 5 3 3 4 4 -{% endif -%} t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 5 5 @@ -298,7 +291,6 @@ t1 ALL FULL JOIN USING t2 | bs = {{ block_size }} 2 5 5 2 5 5 3 4 4 -{% endif -%} t1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -315,7 +307,6 @@ t1 ALL LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -326,7 +317,6 @@ t1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 -{% endif -%} t1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -341,7 +331,6 @@ t1 ANY LEFT JOIN tn2 | bs = {{ block_size }} 2 \N val27 0 3 3 val3 4 t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 val22 0 \N 0 val23 0 \N 0 val24 @@ -352,9 +341,7 @@ t1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} 1 1 4 val11 1 1 4 val12 3 3 4 val3 -{% endif -%} t1 ALL FULL JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 0 \N 0 5 0 \N 0 5 0 \N 0 5 @@ -372,9 +359,8 @@ t1 ALL FULL JOIN tn2 | bs = {{ block_size }} 2 \N 5 0 2 \N 5 0 3 3 4 4 -{% endif -%} -t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} {% if join_algorithm != 'grace_hash' -%} +t1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} 1 4 5 1 4 5 2 5 0 @@ -409,7 +395,6 @@ tn1 ALL LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -420,7 +405,6 @@ tn1 ALL RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -{% endif -%} tn1 ANY INNER JOIN t2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -435,7 +419,6 @@ tn1 ANY LEFT JOIN t2 | bs = {{ block_size }} \N 0 val26 0 \N 0 val27 0 tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -446,9 +429,7 @@ tn1 ANY RIGHT JOIN t2 | bs = {{ block_size }} \N 2 0 val26 \N 2 0 val27 \N 2 0 val28 -{% endif -%} tn1 ALL FULL JOIN t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -466,9 +447,7 @@ tn1 ALL FULL JOIN t2 | bs = {{ block_size }} \N 2 0 5 \N 2 0 5 \N 2 0 5 -{% endif -%} tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 4 5 1 4 5 2 0 5 @@ -486,7 +465,6 @@ tn1 ALL FULL JOIN USING t2 | bs = {{ block_size }} \N 5 0 \N 5 0 \N 5 0 -{% endif -%} tn1 ALL INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 1 1 4 5 @@ -503,7 +481,6 @@ tn1 ALL LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -514,7 +491,6 @@ tn1 ALL RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 -{% endif -%} tn1 ANY INNER JOIN tn2 | bs = {{ block_size }} 1 1 4 5 3 3 4 4 @@ -529,7 +505,6 @@ tn1 ANY LEFT JOIN tn2 | bs = {{ block_size }} \N \N val26 0 \N \N val27 0 tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 val11 1 1 4 val12 3 3 4 val3 @@ -540,9 +515,7 @@ tn1 ANY RIGHT JOIN tn2 | bs = {{ block_size }} \N \N 0 val26 \N \N 0 val27 \N \N 0 val28 -{% endif -%} tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} -{% if join_algorithm != 'grace_hash' -%} 1 1 4 5 1 1 4 5 3 3 4 4 @@ -560,9 +533,8 @@ tn1 ALL FULL JOIN tn2 | bs = {{ block_size }} \N \N 5 0 \N \N 5 0 \N \N 5 0 -{% endif -%} -tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} {% if join_algorithm != 'grace_hash' -%} +tn1 ALL FULL JOIN USING tn2 | bs = {{ block_size }} 1 4 5 1 4 5 3 4 4 diff --git a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 index 613da65421e..f8eb4b1a53e 100644 --- a/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 +++ b/tests/queries/0_stateless/02274_full_sort_join_nodistinct.sql.j2 @@ -16,7 +16,6 @@ INSERT INTO t2 VALUES (1, 'val11'), (1, 'val12'), (2, 'val22'), (2, 'val23'), (2 INSERT INTO tn2 VALUES (1, 'val11'), (1, 'val12'), (NULL, 'val22'), (NULL, 'val23'), (NULL, 'val24'), (NULL, 'val25'), (NULL, 'val26'), (NULL, 'val27'), (NULL, 'val28'), (3, 'val3'); {% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } {% endif %} {% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} @@ -27,7 +26,7 @@ SET join_algorithm = '{{ join_algorithm }}'; SELECT '--- {{ join_algorithm }} ---'; -{% for block_size in range(1, 11) -%} +{% for block_size in range(1, 11, 4) -%} SET max_block_size = {{ block_size }}; {% for t1, t2 in [('t1', 't2'), ('t1', 'tn2'), ('tn1', 't2'), ('tn1', 'tn2')] -%} @@ -47,9 +46,10 @@ SELECT t1.key, t2.key, length(t1.s), t2.s FROM {{ t1 }} AS t1 {{ kind }} RIGHT J SELECT '{{ t1 }} ALL FULL JOIN {{ t2 }} | bs = {{ block_size }}'; SELECT t1.key, t2.key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 {{ kind }} FULL JOIN {{ t2 }} AS t2 ON t1.key == t2.key ORDER BY t1.key, t2.key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} +{% if join_algorithm == 'full_sorting_merge' or t2 != 'tn2' -%} SELECT '{{ t1 }} ALL FULL JOIN USING {{ t2 }} | bs = {{ block_size }}'; SELECT key, length(t1.s), length(t2.s) FROM {{ t1 }} AS t1 ALL FULL JOIN {{ t2 }} AS t2 USING (key) ORDER BY key, length(t1.s), length(t2.s); {{ is_implemented(join_algorithm) }} - +{% endif -%} {% endfor -%} {% endfor -%} SET max_bytes_in_join = 0; diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.reference b/tests/queries/0_stateless/02275_full_sort_join_long.reference index 9ec06aea3e6..73482358d12 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.reference +++ b/tests/queries/0_stateless/02275_full_sort_join_long.reference @@ -41,16 +41,34 @@ ALL INNER ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 ALL INNER 500353531835 500353531835 1000342 1000342 1000342 ALL LEFT 50195752660639 500353531835 10369589 10369589 1000342 ALL RIGHT -skipped +500353531835 684008812186 1367170 1000342 1367170 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 +ANY INNER +199622811843 199622811843 399458 399458 399458 +ANY LEFT +50010619420459 315220291655 10000000 10000000 630753 +ANY RIGHT +316611844056 500267124407 1000000 633172 1000000 diff --git a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 index 7276e77dc16..621352f9c25 100644 --- a/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 +++ b/tests/queries/0_stateless/02275_full_sort_join_long.sql.j2 @@ -22,11 +22,6 @@ INSERT INTO t2 FROM numbers_mt({{ rtable_size }}) ; -{% macro is_implemented(join_algorithm) -%} -{% if join_algorithm == 'grace_hash' %} -- { serverError NOT_IMPLEMENTED } -SELECT 'skipped'; -{% endif -%} -{% endmacro -%} {% for join_algorithm in ['full_sorting_merge', 'grace_hash'] -%} @@ -40,7 +35,6 @@ SET join_algorithm = '{{ join_algorithm }}'; SET max_block_size = {{ block_size }}; -{% if not (kind == 'ANY' and join_algorithm == 'grace_hash') -%} SELECT '{{ kind }} INNER'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 @@ -58,9 +52,8 @@ SELECT '{{ kind }} RIGHT'; SELECT sum(t1.key), sum(t2.key), count(), countIf(t1.key != 0), countIf(t2.key != 0) FROM t1 {{ kind }} RIGHT JOIN t2 ON t1.key == t2.key -; {{ is_implemented(join_algorithm) }} +; -{% endif -%} {% endfor -%} {% endfor -%} From 91dc6a35e17417a44de46d76c0f0214911615244 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 19 Jul 2023 09:18:16 +0800 Subject: [PATCH 0904/2047] update --- src/Interpreters/GraceHashJoin.cpp | 1 - .../Transforms/JoiningTransform.cpp | 19 +++++-------------- src/Processors/Transforms/JoiningTransform.h | 14 ++++---------- src/QueryPipeline/QueryPipelineBuilder.cpp | 5 ++++- 4 files changed, 13 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index f94453293f6..5d72cf20740 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -301,7 +301,6 @@ void GraceHashJoin::initBuckets() bool GraceHashJoin::isSupported(const std::shared_ptr & table_join) { - bool is_asof = (table_join->strictness() == JoinStrictness::Asof); auto kind = table_join->kind(); return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct(); diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index f1ceefbf229..5480fea27a4 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -311,15 +311,10 @@ void FillingRightJoinSideTransform::work() DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform( - Block left_header_, Block output_header_, - size_t max_block_size_, - JoinPtr join_) + NonJoinedStreamBuilder non_joined_stream_builder_) : IProcessor(InputPorts{Block()}, OutputPorts{output_header_}) - , left_header(left_header_) - , output_header(output_header_) - , max_block_size(max_block_size_) - , join(join_) + , non_joined_stream_builder(std::move(non_joined_stream_builder_)) { } @@ -396,15 +391,12 @@ void DelayedJoinedBlocksWorkerTransform::work() return; Block block; - if (!left_delayed_stream_finished) + /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left + if (!task->delayed_blocks->isFinished()) { block = task->delayed_blocks->next(); - if (!block) - { - left_delayed_stream_finished = true; block = nextNonJoinedBlock(); - } } else { @@ -424,7 +416,6 @@ void DelayedJoinedBlocksWorkerTransform::work() void DelayedJoinedBlocksWorkerTransform::resetTask() { task.reset(); - left_delayed_stream_finished = false; non_joined_delayed_stream = nullptr; } @@ -436,7 +427,7 @@ Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock() // To make only one processor could read from non-joined stream seems be a easy way. if (!non_joined_delayed_stream && task && task->left_delayed_stream_finish_counter->isLast()) { - non_joined_delayed_stream = join->getNonJoinedBlocks(left_header, output_header, max_block_size); + non_joined_delayed_stream = non_joined_stream_builder(); } if (non_joined_delayed_stream) diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 10b413ed4e5..5e7403dbbdb 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -152,11 +152,10 @@ private: class DelayedJoinedBlocksWorkerTransform : public IProcessor { public: + using NonJoinedStreamBuilder = std::function; explicit DelayedJoinedBlocksWorkerTransform( - Block left_header_, Block output_header_, - size_t max_block_size_, - JoinPtr join_); + NonJoinedStreamBuilder non_joined_stream_builder_); String getName() const override { return "DelayedJoinedBlocksWorkerTransform"; } @@ -164,15 +163,10 @@ public: void work() override; private: - Block left_header; - Block output_header; - size_t max_block_size; - JoinPtr join; DelayedBlocksTaskPtr task; Chunk output_chunk; - - /// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left - bool left_delayed_stream_finished = false; + /// For building a block stream to access the non-joined rows. + NonJoinedStreamBuilder non_joined_stream_builder; IBlocksStreamPtr non_joined_delayed_stream = nullptr; void resetTask(); diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index ba98d725532..553b18dd57b 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -491,7 +491,10 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe if (delayed_root) { // Process delayed joined blocks when all JoiningTransform are finished. - auto delayed = std::make_shared(left_header, joined_header, max_block_size, join); + auto delayed = std::make_shared( + joined_header, + [left_header, joined_header, max_block_size, join]() + { return join->getNonJoinedBlocks(left_header, joined_header, max_block_size); }); if (delayed->getInputs().size() != 1 || delayed->getOutputs().size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform should have one input and one output"); From 7a0de384d498497fd026283a8232fcb8ed8ea5e6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 16:46:51 +0000 Subject: [PATCH 0905/2047] Cosmetics --- src/Parsers/IAST.h | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 7a8ab36518d..f6b7f91fec8 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -196,20 +196,23 @@ public: bool always_quote_identifiers = false; IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). - - // Newline or whitespace. - char nl_or_ws; + char nl_or_ws; /// Newline or whitespace. FormatSettings(WriteBuffer & ostr_, bool one_line_, bool show_secrets_ = true) - : ostr(ostr_), one_line(one_line_), show_secrets(show_secrets_) + : ostr(ostr_) + , one_line(one_line_) + , show_secrets(show_secrets_) { nl_or_ws = one_line ? ' ' : '\n'; } FormatSettings(WriteBuffer & ostr_, const FormatSettings & other) - : ostr(ostr_), hilite(other.hilite), one_line(other.one_line), - always_quote_identifiers(other.always_quote_identifiers), identifier_quoting_style(other.identifier_quoting_style), - show_secrets(other.show_secrets) + : ostr(ostr_) + , hilite(other.hilite) + , one_line(other.one_line) + , always_quote_identifiers(other.always_quote_identifiers) + , identifier_quoting_style(other.identifier_quoting_style) + , show_secrets(other.show_secrets) { nl_or_ws = one_line ? ' ' : '\n'; } From 35a4fabc2d66ea28b3de3d77df4cfea4b91df870 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 17:03:04 +0000 Subject: [PATCH 0906/2047] Make IAST::FormatSettings more regular --- src/Parsers/IAST.cpp | 4 +++- src/Parsers/IAST.h | 3 +-- src/Parsers/formatAST.cpp | 4 ++-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 0138372ce89..bf4d6fc9dec 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -170,7 +170,9 @@ size_t IAST::checkDepthImpl(size_t max_depth) const String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const { WriteBufferFromOwnString buf; - format({buf, one_line, show_secrets}); + FormatSettings settings(buf, one_line); + settings.show_secrets = show_secrets; + format(settings); return wipeSensitiveDataAndCutToLength(buf.str(), max_length); } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index f6b7f91fec8..13b2e5d9867 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -198,10 +198,9 @@ public: bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). char nl_or_ws; /// Newline or whitespace. - FormatSettings(WriteBuffer & ostr_, bool one_line_, bool show_secrets_ = true) + FormatSettings(WriteBuffer & ostr_, bool one_line_) : ostr(ostr_) , one_line(one_line_) - , show_secrets(show_secrets_) { nl_or_ws = one_line ? ' ' : '\n'; } diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index aa1afe17c75..bc7faf4bd1d 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -6,9 +6,9 @@ namespace DB void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool show_secrets) { - IAST::FormatSettings settings(buf, one_line, show_secrets); + IAST::FormatSettings settings(buf, one_line); settings.hilite = hilite; - + settings.show_secrets = show_secrets; ast.format(settings); } From e5ec6a1523529db3d1b9d7f137997076c8c2adde Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 17:21:03 +0000 Subject: [PATCH 0907/2047] Make IAST::FormatSettings more regular, pt. II --- src/Parsers/IAST.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 13b2e5d9867..8e2971d0355 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -191,8 +191,8 @@ public: struct FormatSettings { WriteBuffer & ostr; - bool hilite = false; bool one_line; + bool hilite = false; bool always_quote_identifiers = false; IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). @@ -207,13 +207,13 @@ public: FormatSettings(WriteBuffer & ostr_, const FormatSettings & other) : ostr(ostr_) - , hilite(other.hilite) , one_line(other.one_line) + , hilite(other.hilite) , always_quote_identifiers(other.always_quote_identifiers) , identifier_quoting_style(other.identifier_quoting_style) , show_secrets(other.show_secrets) + , nl_or_ws(other.nl_or_ws) { - nl_or_ws = one_line ? ' ' : '\n'; } void writeIdentifier(const String & name) const; From 25ddcc256b04de71e84935cb60a53190c114a494 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 17:31:53 +0000 Subject: [PATCH 0908/2047] Make IAST::FormatSettings more regular, pt. III --- src/Parsers/IAST.h | 22 ++++++++++++++----- src/Parsers/formatAST.cpp | 3 +-- src/Parsers/getInsertQuery.cpp | 4 +--- src/Parsers/tests/gtest_format_hiliting.cpp | 3 +-- src/Processors/QueryPlan/ReadFromRemote.cpp | 4 +--- .../MeiliSearch/StorageMeiliSearch.cpp | 7 +++--- src/Storages/StorageDistributed.cpp | 6 ++--- src/Storages/StorageReplicatedMergeTree.cpp | 3 +-- .../transformQueryForExternalDatabase.cpp | 7 +++--- 9 files changed, 31 insertions(+), 28 deletions(-) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 8e2971d0355..d217876459f 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -192,17 +192,27 @@ public: { WriteBuffer & ostr; bool one_line; - bool hilite = false; - bool always_quote_identifiers = false; - IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; - bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). + bool hilite; + bool always_quote_identifiers; + IdentifierQuotingStyle identifier_quoting_style; + bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys). char nl_or_ws; /// Newline or whitespace. - FormatSettings(WriteBuffer & ostr_, bool one_line_) + explicit FormatSettings( + WriteBuffer & ostr_, + bool one_line_, + bool hilite_ = false, + bool always_quote_identifiers_ = false, + IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks, + bool show_secrets_ = true) : ostr(ostr_) , one_line(one_line_) + , hilite(hilite_) + , always_quote_identifiers(always_quote_identifiers_) + , identifier_quoting_style(identifier_quoting_style_) + , show_secrets(show_secrets_) + , nl_or_ws(one_line ? ' ' : '\n') { - nl_or_ws = one_line ? ' ' : '\n'; } FormatSettings(WriteBuffer & ostr_, const FormatSettings & other) diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index bc7faf4bd1d..ae2c4a6fcad 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -6,8 +6,7 @@ namespace DB void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool show_secrets) { - IAST::FormatSettings settings(buf, one_line); - settings.hilite = hilite; + IAST::FormatSettings settings(buf, one_line, hilite); settings.show_secrets = show_secrets; ast.format(settings); } diff --git a/src/Parsers/getInsertQuery.cpp b/src/Parsers/getInsertQuery.cpp index 6f52056dfe2..9d111b147bd 100644 --- a/src/Parsers/getInsertQuery.cpp +++ b/src/Parsers/getInsertQuery.cpp @@ -19,9 +19,7 @@ std::string getInsertQuery(const std::string & db_name, const std::string & tabl query.columns->children.emplace_back(std::make_shared(column.name)); WriteBufferFromOwnString buf; - IAST::FormatSettings settings(buf, true); - settings.always_quote_identifiers = true; - settings.identifier_quoting_style = quoting; + IAST::FormatSettings settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true, /*identifier_quoting_style*/ quoting); query.IAST::format(settings); return buf.str(); } diff --git a/src/Parsers/tests/gtest_format_hiliting.cpp b/src/Parsers/tests/gtest_format_hiliting.cpp index d0ce8f2c897..a4c3ed86182 100644 --- a/src/Parsers/tests/gtest_format_hiliting.cpp +++ b/src/Parsers/tests/gtest_format_hiliting.cpp @@ -51,8 +51,7 @@ void compare(const String & expected, const String & query) ASTPtr ast = parseQuery(parser, query, 0, 0); WriteBufferFromOwnString write_buffer; - IAST::FormatSettings settings(write_buffer, true); - settings.hilite = true; + IAST::FormatSettings settings(write_buffer, true, true); ast->format(settings); ASSERT_PRED2(HiliteComparator::are_equal_with_hilites_removed, expected, write_buffer.str()); diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index ed740e3e242..5cc13f45df4 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -86,9 +86,7 @@ static String formattedAST(const ASTPtr & ast) return {}; WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.hilite = false; - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true); ast->format(ast_format_settings); return buf.str(); } diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp index 5d77fc080a4..aa8b437263a 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp @@ -62,9 +62,10 @@ ColumnsDescription StorageMeiliSearch::getTableStructureFromData(const MeiliSear String convertASTtoStr(ASTPtr ptr) { WriteBufferFromOwnString out; - IAST::FormatSettings settings(out, true); - settings.identifier_quoting_style = IdentifierQuotingStyle::BackticksMySQL; - settings.always_quote_identifiers = IdentifierQuotingStyle::BackticksMySQL != IdentifierQuotingStyle::None; + IAST::FormatSettings settings( + out, /*one_line*/ true, /*hilite*/ false, + /*always_quote_identifiers*/ IdentifierQuotingStyle::BackticksMySQL != IdentifierQuotingStyle::None, + /*identifier_quoting_style*/ IdentifierQuotingStyle::BackticksMySQL); ptr->format(settings); return out.str(); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c46192ab43b..e02d7f32b98 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -906,8 +906,7 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu String new_query_str; { WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers_=*/ true); new_query->IAST::format(ast_format_settings); new_query_str = buf.str(); } @@ -968,8 +967,7 @@ std::optional StorageDistributed::distributedWriteFromClusterStor String new_query_str; { WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true); new_query->IAST::format(ast_format_settings); new_query_str = buf.str(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5f20c497cb8..52f478d7729 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5074,8 +5074,7 @@ std::optional StorageReplicatedMergeTree::distributedWriteFromClu String query_str; { WriteBufferFromOwnString buf; - IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true); - ast_format_settings.always_quote_identifiers = true; + IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true); query.IAST::format(ast_format_settings); query_str = buf.str(); } diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 548b55749d7..375510e62bf 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -334,9 +334,10 @@ String transformQueryForExternalDatabaseImpl( dropAliases(select_ptr); WriteBufferFromOwnString out; - IAST::FormatSettings settings(out, true); - settings.identifier_quoting_style = identifier_quoting_style; - settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None; + IAST::FormatSettings settings( + out, /*one_line*/ true, /*hilite*/ false, + /*always_quote_identifiers*/ identifier_quoting_style != IdentifierQuotingStyle::None, + /*identifier_quoting_style*/ identifier_quoting_style); select->format(settings); From bd761c365a95e97f1a92638f145353d54a4f2db5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jul 2023 18:02:09 +0000 Subject: [PATCH 0909/2047] Make serializeAST() more regular --- src/Disks/getOrCreateDiskFromAST.cpp | 2 +- src/Interpreters/Cache/QueryCache.h | 2 +- src/Interpreters/ThreadStatusExt.cpp | 2 +- src/Parsers/formatAST.cpp | 4 ++-- src/Parsers/formatAST.h | 9 +++++---- src/Parsers/tests/gtest_Parser.cpp | 10 ++++++++-- src/Parsers/tests/gtest_dictionary_parser.cpp | 10 +++++----- .../Transforms/CheckConstraintsTransform.cpp | 4 ++-- src/Storages/ConstraintsDescription.cpp | 2 +- src/Storages/IndicesDescription.cpp | 2 +- src/Storages/ProjectionsDescription.cpp | 2 +- 11 files changed, 28 insertions(+), 21 deletions(-) diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index 81d5b7372f3..a9a0e972bd1 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -32,7 +32,7 @@ namespace /// We need a unique name for a created custom disk, but it needs to be the same /// after table is reattached or server is restarted, so take a hash of the disk /// configuration serialized ast as a disk name suffix. - auto disk_setting_string = serializeAST(function, true); + auto disk_setting_string = serializeAST(function); disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index a67adcc86c9..c24b09c8e46 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -60,7 +60,7 @@ public: /// The SELECT query as plain string, displayed in SYSTEM.QUERY_CACHE. Stored explicitly, i.e. not constructed from the AST, for the /// sole reason that QueryCache-related SETTINGS are pruned from the AST (see removeQueryCacheSettings()) which will look ugly in - /// the SYSTEM.QUERY_CACHE. + /// SYSTEM.QUERY_CACHE. const String query_string; /// Ctor to construct a Key for writing into query cache. diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 5acfe500b1d..398bea26b87 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -518,7 +518,7 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String static String getCleanQueryAst(const ASTPtr q, ContextPtr context) { - String res = serializeAST(*q, true); + String res = serializeAST(*q); if (auto * masker = SensitiveDataMasker::getInstance()) masker->wipeSensitiveData(res); diff --git a/src/Parsers/formatAST.cpp b/src/Parsers/formatAST.cpp index ae2c4a6fcad..9315279eae6 100644 --- a/src/Parsers/formatAST.cpp +++ b/src/Parsers/formatAST.cpp @@ -11,10 +11,10 @@ void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, ast.format(settings); } -String serializeAST(const IAST & ast, bool one_line) +String serializeAST(const IAST & ast) { WriteBufferFromOwnString buf; - formatAST(ast, buf, false, one_line); + formatAST(ast, buf, false, true); return buf.str(); } diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index ebd284fc18a..dd72a59b4a2 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -8,12 +8,13 @@ namespace DB class WriteBuffer; -/** Takes a syntax tree and turns it back into text. - * In case of INSERT query, the data will be missing. - */ +/// Takes a syntax tree and turns it into text. +/// Intended for pretty-printing (multi-line + hiliting). +/// In case of INSERT query, the data will be missing. void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false, bool show_secrets = true); -String serializeAST(const IAST & ast, bool one_line = true); +/// Like formatAST() but intended for serialization w/o pretty-printing (single-line, no hiliting). +String serializeAST(const IAST & ast); inline WriteBuffer & operator<<(WriteBuffer & buf, const IAST & ast) { diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 2795de64b1d..a53de155355 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -64,7 +64,10 @@ TEST_P(ParserTest, parseQuery) if (std::string("CREATE USER or ALTER USER query") != parser->getName() && std::string("ATTACH access entity query") != parser->getName()) { - EXPECT_EQ(expected_ast, serializeAST(*ast->clone(), false)); + WriteBufferFromOwnString buf; + formatAST(*ast->clone(), buf, false, false); + String formatted_ast = buf.str(); + EXPECT_EQ(expected_ast, formatted_ast); } else { @@ -75,7 +78,10 @@ TEST_P(ParserTest, parseQuery) } else { - EXPECT_TRUE(std::regex_match(serializeAST(*ast->clone(), false), std::regex(expected_ast))); + WriteBufferFromOwnString buf; + formatAST(*ast->clone(), buf, false, false); + String formatted_ast = buf.str(); + EXPECT_TRUE(std::regex_match(formatted_ast, std::regex(expected_ast))); } } } diff --git a/src/Parsers/tests/gtest_dictionary_parser.cpp b/src/Parsers/tests/gtest_dictionary_parser.cpp index 22484727ea2..c0a975f7a38 100644 --- a/src/Parsers/tests/gtest_dictionary_parser.cpp +++ b/src/Parsers/tests/gtest_dictionary_parser.cpp @@ -155,7 +155,7 @@ TEST(ParserDictionaryDDL, AttributesWithMultipleProperties) EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); - EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression, true), "(rand() % 100) * 77"); + EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression), "(rand() % 100) * 77"); EXPECT_EQ(attributes_children[0]->as()->hierarchical, false); EXPECT_EQ(attributes_children[1]->as()->hierarchical, true); @@ -201,7 +201,7 @@ TEST(ParserDictionaryDDL, CustomAttributePropertiesOrder) EXPECT_EQ(attributes_children[0]->as()->expression, nullptr); EXPECT_EQ(attributes_children[1]->as()->expression, nullptr); - EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression, true), "(rand() % 100) * 77"); + EXPECT_EQ(serializeAST(*attributes_children[2]->as()->expression), "(rand() % 100) * 77"); EXPECT_EQ(attributes_children[0]->as()->hierarchical, false); EXPECT_EQ(attributes_children[1]->as()->hierarchical, true); @@ -288,7 +288,7 @@ TEST(ParserDictionaryDDL, Formatting) ParserCreateDictionaryQuery parser; ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); ASTCreateQuery * create = ast->as(); - auto str = serializeAST(*create, true); + auto str = serializeAST(*create); EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '')) LIFETIME(MIN 1 MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column MAX third_column)"); } @@ -303,7 +303,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery) EXPECT_TRUE(drop1->is_dictionary); EXPECT_EQ(drop1->getDatabase(), "test"); EXPECT_EQ(drop1->getTable(), "dict1"); - auto str1 = serializeAST(*drop1, true); + auto str1 = serializeAST(*drop1); EXPECT_EQ(input1, str1); String input2 = "DROP DICTIONARY IF EXISTS dict2"; @@ -314,7 +314,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery) EXPECT_TRUE(drop2->is_dictionary); EXPECT_EQ(drop2->getDatabase(), ""); EXPECT_EQ(drop2->getTable(), "dict2"); - auto str2 = serializeAST(*drop2, true); + auto str2 = serializeAST(*drop2); EXPECT_EQ(input2, str2); } diff --git a/src/Processors/Transforms/CheckConstraintsTransform.cpp b/src/Processors/Transforms/CheckConstraintsTransform.cpp index 88f02a3926f..3a6595ea4fb 100644 --- a/src/Processors/Transforms/CheckConstraintsTransform.cpp +++ b/src/Processors/Transforms/CheckConstraintsTransform.cpp @@ -73,7 +73,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk) "Constraint expression returns nullable column that contains null value", backQuote(constraint_ptr->name), table_id.getNameForLogs(), - serializeAST(*(constraint_ptr->expr), true)); + serializeAST(*(constraint_ptr->expr))); result_column = nested_column; } @@ -116,7 +116,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk) backQuote(constraint_ptr->name), table_id.getNameForLogs(), rows_written + row_idx + 1, - serializeAST(*(constraint_ptr->expr), true), + serializeAST(*(constraint_ptr->expr)), column_values_msg); } } diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index db37ac7c4c3..249ed8be428 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -35,7 +35,7 @@ String ConstraintsDescription::toString() const for (const auto & constraint : constraints) list.children.push_back(constraint); - return serializeAST(list, true); + return serializeAST(list); } ConstraintsDescription ConstraintsDescription::parse(const String & str) diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index c7aeaf8e4ef..06518a52c61 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -151,7 +151,7 @@ String IndicesDescription::toString() const for (const auto & index : *this) list.children.push_back(index.definition_ast); - return serializeAST(list, true); + return serializeAST(list); } diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 73fb279d51c..aecf0ac6d00 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -324,7 +324,7 @@ String ProjectionsDescription::toString() const for (const auto & projection : projections) list.children.push_back(projection.definition_ast); - return serializeAST(list, true); + return serializeAST(list); } ProjectionsDescription ProjectionsDescription::parse(const String & str, const ColumnsDescription & columns, ContextPtr query_context) From a24bf14450bdb1dad881330ca168566bf7e1f82f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 20 Jul 2023 10:44:44 +0000 Subject: [PATCH 0910/2047] Use correct ZXID --- src/Coordination/KeeperStateMachine.cpp | 2 +- utils/keeper-data-dumper/main.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5c84f23fc60..a89b608aa69 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -390,7 +390,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) /// maybe some logs were preprocessed with log idx larger than the snapshot idx /// we have to apply them to the new storage - storage->applyUncommittedState(*snapshot_deserialization_result.storage, s.get_last_log_idx()); + storage->applyUncommittedState(*snapshot_deserialization_result.storage, snapshot_deserialization_result.storage->getZXID()); storage = std::move(snapshot_deserialization_result.storage); latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta; cluster_config = snapshot_deserialization_result.cluster_config; diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 51a09b676dc..39d9200f913 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -65,7 +65,7 @@ int main(int argc, char *argv[]) CoordinationSettingsPtr settings = std::make_shared(); KeeperContextPtr keeper_context = std::make_shared(true); keeper_context->setLogDisk(std::make_shared("LogDisk", argv[2])); - keeper_context->setSnapshotDisk(std::make_shared("LogDisk", argv[1])); + keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", argv[1])); auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); From 27921a5d8f4218a92dafb6fdc145bf3891710e3a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 20 Jul 2023 10:48:39 +0000 Subject: [PATCH 0911/2047] Docs: Add another reason for integer promotion rules in ClickHouse --- docs/en/sql-reference/functions/arithmetic-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 64fae0e82f0..054c59d5778 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -6,7 +6,7 @@ sidebar_label: Arithmetic # Arithmetic Functions -The result type of all arithmetic functions is the smallest type which can represent all possible results. Size promotion happens for integers up to 32 bit, e.g. `UInt8 + UInt16 = UInt32`. If one of the inters has 64 or more bits, the result is of the same type as the bigger of the input integers, e.g. `UInt16 + UInt128 = UInt128`. While this introduces a risk of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of 64 bit. +The result type of all arithmetic functions is the smallest type which can represent all possible results. Size promotion happens for integers up to 32 bit, e.g. `UInt8 + UInt16 = UInt32`. If one of the inters has 64 or more bits, the result is of the same type as the bigger of the input integers, e.g. `UInt16 + UInt128 = UInt128`. While this introduces a risk of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of 64 bit. Also, this behavior guarantees compatibility with many other databases which provide 64 bit integers (BIGINT) as the biggest integer type. The result of addition or multiplication of two integers is unsigned unless one of the integers is signed. From e74acda53ec3a7f8a536eb56e4a939935d10f8e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 20 Jul 2023 12:54:42 +0200 Subject: [PATCH 0912/2047] PRQL integration (#50686) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Added prql-lib * Add PRQL parser * Extend stateless tests * Add unit tests for `ParserPRQL` --------- Co-authored-by: Ubuntu Co-authored-by: Ubuntu Co-authored-by: Александр Нам <47687537+seshWCS@users.noreply.github.com> --- rust/CMakeLists.txt | 1 + rust/prql/CMakeLists.txt | 3 + rust/prql/Cargo.lock | 569 ++++++++++++++++++ rust/prql/Cargo.toml | 20 + rust/prql/include/prql.h | 18 + rust/prql/src/lib.rs | 56 ++ src/Client/ClientBase.cpp | 4 + src/Common/config.h.in | 1 + src/Core/SettingsEnums.cpp | 4 +- src/Core/SettingsEnums.h | 1 + src/Interpreters/executeQuery.cpp | 7 +- src/Parsers/CMakeLists.txt | 4 + src/Parsers/PRQL/ParserPRQLQuery.cpp | 86 +++ src/Parsers/PRQL/ParserPRQLQuery.h | 27 + src/Parsers/tests/gtest_Parser.cpp | 20 + src/configure_config.cmake | 3 + .../queries/0_stateless/02766_prql.reference | 19 + tests/queries/0_stateless/02766_prql.sh | 58 ++ 18 files changed, 899 insertions(+), 2 deletions(-) create mode 100644 rust/prql/CMakeLists.txt create mode 100644 rust/prql/Cargo.lock create mode 100644 rust/prql/Cargo.toml create mode 100644 rust/prql/include/prql.h create mode 100644 rust/prql/src/lib.rs create mode 100644 src/Parsers/PRQL/ParserPRQLQuery.cpp create mode 100644 src/Parsers/PRQL/ParserPRQLQuery.h create mode 100644 tests/queries/0_stateless/02766_prql.reference create mode 100755 tests/queries/0_stateless/02766_prql.sh diff --git a/rust/CMakeLists.txt b/rust/CMakeLists.txt index 6700ead9786..41451fe0a1e 100644 --- a/rust/CMakeLists.txt +++ b/rust/CMakeLists.txt @@ -88,3 +88,4 @@ endfunction() add_rust_subdirectory (BLAKE3) add_rust_subdirectory (skim) +add_rust_subdirectory (prql) diff --git a/rust/prql/CMakeLists.txt b/rust/prql/CMakeLists.txt new file mode 100644 index 00000000000..65109d19a81 --- /dev/null +++ b/rust/prql/CMakeLists.txt @@ -0,0 +1,3 @@ +clickhouse_import_crate(MANIFEST_PATH Cargo.toml) +target_include_directories(_ch_rust_prql INTERFACE include) +add_library(ch_rust::prql ALIAS _ch_rust_prql) diff --git a/rust/prql/Cargo.lock b/rust/prql/Cargo.lock new file mode 100644 index 00000000000..da94e4ca852 --- /dev/null +++ b/rust/prql/Cargo.lock @@ -0,0 +1,569 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "_ch_rust_prql" +version = "0.1.0" +dependencies = [ + "prql-compiler", + "serde_json", +] + +[[package]] +name = "addr2line" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4fa78e18c64fce05e902adecd7a5eed15a5e0a3439f7b0e169f0252214865e3" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "ahash" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" +dependencies = [ + "getrandom", + "once_cell", + "version_check", +] + +[[package]] +name = "aho-corasick" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43f6cb1bf222025340178f382c426f13757b2960e89779dfcb319c32542a5a41" +dependencies = [ + "memchr", +] + +[[package]] +name = "anyhow" +version = "1.0.71" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c7d0618f0e0b7e8ff11427422b64564d5fb0be1940354bfe2e0529b18a9d9b8" +dependencies = [ + "backtrace", +] + +[[package]] +name = "ariadne" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "367fd0ad87307588d087544707bc5fbf4805ded96c7db922b70d368fa1cb5702" +dependencies = [ + "unicode-width", + "yansi", +] + +[[package]] +name = "backtrace" +version = "0.3.68" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4319208da049c43661739c5fade2ba182f09d1dc2299b32298d3a31692b17e12" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", +] + +[[package]] +name = "cc" +version = "1.0.79" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "50d30906286121d95be3d479533b458f87493b30a4b5f79a607db8f5d11aa91f" + +[[package]] +name = "cfg-if" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" + +[[package]] +name = "chumsky" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23170228b96236b5a7299057ac284a321457700bc8c41a4476052f0f4ba5349d" +dependencies = [ + "hashbrown 0.12.3", + "stacker", +] + +[[package]] +name = "csv" +version = "1.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "626ae34994d3d8d668f4269922248239db4ae42d538b14c398b74a52208e8086" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "csv-core" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b2466559f260f48ad25fe6317b3c8dac77b5bdb5763ac7d9d6103530663bc90" +dependencies = [ + "memchr", +] + +[[package]] +name = "either" +version = "1.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" + +[[package]] +name = "enum-as-inner" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "equivalent" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "88bffebc5d80432c9b140ee17875ff173a8ab62faad5b257da912bd2f6c1c0a1" + +[[package]] +name = "getrandom" +version = "0.2.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" +dependencies = [ + "cfg-if", + "libc", + "wasi", +] + +[[package]] +name = "gimli" +version = "0.27.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c80984affa11d98d1b88b66ac8853f143217b399d3c74116778ff8fdb4ed2e" + +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" +dependencies = [ + "ahash", +] + +[[package]] +name = "hashbrown" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" + +[[package]] +name = "heck" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" + +[[package]] +name = "indexmap" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d" +dependencies = [ + "equivalent", + "hashbrown 0.14.0", +] + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62b02a5381cc465bd3041d84623d0fa3b66738b52b8e2fc3bab8ad63ab032f4a" + +[[package]] +name = "lazy_static" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" + +[[package]] +name = "libc" +version = "0.2.147" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" + +[[package]] +name = "log" +version = "0.4.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b06a4cde4c0f271a446782e3eff8de789548ce57dbc8eca9292c27f4a42004b4" + +[[package]] +name = "memchr" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" + +[[package]] +name = "minimal-lexical" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" + +[[package]] +name = "miniz_oxide" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +dependencies = [ + "adler", +] + +[[package]] +name = "nom" +version = "7.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d273983c5a657a70a3e8f2a01329822f3b8c8172b73826411a55751e404a0a4a" +dependencies = [ + "memchr", + "minimal-lexical", +] + +[[package]] +name = "object" +version = "0.31.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bda667d9f2b5051b8833f59f3bf748b28ef54f850f4fcb389a252aa383866d1" +dependencies = [ + "memchr", +] + +[[package]] +name = "once_cell" +version = "1.18.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" + +[[package]] +name = "proc-macro2" +version = "1.0.63" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b368fba921b0dce7e60f5e04ec15e565b3303972b42bcfde1d0713b881959eb" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prql-compiler" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c99b52154002ac7f286dd2293c2f8d4e30526c1d396b14deef5ada1deef3c9ff" +dependencies = [ + "anyhow", + "ariadne", + "chumsky", + "csv", + "enum-as-inner", + "itertools", + "lazy_static", + "log", + "once_cell", + "regex", + "semver", + "serde", + "serde_json", + "serde_yaml", + "sqlformat", + "sqlparser", + "strum", + "strum_macros", +] + +[[package]] +name = "psm" +version = "0.1.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5787f7cda34e3033a72192c018bc5883100330f362ef279a8cbccfce8bb4e874" +dependencies = [ + "cc", +] + +[[package]] +name = "quote" +version = "1.0.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "573015e8ab27661678357f27dc26460738fd2b6c86e46f386fde94cb5d913105" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "regex" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89089e897c013b3deb627116ae56a6955a72b8bed395c9526af31c9fe528b484" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa250384981ea14565685dea16a9ccc4d1c541a13f82b9c168572264d1df8c56" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax", +] + +[[package]] +name = "regex-syntax" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2ab07dc67230e4a4718e70fd5c20055a4334b121f1f9db8fe63ef39ce9b8c846" + +[[package]] +name = "rustc-demangle" +version = "0.1.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" + +[[package]] +name = "rustversion" +version = "1.0.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc31bd9b61a32c31f9650d18add92aa83a49ba979c143eefd27fe7177b05bd5f" + +[[package]] +name = "ryu" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe232bdf6be8c8de797b22184ee71118d63780ea42ac85b61d1baa6d3b782ae9" + +[[package]] +name = "semver" +version = "1.0.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bebd363326d05ec3e2f532ab7660680f3b02130d780c299bca73469d521bc0ed" +dependencies = [ + "serde", +] + +[[package]] +name = "serde" +version = "1.0.166" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d01b7404f9d441d3ad40e6a636a7782c377d2abdbe4fa2440e2edcc2f4f10db8" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde_derive" +version = "1.0.166" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5dd83d6dde2b6b2d466e14d9d1acce8816dedee94f735eac6395808b3483c6d6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.23", +] + +[[package]] +name = "serde_json" +version = "1.0.100" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f1e14e89be7aa4c4b78bdbdc9eb5bf8517829a600ae8eaa39a6e1d960b5185c" +dependencies = [ + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "serde_yaml" +version = "0.9.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "452e67b9c20c37fa79df53201dc03839651086ed9bbe92b3ca585ca9fdaa7d85" +dependencies = [ + "indexmap", + "itoa", + "ryu", + "serde", + "unsafe-libyaml", +] + +[[package]] +name = "sqlformat" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c12bc9199d1db8234678b7051747c07f517cdcf019262d1847b94ec8b1aee3e" +dependencies = [ + "itertools", + "nom", + "unicode_categories", +] + +[[package]] +name = "sqlparser" +version = "0.33.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "355dc4d4b6207ca8a3434fc587db0a8016130a574dbcdbfb93d7f7b5bc5b211a" +dependencies = [ + "log", + "serde", +] + +[[package]] +name = "stacker" +version = "0.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c886bd4480155fd3ef527d45e9ac8dd7118a898a46530b7b94c3e21866259fce" +dependencies = [ + "cc", + "cfg-if", + "libc", + "psm", + "winapi", +] + +[[package]] +name = "strum" +version = "0.24.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" +dependencies = [ + "strum_macros", +] + +[[package]] +name = "strum_macros" +version = "0.24.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "rustversion", + "syn 1.0.109", +] + +[[package]] +name = "syn" +version = "1.0.109" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "59fb7d6d8281a51045d62b8eb3a7d1ce347b76f312af50cd3dc0af39c87c1737" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "unicode-ident" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22049a19f4a68748a168c0fc439f9516686aa045927ff767eca0a85101fb6e73" + +[[package]] +name = "unicode-width" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" + +[[package]] +name = "unicode_categories" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39ec24b3121d976906ece63c9daad25b85969647682eee313cb5779fdd69e14e" + +[[package]] +name = "unsafe-libyaml" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1865806a559042e51ab5414598446a5871b561d21b6764f2eabb0dd481d880a6" + +[[package]] +name = "version_check" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" + +[[package]] +name = "wasi" +version = "0.11.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "yansi" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" diff --git a/rust/prql/Cargo.toml b/rust/prql/Cargo.toml new file mode 100644 index 00000000000..314d1b52391 --- /dev/null +++ b/rust/prql/Cargo.toml @@ -0,0 +1,20 @@ +[package] +name = "_ch_rust_prql" +version = "0.1.0" +edition = "2021" + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +prql-compiler = "0.8.1" +serde_json = "1.0" + +[lib] +crate-type = ["staticlib"] + +[profile.release] +debug = true + +[profile.release-thinlto] +inherits = "release" +lto = true diff --git a/rust/prql/include/prql.h b/rust/prql/include/prql.h new file mode 100644 index 00000000000..29158d7f30d --- /dev/null +++ b/rust/prql/include/prql.h @@ -0,0 +1,18 @@ +#pragma once + +#include + +extern "C" { + +/// Converts a PRQL query to an SQL query. +/// @param query is a pointer to the beginning of the PRQL query. +/// @param size is the size of the PRQL query. +/// @param out is a pointer to a uint8_t pointer which will be set to the beginning of the null terminated SQL query or the error message. +/// @param out_size is the size of the string pointed by `out`. +/// @returns zero in case of success, non-zero in case of failure. +int64_t prql_to_sql(const uint8_t * query, uint64_t size, uint8_t ** out, uint64_t * out_size); + +/// Frees the passed in pointer which's memory was allocated by Rust allocators previously. +void prql_free_pointer(uint8_t * ptr_to_free); + +} // extern "C" diff --git a/rust/prql/src/lib.rs b/rust/prql/src/lib.rs new file mode 100644 index 00000000000..fb71d62d527 --- /dev/null +++ b/rust/prql/src/lib.rs @@ -0,0 +1,56 @@ +use prql_compiler::sql::Dialect; +use prql_compiler::{Options, Target}; +use std::ffi::{c_char, CString}; +use std::slice; + +fn set_output(result: String, out: *mut *mut u8, out_size: *mut u64) { + assert!(!out_size.is_null()); + let out_size_ptr = unsafe { &mut *out_size }; + *out_size_ptr = (result.len() + 1).try_into().unwrap(); + + assert!(!out.is_null()); + let out_ptr = unsafe { &mut *out }; + *out_ptr = CString::new(result).unwrap().into_raw() as *mut u8; +} + +#[no_mangle] +pub unsafe extern "C" fn prql_to_sql( + query: *const u8, + size: u64, + out: *mut *mut u8, + out_size: *mut u64, +) -> i64 { + let query_vec = unsafe { slice::from_raw_parts(query, size.try_into().unwrap()) }.to_vec(); + let maybe_prql_query = String::from_utf8(query_vec); + if maybe_prql_query.is_err() { + set_output( + String::from("The PRQL query must be UTF-8 encoded!"), + out, + out_size, + ); + return 1; + } + let prql_query = maybe_prql_query.unwrap(); + let opts = &Options { + format: true, + target: Target::Sql(Some(Dialect::ClickHouse)), + signature_comment: false, + color: false, + }; + let (is_err, res) = match prql_compiler::compile(&prql_query, &opts) { + Ok(sql_str) => (false, sql_str), + Err(err) => (true, err.to_string()), + }; + + set_output(res, out, out_size); + + match is_err { + true => 1, + false => 0, + } +} + +#[no_mangle] +pub unsafe extern "C" fn prql_free_pointer(ptr_to_free: *mut u8) { + std::mem::drop(CString::from_raw(ptr_to_free as *mut c_char)); +} diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 509dfe2e232..f5390037e6b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -46,6 +46,7 @@ #include #include #include +#include #include #include @@ -72,6 +73,7 @@ #include #include #include +#include #include #include "config_version.h" @@ -338,6 +340,8 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu if (dialect == Dialect::kusto) parser = std::make_unique(end, global_context->getSettings().allow_settings_after_format_in_insert); + else if (dialect == Dialect::prql) + parser = std::make_unique(max_length, settings.max_parser_depth); else parser = std::make_unique(end, global_context->getSettings().allow_settings_after_format_in_insert); diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 1cb13d3ae3e..a2c18fc330f 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -54,6 +54,7 @@ #cmakedefine01 USE_BORINGSSL #cmakedefine01 USE_BLAKE3 #cmakedefine01 USE_SKIM +#cmakedefine01 USE_PRQL #cmakedefine01 USE_OPENSSL_INTREE #cmakedefine01 USE_ULID #cmakedefine01 FIU_ENABLE diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 1e2cbce9309..86400954e2f 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -138,7 +138,9 @@ IMPLEMENT_SETTING_ENUM(MsgPackUUIDRepresentation, ErrorCodes::BAD_ARGUMENTS, IMPLEMENT_SETTING_ENUM(Dialect, ErrorCodes::BAD_ARGUMENTS, {{"clickhouse", Dialect::clickhouse}, - {"kusto", Dialect::kusto}}) + {"kusto", Dialect::kusto}, + {"kusto", Dialect::kusto}, + {"prql", Dialect::prql}}) // FIXME: do not add 'kusto_auto' to the list. Maybe remove it from code completely? IMPLEMENT_SETTING_ENUM(ParallelReplicasCustomKeyFilterType, ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index c2783447441..c61afbd2bbf 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -207,6 +207,7 @@ enum class Dialect clickhouse, kusto, kusto_auto, + prql, }; DECLARE_SETTING_ENUM(Dialect) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 4b76d20f31d..66bc0bcb757 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -75,6 +75,7 @@ #include #include +#include namespace ProfileEvents { @@ -702,10 +703,14 @@ static std::tuple executeQueryImpl( /// TODO: parser should fail early when max_query_size limit is reached. ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); } + else if (settings.dialect == Dialect::prql && !internal) + { + ParserPRQLQuery parser(max_query_size, settings.max_parser_depth); + ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); + } else { ParserQuery parser(end, settings.allow_settings_after_format_in_insert); - /// TODO: parser should fail early when max_query_size limit is reached. ast = parseQuery(parser, begin, end, "", max_query_size, settings.max_parser_depth); } diff --git a/src/Parsers/CMakeLists.txt b/src/Parsers/CMakeLists.txt index d5cf2bd4784..d74137f8a91 100644 --- a/src/Parsers/CMakeLists.txt +++ b/src/Parsers/CMakeLists.txt @@ -4,8 +4,12 @@ add_headers_and_sources(clickhouse_parsers .) add_headers_and_sources(clickhouse_parsers ./Access) add_headers_and_sources(clickhouse_parsers ./MySQL) add_headers_and_sources(clickhouse_parsers ./Kusto) +add_headers_and_sources(clickhouse_parsers ./PRQL) add_library(clickhouse_parsers ${clickhouse_parsers_headers} ${clickhouse_parsers_sources}) target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io clickhouse_common_access string_utils) +if (TARGET ch_rust::prql) + target_link_libraries(clickhouse_parsers PRIVATE ch_rust::prql) +endif () if (USE_DEBUG_HELPERS) # CMake generator expression will do insane quoting when it encounters special character like quotes, spaces, etc. diff --git a/src/Parsers/PRQL/ParserPRQLQuery.cpp b/src/Parsers/PRQL/ParserPRQLQuery.cpp new file mode 100644 index 00000000000..b3733b727dc --- /dev/null +++ b/src/Parsers/PRQL/ParserPRQLQuery.cpp @@ -0,0 +1,86 @@ +#include +#include + +#include "Parsers/Lexer.h" +#include "config.h" + +#if USE_PRQL +# include +#endif + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; + extern const int SUPPORT_IS_DISABLED; +} + +bool ParserPRQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserSetQuery set_p; + + if (set_p.parse(pos, node, expected)) + return true; + +#if !USE_PRQL + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, "PRQL is not available. Rust code or PRQL itself may be disabled. Use another dialect!"); +#else + const auto * begin = pos->begin; + + // The same parsers are used in the client and the server, so the parser have to detect the end of a single query in case of multiquery queries + while (!pos->isEnd() && pos->type != TokenType::Semicolon) + ++pos; + + const auto * end = pos->begin; + + uint8_t * sql_query_ptr{nullptr}; + uint64_t sql_query_size{0}; + + const auto res + = prql_to_sql(reinterpret_cast(begin), static_cast(end - begin), &sql_query_ptr, &sql_query_size); + + SCOPE_EXIT({ prql_free_pointer(sql_query_ptr); }); + + const auto * sql_query_char_ptr = reinterpret_cast(sql_query_ptr); + const auto * const original_sql_query_ptr = sql_query_char_ptr; + + if (res != 0) + { + throw Exception(ErrorCodes::SYNTAX_ERROR, "PRQL syntax error: '{}'", sql_query_char_ptr); + } + chassert(sql_query_size > 0); + + ParserQuery query_p(end, false); + String error_message; + node = tryParseQuery( + query_p, + sql_query_char_ptr, + sql_query_char_ptr + sql_query_size - 1, + error_message, + false, + "", + false, + max_query_size, + max_parser_depth); + + if (!node) + throw Exception( + ErrorCodes::SYNTAX_ERROR, + "Error while parsing the SQL query generated from PRQL query :'{}'.\nPRQL Query:'{}'\nSQL query: '{}'", + error_message, + std::string_view{begin, end}, + std::string_view(original_sql_query_ptr, original_sql_query_ptr + sql_query_size)); + + + return true; +#endif +} +} diff --git a/src/Parsers/PRQL/ParserPRQLQuery.h b/src/Parsers/PRQL/ParserPRQLQuery.h new file mode 100644 index 00000000000..4fc450df6b6 --- /dev/null +++ b/src/Parsers/PRQL/ParserPRQLQuery.h @@ -0,0 +1,27 @@ +#pragma once + +#include + +namespace DB +{ +// Even when PRQL is disabled, it is not possible to exclude this parser because changing the dialect via `SET dialect = '...'` queries should succeed. +// Another solution would be disabling setting the dialect to PRQL, but it requires a lot of finicky conditional compiling around the Dialect setting enum. +// Therefore the decision, for now, is to use this parser even when PRQL is disabled to enable users to switch to another dialect. +class ParserPRQLQuery final : public IParserBase +{ +private: + // These fields are not used when PRQL is disabled at build time. + [[maybe_unused]] size_t max_query_size; + [[maybe_unused]] size_t max_parser_depth; + +public: + ParserPRQLQuery(size_t max_query_size_, size_t max_parser_depth_) : max_query_size{max_query_size_}, max_parser_depth{max_parser_depth_} + { + } + + const char * getName() const override { return "PRQL Statement"; } + +protected: + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 2795de64b1d..ef4ef05e35e 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -476,3 +477,22 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest, "SELECT *\nFROM Customers\nWHERE NOT (FirstName ILIKE 'pet%')" } }))); + +static constexpr size_t kDummyMaxQuerySize = 256 * 1024; +static constexpr size_t kDummyMaxParserDepth = 256; + +INSTANTIATE_TEST_SUITE_P( + ParserPRQL, + ParserTest, + ::testing::Combine( + ::testing::Values(std::make_shared(kDummyMaxQuerySize, kDummyMaxParserDepth)), + ::testing::ValuesIn(std::initializer_list{ + { + "from albums\ngroup [author_id] (\n aggregate [first_pushlied = min published]\n)\njoin a=author side:left [==author_id]\njoin p=purchases side:right [==author_id]\ngroup [a.id, p.purchase_id] (\n aggregate [avg_sell = min first_pushlied]\n)", + "WITH table_1 AS\n (\n SELECT\n MIN(published) AS _expr_0,\n author_id\n FROM albums\n GROUP BY author_id\n )\nSELECT\n a.id,\n p.purchase_id,\n MIN(table_0._expr_0) AS avg_sell\nFROM table_1 AS table_0\nLEFT JOIN author AS a ON table_0.author_id = a.author_id\nRIGHT JOIN purchases AS p ON table_0.author_id = p.author_id\nGROUP BY\n a.id,\n p.purchase_id", + }, + { + "from matches\nfilter start_date > @2023-05-30 # Some comment here\nderive [\n some_derived_value_1 = a + (b ?? 0), # And there\n some_derived_value_2 = c + some_derived_value\n]\nfilter some_derived_value_2 > 0\ngroup [country, city] (\n aggregate [\n average some_derived_value_2,\n aggr = max some_derived_value_2,\n ]\n)\nderive place = f\"{city} in {country}\"\nderive country_code = s\"LEFT(country, 2)\"\nsort [aggr, -country]\ntake 1..20", + "WITH\n table_3 AS\n (\n SELECT\n country,\n city,\n c + some_derived_value AS _expr_1\n FROM matches\n WHERE start_date > toDate('2023-05-30')\n ),\n table_1 AS\n (\n SELECT\n country,\n city,\n AVG(_expr_1) AS _expr_0,\n MAX(_expr_1) AS aggr\n FROM table_3 AS table_2\n WHERE _expr_1 > 0\n GROUP BY\n country,\n city\n )\nSELECT\n country,\n city,\n _expr_0,\n aggr,\n CONCAT(city, ' in ', country) AS place,\n LEFT(country, 2) AS country_code\nFROM table_1 AS table_0\nORDER BY\n aggr ASC,\n country DESC\nLIMIT 20", + }, + }))); diff --git a/src/configure_config.cmake b/src/configure_config.cmake index c11a19b36ea..ae6305705c2 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -25,6 +25,9 @@ endif() if (TARGET ch_rust::skim) set(USE_SKIM 1) endif() +if (TARGET ch_rust::prql) + set(USE_PRQL 1) +endif() if (TARGET OpenSSL::SSL) set(USE_SSL 1) endif() diff --git a/tests/queries/0_stateless/02766_prql.reference b/tests/queries/0_stateless/02766_prql.reference new file mode 100644 index 00000000000..90e0b26cee6 --- /dev/null +++ b/tests/queries/0_stateless/02766_prql.reference @@ -0,0 +1,19 @@ +101 Hello, ClickHouse! 2 He +101 Granules are the smallest chunks of data read 2 Gr +102 Insert a lot of rows per batch 2 In +102 Sort your data based on your commonly-used queries 2 So +103 This is an awesome message 2 Th +103 42 +102 4.132209897041321 +--- +101 Hello, ClickHouse! 2019-01-01 00:00:00.000 -1 +101 Granules are the smallest chunks of data read 2019-05-01 00:00:00.000 3.14159 +102 Insert a lot of rows per batch 2019-02-01 00:00:00.000 1.41421 +102 Sort your data based on your commonly-used queries 2019-03-01 00:00:00.000 2.718 +103 This is an awesome message 2019-04-01 00:00:00.000 42 +--- +101 Hello, ClickHouse! 2019-01-01 00:00:00.000 -1 +101 Granules are the smallest chunks of data read 2019-05-01 00:00:00.000 3.14159 +102 Insert a lot of rows per batch 2019-02-01 00:00:00.000 1.41421 +102 Sort your data based on your commonly-used queries 2019-03-01 00:00:00.000 2.718 +103 This is an awesome message 2019-04-01 00:00:00.000 42 diff --git a/tests/queries/0_stateless/02766_prql.sh b/tests/queries/0_stateless/02766_prql.sh new file mode 100755 index 00000000000..f8bbd72af4e --- /dev/null +++ b/tests/queries/0_stateless/02766_prql.sh @@ -0,0 +1,58 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-random-settings + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -n -q " +CREATE TEMPORARY TABLE IF NOT EXISTS aboba +( + user_id UInt32, + message String, + creation_date DateTime64, + metric Float32 +) +ENGINE = MergeTree +ORDER BY user_id; + +INSERT INTO aboba (user_id, message, creation_date, metric) VALUES (101, 'Hello, ClickHouse!', toDateTime('2019-01-01 00:00:00', 3, 'Europe/Amsterdam'), -1.0), (102, 'Insert a lot of rows per batch', toDateTime('2019-02-01 00:00:00', 3, 'Europe/Amsterdam'), 1.41421 ), (102, 'Sort your data based on your commonly-used queries', toDateTime('2019-03-01 00:00:00', 3, 'Europe/Amsterdam'), 2.718), (101, 'Granules are the smallest chunks of data read', toDateTime('2019-05-01 00:00:00', 3, 'Europe/Amsterdam'), 3.14159), (103, 'This is an awesome message', toDateTime('2019-04-01 00:00:00', 3, 'Europe/Amsterdam'), 42); + +SET dialect = 'prql'; + +from aboba +derive [ + a = 2, + b = s\"LEFT(message, 2)\" +] +select [ user_id, message, a, b ]; + +from aboba +filter user_id > 101 +group user_id ( + aggregate [ + metrics = sum metric + ] +); + +SET dialect = 'clickhouse'; + +SELECT '---'; +SELECT + user_id, + message, + toTimeZone(creation_date, 'Europe/Amsterdam') as creation_date, + metric +FROM aboba; +SELECT '---'; + +SET dialect = 'prql'; + +from aboba +select [ user_id, message, metric ] +derive creation_date = s\"toTimeZone(creation_date, 'Europe/Amsterdam')\" +select [ user_id, message, creation_date, metric]; + +from s\"SELECT * FROM system.users\" | select non_existent_column; # {serverError UNKNOWN_IDENTIFIER} +from non_existent_table; # {serverError UNKNOWN_TABLE} +" \ No newline at end of file From 84f6a7336c2d7ac547ad7030c389d4961f4ab8e4 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Thu, 20 Jul 2023 19:03:42 +0800 Subject: [PATCH 0913/2047] Prevent going beyond the index of const_columns_to_remove. --- src/Processors/Transforms/FinishSortingTransform.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index 744d035d0ee..baf898481ab 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -38,11 +38,12 @@ FinishSortingTransform::FinishSortingTransform( /// Remove constants from description_sorted_. SortDescription description_sorted_without_constants; description_sorted_without_constants.reserve(description_sorted_.size()); + size_t num_columns = const_columns_to_remove.size(); for (const auto & column_description : description_sorted_) { auto pos = header.getPositionByName(column_description.column_name); - if (!const_columns_to_remove[pos]){ + if (pos < num_columns && !const_columns_to_remove[pos]){ description_sorted_without_constants.push_back(column_description); } } From cbcd48979cab1a3a4a0f0d5baaf8077164887cf5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 20 Jul 2023 13:04:43 +0200 Subject: [PATCH 0914/2047] Fix race one more time --- programs/server/Server.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d1c1a1d200f..774c3f223a6 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -739,9 +739,10 @@ try [&]() -> std::vector { std::vector metrics; - metrics.reserve(servers_to_start_before_tables.size() + servers.size()); std::lock_guard lock(servers_lock); + metrics.reserve(servers_to_start_before_tables.size() + servers.size()); + for (const auto & server : servers_to_start_before_tables) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); From f2d184cf1b002d18be152880ee2d82e57fed3b26 Mon Sep 17 00:00:00 2001 From: chen768959 <934103231@qq.com> Date: Thu, 20 Jul 2023 19:11:08 +0800 Subject: [PATCH 0915/2047] Consistent style for if statements. --- src/Processors/Transforms/FinishSortingTransform.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index baf898481ab..63a9c3924a2 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -43,9 +43,8 @@ FinishSortingTransform::FinishSortingTransform( { auto pos = header.getPositionByName(column_description.column_name); - if (pos < num_columns && !const_columns_to_remove[pos]){ + if (pos < num_columns && !const_columns_to_remove[pos]) description_sorted_without_constants.push_back(column_description); - } } /// The target description is modified in SortingTransform constructor. /// To avoid doing the same actions with description_sorted just copy it from prefix of target description. From db1b53d1bb8ed6aa71f47010c81a7f3ebb0ae65d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 14:18:48 +0300 Subject: [PATCH 0916/2047] Update 01606_git_import.sh --- tests/queries/0_stateless/01606_git_import.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01606_git_import.sh b/tests/queries/0_stateless/01606_git_import.sh index c9aa2c7d82e..48558d79f93 100755 --- a/tests/queries/0_stateless/01606_git_import.sh +++ b/tests/queries/0_stateless/01606_git_import.sh @@ -13,7 +13,7 @@ cd $CLICKHOUSE_TMP || exit # Protection for network errors for _ in {1..10}; do rm -rf ./clickhouse-odbc - git clone --quiet https://github.com/ClickHouse/clickhouse-odbc.git && pushd clickhouse-odbc > /dev/null && git checkout --quiet 5d84ec591c53cbb272593f024230a052690fdf69 && break + git clone --quiet https://github.com/ClickHouse/clickhouse-odbc.git && pushd clickhouse-odbc 2> /dev/null > /dev/null && git checkout --quiet 5d84ec591c53cbb272593f024230a052690fdf69 && break sleep 1 done From 2b29e3dc83d9ed6747acb4a249c9e1aca9616f21 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 14:22:22 +0300 Subject: [PATCH 0917/2047] Update MergeTreeBackgroundExecutor.cpp (#52261) --- src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 6eab4337162..e497a799274 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -269,7 +269,7 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item) try { ALLOW_ALLOCATIONS_IN_SCOPE; - item->task->getQueryId(); + query_id = item->task->getQueryId(); need_execute_again = item->task->executeStep(); } catch (...) From f53ff5d4f2228b7016af5742ea1ae8f70ef772df Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 14:51:01 +0300 Subject: [PATCH 0918/2047] more fair queue for drop table sync (#52276) --- src/Interpreters/DatabaseCatalog.cpp | 17 ++++++++++++++++- src/Interpreters/DatabaseCatalog.h | 1 + 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 23a67f4bc2f..0e2e30eefee 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -697,6 +697,7 @@ DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_) , loading_dependencies{"LoadingDeps"} , view_dependencies{"ViewDeps"} , log(&Poco::Logger::get("DatabaseCatalog")) + , first_async_drop_in_queue(tables_marked_dropped.end()) { } @@ -959,9 +960,17 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr std::lock_guard lock(tables_marked_dropped_mutex); if (ignore_delay) - tables_marked_dropped.push_front({table_id, table, dropped_metadata_path, drop_time}); + { + /// Insert it before first_async_drop_in_queue, so sync drop queries will have priority over async ones, + /// but the queue will remain fair for multiple sync drop queries. + tables_marked_dropped.emplace(first_async_drop_in_queue, TableMarkedAsDropped{table_id, table, dropped_metadata_path, drop_time}); + } else + { tables_marked_dropped.push_back({table_id, table, dropped_metadata_path, drop_time + drop_delay_sec}); + if (first_async_drop_in_queue == tables_marked_dropped.end()) + --first_async_drop_in_queue; + } tables_marked_dropped_ids.insert(table_id.uuid); CurrentMetrics::add(CurrentMetrics::TablesToDropQueueSize, 1); @@ -1012,6 +1021,8 @@ void DatabaseCatalog::dequeueDroppedTableCleanup(StorageID table_id) /// This maybe throw exception. renameNoReplace(latest_metadata_dropped_path, table_metadata_path); + if (first_async_drop_in_queue == it_dropped_table) + ++first_async_drop_in_queue; tables_marked_dropped.erase(it_dropped_table); [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(dropped_table.table_id.uuid); assert(removed); @@ -1074,6 +1085,8 @@ void DatabaseCatalog::dropTableDataTask() table = std::move(*it); LOG_INFO(log, "Have {} tables in drop queue ({} of them are in use), will try drop {}", tables_marked_dropped.size(), tables_in_use_count, table.table_id.getNameForLogs()); + if (first_async_drop_in_queue == it) + ++first_async_drop_in_queue; tables_marked_dropped.erase(it); /// Schedule the task as soon as possible, while there are suitable tables to drop. schedule_after_ms = 0; @@ -1110,6 +1123,8 @@ void DatabaseCatalog::dropTableDataTask() table.drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()) + drop_error_cooldown_sec; std::lock_guard lock(tables_marked_dropped_mutex); tables_marked_dropped.emplace_back(std::move(table)); + if (first_async_drop_in_queue == tables_marked_dropped.end()) + --first_async_drop_in_queue; /// If list of dropped tables was empty, schedule a task to retry deletion. if (tables_marked_dropped.size() == 1) { diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index d502505027f..805d7786569 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -323,6 +323,7 @@ private: mutable std::mutex ddl_guards_mutex; TablesMarkedAsDropped tables_marked_dropped TSA_GUARDED_BY(tables_marked_dropped_mutex); + TablesMarkedAsDropped::iterator first_async_drop_in_queue TSA_GUARDED_BY(tables_marked_dropped_mutex); std::unordered_set tables_marked_dropped_ids TSA_GUARDED_BY(tables_marked_dropped_mutex); mutable std::mutex tables_marked_dropped_mutex; From fc3be766f7347315a972a9854e25ac72d63dd0e0 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Thu, 20 Jul 2023 19:58:51 +0800 Subject: [PATCH 0919/2047] ignore test --- .../02813_system_events_and_metrics_add_alias.reference | 4 ---- .../0_stateless/02813_system_events_and_metrics_add_alias.sql | 4 ---- 2 files changed, 8 deletions(-) delete mode 100644 tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference delete mode 100644 tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql diff --git a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference deleted file mode 100644 index 93be2764a57..00000000000 --- a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.reference +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE system.metrics\n(\n `metric` String,\n `value` Int64,\n `description` String,\n `name` String\n)\nENGINE = SystemMetrics\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -1 -CREATE TABLE system.events\n(\n `event` String,\n `value` UInt64,\n `description` String,\n `name` String\n)\nENGINE = SystemEvents\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -1 diff --git a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql b/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql deleted file mode 100644 index a2250608f8f..00000000000 --- a/tests/queries/0_stateless/02813_system_events_and_metrics_add_alias.sql +++ /dev/null @@ -1,4 +0,0 @@ -show create table system.metrics; -select equals((select count() from system.metrics where name=metric) as r1, (select count() from system.metrics) as r2); -show create table system.events; -select equals((select count() from system.events where name=event) as r1, (select count() from system.events) as r2); \ No newline at end of file From d16d4449432999cdee3393b1f47b4a7d7c5314a6 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Thu, 20 Jul 2023 12:24:52 +0200 Subject: [PATCH 0920/2047] MaterializedMySQL: Add support of double quoted comments --- src/Parsers/ExpressionElementParsers.cpp | 33 +++++++++++++++++ src/Parsers/ExpressionElementParsers.h | 15 ++++++++ src/Parsers/MySQL/ASTDeclareColumn.cpp | 2 +- .../materialized_with_ddl.py | 35 +++++++++++++++++++ .../test_materialized_mysql_database/test.py | 6 ++++ 5 files changed, 90 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 3a7e8790bb4..0149526da79 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1900,6 +1900,39 @@ bool ParserSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } +bool ParserMySQLComment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (pos->type != TokenType::QuotedIdentifier && pos->type != TokenType::StringLiteral) + return false; + String s; + ReadBufferFromMemory in(pos->begin, pos->size()); + try + { + if (pos->type == TokenType::StringLiteral) + readQuotedStringWithSQLStyle(s, in); + else + readDoubleQuotedStringWithSQLStyle(s, in); + } + catch (const Exception &) + { + expected.add(pos, "string literal or double quoted string"); + return false; + } + + if (in.count() != pos->size()) + { + expected.add(pos, "string literal or double quoted string"); + return false; + } + + auto literal = std::make_shared(s); + literal->begin = pos; + literal->end = ++pos; + node = literal; + return true; +} + + bool ParserMySQLGlobalVariable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (pos->type != TokenType::DoubleAt) diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index cc88faf2653..f33f2d99f71 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -367,6 +367,21 @@ protected: }; +/** MySQL comment: + * CREATE TABLE t ( + * i INT PRIMARY KEY, + * first_name VARCHAR(255) COMMENT 'FIRST_NAME', + * last_name VARCHAR(255) COMMENT "LAST_NAME" + * ) + */ +class ParserMySQLComment : public IParserBase +{ +protected: + const char * getName() const override { return "MySQL comment parser"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + + /** MySQL-style global variable: @@var */ class ParserMySQLGlobalVariable : public IParserBase diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp index e585dcb670c..e5f2b7870e2 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.cpp +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -50,7 +50,7 @@ static inline bool parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, OptionDescribe("PRIMARY KEY", "primary_key", std::make_unique()), OptionDescribe("UNIQUE", "unique_key", std::make_unique()), OptionDescribe("KEY", "primary_key", std::make_unique()), - OptionDescribe("COMMENT", "comment", std::make_unique()), + OptionDescribe("COMMENT", "comment", std::make_unique()), OptionDescribe("CHARACTER SET", "charset_name", std::make_unique()), OptionDescribe("CHARSET", "charset", std::make_unique()), OptionDescribe("COLLATE", "collate", std::make_unique()), diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 8cf9e67bf63..f7a930ec00b 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -1617,6 +1617,41 @@ def materialized_with_column_comments_test(clickhouse_node, mysql_node, service_ mysql_node.query("DROP DATABASE materialized_with_column_comments_test") +def double_quoted_comment(clickhouse_node, mysql_node, service_name): + db = "comment_db" + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"CREATE DATABASE {db}") + mysql_node.query( + f'CREATE TABLE {db}.t1 (i INT PRIMARY KEY, id VARCHAR(255) COMMENT "ID")' + ) + mysql_node.query( + f"CREATE TABLE {db}.t2 (i INT PRIMARY KEY, id VARCHAR(255) COMMENT 'ID')" + ) + clickhouse_node.query( + f"CREATE DATABASE {db} ENGINE = MaterializedMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')" + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t1\nt2\n", + ) + + # incremental + mysql_node.query( + f'CREATE TABLE {db}.t3 (i INT PRIMARY KEY, id VARCHAR(255) COMMENT "ID")' + ) + mysql_node.query( + f"CREATE TABLE {db}.t4 (i INT PRIMARY KEY, id VARCHAR(255) COMMENT 'ID')" + ) + check_query( + clickhouse_node, f"SHOW TABLES FROM {db} FORMAT TSV", "t1\nt2\nt3\nt4\n" + ) + + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + + def materialized_with_enum8_test(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS materialized_with_enum8_test") clickhouse_node.query("DROP DATABASE IF EXISTS materialized_with_enum8_test") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 21316d1a474..0166f7d1d33 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -416,6 +416,12 @@ def test_materialized_with_column_comments( ) +def test_double_quoted_comment(started_cluster, started_mysql_8_0, clickhouse_node): + materialized_with_ddl.double_quoted_comment( + clickhouse_node, started_mysql_8_0, "mysql80" + ) + + def test_materialized_with_enum( started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node ): From fe934d3059936cd203952cfe5881ff7243001ae9 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 20 Jul 2023 12:38:41 +0000 Subject: [PATCH 0921/2047] Make better --- docs/en/engines/table-engines/special/url.md | 2 +- docs/en/operations/settings/settings.md | 6 +++--- docs/en/sql-reference/table-functions/url.md | 4 ++-- src/Core/Settings.h | 2 +- src/Storages/StorageURL.cpp | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index 9f2bf177c96..f556df0a088 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -106,4 +106,4 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. -- [decode_and_encode_path_in_url](/docs/en/operations/settings/settings.md#decode_and_encode_path_in_url) - enables or disables decoding/encoding path in uri. Enabled by default. +- [disable_url_encoding](/docs/en/operations/settings/settings.md#disable_url_encoding) -allows to disable decoding/encoding path in uri. Disabled by default. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index db5d1a2f5d9..d138b07d3ae 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3466,11 +3466,11 @@ Possible values: Default value: `0`. -## decode_and_encode_path_in_url {#decode_and_encode_path_in_url} +## disable_url_encoding {#disable_url_encoding} -Enables or disables decoding/encoding path in uri in [URL](../../engines/table-engines/special/url.md) engine tables. +Allows to disable decoding/encoding path in uri in [URL](../../engines/table-engines/special/url.md) engine tables. -Enabled by default. +Disabled by default. ## database_atomic_wait_for_drop_and_detach_synchronously {#database_atomic_wait_for_drop_and_detach_synchronously} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 96f36f03949..677ed011960 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -56,8 +56,8 @@ Character `|` inside patterns is used to specify failover addresses. They are it ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. -- [decode_and_encode_path_in_url](/docs/en/operations/settings/settings.md#decode_and_encode_path_in_url) - enables or disables decoding/encoding path in uri. Enabled by default. +- [disable_url_encoding](/docs/en/operations/settings/settings.md#disable_url_encoding) - allows to disable decoding/encoding path in uri. Disabled by default. -- **See Also** +**See Also** - [Virtual columns](/docs/en/engines/table-engines/index.md#table_engines-virtual_columns) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ffa72d841be..5dc40494115 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -621,7 +621,7 @@ class IColumn; M(Bool, engine_file_allow_create_multiple_files, false, "Enables or disables creating a new file on each insert in file engine tables if format has suffix.", 0) \ M(Bool, engine_file_skip_empty_files, false, "Allows to skip empty files in file table engine", 0) \ M(Bool, engine_url_skip_empty_files, false, "Allows to skip empty files in url table engine", 0) \ - M(Bool, decode_and_encode_path_in_url, true, "Enables or disables decoding/encoding path in uri in URL table engine", 0) \ + M(Bool, disable_url_encoding, false, " Allows to disable decoding/encoding path in uri in URL table engine", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(Bool, database_replicated_enforce_synchronous_settings, false, "Enforces synchronous waiting for some queries (see also database_atomic_wait_for_drop_and_detach_synchronously, mutation_sync, alter_sync). Not recommended to enable these settings.", 0) \ diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 4cfefbc5527..0c915f54cff 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -389,7 +389,7 @@ std::pair> StorageURLSource: for (; option != end; ++option) { bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end); - auto request_uri = Poco::URI(*option, context->getSettingsRef().decode_and_encode_path_in_url); + auto request_uri = Poco::URI(*option, context->getSettingsRef().disable_url_encoding); for (const auto & [param, value] : params) request_uri.addQueryParameter(param, value); From f6a44f8eedce98bd50ceee72e5fdc4da1a82a43a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 20 Jul 2023 12:40:41 +0000 Subject: [PATCH 0922/2047] Better --- base/poco/Foundation/include/Poco/URI.h | 6 +++-- base/poco/Foundation/src/URI.cpp | 34 +++++++++++++------------ 2 files changed, 22 insertions(+), 18 deletions(-) diff --git a/base/poco/Foundation/include/Poco/URI.h b/base/poco/Foundation/include/Poco/URI.h index 5e6e7efd938..f4505147ced 100644 --- a/base/poco/Foundation/include/Poco/URI.h +++ b/base/poco/Foundation/include/Poco/URI.h @@ -57,7 +57,7 @@ public: URI(); /// Creates an empty URI. - explicit URI(const std::string & uri, bool decode_and_encode_path = true); + explicit URI(const std::string & uri, bool disable_url_encoding = true); /// Parses an URI from the given string. Throws a /// SyntaxException if the uri is not valid. @@ -351,6 +351,8 @@ protected: private: void encodePath(std::string & encodedStr) const; + void decodePath(const std::string & encodedStr); + std::string _scheme; std::string _userInfo; @@ -360,7 +362,7 @@ private: std::string _query; std::string _fragment; - bool _decode_and_encode_path = true; + bool _disable_url_encoding = true; }; diff --git a/base/poco/Foundation/src/URI.cpp b/base/poco/Foundation/src/URI.cpp index 9bad1b39a87..3354c69d188 100644 --- a/base/poco/Foundation/src/URI.cpp +++ b/base/poco/Foundation/src/URI.cpp @@ -37,7 +37,7 @@ URI::URI(): URI::URI(const std::string& uri, bool decode_and_encode_path): - _port(0), _decode_and_encode_path(decode_and_encode_path) + _port(0), _disable_url_encoding(decode_and_encode_path) { parse(uri); } @@ -108,7 +108,7 @@ URI::URI(const URI& uri): _path(uri._path), _query(uri._query), _fragment(uri._fragment), - _decode_and_encode_path(uri._decode_and_encode_path) + _disable_url_encoding(uri._disable_url_encoding) { } @@ -121,7 +121,7 @@ URI::URI(const URI& baseURI, const std::string& relativeURI): _path(baseURI._path), _query(baseURI._query), _fragment(baseURI._fragment), - _decode_and_encode_path(baseURI._decode_and_encode_path) + _disable_url_encoding(baseURI._disable_url_encoding) { resolve(relativeURI); } @@ -153,7 +153,7 @@ URI& URI::operator = (const URI& uri) _path = uri._path; _query = uri._query; _fragment = uri._fragment; - _decode_and_encode_path = uri._decode_and_encode_path; + _disable_url_encoding = uri._disable_url_encoding; } return *this; } @@ -184,7 +184,7 @@ void URI::swap(URI& uri) std::swap(_path, uri._path); std::swap(_query, uri._query); std::swap(_fragment, uri._fragment); - std::swap(_decode_and_encode_path, uri._decode_and_encode_path); + std::swap(_disable_url_encoding, uri._disable_url_encoding); } @@ -317,10 +317,7 @@ void URI::setAuthority(const std::string& authority) void URI::setPath(const std::string& path) { _path.clear(); - if (_decode_and_encode_path) - decode(path, _path); - else - _path = path; + decodePath(path); } @@ -690,10 +687,18 @@ void URI::decode(const std::string& str, std::string& decodedStr, bool plusAsSpa void URI::encodePath(std::string & encodedStr) const { - if (_decode_and_encode_path) - encode(_path, RESERVED_PATH, encodedStr); - else + if (_disable_url_encoding) encodedStr = _path; + else + encode(_path, RESERVED_PATH, encodedStr); +} + +void URI::decodePath(const std::string & encodedStr) +{ + if (_disable_url_encoding) + _path = encodedStr; + else + decode(encodedStr, _path); } bool URI::isWellKnownPort() const @@ -834,10 +839,7 @@ void URI::parsePath(std::string::const_iterator& it, const std::string::const_it { std::string path; while (it != end && *it != '?' && *it != '#') path += *it++; - if (_decode_and_encode_path) - decode(path, _path); - else - _path = path; + decodePath(path); } From 710e9a817de5c5b0f47ea1a0ae0ac29ac8e48cb4 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 20 Jul 2023 12:59:05 +0000 Subject: [PATCH 0923/2047] Fix tests --- src/Functions/structureToFormatSchema.cpp | 2 +- .../0_stateless/02817_structure_to_schema.out | 468 ++++++++++++++++++ .../02817_structure_to_schema.reference | 22 - .../0_stateless/02817_structure_to_schema.sh | 4 +- 4 files changed, 471 insertions(+), 25 deletions(-) create mode 100644 tests/queries/0_stateless/02817_structure_to_schema.out diff --git a/src/Functions/structureToFormatSchema.cpp b/src/Functions/structureToFormatSchema.cpp index f2bd78e4fb7..406da372c04 100644 --- a/src/Functions/structureToFormatSchema.cpp +++ b/src/Functions/structureToFormatSchema.cpp @@ -27,7 +27,7 @@ class FunctionStructureToFormatSchema : public IFunction public: static constexpr auto name = Impl::name; - FunctionStructureToFormatSchema(ContextPtr context_) : context(std::move(context_)) + explicit FunctionStructureToFormatSchema(ContextPtr context_) : context(std::move(context_)) { } diff --git a/tests/queries/0_stateless/02817_structure_to_schema.out b/tests/queries/0_stateless/02817_structure_to_schema.out new file mode 100644 index 00000000000..ad45343139b --- /dev/null +++ b/tests/queries/0_stateless/02817_structure_to_schema.out @@ -0,0 +1,468 @@ +CapnProto +Numbers + +struct Message +{ + int8 @0 : Int8; + uint8 @1 : UInt8; + int16 @2 : Int16; + uint16 @3 : UInt16; + int32 @4 : Int32; + uint32 @5 : UInt32; + int64 @6 : Int64; + uint64 @7 : UInt64; + int128 @8 : Data; + uint128 @9 : Data; + int256 @10 : Data; + uint256 @11 : Data; + float32 @12 : Float32; + float64 @13 : Float64; + decimal32 @14 : Int32; + decimal64 @15 : Int64; + decimal128 @16 : Data; + decimal256 @17 : Data; +} +Dates + +struct Message +{ + data @0 : UInt16; + date32 @1 : Int32; + datetime @2 : UInt32; + datatime64 @3 : Int64; +} +Strings + +struct Message +{ + string @0 : Data; + fixedstring @1 : Data; +} +Special + +struct Message +{ + ipv4 @0 : UInt32; + ipv6 @1 : Data; + uuid @2 : Data; +} +Nullable + +struct Message +{ + struct Nullable + { + union + { + value @0 : UInt32; + null @1 : Void; + } + } + nullable @0 : Nullable; +} +Enums + +struct Message +{ + enum Enum8 + { + v1 @0; + v2 @1; + v3 @2; + v4 @3; + } + enum8 @0 : Enum8; + enum Enum16 + { + v5 @0; + v6 @1; + v7 @2; + v8 @3; + v9 @4; + } + enum16 @1 : Enum16; +} +Arrays + +struct Message +{ + arr1 @0 : List(UInt32); + arr2 @1 : List(List(List(UInt32))); +} +Tuples + +struct Message +{ + struct Tuple1 + { + e1 @0 : UInt32; + e2 @1 : Data; + e3 @2 : UInt32; + } + tuple1 @0 : Tuple1; + struct Tuple2 + { + struct E1 + { + e1 @0 : UInt32; + struct E2 + { + e1 @0 : Data; + e2 @1 : UInt32; + } + e2 @1 : E2; + e3 @2 : Data; + } + e1 @0 : E1; + struct E2 + { + e1 @0 : Data; + e2 @1 : UInt32; + } + e2 @1 : E2; + } + tuple2 @1 : Tuple2; +} +Maps + +struct Message +{ + struct Map1 + { + struct Entry + { + key @0 : Data; + value @1 : UInt32; + } + entries @0 : List(Entry); + } + map1 @0 : Map1; + struct Map2 + { + struct Entry + { + struct Value + { + struct Entry + { + struct Value + { + struct Entry + { + key @0 : Data; + value @1 : UInt32; + } + entries @0 : List(Entry); + } + key @0 : Data; + value @1 : Value; + } + entries @0 : List(Entry); + } + key @0 : Data; + value @1 : Value; + } + entries @0 : List(Entry); + } + map2 @1 : Map2; +} +Complex + +struct Message +{ + struct C1 + { + struct E1 + { + struct Entry + { + struct Value + { + union + { + value @0 : UInt32; + null @1 : Void; + } + } + key @0 : Data; + value @1 : List(List(Value)); + } + entries @0 : List(Entry); + } + e1 @0 : List(E1); + struct E2 + { + struct Entry + { + struct Value + { + struct E1 + { + union + { + value @0 : Data; + null @1 : Void; + } + } + e1 @0 : List(List(E1)); + struct E2 + { + e1 @0 : UInt32; + struct E2 + { + struct E1 + { + union + { + value @0 : Data; + null @1 : Void; + } + } + e1 @0 : List(List(E1)); + e2 @1 : UInt32; + } + e2 @1 : E2; + } + e2 @1 : List(E2); + } + key @0 : Data; + value @1 : Value; + } + entries @0 : List(Entry); + } + e2 @1 : List(E2); + } + c1 @0 : C1; +} +Read/write with no schema +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +Output schema +@0xda5297bc0d5b501a; + +struct Message +{ + number @0 : UInt64; +} +Bad output schema path +2 +2 +Protobuf +Numbers + +message Message +{ + int32 int8 = 1; + uint32 uint8 = 2; + int32 int16 = 3; + uint32 uint16 = 4; + int32 int32 = 5; + uint32 uint32 = 6; + int64 int64 = 7; + uint64 uint64 = 8; + bytes int128 = 9; + bytes uint128 = 10; + bytes int256 = 11; + bytes uint256 = 12; + float float32 = 13; + double float64 = 14; + bytes decimal32 = 15; + bytes decimal64 = 16; + bytes decimal128 = 17; + bytes decimal256 = 18; +} +Dates + +message Message +{ + uint32 data = 1; + int32 date32 = 2; + uint32 datetime = 3; + uint64 datatime64 = 4; +} +Strings + +message Message +{ + bytes string = 1; + bytes fixedstring = 2; +} +Special + +message Message +{ + uint32 ipv4 = 1; + bytes ipv6 = 2; + bytes uuid = 3; +} +Nullable + +message Message +{ + uint32 nullable = 1; +} +Enums + +message Message +{ + enum Enum8 + { + v1 = 0; + v2 = 1; + v3 = 2; + v4 = 3; + } + Enum8 enum8 = 1; + enum Enum16 + { + v5 = 0; + v6 = 1; + v7 = 2; + v8 = 3; + v9 = 4; + } + Enum16 enum16 = 2; +} +Arrays + +message Message +{ + repeated uint32 arr1 = 1; + message Arr2 + { + message Arr2 + { + repeated uint32 arr2 = 1; + } + repeated Arr2 arr2 = 1; + } + repeated Arr2 arr2 = 2; +} +Tuples + +message Message +{ + message Tuple1 + { + uint32 e1 = 1; + bytes e2 = 2; + uint32 e3 = 3; + } + Tuple1 tuple1 = 1; + message Tuple2 + { + message E1 + { + uint32 e1 = 1; + message E2 + { + bytes e1 = 1; + uint32 e2 = 2; + } + E2 e2 = 2; + bytes e3 = 3; + } + E1 e1 = 1; + message E2 + { + bytes e1 = 1; + uint32 e2 = 2; + } + E2 e2 = 2; + } + Tuple2 tuple2 = 2; +} +Maps + +message Message +{ + map map1 = 1; + message Map2Value + { + message Map2ValueValue + { + map map2ValueValue = 1; + } + map map2Value = 1; + } + map map2 = 2; +} +Complex + +message Message +{ + message C1 + { + message E1 + { + message E1Value + { + message E1Value + { + repeated uint32 e1Value = 1; + } + repeated E1Value e1Value = 1; + } + map e1 = 1; + } + repeated E1 e1 = 1; + message E2 + { + message E2Value + { + message E1 + { + repeated bytes e1 = 1; + } + repeated E1 e1 = 1; + message E2 + { + uint32 e1 = 1; + message E2 + { + message E1 + { + repeated bytes e1 = 1; + } + repeated E1 e1 = 1; + uint32 e2 = 2; + } + E2 e2 = 2; + } + repeated E2 e2 = 2; + } + map e2 = 1; + } + repeated E2 e2 = 2; + } + C1 c1 = 1; +} +Read/write with no schema +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +Output schema +syntax = "proto3"; + +message Message +{ + uint64 number = 1; +} +Bad output schema path +2 +2 diff --git a/tests/queries/0_stateless/02817_structure_to_schema.reference b/tests/queries/0_stateless/02817_structure_to_schema.reference index a1aed3f171f..1f39a8ed50e 100644 --- a/tests/queries/0_stateless/02817_structure_to_schema.reference +++ b/tests/queries/0_stateless/02817_structure_to_schema.reference @@ -1,6 +1,5 @@ CapnProto Numbers -@0xfcfddd851150d4a7; struct Message { @@ -24,7 +23,6 @@ struct Message decimal256 @17 : Data; } Dates -@0xf6c4d408fe41b545; struct Message { @@ -34,7 +32,6 @@ struct Message datatime64 @3 : Int64; } Strings -@0xaa8865faa0622d35; struct Message { @@ -42,7 +39,6 @@ struct Message fixedstring @1 : Data; } Special -@0xc4a88da5454bdc56; struct Message { @@ -51,7 +47,6 @@ struct Message uuid @2 : Data; } Nullable -@0xec817ea81ba08bcf; struct Message { @@ -66,7 +61,6 @@ struct Message nullable @0 : Nullable; } Enums -@0x9d57ed1a12d87f28; struct Message { @@ -89,7 +83,6 @@ struct Message enum16 @1 : Enum16; } Arrays -@0xc10c035fde2e533a; struct Message { @@ -97,7 +90,6 @@ struct Message arr2 @1 : List(List(List(UInt32))); } Tuples -@0xfbb3d48432e54b68; struct Message { @@ -132,7 +124,6 @@ struct Message tuple2 @1 : Tuple2; } Maps -@0xfdb6e3129839665e; struct Message { @@ -176,7 +167,6 @@ struct Message map2 @1 : Map2; } Complex -@0x8bfcfe1141c8e85c; struct Message { @@ -256,7 +246,6 @@ Read/write with no schema 8 9 Output schema -@0xa43428a60b94a646; struct Message { @@ -267,7 +256,6 @@ Bad output schema path 2 Protobuf Numbers -syntax = "proto3"; message Message { @@ -291,7 +279,6 @@ message Message bytes decimal256 = 18; } Dates -syntax = "proto3"; message Message { @@ -301,7 +288,6 @@ message Message uint64 datatime64 = 4; } Strings -syntax = "proto3"; message Message { @@ -309,7 +295,6 @@ message Message bytes fixedstring = 2; } Special -syntax = "proto3"; message Message { @@ -318,14 +303,12 @@ message Message bytes uuid = 3; } Nullable -syntax = "proto3"; message Message { uint32 nullable = 1; } Enums -syntax = "proto3"; message Message { @@ -348,7 +331,6 @@ message Message Enum16 enum16 = 2; } Arrays -syntax = "proto3"; message Message { @@ -364,7 +346,6 @@ message Message repeated Arr2 arr2 = 2; } Tuples -syntax = "proto3"; message Message { @@ -399,7 +380,6 @@ message Message Tuple2 tuple2 = 2; } Maps -syntax = "proto3"; message Message { @@ -415,7 +395,6 @@ message Message map map2 = 2; } Complex -syntax = "proto3"; message Message { @@ -477,7 +456,6 @@ Read/write with no schema 8 9 Output schema -syntax = "proto3"; message Message { diff --git a/tests/queries/0_stateless/02817_structure_to_schema.sh b/tests/queries/0_stateless/02817_structure_to_schema.sh index 76c5b1a1d85..3b79fa099a8 100755 --- a/tests/queries/0_stateless/02817_structure_to_schema.sh +++ b/tests/queries/0_stateless/02817_structure_to_schema.sh @@ -15,7 +15,7 @@ function test_structure() structure=$3 $CLICKHOUSE_LOCAL -q "select structureTo${format}Schema('$structure') format TSVRaw" > $SCHEMA_FILE.$ext - cat $SCHEMA_FILE.$ext + tail -n +2 $SCHEMA_FILE.$ext $CLICKHOUSE_LOCAL -q "select * from generateRandom('$structure', 42) limit 10 format $format settings format_schema='$SCHEMA_FILE:Message', format_capn_proto_enum_comparising_mode='by_names'" > $DATA_FILE $CLICKHOUSE_LOCAL -q "select * from file('$DATA_FILE', $format, '$structure') format Null settings format_schema='$SCHEMA_FILE:Message', format_capn_proto_enum_comparising_mode='by_names'" @@ -75,7 +75,7 @@ function test_format() echo "Output schema" $CLICKHOUSE_LOCAL -q "select * from numbers(10) format $format settings output_format_schema='$SCHEMA_FILE.$ext'" > $DATA_FILE - cat $SCHEMA_FILE.$ext + tail -n +2 $SCHEMA_FILE.$ext echo "Bad output schema path" $CLICKHOUSE_CLIENT -q "insert into function file('$DATA_FILE', $format) select * from numbers(10) settings output_format_schema='/tmp/schema.$ext'" 2>&1 | grep "BAD_ARGUMENTS" -c From 3c9e46b557a882085fdcdce5d74ad12329457db3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 16:19:12 +0300 Subject: [PATCH 0924/2047] Update ci-slack-bot.py --- utils/ci-slack-bot/ci-slack-bot.py | 43 ++++++++++++++++++++++++++---- 1 file changed, 38 insertions(+), 5 deletions(-) diff --git a/utils/ci-slack-bot/ci-slack-bot.py b/utils/ci-slack-bot/ci-slack-bot.py index 6e694b4fdbd..0fb12e89ce9 100755 --- a/utils/ci-slack-bot/ci-slack-bot.py +++ b/utils/ci-slack-bot/ci-slack-bot.py @@ -26,10 +26,11 @@ else: DRY_RUN_MARK = "" -MAX_FAILURES_DEFAULT = 40 +MAX_FAILURES_DEFAULT = 30 SLACK_URL_DEFAULT = DRY_RUN_MARK -FLAKY_ALERT_PROBABILITY = 0.20 +FLAKY_ALERT_PROBABILITY = 0.50 +REPORT_NO_FAILURES_PROBABILITY = 0.99 MAX_TESTS_TO_REPORT = 4 @@ -89,6 +90,22 @@ WHERE 1 AND check_name ILIKE check_name_pattern """ +# Returns percentage of failed checks (once per day, at noon) +FAILED_CHECKS_PERCENTAGE_QUERY = """ +SELECT if(toHour(now('Europe/Amsterdam')) = 12, v, 0) +FROM +( + SELECT + countDistinctIf((commit_sha, check_name), (test_status LIKE 'F%') AND (check_status != 'success')) + / countDistinct((commit_sha, check_name)) AS v + FROM checks + WHERE 1 + AND (pull_request_number = 0) + AND (test_status != 'SKIPPED') + AND (check_start_time > (now() - toIntervalDay(1))) +) +""" + # It shows all recent failures of the specified test (helps to find when it started) ALL_RECENT_FAILURES_QUERY = """ WITH @@ -202,9 +219,9 @@ def get_too_many_failures_message_impl(failures_count): curr_failures = int(failures_count[0][0]) prev_failures = int(failures_count[0][1]) if curr_failures == 0 and prev_failures != 0: - return ( - "Looks like CI is completely broken: there are *no failures* at all... 0_o" - ) + if random.random() < REPORT_NO_FAILURES_PROBABILITY: + return None + return "Wow, there are *no failures* at all... 0_o" if curr_failures < MAX_FAILURES: return None if prev_failures < MAX_FAILURES: @@ -227,6 +244,19 @@ def get_too_many_failures_message(failures_count): return msg +def get_failed_checks_percentage_message(percentage): + p = percentage[0][0] * 100 + + # Always report more than 1% of failed checks + # For <= 1%: higher percentage of failures == higher probability + if p <= random.random(): + return None + + msg = ":alert: " if p > 1 else "Only " if p < 0.5 else "" + msg += "*{0:.2f}%* of all checks in master have failed yesterday".format(p) + return msg + + def split_slack_message(long_message): lines = long_message.split("\n") messages = [] @@ -280,6 +310,9 @@ def query_and_alert_if_needed(query, get_message_func): def check_and_alert(): query_and_alert_if_needed(NEW_BROKEN_TESTS_QUERY, get_new_broken_tests_message) query_and_alert_if_needed(COUNT_FAILURES_QUERY, get_too_many_failures_message) + query_and_alert_if_needed( + FAILED_CHECKS_PERCENTAGE_QUERY, get_failed_checks_percentage_message + ) def lambda_handler(event, context): From 8649c84461f3c27bdf9fcab4db1884b21603dc2e Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 20 Jul 2023 13:28:37 +0000 Subject: [PATCH 0925/2047] Remove conditional linking --- utils/config-processor/CMakeLists.txt | 6 +----- utils/keeper-bench/CMakeLists.txt | 6 +----- 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt index 4394083a1c3..80c3535ef4e 100644 --- a/utils/config-processor/CMakeLists.txt +++ b/utils/config-processor/CMakeLists.txt @@ -1,6 +1,2 @@ clickhouse_add_executable (config-processor config-processor.cpp) -if (ENABLE_SSL) - target_link_libraries(config-processor PRIVATE dbms) -else () - target_link_libraries(config-processor PRIVATE clickhouse_common_config_no_zookeeper_log) -endif () +target_link_libraries(config-processor PRIVATE dbms) diff --git a/utils/keeper-bench/CMakeLists.txt b/utils/keeper-bench/CMakeLists.txt index e8daec9e164..5514c34f4ef 100644 --- a/utils/keeper-bench/CMakeLists.txt +++ b/utils/keeper-bench/CMakeLists.txt @@ -4,9 +4,5 @@ if (NOT TARGET ch_contrib::rapidjson) endif () clickhouse_add_executable(keeper-bench Generator.cpp Runner.cpp Stats.cpp main.cpp) -if (ENABLE_SSL) - target_link_libraries(keeper-bench PRIVATE dbms) -else () - target_link_libraries(keeper-bench PRIVATE clickhouse_common_config_no_zookeeper_log) -endif () +target_link_libraries(keeper-bench PRIVATE dbms) target_link_libraries(keeper-bench PRIVATE ch_contrib::rapidjson) From f997adfe27e1bd3bb772857fb11fae962c373b9e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 20 Jul 2023 14:02:55 +0000 Subject: [PATCH 0926/2047] Retry if sessions not closed because missing leader --- src/Coordination/KeeperDispatcher.cpp | 37 +++++++++------- tests/integration/test_keeper_session/test.py | 42 ++++++++++++++----- 2 files changed, 54 insertions(+), 25 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9d9df5c7f30..dfb621eb0ad 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -473,23 +473,30 @@ void KeeperDispatcher::shutdown() session_to_response_callback.clear(); } - // if there is no leader, there is no reason to do CLOSE because it's a write request - if (server && hasLeader() && !close_requests.empty()) + if (server && !close_requests.empty()) { - LOG_INFO(log, "Trying to close {} session(s)", close_requests.size()); - const auto raft_result = server->putRequestBatch(close_requests); - auto sessions_closing_done_promise = std::make_shared>(); - auto sessions_closing_done = sessions_closing_done_promise->get_future(); - raft_result->when_ready([my_sessions_closing_done_promise = std::move(sessions_closing_done_promise)]( - nuraft::cmd_result> & /*result*/, - nuraft::ptr & /*exception*/) { my_sessions_closing_done_promise->set_value(); }); + // if there is no leader, there is no reason to do CLOSE because it's a write request + if (hasLeader()) + { + LOG_INFO(log, "Trying to close {} session(s)", close_requests.size()); + const auto raft_result = server->putRequestBatch(close_requests); + auto sessions_closing_done_promise = std::make_shared>(); + auto sessions_closing_done = sessions_closing_done_promise->get_future(); + raft_result->when_ready([my_sessions_closing_done_promise = std::move(sessions_closing_done_promise)]( + nuraft::cmd_result> & /*result*/, + nuraft::ptr & /*exception*/) { my_sessions_closing_done_promise->set_value(); }); - auto session_shutdown_timeout = configuration_and_settings->coordination_settings->session_shutdown_timeout.totalMilliseconds(); - if (sessions_closing_done.wait_for(std::chrono::milliseconds(session_shutdown_timeout)) != std::future_status::ready) - LOG_WARNING( - log, - "Failed to close sessions in {}ms. If they are not closed, they will be closed after session timeout.", - session_shutdown_timeout); + auto session_shutdown_timeout = configuration_and_settings->coordination_settings->session_shutdown_timeout.totalMilliseconds(); + if (sessions_closing_done.wait_for(std::chrono::milliseconds(session_shutdown_timeout)) != std::future_status::ready) + LOG_WARNING( + log, + "Failed to close sessions in {}ms. If they are not closed, they will be closed after session timeout.", + session_shutdown_timeout); + } + else + { + LOG_INFO(log, "Sessions cannot be closed during shutdown because there is no active leader"); + } } if (server) diff --git a/tests/integration/test_keeper_session/test.py b/tests/integration/test_keeper_session/test.py index e57057a8258..68147865cd2 100644 --- a/tests/integration/test_keeper_session/test.py +++ b/tests/integration/test_keeper_session/test.py @@ -6,6 +6,7 @@ import socket import struct from kazoo.client import KazooClient +from kazoo.exceptions import NoNodeError # from kazoo.protocol.serialization import Connect, read_buffer, write_buffer @@ -162,17 +163,38 @@ def test_session_timeout(started_cluster): def test_session_close_shutdown(started_cluster): wait_nodes() - node1_zk = get_fake_zk(node1.name) - node2_zk = get_fake_zk(node2.name) + node1_zk = None + node2_zk = None + for i in range(20): + node1_zk = get_fake_zk(node1.name) + node2_zk = get_fake_zk(node2.name) - eph_node = "/test_node" - node2_zk.create(eph_node, ephemeral=True) - node1_zk.sync(eph_node) - assert node1_zk.exists(eph_node) != None + eph_node = "/test_node" + node2_zk.create(eph_node, ephemeral=True) + node1_zk.sync(eph_node) - # shutdown while session is active - node2.stop_clickhouse() + node1_zk.exists(eph_node) != None - assert node1_zk.exists(eph_node) == None + # restart while session is active so it's closed during shutdown + node2.restart_clickhouse() - node2.start_clickhouse() + if node1_zk.exists(eph_node) == None: + break + + assert node2.contains_in_log("Sessions cannot be closed during shutdown because there is no active leader") + + try: + node1_zk.delete(eph_node) + except NoNodeError: + pass + + assert node1_zk.exists(eph_node) == None + + destroy_zk_client(node1_zk) + node1_zk = None + destroy_zk_client(node2_zk) + node2_zk = None + + time.sleep(1) + else: + assert False, "Session wasn't properly cleaned up on shutdown" \ No newline at end of file From 5decb1f5c555d2465724f9bc3c555c157f9deb81 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 20 Jul 2023 14:11:11 +0000 Subject: [PATCH 0927/2047] Automatic style fix --- tests/integration/test_keeper_session/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_session/test.py b/tests/integration/test_keeper_session/test.py index 68147865cd2..cd012ad6e9e 100644 --- a/tests/integration/test_keeper_session/test.py +++ b/tests/integration/test_keeper_session/test.py @@ -181,7 +181,9 @@ def test_session_close_shutdown(started_cluster): if node1_zk.exists(eph_node) == None: break - assert node2.contains_in_log("Sessions cannot be closed during shutdown because there is no active leader") + assert node2.contains_in_log( + "Sessions cannot be closed during shutdown because there is no active leader" + ) try: node1_zk.delete(eph_node) @@ -197,4 +199,4 @@ def test_session_close_shutdown(started_cluster): time.sleep(1) else: - assert False, "Session wasn't properly cleaned up on shutdown" \ No newline at end of file + assert False, "Session wasn't properly cleaned up on shutdown" From 046bf55dc084d4df91ecfddb8e22aa6f9300fa43 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 20 Jul 2023 14:17:33 +0000 Subject: [PATCH 0928/2047] Incorporate feedback --- .../functions/arithmetic-functions.md | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 054c59d5778..69f1816b7df 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -6,9 +6,20 @@ sidebar_label: Arithmetic # Arithmetic Functions -The result type of all arithmetic functions is the smallest type which can represent all possible results. Size promotion happens for integers up to 32 bit, e.g. `UInt8 + UInt16 = UInt32`. If one of the inters has 64 or more bits, the result is of the same type as the bigger of the input integers, e.g. `UInt16 + UInt128 = UInt128`. While this introduces a risk of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of 64 bit. Also, this behavior guarantees compatibility with many other databases which provide 64 bit integers (BIGINT) as the biggest integer type. +Arithmetic functions work for any two operands of type `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`, `Float32`, or `Float64`. -The result of addition or multiplication of two integers is unsigned unless one of the integers is signed. +Before performing the operation, both operands are casted to the result type. The result type is determined as follows (unless specified +differently in the function documentation below): +- If both operands are up to 32 bits wide, the size of the result type will be the size of the next bigger type following the bigger of the + two operands (integer size promotion). For example, `UInt8 + UInt16 = UInt32` or `Float32 * Float32 = Float64`. +- If one of the operands has 64 or more bits, the size of the result type will be the same size as the bigger of the two operands. For + example, `UInt32 + UInt128 = UInt128` or `Float32 * Float64 = Float64`. +- If one of the operands is signed, the result type will also be signed, otherwise it will be signed. For example, `UInt32 * Int32 = Int64`. + +These rules make sure that the result type will be the smallest type which can represent all possible results. While this introduces a risk +of overflows around the value range boundary, it ensures that calculations are performed quickly using the maximum native integer width of +64 bit. This behavior also guarantees compatibility with many other databases which provide 64 bit integers (BIGINT) as the biggest integer +type. Example: @@ -22,8 +33,6 @@ SELECT toTypeName(0), toTypeName(0 + 0), toTypeName(0 + 0 + 0), toTypeName(0 + 0 └───────────────┴────────────────────────┴─────────────────────────────────┴──────────────────────────────────────────┘ ``` -Arithmetic functions work for any pair of `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`, `Float32`, or `Float64` values. - Overflows are produced the same way as in C++. ## plus @@ -68,7 +77,7 @@ Alias: `a \* b` (operator) ## divide -Calculates the quotient of two values `a` and `b`. The result is always a floating-point value. If you need integer division, you can use the `intDiv` function. +Calculates the quotient of two values `a` and `b`. The result type is always [Float64](../../sql-reference/data-types/float.md). Integer division is provided by the `intDiv` function. Division by 0 returns `inf`, `-inf`, or `nan`. @@ -84,7 +93,7 @@ Alias: `a / b` (operator) Performs an integer division of two values `a` by `b`, i.e. computes the quotient rounded down to the next smallest integer. -The result has the same type as the dividend (the first parameter). +The result has the same width as the dividend (the first parameter). An exception is thrown when dividing by zero, when the quotient does not fit in the range of the dividend, or when dividing a minimal negative number by minus one. @@ -135,7 +144,7 @@ intDivOrZero(a, b) Calculates the remainder of the division of two values `a` by `b`. -The result type is an integer if both inputs are integers. If one of the inputs is a floating-point number, the result is a floating-point number. +The result type is an integer if both inputs are integers. If one of the inputs is a floating-point number, the result type is [Float64](../../sql-reference/data-types/float.md). The remainder is computed like in C++. Truncated division is used for negative numbers. From 8adf57a6981610936acc84f3c69342682952ff0a Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 20 Jul 2023 14:18:32 +0000 Subject: [PATCH 0929/2047] Fix text in comments and improve exception handling --- src/Common/examples/encrypt_decrypt.cpp | 2 +- tests/integration/test_config_decryption/test_wrong_settings.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/examples/encrypt_decrypt.cpp b/src/Common/examples/encrypt_decrypt.cpp index 542e173deb9..2d8c5a5f61f 100644 --- a/src/Common/examples/encrypt_decrypt.cpp +++ b/src/Common/examples/encrypt_decrypt.cpp @@ -3,7 +3,7 @@ #include #include -/** This test program encrypts or decrypts text values using AES_128_GCM_SIV or AES_256_GCM_SIV codecs. +/** This test program encrypts or decrypts text values using a symmetric encryption codec like AES_128_GCM_SIV or AES_256_GCM_SIV. * Keys for codecs are loaded from section of configuration file. * * How to use: diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index e86f7fa9b39..e0fbd4b2948 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -10,7 +10,7 @@ def start_clickhouse(config, err_msg): cluster.start() except Exception as e: caught_exception = str(e) - assert caught_exception.find(err_msg) != -1 + assert err_msg in caught_exception def test_wrong_method(): From e467264588a6435199879fd89d1dc995c9e37c63 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 17:56:30 +0300 Subject: [PATCH 0930/2047] Update src/IO/HTTPCommon.cpp --- src/IO/HTTPCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index a5816911c09..ddd7ccbe483 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -81,7 +81,7 @@ namespace Session::close(); LOG_TRACE( log, - "Last ip ({}) is unreachable for {}:{}. Will try another resolved address.", + "Last ip ({}) is unreachable for {}:{}. Will try another resolved address.", Session::getResolvedHost(), Session::getHost(), Session::getPort()); From c0aa3e456705e3ef75ed09683f4e9ed6d9151917 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 17:59:43 +0300 Subject: [PATCH 0931/2047] Update ci-slack-bot.py --- utils/ci-slack-bot/ci-slack-bot.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/ci-slack-bot/ci-slack-bot.py b/utils/ci-slack-bot/ci-slack-bot.py index 0fb12e89ce9..ea883e3cda3 100755 --- a/utils/ci-slack-bot/ci-slack-bot.py +++ b/utils/ci-slack-bot/ci-slack-bot.py @@ -245,7 +245,7 @@ def get_too_many_failures_message(failures_count): def get_failed_checks_percentage_message(percentage): - p = percentage[0][0] * 100 + p = float(percentage[0][0]) * 100 # Always report more than 1% of failed checks # For <= 1%: higher percentage of failures == higher probability From ea252e2f612afd9e83c1aa000af945eebbe18a16 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Jul 2023 15:05:07 +0000 Subject: [PATCH 0932/2047] Disable analyzer setting in backward_compatibility integration tests. --- tests/integration/helpers/cluster.py | 6 +++++- tests/integration/test_backward_compatibility/test.py | 2 ++ .../test_aggregate_fixed_key.py | 5 +++-- .../test_aggregate_function_state.py | 6 ++++-- .../test_backward_compatibility/test_convert_ordinary.py | 1 + .../test_backward_compatibility/test_cte_distributed.py | 7 ++++--- .../test_data_skipping_indices.py | 1 + .../test_backward_compatibility/test_functions.py | 3 ++- .../test_in_memory_parts_still_read.py | 1 + .../test_insert_profile_events.py | 3 ++- .../test_ip_types_binary_compatibility.py | 1 + .../test_memory_bound_aggregation.py | 4 +++- .../test_normalized_count_comparison.py | 3 ++- .../test_select_aggregate_alias_column.py | 3 ++- .../test_short_strings_aggregation.py | 4 +++- .../test_vertical_merges_from_compact_parts.py | 2 ++ 16 files changed, 38 insertions(+), 14 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c52442ecb9c..c85fbb8ad9e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1533,6 +1533,7 @@ class ClickHouseCluster: with_jdbc_bridge=False, with_hive=False, with_coredns=False, + allow_analyzer=True, hostname=None, env_variables=None, image="clickhouse/integration-test", @@ -1630,6 +1631,7 @@ class ClickHouseCluster: with_hive=with_hive, with_coredns=with_coredns, with_cassandra=with_cassandra, + allow_analyzer=allow_analyzer, server_bin_path=self.server_bin_path, odbc_bridge_bin_path=self.odbc_bridge_bin_path, library_bridge_bin_path=self.library_bridge_bin_path, @@ -3169,6 +3171,7 @@ class ClickHouseInstance: with_hive, with_coredns, with_cassandra, + allow_analyzer, server_bin_path, odbc_bridge_bin_path, library_bridge_bin_path, @@ -3256,6 +3259,7 @@ class ClickHouseInstance: self.with_hive = with_hive self.with_coredns = with_coredns self.coredns_config_dir = p.abspath(p.join(base_path, "coredns_config")) + self.allow_analyzer = allow_analyzer self.main_config_name = main_config_name self.users_config_name = users_config_name @@ -4227,7 +4231,7 @@ class ClickHouseInstance: ) write_embedded_config("0_common_instance_users.xml", users_d_dir) - if os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None: + if os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None and self.allow_analyzer: write_embedded_config("0_common_enable_analyzer.xml", users_d_dir) if len(self.custom_dictionaries_paths): diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index ea1d3ab9c07..c3d3b8aad34 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -10,11 +10,13 @@ node1 = cluster.add_instance( tag="19.17.8.54", stay_alive=True, with_installed_binary=True, + allow_analyzer=False ) node2 = cluster.add_instance( "node2", main_configs=["configs/wide_parts_only.xml", "configs/no_compress_marks.xml"], with_zookeeper=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index 01c9736c354..cf258987cbf 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -9,9 +9,10 @@ node1 = cluster.add_instance( image="yandex/clickhouse-server", tag="21.3", with_installed_binary=True, + allow_analyzer=False, ) -node2 = cluster.add_instance("node2", with_zookeeper=True) -node3 = cluster.add_instance("node3", with_zookeeper=True) +node2 = cluster.add_instance("node2", with_zookeeper=True, allow_analyzer=False) +node3 = cluster.add_instance("node3", with_zookeeper=True, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_aggregate_function_state.py b/tests/integration/test_backward_compatibility/test_aggregate_function_state.py index 1f6d405603a..3a936239cc8 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_function_state.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_function_state.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", @@ -18,9 +19,10 @@ node2 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) -node3 = cluster.add_instance("node3", with_zookeeper=False) -node4 = cluster.add_instance("node4", with_zookeeper=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) +node4 = cluster.add_instance("node4", with_zookeeper=False, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_convert_ordinary.py b/tests/integration/test_backward_compatibility/test_convert_ordinary.py index 8b1afd358eb..36facdd59b1 100644 --- a/tests/integration/test_backward_compatibility/test_convert_ordinary.py +++ b/tests/integration/test_backward_compatibility/test_convert_ordinary.py @@ -9,6 +9,7 @@ node = cluster.add_instance( stay_alive=True, with_zookeeper=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_cte_distributed.py b/tests/integration/test_backward_compatibility/test_cte_distributed.py index 7ea0d2d9f21..c68468aad75 100644 --- a/tests/integration/test_backward_compatibility/test_cte_distributed.py +++ b/tests/integration/test_backward_compatibility/test_cte_distributed.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,6 +11,7 @@ node2 = cluster.add_instance( tag="21.7.3.14", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) @@ -31,7 +32,7 @@ WITH quantile(0.05)(cnt) as p05, quantile(0.95)(cnt) as p95, p95 - p05 as inter_percentile_range -SELECT +SELECT sum(cnt) as total_requests, count() as data_points, inter_percentile_range @@ -49,7 +50,7 @@ WITH quantile(0.05)(cnt) as p05, quantile(0.95)(cnt) as p95, p95 - p05 as inter_percentile_range -SELECT +SELECT sum(cnt) as total_requests, count() as data_points, inter_percentile_range diff --git a/tests/integration/test_backward_compatibility/test_data_skipping_indices.py b/tests/integration/test_backward_compatibility/test_data_skipping_indices.py index c65dc6d3841..46ab27d2ab0 100644 --- a/tests/integration/test_backward_compatibility/test_data_skipping_indices.py +++ b/tests/integration/test_backward_compatibility/test_data_skipping_indices.py @@ -12,6 +12,7 @@ node = cluster.add_instance( tag="21.6", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index afb19901e74..fa24b146fec 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -9,7 +9,7 @@ from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) -upstream = cluster.add_instance("upstream") +upstream = cluster.add_instance("upstream", allow_analyzer=False) backward = cluster.add_instance( "backward", image="clickhouse/clickhouse-server", @@ -19,6 +19,7 @@ backward = cluster.add_instance( # Affected at least: singleValueOrNull, last_value, min, max, any, anyLast, anyHeavy, first_value, argMin, argMax tag="22.6", with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py b/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py index d55f155918e..cd67f1f6344 100644 --- a/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py +++ b/tests/integration/test_backward_compatibility/test_in_memory_parts_still_read.py @@ -12,6 +12,7 @@ node = cluster.add_instance( tag="23.4", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_insert_profile_events.py b/tests/integration/test_backward_compatibility/test_insert_profile_events.py index 0fd453e57d4..8564c6b5952 100644 --- a/tests/integration/test_backward_compatibility/test_insert_profile_events.py +++ b/tests/integration/test_backward_compatibility/test_insert_profile_events.py @@ -7,12 +7,13 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -upstream_node = cluster.add_instance("upstream_node") +upstream_node = cluster.add_instance("upstream_node", allow_analyzer=False) old_node = cluster.add_instance( "old_node", image="clickhouse/clickhouse-server", tag="22.5.1.2079", with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py b/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py index bb40dff27ac..04016755a24 100644 --- a/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py +++ b/tests/integration/test_backward_compatibility/test_ip_types_binary_compatibility.py @@ -10,6 +10,7 @@ node_22_6 = cluster.add_instance( tag="22.6", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py index d76c4eba409..96b41c81384 100644 --- a/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_memory_bound_aggregation.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( tag="21.1", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", @@ -18,8 +19,9 @@ node2 = cluster.add_instance( tag="21.1", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) -node3 = cluster.add_instance("node3", with_zookeeper=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py index fcdedd29dad..3cd708d5029 100644 --- a/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py +++ b/tests/integration/test_backward_compatibility/test_normalized_count_comparison.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,6 +11,7 @@ node2 = cluster.add_instance( tag="21.7.2.7", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py index 8bdae54a889..7e10b6ab430 100644 --- a/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py +++ b/tests/integration/test_backward_compatibility/test_select_aggregate_alias_column.py @@ -3,7 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance("node1", with_zookeeper=False) +node1 = cluster.add_instance("node1", with_zookeeper=False, allow_analyzer=False) node2 = cluster.add_instance( "node2", with_zookeeper=False, @@ -11,6 +11,7 @@ node2 = cluster.add_instance( tag="21.7.2.7", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py index 17a7282b7b5..e4fda618031 100644 --- a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py @@ -10,6 +10,7 @@ node1 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", @@ -18,8 +19,9 @@ node2 = cluster.add_instance( tag="19.16.9.37", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) -node3 = cluster.add_instance("node3", with_zookeeper=False) +node3 = cluster.add_instance("node3", with_zookeeper=False, allow_analyzer=False) @pytest.fixture(scope="module") diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index 3d006caad0d..82ffcc20b60 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -11,12 +11,14 @@ node_old = cluster.add_instance( stay_alive=True, with_installed_binary=True, with_zookeeper=True, + allow_analyzer=False, ) node_new = cluster.add_instance( "node2", main_configs=["configs/no_compress_marks.xml"], with_zookeeper=True, stay_alive=True, + allow_analyzer=False, ) From a26de1b370e8c09c548528ffbe3337cbf2340012 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 20 Jul 2023 15:12:55 +0000 Subject: [PATCH 0933/2047] Automatic style fix --- tests/integration/helpers/cluster.py | 5 ++++- tests/integration/test_backward_compatibility/test.py | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c85fbb8ad9e..0ac2f330b1e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4231,7 +4231,10 @@ class ClickHouseInstance: ) write_embedded_config("0_common_instance_users.xml", users_d_dir) - if os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None and self.allow_analyzer: + if ( + os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None + and self.allow_analyzer + ): write_embedded_config("0_common_enable_analyzer.xml", users_d_dir) if len(self.custom_dictionaries_paths): diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index c3d3b8aad34..6f21b184a95 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -10,7 +10,7 @@ node1 = cluster.add_instance( tag="19.17.8.54", stay_alive=True, with_installed_binary=True, - allow_analyzer=False + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", From e6624a07e4fe938b55dd6bc5d8cbabd0ed93d2d7 Mon Sep 17 00:00:00 2001 From: AlexBykovski Date: Thu, 20 Jul 2023 18:54:48 +0300 Subject: [PATCH 0934/2047] Update build-osx.md syntax error in command for compiler for OSx compilation --- docs/ru/development/build-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/development/build-osx.md b/docs/ru/development/build-osx.md index 9a1f9c9347d..6b4e612b13f 100644 --- a/docs/ru/development/build-osx.md +++ b/docs/ru/development/build-osx.md @@ -68,7 +68,7 @@ $ /bin/bash -c "$(curl -fsSL https://raw.githubusercontent.com/Homebrew/install/ $ rm -rf build $ mkdir build $ cd build - $ cmake -DCMAKE_C_COMPILER=$(brew --prefix llvm)/bin/clang -DCMAKE_CXX_COMPILER==$(brew --prefix llvm)/bin/clang++ -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. + $ cmake -DCMAKE_C_COMPILER=$(brew --prefix llvm)/bin/clang -DCMAKE_CXX_COMPILER=$(brew --prefix llvm)/bin/clang++ -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. $ cmake -DCMAKE_C_COMPILER=$(brew --prefix llvm)/bin/clang -DCMAKE_CXX_COMPILER=$(brew --prefix llvm)/bin/clang++ -DCMAKE_BUILD_TYPE=RelWithDebInfo -DENABLE_JEMALLOC=OFF .. $ cmake --build . --config RelWithDebInfo $ cd .. From 97e54d6ebaa174f8d2ae291ddec20fd879b29bfa Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 19 Jul 2023 07:13:25 +0000 Subject: [PATCH 0935/2047] Fix test_backup_restore_on_cluster flakiness caused by missing replica syncs --- tests/integration/test_backup_restore_on_cluster/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_backup_restore_on_cluster/test.py b/tests/integration/test_backup_restore_on_cluster/test.py index 6af3a7dbab8..39496b8a5c8 100644 --- a/tests/integration/test_backup_restore_on_cluster/test.py +++ b/tests/integration/test_backup_restore_on_cluster/test.py @@ -580,6 +580,7 @@ def test_required_privileges(): node1.query( f"RESTORE TABLE tbl AS tbl2 ON CLUSTER 'cluster' FROM {backup_name}", user="u1" ) + node2.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl2") assert node2.query("SELECT * FROM tbl2") == "100\n" @@ -593,6 +594,7 @@ def test_required_privileges(): node1.query("GRANT INSERT, CREATE TABLE ON tbl TO u1") node1.query(f"RESTORE ALL ON CLUSTER 'cluster' FROM {backup_name}", user="u1") + node2.query("SYSTEM SYNC REPLICA ON CLUSTER 'cluster' tbl") assert node2.query("SELECT * FROM tbl") == "100\n" From 13f8d72f54433a790f3efcb054db389e4fdd53f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 20 Jul 2023 17:46:22 +0200 Subject: [PATCH 0936/2047] Wait for zero copy replication lock even if some disks don't support it --- .../MergeTree/MergeFromLogEntryTask.cpp | 8 +++++-- .../MergeTree/MutateFromLogEntryTask.cpp | 6 ++++- .../ReplicatedMergeMutateTaskBase.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 22 +++++++++++++++---- 4 files changed, 30 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 9f54c554c85..883cfee89c8 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -230,7 +230,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() /// the fast replica is not overloaded because amount of executing merges doesn't affect the ability to acquire locks for new merges. /// /// So here we trying to solve it with the simplest solution -- sleep random time up to 500ms for 1GB part and up to 7 seconds for 300GB part. - /// It can sound too much, but we are trying to aquite these locks in background tasks which can be scheduled each 5 seconds or so. + /// It can sound too much, but we are trying to acquire these locks in background tasks which can be scheduled each 5 seconds or so. double start_to_sleep_seconds = std::logf(storage_settings_ptr->zero_copy_merge_mutation_min_parts_size_sleep_before_lock.value); uint64_t right_border_to_sleep_ms = static_cast((std::log(estimated_space_for_merge) - start_to_sleep_seconds + 0.5) * 1000); uint64_t time_to_sleep_milliseconds = std::min(10000UL, std::uniform_int_distribution(1, 1 + right_border_to_sleep_ms)(rng)); @@ -245,7 +245,11 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() if (!zero_copy_lock || !zero_copy_lock->isLocked()) { - LOG_DEBUG(log, "Merge of part {} started by some other replica, will wait it and fetch merged part", entry.new_part_name); + LOG_DEBUG( + log, + "Merge of part {} started by some other replica, will wait for it and fetch merged part. Number of tries {}", + entry.new_part_name, + entry.num_tries); storage.watchZeroCopyLock(entry.new_part_name, disk); /// Don't check for missing part -- it's missing because other replica still not /// finished merge. diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 6cb9d50436e..164b541d2b8 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -154,8 +154,12 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() if (!zero_copy_lock || !zero_copy_lock->isLocked()) { + LOG_DEBUG( + log, + "Mutation of part {} started by some other replica, will wait for it and mutated merged part. Number of tries {}", + entry.new_part_name, + entry.num_tries); storage.watchZeroCopyLock(entry.new_part_name, disk); - LOG_DEBUG(log, "Mutation of part {} started by some other replica, will wait it and mutated merged part", entry.new_part_name); return PrepareResult{ .prepared_successfully = false, diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index b4748ee77ea..6ad77119016 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -174,7 +174,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() part_log_writer = prepare_result.part_log_writer; - /// Avoid resheduling, execute fetch here, in the same thread. + /// Avoid rescheduling, execute fetch here, in the same thread. if (!prepare_result.prepared_successfully) return execute_fetch(prepare_result.need_to_check_missing_part_in_fetch); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 07f46c07466..3264de850a0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1370,13 +1370,27 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( if (data_settings->allow_remote_fs_zero_copy_replication) { auto disks = storage.getDisks(); - bool only_s3_storage = true; + DiskPtr disk_with_zero_copy = nullptr; for (const auto & disk : disks) - if (!disk->supportZeroCopyReplication()) - only_s3_storage = false; + { + if (disk->supportZeroCopyReplication()) + { + disk_with_zero_copy = disk; + break; + } + } + /// Technically speaking if there are more than one disk that could store the part (a local hot + cloud cold) + /// It would be possible for the merge to happen concurrently with other replica if the other replica is doing + /// a merge using zero-copy and the cloud storage, and the local replica uses the local storage instead + /// The question is, is it worth keep retrying to do the merge over and over for the opportunity to do + /// double the work? Probably not + /// So what we do is that, even if hot merge could happen, check the zero copy lock anyway. + /// Keep in mind that for the zero copy lock check to happen (via existing_zero_copy_locks) we need to + /// have failed first because of it and added it via watchZeroCopyLock. Considering we've already tried to + /// use cloud storage and zero-copy replication, the most likely scenario is that we'll try again String replica_to_execute_merge; - if (!disks.empty() && only_s3_storage && storage.checkZeroCopyLockExists(entry.new_part_name, disks[0], replica_to_execute_merge)) + if (disk_with_zero_copy && storage.checkZeroCopyLockExists(entry.new_part_name, disk_with_zero_copy, replica_to_execute_merge)) { constexpr auto fmt_string = "Not executing merge/mutation for the part {}, waiting for {} to execute it and will fetch after."; out_postpone_reason = fmt::format(fmt_string, entry.new_part_name, replica_to_execute_merge); From ed59870f92fa2893c9c105eaaeff82b1efaede22 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Krzysztof=20G=C3=B3ralski?= Date: Thu, 20 Jul 2023 18:04:58 +0200 Subject: [PATCH 0937/2047] Update LRUFileCachePriority.cpp --- src/Interpreters/Cache/LRUFileCachePriority.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 18862e154da..33e567b7a76 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -7,6 +7,7 @@ namespace CurrentMetrics { extern const Metric FilesystemCacheSize; + extern const Metric FilesystemCacheSizeLimit; extern const Metric FilesystemCacheElements; } @@ -101,6 +102,7 @@ void LRUFileCachePriority::updateSize(int64_t size) { current_size += size; CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size); + CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, getSizeLimit()); } void LRUFileCachePriority::updateElementsCount(int64_t num) From b3c42a1171e3f631e8985b80fc3c822c7ac87dd1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Krzysztof=20G=C3=B3ralski?= Date: Thu, 20 Jul 2023 18:06:54 +0200 Subject: [PATCH 0938/2047] Update CurrentMetrics.cpp with FilesystemCacheSizeLimit metric --- src/Common/CurrentMetrics.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 626b43aea2c..583b13cf79d 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -187,6 +187,7 @@ M(CacheFileSegments, "Number of existing cache file segments") \ M(CacheDetachedFileSegments, "Number of existing detached cache file segments") \ M(FilesystemCacheSize, "Filesystem cache size in bytes") \ + M(FilesystemCacheSizeLimit, "Filesystem cache size limit in bytes") \ M(FilesystemCacheElements, "Filesystem cache elements (file segments)") \ M(FilesystemCacheDownloadQueueElements, "Filesystem cache elements in download queue") \ M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \ From d2195cff1101f5035148e9c0f7672bfb83eeb693 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 20 Jul 2023 18:21:37 +0200 Subject: [PATCH 0939/2047] Small fixes --- src/Storages/S3Queue/S3QueueHolder.cpp | 15 +++++++-------- src/Storages/S3Queue/S3QueueHolder.h | 8 ++++---- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 8 ++++---- src/Storages/S3Queue/S3QueueTableMetadata.h | 4 ++-- 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 7a984cd418f..0900c84a2e8 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -74,9 +74,9 @@ S3QueueHolder::S3QueueProcessedCollection::S3QueueProcessedCollection(const UInt { } -void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) +void S3QueueHolder::S3QueueProcessedCollection::parse(const String & collection_str) { - ReadBufferFromString buf(s); + ReadBufferFromString buf(collection_str); read(buf); // Remove old items if (max_age > 0) @@ -111,9 +111,9 @@ S3QueueHolder::S3QueueFailedCollection::S3QueueFailedCollection(const UInt64 & m { } -void S3QueueHolder::S3QueueFailedCollection::parse(const String & s) +void S3QueueHolder::S3QueueFailedCollection::parse(const String & collection_str) { - ReadBufferFromString buf(s); + ReadBufferFromString buf(collection_str); read(buf); } @@ -124,8 +124,7 @@ bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) = std::find_if(files.begin(), files.end(), [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); if (failed_it != files.end()) { - failed_it->retries_count--; - if (failed_it->retries_count == 0) + if (failed_it->retries_count == 0 || --failed_it->retries_count == 0) { return false; } @@ -138,7 +137,7 @@ bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) return true; } -S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueFailedCollection::getFilesWithoutRetries() +S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueFailedCollection::getFileNames() { S3FilesCollection failed_keys; for (const auto & pair : files) @@ -233,7 +232,7 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() auto failed_collection = S3QueueFailedCollection(max_loading_retries); failed_collection.parse(failed_files); - return failed_collection.getFilesWithoutRetries(); + return failed_collection.getFileNames(); } String S3QueueHolder::getMaxProcessedFile() diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index 1292ebadc60..83edd8a2e74 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -45,7 +45,7 @@ public: String toString() const; S3FilesCollection getFileNames(); - virtual void parse(const String & s) = 0; + virtual void parse(const String & collection_str) = 0; protected: TrackedFiles files; @@ -59,7 +59,7 @@ public: public: S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_); - void parse(const String & s) override; + void parse(const String & collection_str) override; void add(const String & file_name); private: @@ -72,10 +72,10 @@ public: public: S3QueueFailedCollection(const UInt64 & max_retries_count_); - void parse(const String & s) override; + void parse(const String & collection_str) override; bool add(const String & file_name); - S3FilesCollection getFilesWithoutRetries(); + S3FilesCollection getFileNames(); private: UInt64 max_retries_count; diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 5cfd1ee2d35..23eebb6ded9 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -43,10 +43,10 @@ String S3QueueTableMetadata::toString() const return oss.str(); } -void S3QueueTableMetadata::read(const String & s) +void S3QueueTableMetadata::read(const String & metadata_str) { Poco::JSON::Parser parser; - auto json = parser.parse(s).extract(); + auto json = parser.parse(metadata_str).extract(); after_processing = json->getValue("after_processing"); mode = json->getValue("mode"); s3queue_tracked_files_limit = json->getValue("s3queue_tracked_files_limit"); @@ -54,10 +54,10 @@ void S3QueueTableMetadata::read(const String & s) format_name = json->getValue("format_name"); } -S3QueueTableMetadata S3QueueTableMetadata::parse(const String & s) +S3QueueTableMetadata S3QueueTableMetadata::parse(const String & metadata_str) { S3QueueTableMetadata metadata; - metadata.read(s); + metadata.read(metadata_str); return metadata; } diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index d11b3cad00a..4b6fbc54825 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -26,8 +26,8 @@ struct S3QueueTableMetadata S3QueueTableMetadata() = default; S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings); - void read(const String & s); - static S3QueueTableMetadata parse(const String & s); + void read(const String & metadata_str); + static S3QueueTableMetadata parse(const String & metadata_str); String toString() const; From 920887f315e108da3b385986dee329a28aed65fb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Jul 2023 16:43:59 +0000 Subject: [PATCH 0940/2047] Done --- .../test_replicated_merge_tree_encrypted_disk/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py b/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py index 05d7bbb7282..25d30eb9c82 100644 --- a/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py +++ b/tests/integration/test_replicated_merge_tree_encrypted_disk/test.py @@ -67,6 +67,8 @@ def optimize_table(): def check_table(): expected = [[1, "str1"], [2, "str2"]] + node1.query("SYSTEM SYNC REPLICA tbl LIGHTWEIGHT") + node2.query("SYSTEM SYNC REPLICA tbl LIGHTWEIGHT") assert node1.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) assert node2.query("SELECT * FROM tbl ORDER BY id") == TSV(expected) assert node1.query("CHECK TABLE tbl") == "1\n" From 6b3a508a23e62d5459ad2a19a3bfc91ca96ccb8f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Jul 2023 16:52:45 +0000 Subject: [PATCH 0941/2047] Done --- tests/queries/0_stateless/02122_parallel_formatting.lib | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02122_parallel_formatting.lib b/tests/queries/0_stateless/02122_parallel_formatting.lib index 56119012788..5175e004cc5 100755 --- a/tests/queries/0_stateless/02122_parallel_formatting.lib +++ b/tests/queries/0_stateless/02122_parallel_formatting.lib @@ -11,14 +11,14 @@ non_parallel_file=$CLICKHOUSE_TMP/$CLICKHOUSE_TEST_UNIQUE_NAME"_non_parallel" format=$1 echo $format-1 -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_write_statistics=0 --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_write_statistics=0 --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file diff $non_parallel_file $parallel_file echo $format-2 -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file -$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_write_statistics=0 --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file +$CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals order by number limit 190000 format $format" --extremes=1 --output_format_write_statistics=0 --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file diff $non_parallel_file $parallel_file From 500f1e6757b721ecc8733f5e8bf41c765a631918 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 20 Jul 2023 18:55:41 +0200 Subject: [PATCH 0942/2047] Follow up to #49698 --- .../PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index adbc95a2cf2..f2923b60bfd 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -22,7 +22,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int POSTGRESQL_REPLICATION_INTERNAL_ERROR; extern const int BAD_ARGUMENTS; - extern const int TOO_MANY_PARTS; } MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( @@ -591,11 +590,8 @@ void MaterializedPostgreSQLConsumer::syncTables() } catch (DB::Exception & e) { - if (e.code() == ErrorCodes::TOO_MANY_PARTS) - { - /// Retry this buffer later. - storage_data.buffer.columns = result_rows.mutateColumns(); - } + /// Retry this buffer later. + storage_data.buffer.columns = result_rows.mutateColumns(); throw; } From 045ecdf71f544cafb4f5c3eda20ee6f9d593f614 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 20 Jul 2023 19:03:06 +0200 Subject: [PATCH 0943/2047] Update src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index f2923b60bfd..d01746ddf1b 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -588,7 +588,7 @@ void MaterializedPostgreSQLConsumer::syncTables() executor.execute(); } } - catch (DB::Exception & e) + catch (...) { /// Retry this buffer later. storage_data.buffer.columns = result_rows.mutateColumns(); From 09e6bbc0e2ac634cde658b9c53e599d124d0a3d8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jul 2023 21:10:59 +0300 Subject: [PATCH 0944/2047] Update DataPartsExchange.cpp --- src/Storages/MergeTree/DataPartsExchange.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 23bbc1c7f9d..6a3bf2940e9 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -353,8 +353,14 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) { /// It is important to include Outdated parts here because remote replicas cannot reliably /// determine the local state of the part, so queries for the parts in these states are completely normal. - auto part = data.getPartIfExists( - name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + MergeTreeData::DataPartPtr part; + + /// Ephemeral zero-copy lock may be lost for PreActive parts + bool zero_copy_enabled = data.getSettings()->allow_remote_fs_zero_copy_replication; + if (zero_copy_enabled) + part = data.getPartIfExists(name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + else + part = data.getPartIfExists(name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (part) return part; From c8b128aad442c763f1b85c20b8389d240466f97f Mon Sep 17 00:00:00 2001 From: Yury Bogomolov Date: Fri, 21 Jul 2023 01:06:49 +0400 Subject: [PATCH 0945/2047] add predefined macros support in kafka engine settings --- src/Storages/Kafka/StorageKafka.cpp | 15 ++++--- src/Storages/Kafka/StorageKafka.h | 2 + tests/integration/test_storage_kafka/test.py | 42 ++++++++++++++++++++ 3 files changed, 54 insertions(+), 5 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 2aba76c1a3f..8cc4fd90f8d 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -250,15 +250,20 @@ StorageKafka::StorageKafka( : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , kafka_settings(std::move(kafka_settings_)) - , topics(parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value))) - , brokers(getContext()->getMacros()->expand(kafka_settings->kafka_broker_list.value)) - , group(getContext()->getMacros()->expand(kafka_settings->kafka_group_name.value)) + , macros_info([&table_id_](){ + Macros::MacroExpansionInfo info; + info.table_id = table_id_; + return info; + }()) + , topics(parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) + , brokers(getContext()->getMacros()->expand(kafka_settings->kafka_broker_list.value, macros_info)) + , group(getContext()->getMacros()->expand(kafka_settings->kafka_group_name.value, macros_info)) , client_id( kafka_settings->kafka_client_id.value.empty() ? getDefaultClientId(table_id_) - : getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value)) + : getContext()->getMacros()->expand(kafka_settings->kafka_client_id.value, macros_info)) , format_name(getContext()->getMacros()->expand(kafka_settings->kafka_format.value)) , max_rows_per_message(kafka_settings->kafka_max_rows_per_message.value) - , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value)) + , schema_name(getContext()->getMacros()->expand(kafka_settings->kafka_schema.value, macros_info)) , num_consumers(kafka_settings->kafka_num_consumers.value) , log(&Poco::Logger::get("StorageKafka (" + table_id_.table_name + ")")) , semaphore(0, static_cast(num_consumers)) diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 09aa091ef18..72875e77b12 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -79,6 +80,7 @@ public: private: // Configuration and state std::unique_ptr kafka_settings; + Macros::MacroExpansionInfo macros_info; const Names topics; const String brokers; const String group; diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 9a6d3e0513c..621fc72a607 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -444,6 +444,48 @@ def test_kafka_settings_new_syntax(kafka_cluster): assert members[0]["client_id"] == "instance test 1234" +def test_kafka_settings_predefined_macros(kafka_cluster): + instance.query( + """ + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = '{kafka_broker}:19092', + kafka_topic_list = '{database}_{table}_topic', + kafka_group_name = '{database}_{table}_group', + kafka_format = '{kafka_format_json_each_row}', + kafka_row_delimiter = '\\n', + kafka_commit_on_select = 1, + kafka_client_id = '{database}_{table} test 1234', + kafka_skip_broken_messages = 1; + """ + ) + + messages = [] + for i in range(25): + messages.append(json.dumps({"key": i, "value": i})) + kafka_produce(kafka_cluster, "test_kafka", messages) + + # Insert couple of malformed messages. + kafka_produce(kafka_cluster, "test_kafka", ["}{very_broken_message,"]) + kafka_produce(kafka_cluster, "test_kafka", ["}another{very_broken_message,"]) + + messages = [] + for i in range(25, 50): + messages.append(json.dumps({"key": i, "value": i})) + kafka_produce(kafka_cluster, "test_kafka", messages) + + result = "" + while True: + result += instance.query("SELECT * FROM test.kafka", ignore_error=True) + if kafka_check_result(result): + break + + kafka_check_result(result, True) + + members = describe_consumer_group(kafka_cluster, "new") + assert members[0]["client_id"] == "test_kafka test 1234" + + def test_kafka_json_as_string(kafka_cluster): kafka_produce( kafka_cluster, From 24371c33bfd5037455cb025b057fb413ee1be396 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Fri, 21 Jul 2023 09:24:16 +0800 Subject: [PATCH 0946/2047] remove DelayedBlocksTask::finish --- src/Processors/Transforms/JoiningTransform.cpp | 3 ++- src/Processors/Transforms/JoiningTransform.h | 3 +-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 5480fea27a4..4e7868ea1c2 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -375,7 +375,8 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() input.setNotNeeded(); } - if (task->finished) + // When delayed_blocks is nullptr, it means that all buckets have been joined. + if (!task->delayed_blocks) { input.close(); output.finish(); diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index 5e7403dbbdb..a308af03662 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -115,7 +115,7 @@ class DelayedBlocksTask : public ChunkInfo { public: - explicit DelayedBlocksTask() : finished(true) {} + DelayedBlocksTask() = default; explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) : delayed_blocks(std::move(delayed_blocks_)) , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) @@ -125,7 +125,6 @@ public: IBlocksStreamPtr delayed_blocks = nullptr; JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter = nullptr; - bool finished = false; }; using DelayedBlocksTaskPtr = std::shared_ptr; From 2a00e2aa6ae7c417fdd5aafd63225ef44460de6f Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 20 Jul 2023 22:20:36 -0400 Subject: [PATCH 0947/2047] Implement `MOVE` query for the access entities. --- src/Access/AccessControl.cpp | 4 +- src/Access/AccessControl.h | 2 +- src/Access/DiskAccessStorage.cpp | 4 +- src/Access/DiskAccessStorage.h | 2 +- src/Access/IAccessStorage.cpp | 46 +++++++-- src/Access/IAccessStorage.h | 21 +++- src/Access/MemoryAccessStorage.cpp | 4 +- src/Access/MemoryAccessStorage.h | 2 +- src/Access/MultipleAccessStorage.cpp | 40 +++++++- src/Access/MultipleAccessStorage.h | 9 +- src/Access/ReplicatedAccessStorage.cpp | 4 +- src/Access/ReplicatedAccessStorage.h | 2 +- .../Access/InterpreterCreateQuotaQuery.cpp | 36 +++++-- .../Access/InterpreterCreateRoleQuery.cpp | 8 +- .../InterpreterCreateRowPolicyQuery.cpp | 38 ++++++-- .../InterpreterCreateSettingsProfileQuery.cpp | 37 +++++-- .../Access/InterpreterCreateUserQuery.cpp | 33 +++++-- .../InterpreterMoveAccessEntityQuery.cpp | 96 +++++++++++++++++++ .../Access/InterpreterMoveAccessEntityQuery.h | 24 +++++ src/Interpreters/InterpreterFactory.cpp | 6 ++ src/Parsers/Access/ASTCreateQuotaQuery.cpp | 6 ++ src/Parsers/Access/ASTCreateQuotaQuery.h | 1 + src/Parsers/Access/ASTCreateRoleQuery.cpp | 2 +- .../Access/ASTCreateRowPolicyQuery.cpp | 5 + src/Parsers/Access/ASTCreateRowPolicyQuery.h | 1 + .../Access/ASTCreateSettingsProfileQuery.cpp | 6 ++ .../Access/ASTCreateSettingsProfileQuery.h | 1 + src/Parsers/Access/ASTCreateUserQuery.cpp | 5 + src/Parsers/Access/ASTCreateUserQuery.h | 1 + .../Access/ASTDropAccessEntityQuery.cpp | 4 +- .../Access/ASTMoveAccessEntityQuery.cpp | 64 +++++++++++++ src/Parsers/Access/ASTMoveAccessEntityQuery.h | 32 +++++++ src/Parsers/Access/ParserCreateQuotaQuery.cpp | 6 ++ src/Parsers/Access/ParserCreateRoleQuery.cpp | 6 +- .../Access/ParserCreateRowPolicyQuery.cpp | 6 ++ .../ParserCreateSettingsProfileQuery.cpp | 6 ++ src/Parsers/Access/ParserCreateUserQuery.cpp | 5 + .../Access/ParserMoveAccessEntityQuery.cpp | 93 ++++++++++++++++++ .../Access/ParserMoveAccessEntityQuery.h | 17 ++++ src/Parsers/Access/parseUserName.h | 3 +- src/Parsers/IAST.h | 1 + src/Parsers/ParserQuery.cpp | 3 + .../integration/test_grant_and_revoke/test.py | 4 +- .../configs/memory.xml | 5 + .../configs/roles.xml | 4 - .../test.py | 67 ++++++++++++- tests/integration/test_role/test.py | 2 +- .../integration/test_settings_profile/test.py | 14 +-- .../integration/test_user_directories/test.py | 46 ++++----- 49 files changed, 731 insertions(+), 103 deletions(-) create mode 100644 src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp create mode 100644 src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h create mode 100644 src/Parsers/Access/ASTMoveAccessEntityQuery.cpp create mode 100644 src/Parsers/Access/ASTMoveAccessEntityQuery.h create mode 100644 src/Parsers/Access/ParserMoveAccessEntityQuery.cpp create mode 100644 src/Parsers/Access/ParserMoveAccessEntityQuery.h create mode 100644 tests/integration/test_multi_access_storage_role_management/configs/memory.xml diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 0c61a9e3e1b..22b00d5dde7 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -525,9 +525,9 @@ scope_guard AccessControl::subscribeForChanges(const std::vector & ids, co return changes_notifier->subscribeForChanges(ids, handler); } -std::optional AccessControl::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +std::optional AccessControl::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - auto id = MultipleAccessStorage::insertImpl(entity, replace_if_exists, throw_if_exists); + auto id = MultipleAccessStorage::insertImpl(entity, replace_if_exists, throw_if_exists, set_id); if (id) changes_notifier->sendNotifications(); return id; diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 2a8293a49e7..85feaa70462 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -229,7 +229,7 @@ private: class CustomSettingsPrefixes; class PasswordComplexityRules; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 938881fafff..acbd7b12569 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -498,9 +498,9 @@ std::optional> DiskAccessStorage::readNameWi } -std::optional DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +std::optional DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - UUID id = generateRandomID(); + UUID id = set_id ? *set_id : generateRandomID(); if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true)) return id; diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index e3bd32c0292..3541c3228e3 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -39,7 +39,7 @@ private: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 9468e8d220a..11bc97f6cc3 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -93,6 +93,17 @@ String IAccessStorage::readName(const UUID & id) const } +bool IAccessStorage::exists(const std::vector & ids) const +{ + for (const auto & id : ids) + { + if (!exists(id)) + return false; + } + + return true; +} + std::optional IAccessStorage::readName(const UUID & id, bool throw_if_not_exists) const { if (auto name_and_type = readNameWithType(id, throw_if_not_exists)) @@ -167,15 +178,26 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity) return *insert(entity, /* replace_if_exists = */ false, /* throw_if_exists = */ true); } - std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return insertImpl(entity, replace_if_exists, throw_if_exists); + return *insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); } +std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) +{ + return insertImpl(entity, replace_if_exists, throw_if_exists, set_id); +} std::vector IAccessStorage::insert(const std::vector & multiple_entities, bool replace_if_exists, bool throw_if_exists) { + return insert(multiple_entities, /* ids = */ {}, replace_if_exists, throw_if_exists); +} + +std::vector IAccessStorage::insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists, bool throw_if_exists) +{ + if (!ids.empty()) + assert(multiple_entities.size() == ids.size()); + if (multiple_entities.empty()) return {}; @@ -189,16 +211,24 @@ std::vector IAccessStorage::insert(const std::vector & mu std::vector successfully_inserted; try { - std::vector ids; - for (const auto & entity : multiple_entities) + std::vector new_ids; + for (size_t i = 0; i < multiple_entities.size(); ++i) { - if (auto id = insertImpl(entity, replace_if_exists, throw_if_exists)) + const auto & entity = multiple_entities[i]; + + std::optional id; + if (!ids.empty()) + id = ids[i]; + + auto new_id = insertImpl(entity, replace_if_exists, throw_if_exists, id); + + if (new_id) { successfully_inserted.push_back(entity); - ids.push_back(*id); + new_ids.push_back(*new_id); } } - return ids; + return new_ids; } catch (Exception & e) { @@ -244,7 +274,7 @@ std::vector IAccessStorage::insertOrReplace(const std::vector IAccessStorage::insertImpl(const AccessEntityPtr & entity, bool, bool) +std::optional IAccessStorage::insertImpl(const AccessEntityPtr & entity, bool, bool, std::optional) { if (isReadOnly()) throwReadonlyCannotInsert(entity->getType(), entity->getName()); diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index aa3947201e7..894220e95ad 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -92,6 +92,7 @@ public: /// Returns whether there is an entity with such identifier in the storage. virtual bool exists(const UUID & id) const = 0; + bool exists(const std::vector & ids) const; /// Reads an entity. Throws an exception if not found. template @@ -100,6 +101,9 @@ public: template std::shared_ptr read(const String & name, bool throw_if_not_exists = true) const; + template + std::vector read(const std::vector & ids, bool throw_if_not_exists = true) const; + /// Reads an entity. Returns nullptr if not found. template std::shared_ptr tryRead(const UUID & id) const; @@ -128,7 +132,9 @@ public: /// Throws an exception if the specified name already exists. UUID insert(const AccessEntityPtr & entity); std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); std::vector insert(const std::vector & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true); + std::vector insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists = false, bool throw_if_exists = true); /// Inserts an entity to the storage. Returns ID of a new entry in the storage. std::optional tryInsert(const AccessEntityPtr & entity); @@ -179,7 +185,7 @@ protected: virtual std::vector findAllImpl(AccessEntityType type) const = 0; virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0; virtual std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const; - virtual std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + virtual std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); virtual bool removeImpl(const UUID & id, bool throw_if_not_exists); virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); virtual std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const; @@ -240,6 +246,19 @@ std::shared_ptr IAccessStorage::read(const String & name, bo } +template +std::vector IAccessStorage::read(const std::vector & ids, bool throw_if_not_exists) const +{ + std::vector result; + result.reserve(ids.size()); + + for (const auto & id : ids) + result.push_back(read(id, throw_if_not_exists)); + + return result; +} + + template std::shared_ptr IAccessStorage::tryRead(const UUID & id) const { diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp index c4192ee4552..16f3b3793a0 100644 --- a/src/Access/MemoryAccessStorage.cpp +++ b/src/Access/MemoryAccessStorage.cpp @@ -63,9 +63,9 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not } -std::optional MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +std::optional MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - UUID id = generateRandomID(); + UUID id = set_id ? *set_id : generateRandomID(); if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) return id; diff --git a/src/Access/MemoryAccessStorage.h b/src/Access/MemoryAccessStorage.h index b63132147da..7c19ab9c4ab 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -44,7 +44,7 @@ private: std::optional findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index f36beb3f80b..169dc889509 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -213,6 +213,42 @@ ConstStoragePtr MultipleAccessStorage::getStorageByName(const DB::String & stora return const_cast(this)->getStorageByName(storage_name); } +StoragePtr MultipleAccessStorage::findExcludingStorage(AccessEntityType type, const DB::String & name, DB::MultipleAccessStorage::StoragePtr exclude) const +{ + auto storages = getStoragesInternal(); + for (const auto & storage : *storages) + { + if (storage == exclude) + continue; + + if (storage->find(type, name)) + return storage; + } + + return nullptr; +} + +void MultipleAccessStorage::moveAccessEntities(const std::vector & ids, const String & source_storage_name, const String & destination_storage_name) +{ + auto source_storage = findStorageByName(source_storage_name); + auto destination_storage = findStorageByName(destination_storage_name); + + auto to_move = source_storage->read(ids); + source_storage->remove(ids); + + try + { + destination_storage->insert(to_move, ids); + } + catch (Exception & e) + { + e.addMessage("while moving access entities"); + + source_storage->insert(to_move, ids); + throw; + } +} + AccessEntityPtr MultipleAccessStorage::readImpl(const UUID & id, bool throw_if_not_exists) const { if (auto storage = findStorage(id)) @@ -280,7 +316,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode) } -std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { std::shared_ptr storage_for_insertion; @@ -303,7 +339,7 @@ std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & en getStorageName()); } - auto id = storage_for_insertion->insert(entity, replace_if_exists, throw_if_exists); + auto id = storage_for_insertion->insert(entity, replace_if_exists, throw_if_exists, set_id); if (id) { std::lock_guard lock{mutex}; diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 1898c6ba5d3..1d95296b796 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -46,6 +46,11 @@ public: ConstStoragePtr getStorageByName(const String & storage_name) const; StoragePtr getStorageByName(const String & storage_name); + /// Search for an access entity storage, excluding one. Returns nullptr if not found. + StoragePtr findExcludingStorage(AccessEntityType type, const String & name, StoragePtr exclude) const; + + void moveAccessEntities(const std::vector & ids, const String & source_storage_name, const String & destination_storage_name); + bool exists(const UUID & id) const override; bool isBackupAllowed() const override; @@ -58,7 +63,7 @@ protected: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; @@ -70,6 +75,8 @@ private: std::shared_ptr nested_storages TSA_GUARDED_BY(mutex); mutable CacheBase ids_cache TSA_GUARDED_BY(mutex); mutable std::mutex mutex; + + mutable std::mutex move_mutex; }; } diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index 56d68be9268..efa9dbbdfaa 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -108,9 +108,9 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function) } } -std::optional ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +std::optional ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - const UUID id = generateRandomID(); + const UUID id = set_id ? *set_id : generateRandomID(); if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) return id; diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index 555d58e6b04..0a0c53d3bc4 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -46,7 +46,7 @@ private: std::unique_ptr watching_thread; std::shared_ptr> watched_queue; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp index 83a620d11c6..e271497ff5c 100644 --- a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp @@ -14,6 +14,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateQuotaFromQueryImpl( @@ -90,6 +96,15 @@ BlockIO InterpreterCreateQuotaQuery::execute() if (query.roles) roles_from_query = RolesOrUsersSet{*query.roles, access_control, getContext()->getUserID()}; + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -100,11 +115,11 @@ BlockIO InterpreterCreateQuotaQuery::execute() }; if (query.if_exists) { - auto ids = access_control.find(query.names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(query.names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(query.names), update_func); + storage->update(storage->getIDs(query.names), update_func); } else { @@ -116,12 +131,21 @@ BlockIO InterpreterCreateQuotaQuery::execute() new_quotas.emplace_back(std::move(new_quota)); } + if (!query.storage_name.empty()) + { + for (const auto & name : query.names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::QUOTA, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Quota {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_quotas); + storage->tryInsert(new_quotas); else if (query.or_replace) - access_control.insertOrReplace(new_quotas); + storage->insertOrReplace(new_quotas); else - access_control.insert(new_quotas); + storage->insert(new_quotas); } return {}; diff --git a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index 87abe002d3d..f6c3103a7a9 100644 --- a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -97,13 +97,7 @@ BlockIO InterpreterCreateRoleQuery::execute() { for (const auto & name : query.names) { - auto id = access_control.find(name); - - if (!id) - continue; - - auto another_storage_ptr = access_control.findStorage(*id); - if (another_storage_ptr != storage_ptr) + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::ROLE, name, storage_ptr)) throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Role {} already exists in storage {}", name, another_storage_ptr->getStorageName()); } } diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index 87dc9039c17..a938d7afc16 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -14,6 +14,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateRowPolicyFromQueryImpl( @@ -66,6 +72,16 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() if (query.roles) roles_from_query = RolesOrUsersSet{*query.roles, access_control, getContext()->getUserID()}; + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + + Strings names = query.names->toStrings(); if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -74,14 +90,13 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() updateRowPolicyFromQueryImpl(*updated_policy, query, {}, roles_from_query); return updated_policy; }; - Strings names = query.names->toStrings(); if (query.if_exists) { - auto ids = access_control.find(names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(names), update_func); + storage->update(storage->getIDs(names), update_func); } else { @@ -93,12 +108,21 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() new_policies.emplace_back(std::move(new_policy)); } + if (!query.storage_name.empty()) + { + for (const auto & name : names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::ROW_POLICY, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Row policy {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_policies); + storage->tryInsert(new_policies); else if (query.or_replace) - access_control.insertOrReplace(new_policies); + storage->insertOrReplace(new_policies); else - access_control.insert(new_policies); + storage->insert(new_policies); } return {}; diff --git a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp index 0727b6f2182..bac0146e821 100644 --- a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp @@ -10,6 +10,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateSettingsProfileFromQueryImpl( @@ -67,6 +73,16 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() if (query.to_roles) roles_from_query = RolesOrUsersSet{*query.to_roles, access_control, getContext()->getUserID()}; + + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -77,11 +93,11 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() }; if (query.if_exists) { - auto ids = access_control.find(query.names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(query.names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(query.names), update_func); + storage->update(storage->getIDs(query.names), update_func); } else { @@ -93,12 +109,21 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() new_profiles.emplace_back(std::move(new_profile)); } + if (!query.storage_name.empty()) + { + for (const auto & name : query.names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::SETTINGS_PROFILE, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Settings profile {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_profiles); + storage->tryInsert(new_profiles); else if (query.or_replace) - access_control.insertOrReplace(new_profiles); + storage->insertOrReplace(new_profiles); else - access_control.insert(new_profiles); + storage->insert(new_profiles); } return {}; diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 165937560cc..f1d921a9eb2 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -17,6 +17,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int ACCESS_ENTITY_ALREADY_EXISTS; } namespace { @@ -139,6 +140,16 @@ BlockIO InterpreterCreateUserQuery::execute() if (!query.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext()); + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + + Strings names = query.names->toStrings(); if (query.alter) { std::optional grantees_from_query; @@ -152,14 +163,13 @@ BlockIO InterpreterCreateUserQuery::execute() return updated_user; }; - Strings names = query.names->toStrings(); if (query.if_exists) { - auto ids = access_control.find(names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(names), update_func); + storage->update(storage->getIDs(names), update_func); } else { @@ -171,13 +181,22 @@ BlockIO InterpreterCreateUserQuery::execute() new_users.emplace_back(std::move(new_user)); } + if (!query.storage_name.empty()) + { + for (const auto & name : names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::USER, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "User {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + std::vector ids; if (query.if_not_exists) - ids = access_control.tryInsert(new_users); + ids = storage->tryInsert(new_users); else if (query.or_replace) - ids = access_control.insertOrReplace(new_users); + ids = storage->insertOrReplace(new_users); else - ids = access_control.insert(new_users); + ids = storage->insert(new_users); if (query.grantees) { diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp new file mode 100644 index 00000000000..ac7ff4efcb6 --- /dev/null +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp @@ -0,0 +1,96 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int ACCESS_ENTITY_NOT_FOUND; +} + + +BlockIO InterpreterMoveAccessEntityQuery::execute() +{ + auto & query = query_ptr->as(); + auto & access_control = getContext()->getAccessControl(); + getContext()->checkAccess(getRequiredAccess()); + + if (!query.cluster.empty()) + return executeDDLQueryOnCluster(query_ptr, getContext()); + + query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); + + std::vector ids; + if (query.type == AccessEntityType::ROW_POLICY) + ids = access_control.find(query.type, query.row_policy_names->toStrings()); + else + ids = access_control.find(query.type, query.names); + + if (ids.empty()) + return {}; + + /// Validate that all entities are from the same storage. + const auto source_storage = access_control.findStorage(ids.front()); + if (!source_storage->exists(ids)) + throw Exception(ErrorCodes::ACCESS_ENTITY_NOT_FOUND, "All access entities must be from the same storage in order to be moved"); + + access_control.moveAccessEntities(ids, source_storage->getStorageName(), query.storage_name); + return {}; +} + + +AccessRightsElements InterpreterMoveAccessEntityQuery::getRequiredAccess() const +{ + const auto & query = query_ptr->as(); + AccessRightsElements res; + switch (query.type) + { + case AccessEntityType::USER: + { + res.emplace_back(AccessType::DROP_USER); + res.emplace_back(AccessType::CREATE_USER); + return res; + } + case AccessEntityType::ROLE: + { + res.emplace_back(AccessType::DROP_ROLE); + res.emplace_back(AccessType::CREATE_ROLE); + return res; + } + case AccessEntityType::SETTINGS_PROFILE: + { + res.emplace_back(AccessType::DROP_SETTINGS_PROFILE); + res.emplace_back(AccessType::CREATE_SETTINGS_PROFILE); + return res; + } + case AccessEntityType::ROW_POLICY: + { + if (query.row_policy_names) + { + for (const auto & row_policy_name : query.row_policy_names->full_names) + { + res.emplace_back(AccessType::DROP_ROW_POLICY, row_policy_name.database, row_policy_name.table_name); + res.emplace_back(AccessType::CREATE_ROW_POLICY, row_policy_name.database, row_policy_name.table_name); + } + } + return res; + } + case AccessEntityType::QUOTA: + { + res.emplace_back(AccessType::DROP_QUOTA); + res.emplace_back(AccessType::CREATE_QUOTA); + return res; + } + case AccessEntityType::MAX: + break; + } + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: type is not supported by DROP query", toString(query.type)); +} + +} diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h new file mode 100644 index 00000000000..1a70bf7c065 --- /dev/null +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class AccessRightsElements; + +class InterpreterMoveAccessEntityQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterMoveAccessEntityQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + AccessRightsElements getRequiredAccess() const; + + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 9cd1f2a251c..27c94119750 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -96,6 +97,7 @@ #include #include #include +#include #include #include #include @@ -314,6 +316,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/src/Parsers/Access/ASTCreateQuotaQuery.cpp b/src/Parsers/Access/ASTCreateQuotaQuery.cpp index 56abedf5235..091b62b0a9f 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ASTCreateQuotaQuery.cpp @@ -170,6 +170,12 @@ void ASTCreateQuotaQuery::formatImpl(const FormatSettings & settings, FormatStat settings.ostr << (settings.hilite ? hilite_keyword : "") << " OR REPLACE" << (settings.hilite ? hilite_none : ""); formatNames(names, settings); + + if (!storage_name.empty()) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") + << " IN " << (settings.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(settings); if (!new_name.empty()) diff --git a/src/Parsers/Access/ASTCreateQuotaQuery.h b/src/Parsers/Access/ASTCreateQuotaQuery.h index f5eb59800ec..aecbbb00f9a 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.h +++ b/src/Parsers/Access/ASTCreateQuotaQuery.h @@ -38,6 +38,7 @@ public: Strings names; String new_name; std::optional key_type; + String storage_name; struct Limits { diff --git a/src/Parsers/Access/ASTCreateRoleQuery.cpp b/src/Parsers/Access/ASTCreateRoleQuery.cpp index 14946f2d3cd..eeeb34c97e4 100644 --- a/src/Parsers/Access/ASTCreateRoleQuery.cpp +++ b/src/Parsers/Access/ASTCreateRoleQuery.cpp @@ -74,7 +74,7 @@ void ASTCreateRoleQuery::formatImpl(const FormatSettings & format, FormatState & if (!storage_name.empty()) format.ostr << (format.hilite ? IAST::hilite_keyword : "") - << " AT " << (format.hilite ? IAST::hilite_none : "") + << " IN " << (format.hilite ? IAST::hilite_none : "") << backQuoteIfNeed(storage_name); formatOnCluster(format); diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp index ca888be2cfe..a423581e300 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp @@ -168,6 +168,11 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format settings.ostr << " "; names->format(settings); + if (!storage_name.empty()) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") + << " IN " << (settings.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(settings); assert(names->cluster.empty()); diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.h b/src/Parsers/Access/ASTCreateRowPolicyQuery.h index 32d98fab822..3f2418e7287 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.h +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.h @@ -35,6 +35,7 @@ public: bool if_exists = false; bool if_not_exists = false; bool or_replace = false; + String storage_name; std::shared_ptr names; String new_short_name; diff --git a/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp b/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp index 56ddef433ef..3379486d789 100644 --- a/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp +++ b/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp @@ -81,6 +81,12 @@ void ASTCreateSettingsProfileQuery::formatImpl(const FormatSettings & format, Fo format.ostr << (format.hilite ? hilite_keyword : "") << " OR REPLACE" << (format.hilite ? hilite_none : ""); formatNames(names, format); + + if (!storage_name.empty()) + format.ostr << (format.hilite ? IAST::hilite_keyword : "") + << " IN " << (format.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(format); if (!new_name.empty()) diff --git a/src/Parsers/Access/ASTCreateSettingsProfileQuery.h b/src/Parsers/Access/ASTCreateSettingsProfileQuery.h index c1a64998f29..be01aae1e26 100644 --- a/src/Parsers/Access/ASTCreateSettingsProfileQuery.h +++ b/src/Parsers/Access/ASTCreateSettingsProfileQuery.h @@ -28,6 +28,7 @@ public: bool if_exists = false; bool if_not_exists = false; bool or_replace = false; + String storage_name; Strings names; String new_name; diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index 0611545adf0..dbb4c1c6d57 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -208,6 +208,11 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & format.ostr << " "; names->format(format); + if (!storage_name.empty()) + format.ostr << (format.hilite ? IAST::hilite_keyword : "") + << " IN " << (format.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(format); if (new_name) diff --git a/src/Parsers/Access/ASTCreateUserQuery.h b/src/Parsers/Access/ASTCreateUserQuery.h index 62ddbfd0040..06940e67d91 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.h +++ b/src/Parsers/Access/ASTCreateUserQuery.h @@ -45,6 +45,7 @@ public: std::shared_ptr names; std::optional new_name; + String storage_name; std::shared_ptr auth_data; diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp index 02b29f3fd2c..bcd7105d0e9 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp @@ -54,8 +54,8 @@ void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, Forma formatNames(names, settings); if (!storage_name.empty()) - settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") - << " FROM " << (settings.hilite ? IAST::hilite_none : "") + settings.ostr << (settings.hilite ? hilite_keyword : "") + << " FROM " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(storage_name); formatOnCluster(settings); diff --git a/src/Parsers/Access/ASTMoveAccessEntityQuery.cpp b/src/Parsers/Access/ASTMoveAccessEntityQuery.cpp new file mode 100644 index 00000000000..285f07854c0 --- /dev/null +++ b/src/Parsers/Access/ASTMoveAccessEntityQuery.cpp @@ -0,0 +1,64 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + void formatNames(const Strings & names, const IAST::FormatSettings & settings) + { + bool need_comma = false; + for (const auto & name : names) + { + if (std::exchange(need_comma, true)) + settings.ostr << ','; + settings.ostr << ' ' << backQuoteIfNeed(name); + } + } +} + +String ASTMoveAccessEntityQuery::getID(char) const +{ + return String("MOVE ") + toString(type) + " query"; +} + +ASTPtr ASTMoveAccessEntityQuery::clone() const +{ + auto res = std::make_shared(*this); + + if (row_policy_names) + res->row_policy_names = std::static_pointer_cast(row_policy_names->clone()); + + return res; +} + +void ASTMoveAccessEntityQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") + << "MOVE " << AccessEntityTypeInfo::get(type).name + << (settings.hilite ? hilite_none : ""); + + if (type == AccessEntityType::ROW_POLICY) + { + settings.ostr << " "; + row_policy_names->format(settings); + } + else + formatNames(names, settings); + + settings.ostr << (settings.hilite ? hilite_keyword : "") + << " TO " << (settings.hilite ? hilite_none : "") + << backQuoteIfNeed(storage_name); + + formatOnCluster(settings); +} + +void ASTMoveAccessEntityQuery::replaceEmptyDatabase(const String & current_database) const +{ + if (row_policy_names) + row_policy_names->replaceEmptyDatabase(current_database); +} +} diff --git a/src/Parsers/Access/ASTMoveAccessEntityQuery.h b/src/Parsers/Access/ASTMoveAccessEntityQuery.h new file mode 100644 index 00000000000..aa2b3b0f98c --- /dev/null +++ b/src/Parsers/Access/ASTMoveAccessEntityQuery.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +class ASTRowPolicyNames; + +/** MOVE {USER | ROLE | QUOTA | [ROW] POLICY | [SETTINGS] PROFILE} [IF EXISTS] name [,...] [ON [database.]table [,...]] TO storage_name + */ +class ASTMoveAccessEntityQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + AccessEntityType type; + Strings names; + std::shared_ptr row_policy_names; + + String storage_name; + + String getID(char) const override; + ASTPtr clone() const override; + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + void replaceEmptyDatabase(const String & current_database) const; + + QueryKind getQueryKind() const override { return QueryKind::Move; } +}; +} diff --git a/src/Parsers/Access/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp index 2330ba7771c..05c2820b012 100644 --- a/src/Parsers/Access/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -288,6 +289,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe std::optional key_type; std::vector all_limits; String cluster; + String storage_name; while (true) { @@ -310,6 +312,9 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -332,6 +337,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->key_type = key_type; query->all_limits = std::move(all_limits); query->roles = std::move(roles); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserCreateRoleQuery.cpp b/src/Parsers/Access/ParserCreateRoleQuery.cpp index de9e5baed18..b548c07ec70 100644 --- a/src/Parsers/Access/ParserCreateRoleQuery.cpp +++ b/src/Parsers/Access/ParserCreateRoleQuery.cpp @@ -93,9 +93,6 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec String cluster; String storage_name; - if (ParserKeyword{"AT"}.ignore(pos, expected)) - parseStorageName(pos, expected, storage_name); - while (true) { if (alter && new_name.empty() && (names.size() == 1) && parseRenameTo(pos, expected, new_name)) @@ -114,6 +111,9 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } diff --git a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index 2c25fc14e7d..b2177ca4ade 100644 --- a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -245,6 +246,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & String new_short_name; std::optional is_restrictive; std::vector> filters; + String storage_name; while (true) { @@ -271,6 +273,9 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -294,6 +299,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->is_restrictive = is_restrictive; query->filters = std::move(filters); query->roles = std::move(roles); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp index c58a3035dc6..634ff68427a 100644 --- a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp +++ b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -111,6 +112,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec String new_name; std::shared_ptr settings; String cluster; + String storage_name; while (true) { @@ -130,6 +132,9 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -152,6 +157,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec query->new_name = std::move(new_name); query->settings = std::move(settings); query->to_roles = std::move(to_roles); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 0344fb99c04..d95d41296dc 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -414,6 +414,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec std::shared_ptr grantees; std::shared_ptr default_database; String cluster; + String storage_name; while (true) { @@ -480,6 +481,9 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } } + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -514,6 +518,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->settings = std::move(settings); query->grantees = std::move(grantees); query->default_database = std::move(default_database); + query->storage_name = std::move(storage_name); if (query->auth_data) query->children.push_back(query->auth_data); diff --git a/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp new file mode 100644 index 00000000000..6ac41b77b84 --- /dev/null +++ b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp @@ -0,0 +1,93 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + bool parseEntityType(IParserBase::Pos & pos, Expected & expected, AccessEntityType & type) + { + for (auto i : collections::range(AccessEntityType::MAX)) + { + const auto & type_info = AccessEntityTypeInfo::get(i); + if (ParserKeyword{type_info.name}.ignore(pos, expected) + || (!type_info.alias.empty() && ParserKeyword{type_info.alias}.ignore(pos, expected))) + { + type = i; + return true; + } + } + return false; + } + + + bool parseOnCluster(IParserBase::Pos & pos, Expected & expected, String & cluster) + { + return IParserBase::wrapParseImpl(pos, [&] + { + return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected); + }); + } +} + + +bool ParserMoveAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword{"MOVE"}.ignore(pos, expected)) + return false; + + AccessEntityType type; + if (!parseEntityType(pos, expected, type)) + return false; + + Strings names; + std::shared_ptr row_policy_names; + String storage_name; + String cluster; + + if ((type == AccessEntityType::USER) || (type == AccessEntityType::ROLE)) + { + if (!parseUserNames(pos, expected, names)) + return false; + } + else if (type == AccessEntityType::ROW_POLICY) + { + ParserRowPolicyNames parser; + ASTPtr ast; + parser.allowOnCluster(); + if (!parser.parse(pos, ast, expected)) + return false; + row_policy_names = typeid_cast>(ast); + cluster = std::exchange(row_policy_names->cluster, ""); + } + else + { + if (!parseIdentifiersOrStringLiterals(pos, expected, names)) + return false; + } + + if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseStorageName(pos, expected, storage_name)) + return false; + + if (cluster.empty()) + parseOnCluster(pos, expected, cluster); + + auto query = std::make_shared(); + node = query; + + query->type = type; + query->cluster = std::move(cluster); + query->names = std::move(names); + query->row_policy_names = std::move(row_policy_names); + query->storage_name = std::move(storage_name); + + return true; +} +} diff --git a/src/Parsers/Access/ParserMoveAccessEntityQuery.h b/src/Parsers/Access/ParserMoveAccessEntityQuery.h new file mode 100644 index 00000000000..3accec883dd --- /dev/null +++ b/src/Parsers/Access/ParserMoveAccessEntityQuery.h @@ -0,0 +1,17 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses queries like + * MOVE {USER | ROLE | QUOTA | [ROW] POLICY | [SETTINGS] PROFILE} [IF EXISTS] name [,...] [ON [database.]table [,...]] TO storage_name + */ +class ParserMoveAccessEntityQuery : public IParserBase +{ +protected: + const char * getName() const override { return "MOVE access entity query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/Access/parseUserName.h b/src/Parsers/Access/parseUserName.h index 717911bf2f1..64fed2ce21c 100644 --- a/src/Parsers/Access/parseUserName.h +++ b/src/Parsers/Access/parseUserName.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -36,7 +37,7 @@ inline bool parseRoleNames(IParser::Pos & pos, Expected & expected, Strings & ro inline bool parseStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) { - return parseUserName(pos, expected, storage_name); + return parseIdentifierOrStringLiteral(pos, expected, storage_name); } } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index aa5302a15b9..92fb8ed275b 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -290,6 +290,7 @@ public: Alter, Grant, Revoke, + Move, System, Set, Use, diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 77e7b58e6b1..7ed69940bed 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include @@ -54,6 +55,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateIndexQuery create_index_p; ParserDropIndexQuery drop_index_p; ParserDropAccessEntityQuery drop_access_entity_p; + ParserMoveAccessEntityQuery move_access_entity_p; ParserGrantQuery grant_p; ParserSetRoleQuery set_role_p; ParserExternalDDLQuery external_ddl_p; @@ -80,6 +82,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_index_p.parse(pos, node, expected) || drop_index_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) + || move_access_entity_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected) || external_ddl_p.parse(pos, node, expected) || transaction_control_p.parse(pos, node, expected) diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index ee5d4b5df93..8e040a18a88 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -471,7 +471,7 @@ def test_introspection(): [ [ "A", - "local directory", + "local_directory", "no_password", "{}", "['::/0']", @@ -484,7 +484,7 @@ def test_introspection(): ], [ "B", - "local directory", + "local_directory", "no_password", "{}", "['::/0']", diff --git a/tests/integration/test_multi_access_storage_role_management/configs/memory.xml b/tests/integration/test_multi_access_storage_role_management/configs/memory.xml new file mode 100644 index 00000000000..c2f60c8bbc1 --- /dev/null +++ b/tests/integration/test_multi_access_storage_role_management/configs/memory.xml @@ -0,0 +1,5 @@ + + + + + diff --git a/tests/integration/test_multi_access_storage_role_management/configs/roles.xml b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml index 672c7e24728..faaf1ee27ae 100644 --- a/tests/integration/test_multi_access_storage_role_management/configs/roles.xml +++ b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml @@ -1,8 +1,4 @@ - - - - diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index d9a4a73c56b..fb4bbd6b718 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -8,6 +8,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", stay_alive=True, + main_configs=["configs/memory.xml"] ) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -28,6 +29,68 @@ def started_cluster(): cluster.shutdown() +def execute_test_for_access_type(access_type: str, system_table_name: str): + node.query(f"CREATE {access_type} test1 IN local_directory") + node.query(f"CREATE {access_type} test2 IN local_directory") + node.query(f"CREATE {access_type} test3 IN local_directory") + + node.query(f"CREATE {access_type} test4 IN memory") + node.query(f"CREATE {access_type} test5 IN memory") + node.query(f"CREATE {access_type} test6 IN memory") + + # Already exists + with pytest.raises(QueryRuntimeException): + node.query(f"CREATE {access_type} test1 IN memory") + + node.query(f"MOVE {access_type} test1 TO memory") + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test1'") == TSV(["memory"]) + + node.query(f"MOVE {access_type} test2 TO local_directory") + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["local_directory"]) + + node.query(f"MOVE {access_type} test2,test3 TO memory") + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["memory"]) + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test3'") == TSV(["memory"]) + + node.query(f"MOVE {access_type} test4,test5 TO local_directory") + + # Different storages + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test4,test1 TO memory") + + # Doesn't exist + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test7 TO local_directory") + + # Storage doesn't exist + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test6 TO non_existing_storage") + + # Unwriteable storage + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test6 TO users_xml") + + +def test_roles(): + execute_test_for_access_type("ROLE", "roles") + + +def test_users(): + execute_test_for_access_type("USER", "users") + + +def test_settings_profiles(): + execute_test_for_access_type("SETTINGS PROFILE", "settings_profiles") + + +def test_quotas(): + execute_test_for_access_type("QUOTA", "quotas") + + +def test_row_policies(): + execute_test_for_access_type("ROW POLICY", "row_policies") + + def test_role_from_different_storages(): node.query("CREATE ROLE default_role") node.query("GRANT SELECT ON system.* TO default_role") @@ -72,9 +135,9 @@ def test_role_from_different_storages(): # Already exists with pytest.raises(QueryRuntimeException): - node.query("CREATE ROLE default_role AT memory") + node.query("CREATE ROLE default_role IN memory") - node.query("CREATE ROLE other_role AT memory") + node.query("CREATE ROLE other_role IN memory") assert node.query( "SELECT storage FROM system.roles WHERE name = 'other_role'" diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 9a352f43eab..43a402d9fda 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -287,7 +287,7 @@ def test_introspection(): assert instance.query( "SELECT name, storage from system.roles WHERE name IN ('R1', 'R2') ORDER BY name" - ) == TSV([["R1", "local directory"], ["R2", "local directory"]]) + ) == TSV([["R1", "local_directory"], ["R2", "local_directory"]]) assert instance.query( "SELECT * from system.grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, access_type, database, table, column, is_partial_revoke, grant_option" diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index 13acc3f1042..8770cc82064 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -88,7 +88,7 @@ def test_smoke(): ) ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 1, 0, "['robin']", "[]"] + ["xyz", "local_directory", 1, 0, "['robin']", "[]"] ] assert system_settings_profile_elements(profile_name="xyz") == [ [ @@ -120,7 +120,7 @@ def test_smoke(): instance.query("SET max_memory_usage = 80000000", user="robin") instance.query("SET max_memory_usage = 120000000", user="robin") assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 1, 0, "[]", "[]"] + ["xyz", "local_directory", 1, 0, "[]", "[]"] ] assert system_settings_profile_elements(user_name="robin") == [] @@ -201,7 +201,7 @@ def test_settings_from_granted_role(): ) ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 2, 0, "[]", "[]"] + ["xyz", "local_directory", 2, 0, "[]", "[]"] ] assert system_settings_profile_elements(profile_name="xyz") == [ [ @@ -276,7 +276,7 @@ def test_settings_from_granted_role(): ) ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 2, 0, "['worker']", "[]"] + ["xyz", "local_directory", 2, 0, "['worker']", "[]"] ] instance.query("ALTER SETTINGS PROFILE xyz TO NONE") @@ -293,7 +293,7 @@ def test_settings_from_granted_role(): ) instance.query("SET max_memory_usage = 120000000", user="robin") assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 2, 0, "[]", "[]"] + ["xyz", "local_directory", 2, 0, "[]", "[]"] ] @@ -323,7 +323,7 @@ def test_inheritance(): ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 1, 0, "[]", "[]"] + ["xyz", "local_directory", 1, 0, "[]", "[]"] ] assert system_settings_profile_elements(profile_name="xyz") == [ [ @@ -340,7 +340,7 @@ def test_inheritance(): ] ] assert system_settings_profile("alpha") == [ - ["alpha", "local directory", 1, 0, "['robin']", "[]"] + ["alpha", "local_directory", 1, 0, "['robin']", "[]"] ] assert system_settings_profile_elements(profile_name="alpha") == [ ["alpha", "\\N", "\\N", 0, "\\N", "\\N", "\\N", "\\N", "\\N", "xyz"] diff --git a/tests/integration/test_user_directories/test.py b/tests/integration/test_user_directories/test.py index 45afb86f464..704fb30b2fd 100644 --- a/tests/integration/test_user_directories/test.py +++ b/tests/integration/test_user_directories/test.py @@ -38,14 +38,14 @@ def test_old_style(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users2.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access2\\\\/"}', 2, ], @@ -62,20 +62,20 @@ def test_local_directories(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users3.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access3\\\\/"}', 2, ], [ "local directory (ro)", - "local directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access3-ro\\\\/","readonly":true}', 3, ], @@ -92,8 +92,8 @@ def test_relative_path(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users4.xml"}', 1, ] @@ -110,8 +110,8 @@ def test_memory(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users5.xml"}', 1, ], @@ -129,20 +129,20 @@ def test_mixed_style(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users6.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access6\\\\/"}', 2, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access6a\\\\/"}', 3, ], @@ -160,14 +160,14 @@ def test_duplicates(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users7.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access7\\\\/"}', 2, ], From a8c08b34a6ca179d2b89da87d920916978bf4129 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 21 Jul 2023 02:39:21 +0000 Subject: [PATCH 0948/2047] Automatic style fix --- .../test.py | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index fb4bbd6b718..4c2696158ec 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -6,9 +6,7 @@ from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", - stay_alive=True, - main_configs=["configs/memory.xml"] + "node", stay_alive=True, main_configs=["configs/memory.xml"] ) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -43,14 +41,22 @@ def execute_test_for_access_type(access_type: str, system_table_name: str): node.query(f"CREATE {access_type} test1 IN memory") node.query(f"MOVE {access_type} test1 TO memory") - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test1'") == TSV(["memory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test1'" + ) == TSV(["memory"]) node.query(f"MOVE {access_type} test2 TO local_directory") - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["local_directory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'" + ) == TSV(["local_directory"]) node.query(f"MOVE {access_type} test2,test3 TO memory") - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["memory"]) - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test3'") == TSV(["memory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'" + ) == TSV(["memory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test3'" + ) == TSV(["memory"]) node.query(f"MOVE {access_type} test4,test5 TO local_directory") From f0e277f94a642647cfd3eb5ebc722b486d9203b0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 21 Jul 2023 06:45:35 +0200 Subject: [PATCH 0949/2047] Rename TaskStatsInfoGetter into NetlinkMetricsProvider There is ProcfsMetricsProvider, so by analogy to it. Signed-off-by: Azat Khuzhin --- src/Common/CurrentThread.cpp | 1 - ...oGetter.cpp => NetlinkMetricsProvider.cpp} | 22 +++++++++---------- ...sInfoGetter.h => NetlinkMetricsProvider.h} | 6 ++--- src/Common/ThreadProfileEvents.cpp | 6 ++--- src/Disks/IO/ThreadPoolReader.cpp | 2 +- src/IO/ReadBufferFromFileDescriptor.cpp | 2 +- src/IO/SynchronousReader.cpp | 2 +- 7 files changed, 20 insertions(+), 21 deletions(-) rename src/Common/{TaskStatsInfoGetter.cpp => NetlinkMetricsProvider.cpp} (93%) rename src/Common/{TaskStatsInfoGetter.h => NetlinkMetricsProvider.h} (85%) diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 057b1eeda12..ac5b712279e 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -3,7 +3,6 @@ #include "CurrentThread.h" #include #include -#include #include #include #include diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/NetlinkMetricsProvider.cpp similarity index 93% rename from src/Common/TaskStatsInfoGetter.cpp rename to src/Common/NetlinkMetricsProvider.cpp index 867a50c8cce..4c228bcc6fc 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/NetlinkMetricsProvider.cpp @@ -1,4 +1,4 @@ -#include "TaskStatsInfoGetter.h" +#include "NetlinkMetricsProvider.h" #include #include #include @@ -200,7 +200,7 @@ bool checkPermissionsImpl() if (!res) return false; - /// Check that we can successfully initialize TaskStatsInfoGetter. + /// Check that we can successfully initialize NetlinkMetricsProvider. /// It will ask about family id through Netlink. /// On some LXC containers we have capability but we still cannot use Netlink. /// There is an evidence that Linux fedora-riscv 6.1.22 gives something strange instead of the expected result. @@ -208,7 +208,7 @@ bool checkPermissionsImpl() try { ::taskstats stats{}; - TaskStatsInfoGetter().getStat(stats, static_cast(getThreadId())); + NetlinkMetricsProvider().getStat(stats, static_cast(getThreadId())); } catch (const Exception & e) { @@ -244,14 +244,14 @@ UInt16 getFamilyId(int fd) } -bool TaskStatsInfoGetter::checkPermissions() +bool NetlinkMetricsProvider::checkPermissions() { static bool res = checkPermissionsImpl(); return res; } -TaskStatsInfoGetter::TaskStatsInfoGetter() +NetlinkMetricsProvider::NetlinkMetricsProvider() { netlink_socket_fd = ::socket(PF_NETLINK, SOCK_RAW, NETLINK_GENERIC); if (netlink_socket_fd < 0) @@ -293,7 +293,7 @@ TaskStatsInfoGetter::TaskStatsInfoGetter() } -void TaskStatsInfoGetter::getStat(::taskstats & out_stats, pid_t tid) const +void NetlinkMetricsProvider::getStat(::taskstats & out_stats, pid_t tid) const { NetlinkMessage answer = query(netlink_socket_fd, taskstats_family_id, tid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &tid, sizeof(tid)); @@ -318,7 +318,7 @@ void TaskStatsInfoGetter::getStat(::taskstats & out_stats, pid_t tid) const } -TaskStatsInfoGetter::~TaskStatsInfoGetter() +NetlinkMetricsProvider::~NetlinkMetricsProvider() { if (netlink_socket_fd >= 0) { @@ -335,15 +335,15 @@ TaskStatsInfoGetter::~TaskStatsInfoGetter() namespace DB { -bool TaskStatsInfoGetter::checkPermissions() +bool NetlinkMetricsProvider::checkPermissions() { return false; } -TaskStatsInfoGetter::TaskStatsInfoGetter() = default; -TaskStatsInfoGetter::~TaskStatsInfoGetter() = default; +NetlinkMetricsProvider::NetlinkMetricsProvider() = default; +NetlinkMetricsProvider::~NetlinkMetricsProvider() = default; -void TaskStatsInfoGetter::getStat(::taskstats &, pid_t) const +void NetlinkMetricsProvider::getStat(::taskstats &, pid_t) const { } diff --git a/src/Common/TaskStatsInfoGetter.h b/src/Common/NetlinkMetricsProvider.h similarity index 85% rename from src/Common/TaskStatsInfoGetter.h rename to src/Common/NetlinkMetricsProvider.h index 66655d7ad0d..8a54f33be80 100644 --- a/src/Common/TaskStatsInfoGetter.h +++ b/src/Common/NetlinkMetricsProvider.h @@ -15,11 +15,11 @@ namespace DB /// /// [1]: https://elixir.bootlin.com/linux/v5.18-rc4/source/kernel/tsacct.c#L101 /// -class TaskStatsInfoGetter : private boost::noncopyable +class NetlinkMetricsProvider : private boost::noncopyable { public: - TaskStatsInfoGetter(); - ~TaskStatsInfoGetter(); + NetlinkMetricsProvider(); + ~NetlinkMetricsProvider(); void getStat(::taskstats & out_stats, pid_t tid) const; diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index a94fd81559a..256f53df011 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -2,7 +2,7 @@ #if defined(OS_LINUX) -#include "TaskStatsInfoGetter.h" +#include "NetlinkMetricsProvider.h" #include "ProcfsMetricsProvider.h" #include "hasLinuxCapability.h" @@ -99,7 +99,7 @@ TasksStatsCounters::MetricsProvider TasksStatsCounters::findBestAvailableProvide static std::optional provider = []() -> MetricsProvider { - if (TaskStatsInfoGetter::checkPermissions()) + if (NetlinkMetricsProvider::checkPermissions()) { return MetricsProvider::Netlink; } @@ -119,7 +119,7 @@ TasksStatsCounters::TasksStatsCounters(const UInt64 tid, const MetricsProvider p switch (provider) { case MetricsProvider::Netlink: - stats_getter = [metrics_provider = std::make_shared(), tid]() + stats_getter = [metrics_provider = std::make_shared(), tid]() { ::taskstats result{}; metrics_provider->getStat(result, static_cast(tid)); diff --git a/src/Disks/IO/ThreadPoolReader.cpp b/src/Disks/IO/ThreadPoolReader.cpp index effa19bc1af..cd3f2d8dea0 100644 --- a/src/Disks/IO/ThreadPoolReader.cpp +++ b/src/Disks/IO/ThreadPoolReader.cpp @@ -114,7 +114,7 @@ std::future ThreadPoolReader::submit(Request reques /// It reports real time spent including the time spent while thread was preempted doing nothing. /// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables). /// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it - /// (TaskStatsInfoGetter has about 500K RPS). + /// (NetlinkMetricsProvider has about 500K RPS). Stopwatch watch(CLOCK_MONOTONIC); SCOPE_EXIT({ diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 67bc01279c3..6c0c1681a4c 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -95,7 +95,7 @@ size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_ /// It reports real time spent including the time spent while thread was preempted doing nothing. /// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables). /// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it - /// (TaskStatsInfoGetter has about 500K RPS). + /// (NetlinkMetricsProvider has about 500K RPS). watch.stop(); ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds()); diff --git a/src/IO/SynchronousReader.cpp b/src/IO/SynchronousReader.cpp index 7cef3bd8963..e1c654e48a3 100644 --- a/src/IO/SynchronousReader.cpp +++ b/src/IO/SynchronousReader.cpp @@ -78,7 +78,7 @@ std::future SynchronousReader::submit(Request reque /// It reports real time spent including the time spent while thread was preempted doing nothing. /// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables). /// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it - /// (TaskStatsInfoGetter has about 500K RPS). + /// (NetlinkMetricsProvider has about 500K RPS). watch.stop(); ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds()); From 19c1f392865c8a8c365a23187943eb4f137674b7 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 05:59:25 +0000 Subject: [PATCH 0950/2047] Refactor code to support storage class only via settings --- src/Backups/registerBackupEngineS3.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 219973d8fc4..34a6c904b8c 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -52,7 +52,6 @@ void registerBackupEngineS3(BackupFactory & factory) const auto & args = params.backup_info.args; String s3_uri, access_key_id, secret_access_key; - String storage_class = params.s3_storage_class; if (!id_arg.empty()) { @@ -77,17 +76,15 @@ void registerBackupEngineS3(BackupFactory & factory) } else { - if ((args.size() != 1) && (args.size() != 3) && (args.size() != 4)) + if ((args.size() != 1) && (args.size() != 3)) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Backup S3 requires 1, 3 or 4 arguments: url, [access_key_id, secret_access_key, [storage_class]]"); + "Backup S3 requires 1 or 3 arguments: url, [access_key_id, secret_access_key]"); s3_uri = args[0].safeGet(); if (args.size() >= 3) { access_key_id = args[1].safeGet(); secret_access_key = args[2].safeGet(); - if (args.size() == 4) - storage_class = args[3].safeGet(); } } @@ -115,7 +112,7 @@ void registerBackupEngineS3(BackupFactory & factory) } else { - auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, storage_class, params.context); + auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.s3_storage_class, params.context); return std::make_unique( backup_name_for_logging, From 51e81b37a4158e31b61c5ae8d993a2dee0ae16e3 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 7 Jul 2023 18:45:13 +0200 Subject: [PATCH 0951/2047] Move condtions with columns from PK to the end of PREWHERE chain --- .../MergeTree/MergeTreeWhereOptimizer.cpp | 31 +++++++++++++++++++ .../MergeTree/MergeTreeWhereOptimizer.h | 8 ++++- 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 1620ba98d58..4ff58c1fc86 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -22,6 +22,33 @@ namespace DB /// This is used to assume that condition is likely to have good selectivity. static constexpr auto threshold = 2; +static NameToIndexMap fillNamesPositions(const Names & names) +{ + NameToIndexMap names_positions; + + for (size_t position = 0; position < names.size(); ++position) + { + const auto & name = names[position]; + names_positions[name] = position; + } + + return names_positions; +} + +/// Find minimal position of any of the column in primary key. +static Int64 findMinPosition(const NameSet & condition_table_columns, const NameToIndexMap & primary_key_positions) +{ + Int64 min_position = std::numeric_limits::max() - 1; + + for (const auto & column : condition_table_columns) + { + auto it = primary_key_positions.find(column); + if (it != primary_key_positions.end()) + min_position = std::min(min_position, static_cast(it->second)); + } + + return min_position; +} MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( std::unordered_map column_sizes_, @@ -35,6 +62,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( , supported_columns{supported_columns_} , sorting_key_names{NameSet( metadata_snapshot->getSortingKey().column_names.begin(), metadata_snapshot->getSortingKey().column_names.end())} + , primary_key_names_positions(fillNamesPositions(metadata_snapshot->getPrimaryKey().column_names)) , log{log_} , column_sizes{std::move(column_sizes_)} { @@ -234,6 +262,9 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree if (cond.viable) cond.good = isConditionGood(node, table_columns); + /// Find min position in PK of any column that is used in this condition. + cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); + res.emplace_back(std::move(cond)); } } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 18555a72db1..8ab21471aeb 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -72,9 +72,14 @@ private: /// Does the condition presumably have good selectivity? bool good = false; + /// Does the condition contain primary key column? + /// If so, it is better to move it further to the end of PREWHERE chain depending on minimal position in PK of any + /// column in this condition because this condition have bigger chances to be already satisfied by PK analysis. + Int64 min_position_in_primary_key = std::numeric_limits::max() - 1; + auto tuple() const { - return std::make_tuple(!viable, !good, columns_size, table_columns.size()); + return std::make_tuple(!viable, !good, -min_position_in_primary_key, columns_size, table_columns.size()); } /// Is condition a better candidate for moving to PREWHERE? @@ -141,6 +146,7 @@ private: const Names queried_columns; const std::optional supported_columns; const NameSet sorting_key_names; + const NameToIndexMap primary_key_names_positions; Poco::Logger * log; std::unordered_map column_sizes; UInt64 total_size_of_queried_columns = 0; From fe7cff5c1cfee89d411ec79e7a3e7603ff831782 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 19 Jul 2023 22:45:04 +0200 Subject: [PATCH 0952/2047] Added move_primary_key_columns_to_end_of_prewhere setting --- src/Core/Settings.h | 1 + src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 11 +++++++++-- src/Storages/MergeTree/MergeTreeWhereOptimizer.h | 1 + 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6fb26994d2f..5a1f4b46223 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -129,6 +129,7 @@ class IColumn; M(Bool, optimize_move_to_prewhere_if_final, false, "If query has `FINAL`, the optimization `move_to_prewhere` is not always correct and it is enabled only if both settings `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are turned on", 0) \ M(Bool, move_all_conditions_to_prewhere, true, "Move all viable conditions from WHERE to PREWHERE", 0) \ M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ + M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \ \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 4ff58c1fc86..5efb7286685 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -88,6 +88,7 @@ void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, cons where_optimizer_context.context = context; where_optimizer_context.array_joined_names = determineArrayJoinedNames(select); where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; + where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.is_final = select.final(); RPNBuilderTreeContext tree_context(context, std::move(block_with_constants), {} /*prepared_sets*/); @@ -117,6 +118,7 @@ std::optional MergeTreeWhe where_optimizer_context.context = context; where_optimizer_context.array_joined_names = {}; where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; + where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.is_final = is_final; RPNBuilderTreeContext tree_context(context); @@ -262,8 +264,13 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree if (cond.viable) cond.good = isConditionGood(node, table_columns); - /// Find min position in PK of any column that is used in this condition. - cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); + if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere) + { + /// Consider all conditions good with this setting enabled. + cond.good = cond.viable; + /// Find min position in PK of any column that is used in this condition. + cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions); + } res.emplace_back(std::move(cond)); } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 8ab21471aeb..fb5e84b67c6 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -96,6 +96,7 @@ private: ContextPtr context; NameSet array_joined_names; bool move_all_conditions_to_prewhere = false; + bool move_primary_key_columns_to_end_of_prewhere = false; bool is_final = false; }; From 695a6b7bb3b6b22a0169e8f786b8cff8ffccc62b Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 06:18:54 +0000 Subject: [PATCH 0953/2047] Add integration test --- .../test_backup_s3_storage_class/__init__.py | 0 .../test_backup_s3_storage_class/test.py | 51 +++++++++++++++++++ 2 files changed, 51 insertions(+) create mode 100644 tests/integration/test_backup_s3_storage_class/__init__.py create mode 100644 tests/integration/test_backup_s3_storage_class/test.py diff --git a/tests/integration/test_backup_s3_storage_class/__init__.py b/tests/integration/test_backup_s3_storage_class/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_backup_s3_storage_class/test.py b/tests/integration/test_backup_s3_storage_class/test.py new file mode 100644 index 00000000000..a4c93d40f0c --- /dev/null +++ b/tests/integration/test_backup_s3_storage_class/test.py @@ -0,0 +1,51 @@ +import os +import logging + +import pytest +from helpers.cluster import ClickHouseCluster + +logging.getLogger().setLevel(logging.INFO) +logging.getLogger().addHandler(logging.StreamHandler()) + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + stay_alive=True, + with_minio=True, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_backup_s3_storage_class(started_cluster): + node.query( + """ + CREATE TABLE test_s3_storage_class + ( + `id` UInt64, + `value` String + ) + ENGINE = MergeTree + ORDER BY id; + """, + ) + node.query( + """ + INSERT INTO test_s3_storage_class VALUES (1, 'a'); + """, + ) + result = node.query( + """ + BACKUP TABLE test_s3_storage_class TO S3('http://minio1:9001/root/data', 'minio', 'minio123') + SETTINGS s3_storage_class='STANDARD'; + """ + ) + + assert True From 9b0eb9cdd709418c3782ae2468693b294e81a0cd Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 16 Jul 2023 23:34:44 +0800 Subject: [PATCH 0954/2047] ignore ast opt when doing projection calc --- src/Storages/ProjectionsDescription.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 73fb279d51c..086355b6a79 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -298,6 +298,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) SelectQueryOptions{ type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns : QueryProcessingStage::WithMergeableState} + .ignoreASTOptimizations() .ignoreSettingConstraints()) .buildQueryPipeline(); builder.resize(1); From 696818b340d88667a214674f1df483b8c9e827d9 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 16 Jul 2023 23:35:18 +0800 Subject: [PATCH 0955/2047] Don't check monotonicity when analyze projections --- src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp | 4 ++-- src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h | 2 +- .../Optimizations/optimizeUseAggregateProjection.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp index c9cf46aaeca..1c18465e1e1 100644 --- a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp +++ b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp @@ -8,7 +8,7 @@ namespace DB { -MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag) +MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity) { using Parents = std::set; std::unordered_map inner_parents; @@ -182,7 +182,7 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG } } - if (!match.node && frame.node->function_base->hasInformationAboutMonotonicity()) + if (!match.node && check_monotonicity && frame.node->function_base->hasInformationAboutMonotonicity()) { size_t num_const_args = 0; const ActionsDAG::Node * monotonic_child = nullptr; diff --git a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h index dd689cba46b..223fc40e33f 100644 --- a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h +++ b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.h @@ -39,5 +39,5 @@ struct MatchedTrees using Matches = std::unordered_map; }; -MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag); +MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity = true); } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index f183bdca7a9..cf88de19f03 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -287,7 +287,7 @@ ActionsDAGPtr analyzeAggregateProjection( { auto proj_index = buildDAGIndex(*info.before_aggregation); - MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, *query.dag); + MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, *query.dag, false /* check_monotonicity */); // for (const auto & [node, match] : matches) // { From 60488e23912ba29ca0e75e2a39b1902517244e6b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 16 Jul 2023 23:38:42 +0800 Subject: [PATCH 0956/2047] Add tests --- ...ggregate_projection_with_monotonic_key_expr.reference | 1 + ...1710_aggregate_projection_with_monotonic_key_expr.sql | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql new file mode 100644 index 00000000000..c3109553f63 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS t0; + +CREATE TABLE t0 (c0 Int16, projection h (SELECT min(c0), max(c0), count() GROUP BY -c0)) ENGINE = MergeTree ORDER BY (); + +INSERT INTO t0(c0) VALUES (1); + +SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0); + +DROP TABLE t0; From d7bb006c231c3960e9c9f7a6f07cd8ba299d6422 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jul 2023 18:02:02 +0000 Subject: [PATCH 0957/2047] Fix monotonic chain for read-in-order as well. --- src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp | 7 ++++++- ..._aggregate_projection_with_monotonic_key_expr.reference | 4 ++++ .../01710_aggregate_projection_with_monotonic_key_expr.sql | 7 +++++++ 3 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp index 1c18465e1e1..787a106200a 100644 --- a/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp +++ b/src/Processors/QueryPlan/Optimizations/actionsDAGUtils.cpp @@ -75,7 +75,12 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG } /// A node from found match may be nullptr. /// It means that node is visited, but no match was found. - frame.mapped_children.push_back(it->second.node); + if (it->second.monotonicity) + /// Ignore a match with monotonicity. + frame.mapped_children.push_back(nullptr); + else + frame.mapped_children.push_back(it->second.node); + } if (frame.mapped_children.size() < frame.node->children.size()) diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference index d00491fd7e5..06e9efbe839 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -1 +1,5 @@ 1 +1 +-1 +1 +-1 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql index c3109553f63..bed43ef6630 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS t0; +DROP TABLE IF EXISTS t1; CREATE TABLE t0 (c0 Int16, projection h (SELECT min(c0), max(c0), count() GROUP BY -c0)) ENGINE = MergeTree ORDER BY (); @@ -6,4 +7,10 @@ INSERT INTO t0(c0) VALUES (1); SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0); +create table t1 (c0 Int32) engine = MergeTree order by sin(c0); +insert into t1 values (-1), (1); +select c0 from t1 order by sin(-c0) settings optimize_read_in_order=0; +select c0 from t1 order by sin(-c0) settings optimize_read_in_order=1; + DROP TABLE t0; +DROP TABLE t1; From ec223372848014b79990bc05318862b8f8e76212 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 10:54:26 +0800 Subject: [PATCH 0958/2047] Fix index analysis with indexHint as well --- src/Interpreters/ActionsVisitor.cpp | 8 ++++++++ ...regate_projection_with_monotonic_key_expr.reference | 1 + ...10_aggregate_projection_with_monotonic_key_expr.sql | 10 +++++++++- 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index efab11003f5..b769011e3d4 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -976,7 +976,15 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (node.name == "indexHint") { if (data.only_consts) + { + /// We need to collect constants inside `indexHint` for index analysis. + if (node.arguments) + { + for (const auto & arg : node.arguments->children) + visit(arg, data); + } return; + } /// Here we create a separate DAG for indexHint condition. /// It will be used only for index analysis. diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference index 06e9efbe839..1c8b399e790 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -3,3 +3,4 @@ -1 1 -1 +0 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql index bed43ef6630..5cd8ec87b4b 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -1,16 +1,24 @@ DROP TABLE IF EXISTS t0; DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; CREATE TABLE t0 (c0 Int16, projection h (SELECT min(c0), max(c0), count() GROUP BY -c0)) ENGINE = MergeTree ORDER BY (); INSERT INTO t0(c0) VALUES (1); -SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0); +SELECT count() FROM t0 GROUP BY gcd(-sign(c0), -c0) SETTINGS optimize_use_implicit_projections = 1; create table t1 (c0 Int32) engine = MergeTree order by sin(c0); insert into t1 values (-1), (1); select c0 from t1 order by sin(-c0) settings optimize_read_in_order=0; select c0 from t1 order by sin(-c0) settings optimize_read_in_order=1; +CREATE TABLE t2 (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; + +INSERT INTO t2 FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); + +SELECT count() FROM t2 WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; + DROP TABLE t0; DROP TABLE t1; +DROP TABLE t2; From 5560603321319243180a7d38b17be29e2f69cf30 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 10:56:37 +0800 Subject: [PATCH 0959/2047] optimize_use_implicit_projections=1 by default --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6fb26994d2f..b8207b142d3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -577,7 +577,7 @@ class IColumn; M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) ALIAS(allow_experimental_projection_optimization) \ - M(Bool, optimize_use_implicit_projections, false, "Automatically choose implicit projections to perform SELECT query", 0) \ + M(Bool, optimize_use_implicit_projections, true, "Automatically choose implicit projections to perform SELECT query", 0) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 3e58750e1d2..2886cdd288d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -80,7 +80,6 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { - {"23.7", {{"optimize_use_implicit_projections", true, false, "Disable implicit projections due to unexpected results."}}}, {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, From 68e7583dbfaca500757ba0b8e3d3d859b89accfa Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 21 Jul 2023 14:53:06 +0800 Subject: [PATCH 0960/2047] reorganize tests and add some comments --- .../Optimizations/optimizeUseAggregateProjection.cpp | 3 +++ src/Storages/MergeTree/MergeTreeData.cpp | 4 +++- ...aggregate_projection_with_monotonic_key_expr.reference | 1 - ...01710_aggregate_projection_with_monotonic_key_expr.sql | 7 ------- tests/queries/0_stateless/01739_index_hint.reference | 5 +++++ tests/queries/0_stateless/01739_index_hint.sql | 8 ++++++++ 6 files changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index cf88de19f03..e611bb5b2ef 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -497,6 +497,9 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( // LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection sample block 2 {}", block.dumpStructure()); + // minmax_count_projection cannot be used used when there is no data to process, because + // it will produce incorrect result during constant aggregation. + // See https://github.com/ClickHouse/ClickHouse/issues/36728 if (block) { MinMaxProjectionCandidate minmax; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9c5e45aa488..6c1375ecc1d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7027,7 +7027,9 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg max_added_blocks.get(), query_context); - // minmax_count_projection should not be used when there is no data to process. + // minmax_count_projection cannot be used used when there is no data to process, because + // it will produce incorrect result during constant aggregation. + // See https://github.com/ClickHouse/ClickHouse/issues/36728 if (!query_info.minmax_count_projection_block) return; diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference index 1c8b399e790..06e9efbe839 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.reference @@ -3,4 +3,3 @@ -1 1 -1 -0 diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql index 5cd8ec87b4b..51dafb07b91 100644 --- a/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_monotonic_key_expr.sql @@ -13,12 +13,5 @@ insert into t1 values (-1), (1); select c0 from t1 order by sin(-c0) settings optimize_read_in_order=0; select c0 from t1 order by sin(-c0) settings optimize_read_in_order=1; -CREATE TABLE t2 (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; - -INSERT INTO t2 FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); - -SELECT count() FROM t2 WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; - DROP TABLE t0; DROP TABLE t1; -DROP TABLE t2; diff --git a/tests/queries/0_stateless/01739_index_hint.reference b/tests/queries/0_stateless/01739_index_hint.reference index 3a4b380de65..766dff8c7b0 100644 --- a/tests/queries/0_stateless/01739_index_hint.reference +++ b/tests/queries/0_stateless/01739_index_hint.reference @@ -33,3 +33,8 @@ insert into XXXX select number*60, 0 from numbers(100000); SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_use_implicit_projections = 1; 100000 drop table XXXX; +CREATE TABLE XXXX (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; +INSERT INTO XXXX FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; +0 +drop table XXXX; diff --git a/tests/queries/0_stateless/01739_index_hint.sql b/tests/queries/0_stateless/01739_index_hint.sql index e1e66c630e1..77c2760535d 100644 --- a/tests/queries/0_stateless/01739_index_hint.sql +++ b/tests/queries/0_stateless/01739_index_hint.sql @@ -33,3 +33,11 @@ insert into XXXX select number*60, 0 from numbers(100000); SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_use_implicit_projections = 1; drop table XXXX; + +CREATE TABLE XXXX (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; + +INSERT INTO XXXX FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); + +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; + +drop table XXXX; From 0c2ea94efeeb9a68448d44c1eba08ad3898ef99f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 21 Jul 2023 09:57:32 +0200 Subject: [PATCH 0961/2047] Update test --- .../0_stateless/02156_storage_merge_prewhere.reference | 2 +- tests/queries/0_stateless/02156_storage_merge_prewhere.sql | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 30f9b1ab175..74ba452d783 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -1,6 +1,6 @@ SELECT count() FROM t_02156_merge1 -PREWHERE (k = 3) AND notEmpty(v) +PREWHERE notEmpty(v) AND (k = 3) 2 SELECT count() FROM t_02156_merge2 diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql index b75d3fa22e5..83d88a68d9b 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql @@ -1,4 +1,5 @@ SET optimize_move_to_prewhere = 1; +SET enable_multiple_prewhere_read_steps = 1; DROP TABLE IF EXISTS t_02156_mt1; DROP TABLE IF EXISTS t_02156_mt2; @@ -8,8 +9,8 @@ DROP TABLE IF EXISTS t_02156_merge1; DROP TABLE IF EXISTS t_02156_merge2; DROP TABLE IF EXISTS t_02156_merge3; -CREATE TABLE t_02156_mt1 (k UInt32, v String) ENGINE = MergeTree ORDER BY k; -CREATE TABLE t_02156_mt2 (k UInt32, v String) ENGINE = MergeTree ORDER BY k; +CREATE TABLE t_02156_mt1 (k UInt32, v String) ENGINE = MergeTree ORDER BY k SETTINGS min_bytes_for_wide_part=0; +CREATE TABLE t_02156_mt2 (k UInt32, v String) ENGINE = MergeTree ORDER BY k SETTINGS min_bytes_for_wide_part=0; CREATE TABLE t_02156_log (k UInt32, v String) ENGINE = Log; CREATE TABLE t_02156_dist (k UInt32, v String) ENGINE = Distributed(test_shard_localhost, currentDatabase(), t_02156_mt1); From d2dba496bf0c703178758b1c534c0914044d2094 Mon Sep 17 00:00:00 2001 From: StianBerger <111980234+StianBerger@users.noreply.github.com> Date: Fri, 21 Jul 2023 10:26:01 +0200 Subject: [PATCH 0962/2047] Update date-time-functions.md formatDateTime %r for 12-hour time, mentioned %H in equivalent, which is 24H. Replaced with %h. --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index c6b978506a1..87d84425029 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1449,7 +1449,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %n | new-line character (‘’) | | | %p | AM or PM designation | PM | | %Q | Quarter (1-4) | 1 | -| %r | 12-hour HH:MM AM/PM time, equivalent to %H:%i %p | 10:30 PM | +| %r | 12-hour HH:MM AM/PM time, equivalent to %h:%i %p | 10:30 PM | | %R | 24-hour HH:MM time, equivalent to %H:%i | 22:33 | | %s | second (00-59) | 44 | | %S | second (00-59) | 44 | From 53d77e6b1397e3621a81fc88da76aa9bac72ad75 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Jul 2023 21:28:17 +0800 Subject: [PATCH 0963/2047] Add back missing projection QueryAccessInfo. --- src/Interpreters/Context.cpp | 13 ++-- src/Interpreters/Context.h | 1 + .../optimizeUseAggregateProjection.cpp | 18 +++++- .../optimizeUseNormalProjection.cpp | 9 ++- .../QueryPlan/ReadFromMergeTree.cpp | 4 ++ .../QueryPlan/ReadFromPreparedSource.cpp | 8 ++- .../QueryPlan/ReadFromPreparedSource.h | 3 +- ...0_query_log_with_projection_info.reference | 3 + .../01710_query_log_with_projection_info.sql | 64 +++++++++++++++++++ 9 files changed, 114 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/01710_query_log_with_projection_info.reference create mode 100644 tests/queries/0_stateless/01710_query_log_with_projection_info.sql diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9e4d1e8d1e2..434fc1adb40 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1461,15 +1461,20 @@ void Context::addQueryAccessInfo( void Context::addQueryAccessInfo(const Names & partition_names) { if (isGlobalContext()) - { throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info"); - } std::lock_guard lock(query_access_info.mutex); for (const auto & partition_name : partition_names) - { query_access_info.partitions.emplace(partition_name); - } +} + +void Context::addQueryAccessInfo(const String & qualified_projection_name) +{ + if (isGlobalContext()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info"); + + std::lock_guard lock(query_access_info.mutex); + query_access_info.projections.emplace(qualified_projection_name); } void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3a8d41bf130..3ce899bfb77 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -657,6 +657,7 @@ public: const String & projection_name = {}, const String & view_name = {}); void addQueryAccessInfo(const Names & partition_names); + void addQueryAccessInfo(const String & qualified_projection_name); /// Supported factories for records in query_log diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index f183bdca7a9..8c85435138c 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -625,7 +625,14 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & // candidates.minmax_projection->block.dumpStructure()); Pipe pipe(std::make_shared(std::move(candidates.minmax_projection->block))); - projection_reading = std::make_unique(std::move(pipe)); + projection_reading = std::make_unique( + std::move(pipe), + context, + query_info.is_internal ? "" + : fmt::format( + "{}.{}", + reading->getMergeTreeData().getStorageID().getFullTableName(), + backQuoteIfNeed(candidates.minmax_projection->candidate.projection->name))); has_ordinary_parts = !candidates.minmax_projection->normal_parts.empty(); if (has_ordinary_parts) @@ -658,7 +665,14 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & { auto header = proj_snapshot->getSampleBlockForColumns(best_candidate->dag->getRequiredColumnsNames()); Pipe pipe(std::make_shared(std::move(header))); - projection_reading = std::make_unique(std::move(pipe)); + projection_reading = std::make_unique( + std::move(pipe), + context, + query_info.is_internal ? "" + : fmt::format( + "{}.{}", + reading->getMergeTreeData().getStorageID().getFullTableName(), + backQuoteIfNeed(best_candidate->projection->name))); } has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index dd7a5d449bc..71db561e1c9 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -183,7 +183,14 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) if (!projection_reading) { Pipe pipe(std::make_shared(proj_snapshot->getSampleBlockForColumns(required_columns))); - projection_reading = std::make_unique(std::move(pipe)); + projection_reading = std::make_unique( + std::move(pipe), + context, + query_info.is_internal ? "" + : fmt::format( + "{}.{}", + reading->getMergeTreeData().getStorageID().getFullTableName(), + backQuoteIfNeed(best_candidate->projection->name))); } bool has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 13de5d1d140..82f47cc61d5 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1761,6 +1761,10 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons fmt::format("{}.{}", data.getStorageID().getFullNameNotQuoted(), part.data_part->info.partition_id)); } context->getQueryContext()->addQueryAccessInfo(partition_names); + + if (storage_snapshot->projection) + context->getQueryContext()->addQueryAccessInfo( + fmt::format("{}.{}", data.getStorageID().getFullTableName(), backQuoteIfNeed(storage_snapshot->projection->name))); } ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts); diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index 7446203ec35..d50eec47ca8 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -1,17 +1,23 @@ +#include #include #include namespace DB { -ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_) +ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, ContextPtr context_, const String & qualified_projection_name_) : ISourceStep(DataStream{.header = pipe_.getHeader()}) , pipe(std::move(pipe_)) + , context(context_) + , qualified_projection_name(qualified_projection_name_) { } void ReadFromPreparedSource::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { + if (context && context->hasQueryContext() && !qualified_projection_name.empty()) + context->getQueryContext()->addQueryAccessInfo(qualified_projection_name); + for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.h b/src/Processors/QueryPlan/ReadFromPreparedSource.h index 05e3ebd5102..5e64dcb7a4f 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -9,7 +9,7 @@ namespace DB class ReadFromPreparedSource : public ISourceStep { public: - explicit ReadFromPreparedSource(Pipe pipe_); + explicit ReadFromPreparedSource(Pipe pipe_, ContextPtr context_ = nullptr, const String & qualified_projection_name_ = ""); String getName() const override { return "ReadFromPreparedSource"; } @@ -18,6 +18,7 @@ public: protected: Pipe pipe; ContextPtr context; + String qualified_projection_name; }; class ReadFromStorageStep : public ReadFromPreparedSource diff --git a/tests/queries/0_stateless/01710_query_log_with_projection_info.reference b/tests/queries/0_stateless/01710_query_log_with_projection_info.reference new file mode 100644 index 00000000000..9c2e9df6662 --- /dev/null +++ b/tests/queries/0_stateless/01710_query_log_with_projection_info.reference @@ -0,0 +1,3 @@ +t.t_normal +t.t_agg +t._minmax_count_projection diff --git a/tests/queries/0_stateless/01710_query_log_with_projection_info.sql b/tests/queries/0_stateless/01710_query_log_with_projection_info.sql new file mode 100644 index 00000000000..25e7e8fed60 --- /dev/null +++ b/tests/queries/0_stateless/01710_query_log_with_projection_info.sql @@ -0,0 +1,64 @@ +set log_queries=1; +set log_queries_min_type='QUERY_FINISH'; +set optimize_use_implicit_projections=1; + +DROP TABLE IF EXISTS t; + +CREATE TABLE t +( + `id` UInt64, + `id2` UInt64, + `id3` UInt64, + PROJECTION t_normal + ( + SELECT + id, + id2, + id3 + ORDER BY + id2, + id, + id3 + ), + PROJECTION t_agg + ( + SELECT + sum(id3) + GROUP BY id2 + ) +) +ENGINE = MergeTree +ORDER BY id +SETTINGS index_granularity = 8; + +insert into t SELECT number, -number, number FROM numbers(10000); + +SELECT * FROM t WHERE id2 = 3 FORMAT Null; +SELECT sum(id3) FROM t GROUP BY id2 FORMAT Null; +SELECT min(id) FROM t FORMAT Null; + +SYSTEM FLUSH LOGS; + +SELECT + --Remove the prefix string which is a mutable database name. + arrayStringConcat(arrayPopFront(splitByString('.', projections[1])), '.') +FROM + system.query_log +WHERE + current_database=currentDatabase() and query = 'SELECT * FROM t WHERE id2 = 3 FORMAT Null;'; + +SELECT + --Remove the prefix string which is a mutable database name. + arrayStringConcat(arrayPopFront(splitByString('.', projections[1])), '.') +FROM + system.query_log +WHERE + current_database=currentDatabase() and query = 'SELECT sum(id3) FROM t GROUP BY id2 FORMAT Null;'; + +SELECT + --Remove the prefix string which is a mutable database name. + arrayStringConcat(arrayPopFront(splitByString('.', projections[1])), '.') +FROM + system.query_log +WHERE + current_database=currentDatabase() and query = 'SELECT min(id) FROM t FORMAT Null;'; From 8187118232371630fb10ee4062b8a52285003fa0 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 20 Jul 2023 11:12:22 +0800 Subject: [PATCH 0964/2047] Better code --- src/Interpreters/Context.cpp | 8 ++++++-- src/Interpreters/Context.h | 9 ++++++++- .../optimizeUseAggregateProjection.cpp | 18 ++++++++---------- .../optimizeUseNormalProjection.cpp | 9 ++++----- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- .../QueryPlan/ReadFromPreparedSource.cpp | 9 ++++----- .../QueryPlan/ReadFromPreparedSource.h | 7 +++++-- 7 files changed, 36 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 434fc1adb40..cc1277e08b9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1468,13 +1468,17 @@ void Context::addQueryAccessInfo(const Names & partition_names) query_access_info.partitions.emplace(partition_name); } -void Context::addQueryAccessInfo(const String & qualified_projection_name) +void Context::addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name) { + if (!qualified_projection_name) + return; + if (isGlobalContext()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have query access info"); std::lock_guard lock(query_access_info.mutex); - query_access_info.projections.emplace(qualified_projection_name); + query_access_info.projections.emplace(fmt::format( + "{}.{}", qualified_projection_name.storage_id.getFullTableName(), backQuoteIfNeed(qualified_projection_name.projection_name))); } void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3ce899bfb77..fa210f04451 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -657,7 +657,14 @@ public: const String & projection_name = {}, const String & view_name = {}); void addQueryAccessInfo(const Names & partition_names); - void addQueryAccessInfo(const String & qualified_projection_name); + + struct QualifiedProjectionName + { + StorageID storage_id = StorageID::createEmpty(); + String projection_name; + explicit operator bool() const { return !projection_name.empty(); } + }; + void addQueryAccessInfo(const QualifiedProjectionName & qualified_projection_name); /// Supported factories for records in query_log diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 8c85435138c..fa6a7f5b8ea 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -628,11 +628,10 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & projection_reading = std::make_unique( std::move(pipe), context, - query_info.is_internal ? "" - : fmt::format( - "{}.{}", - reading->getMergeTreeData().getStorageID().getFullTableName(), - backQuoteIfNeed(candidates.minmax_projection->candidate.projection->name))); + query_info.is_internal ? Context::QualifiedProjectionName{} + : Context::QualifiedProjectionName{ + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = candidates.minmax_projection->candidate.projection->name}); has_ordinary_parts = !candidates.minmax_projection->normal_parts.empty(); if (has_ordinary_parts) @@ -668,11 +667,10 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & projection_reading = std::make_unique( std::move(pipe), context, - query_info.is_internal ? "" - : fmt::format( - "{}.{}", - reading->getMergeTreeData().getStorageID().getFullTableName(), - backQuoteIfNeed(best_candidate->projection->name))); + query_info.is_internal + ? Context::QualifiedProjectionName{} + : Context::QualifiedProjectionName{ + .storage_id = reading->getMergeTreeData().getStorageID(), .projection_name = best_candidate->projection->name}); } has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 71db561e1c9..93d1be20e81 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -186,11 +186,10 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) projection_reading = std::make_unique( std::move(pipe), context, - query_info.is_internal ? "" - : fmt::format( - "{}.{}", - reading->getMergeTreeData().getStorageID().getFullTableName(), - backQuoteIfNeed(best_candidate->projection->name))); + query_info.is_internal + ? Context::QualifiedProjectionName{} + : Context::QualifiedProjectionName{ + .storage_id = reading->getMergeTreeData().getStorageID(), .projection_name = best_candidate->projection->name}); } bool has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 82f47cc61d5..2d2412f7e36 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1764,7 +1764,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (storage_snapshot->projection) context->getQueryContext()->addQueryAccessInfo( - fmt::format("{}.{}", data.getStorageID().getFullTableName(), backQuoteIfNeed(storage_snapshot->projection->name))); + Context::QualifiedProjectionName{.storage_id = data.getStorageID(), .projection_name = storage_snapshot->projection->name}); } ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts); diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index d50eec47ca8..a24c4dbe4d0 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -1,21 +1,20 @@ -#include #include #include namespace DB { -ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, ContextPtr context_, const String & qualified_projection_name_) +ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, ContextPtr context_, Context::QualifiedProjectionName qualified_projection_name_) : ISourceStep(DataStream{.header = pipe_.getHeader()}) , pipe(std::move(pipe_)) - , context(context_) - , qualified_projection_name(qualified_projection_name_) + , context(std::move(context_)) + , qualified_projection_name(std::move(qualified_projection_name_)) { } void ReadFromPreparedSource::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - if (context && context->hasQueryContext() && !qualified_projection_name.empty()) + if (context && context->hasQueryContext()) context->getQueryContext()->addQueryAccessInfo(qualified_projection_name); for (const auto & processor : pipe.getProcessors()) diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.h b/src/Processors/QueryPlan/ReadFromPreparedSource.h index 5e64dcb7a4f..2606f501009 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -1,4 +1,6 @@ #pragma once + +#include #include #include @@ -9,7 +11,8 @@ namespace DB class ReadFromPreparedSource : public ISourceStep { public: - explicit ReadFromPreparedSource(Pipe pipe_, ContextPtr context_ = nullptr, const String & qualified_projection_name_ = ""); + explicit ReadFromPreparedSource( + Pipe pipe_, ContextPtr context_ = nullptr, Context::QualifiedProjectionName qualified_projection_name_ = {}); String getName() const override { return "ReadFromPreparedSource"; } @@ -18,7 +21,7 @@ public: protected: Pipe pipe; ContextPtr context; - String qualified_projection_name; + Context::QualifiedProjectionName qualified_projection_name; }; class ReadFromStorageStep : public ReadFromPreparedSource From 2cad81731be0443b50e66e43fb68b2b064d67a77 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 21 Jul 2023 16:46:56 +0800 Subject: [PATCH 0965/2047] Try to fix style issues --- .../optimizeUseAggregateProjection.cpp | 19 ++++++++++++------- .../optimizeUseNormalProjection.cpp | 7 +++++-- 2 files changed, 17 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index fa6a7f5b8ea..53f47bcdf95 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -628,11 +628,13 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & projection_reading = std::make_unique( std::move(pipe), context, - query_info.is_internal ? Context::QualifiedProjectionName{} - : Context::QualifiedProjectionName{ - .storage_id = reading->getMergeTreeData().getStorageID(), - .projection_name = candidates.minmax_projection->candidate.projection->name}); - + query_info.is_internal + ? Context::QualifiedProjectionName{} + : Context::QualifiedProjectionName + { + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = candidates.minmax_projection->candidate.projection->name, + }); has_ordinary_parts = !candidates.minmax_projection->normal_parts.empty(); if (has_ordinary_parts) reading->resetParts(std::move(candidates.minmax_projection->normal_parts)); @@ -669,8 +671,11 @@ bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & context, query_info.is_internal ? Context::QualifiedProjectionName{} - : Context::QualifiedProjectionName{ - .storage_id = reading->getMergeTreeData().getStorageID(), .projection_name = best_candidate->projection->name}); + : Context::QualifiedProjectionName + { + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = best_candidate->projection->name, + }); } has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index 93d1be20e81..f6ace6f8025 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -188,8 +188,11 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) context, query_info.is_internal ? Context::QualifiedProjectionName{} - : Context::QualifiedProjectionName{ - .storage_id = reading->getMergeTreeData().getStorageID(), .projection_name = best_candidate->projection->name}); + : Context::QualifiedProjectionName + { + .storage_id = reading->getMergeTreeData().getStorageID(), + .projection_name = best_candidate->projection->name, + }); } bool has_ordinary_parts = best_candidate->merge_tree_ordinary_select_result_ptr != nullptr; From 6cc8a9af5c60a18a2d5ab636205993701e7fbb9a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 21 Jul 2023 11:19:12 +0200 Subject: [PATCH 0966/2047] Fixed named collection tests --- .../test_storage_azure_blob_storage/test_cluster.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_storage_azure_blob_storage/test_cluster.py b/tests/integration/test_storage_azure_blob_storage/test_cluster.py index 15ac5590cbe..48b76061472 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -26,16 +26,19 @@ def cluster(): cluster.add_instance( "node_0", main_configs=["configs/named_collections.xml", "configs/cluster.xml"], + user_configs=["configs/disable_profilers.xml", "configs/users.xml"], with_azurite=True, ) cluster.add_instance( "node_1", main_configs=["configs/named_collections.xml", "configs/cluster.xml"], + user_configs=["configs/disable_profilers.xml", "configs/users.xml"], with_azurite=True, ) cluster.add_instance( "node_2", main_configs=["configs/named_collections.xml", "configs/cluster.xml"], + user_configs=["configs/disable_profilers.xml", "configs/users.xml"], with_azurite=True, ) cluster.start() From 5fa45bdbeaef99ba6a7db894d89dc749b7ac3f97 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Krzysztof=20G=C3=B3ralski?= Date: Fri, 21 Jul 2023 12:12:34 +0200 Subject: [PATCH 0967/2047] Setting the metric FilesystemCacheSizeLimit in LRUFileCachePriority.h --- src/Interpreters/Cache/LRUFileCachePriority.h | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index e0d7d45062a..662a76968bc 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -5,6 +5,12 @@ #include #include +namespace CurrentMetrics +{ + extern const Metric FilesystemCacheSizeLimit; +} + + namespace DB { @@ -18,7 +24,9 @@ private: using LRUQueueIterator = typename LRUQueue::iterator; public: - LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) {} + LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) { + CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, max_size_); + } size_t getSize(const CacheGuard::Lock &) const override { return current_size; } From 930d45303c5b96b7553d611e82e0c94215ef5705 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Krzysztof=20G=C3=B3ralski?= Date: Fri, 21 Jul 2023 12:13:38 +0200 Subject: [PATCH 0968/2047] removing the metric set from LRUFileCachePriority.cpp --- src/Interpreters/Cache/LRUFileCachePriority.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.cpp b/src/Interpreters/Cache/LRUFileCachePriority.cpp index 33e567b7a76..18862e154da 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.cpp +++ b/src/Interpreters/Cache/LRUFileCachePriority.cpp @@ -7,7 +7,6 @@ namespace CurrentMetrics { extern const Metric FilesystemCacheSize; - extern const Metric FilesystemCacheSizeLimit; extern const Metric FilesystemCacheElements; } @@ -102,7 +101,6 @@ void LRUFileCachePriority::updateSize(int64_t size) { current_size += size; CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size); - CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, getSizeLimit()); } void LRUFileCachePriority::updateElementsCount(int64_t num) From 3412dd225919f3850dfb4c0f8647e74e6630e31f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Krzysztof=20G=C3=B3ralski?= Date: Fri, 21 Jul 2023 12:14:30 +0200 Subject: [PATCH 0969/2047] removed unnecessary whitespace --- src/Interpreters/Cache/LRUFileCachePriority.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 662a76968bc..9396070b792 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -10,7 +10,6 @@ namespace CurrentMetrics extern const Metric FilesystemCacheSizeLimit; } - namespace DB { From 714a3a8d121326e2d908648bfc0e76e09f6e0815 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 21 Jul 2023 12:23:22 +0200 Subject: [PATCH 0970/2047] Don't do it on drop --- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 06f5330f6d9..841b646a126 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3994,6 +3994,13 @@ void StorageReplicatedMergeTree::addLastSentPart(const MergeTreePartInfo & info) void StorageReplicatedMergeTree::waitForUniquePartsToBeFetchedByOtherReplicas(StorageReplicatedMergeTree::ShutdownDeadline shutdown_deadline_) { + /// Will be true in case in case of query + if (CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr) + { + LOG_TRACE(log, "Will not wait for unique parts to be fetched by other replicas because shutdown called from DROP/DETACH query"); + return; + } + if (!shutdown_called.load()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Called waitForUniquePartsToBeFetchedByOtherReplicas before shutdown, it's a bug"); @@ -4951,7 +4958,6 @@ void StorageReplicatedMergeTree::shutdown() flushAndPrepareForShutdown(); - auto settings_ptr = getSettings(); if (!shutdown_deadline.has_value()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Shutdown deadline is not set in shutdown"); @@ -6311,7 +6317,7 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( const auto & stop_waiting = [&]() { - bool stop_waiting_itself = waiting_itself && partial_shutdown_called; + bool stop_waiting_itself = waiting_itself && (partial_shutdown_called || shutdown_prepared_called || shutdown_called); bool timeout_exceeded = check_timeout && wait_for_inactive_timeout < time_waiting.elapsedSeconds(); bool stop_waiting_inactive = (!wait_for_inactive || timeout_exceeded) && !getZooKeeper()->exists(fs::path(table_zookeeper_path) / "replicas" / replica / "is_active"); From e638a9ecd3cebe83c0c3997b19e0e73d1fb14639 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 21 Jul 2023 12:24:36 +0200 Subject: [PATCH 0971/2047] Fix style check --- src/Interpreters/Cache/LRUFileCachePriority.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 9396070b792..e041e59a91a 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -23,7 +23,8 @@ private: using LRUQueueIterator = typename LRUQueue::iterator; public: - LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) { + LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) + { CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, max_size_); } From e181e602a7e7704160a30e507f2df19e193680fd Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 21 Jul 2023 12:43:32 +0200 Subject: [PATCH 0972/2047] Update ci_config.py --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index dddcd4b7121..75aabde38b7 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -175,7 +175,7 @@ CI_CONFIG = { }, "binary_riscv64": { "compiler": "clang-16-riscv64", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "riscv64", From 3815a6aa62414fcd4e51090450a08bc368fd8c86 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 21 Jul 2023 11:14:39 +0000 Subject: [PATCH 0973/2047] Make test_keeper_force_recovery more stable --- .../configs/enable_keeper1.xml | 2 ++ .../configs/enable_keeper2.xml | 2 ++ .../configs/enable_keeper3.xml | 2 ++ .../configs/enable_keeper4.xml | 2 ++ .../configs/enable_keeper5.xml | 2 ++ .../configs/enable_keeper6.xml | 2 ++ .../configs/enable_keeper7.xml | 2 ++ .../configs/enable_keeper8.xml | 2 ++ .../configs/recovered_keeper1.xml | 2 ++ .../configs/use_keeper.xml | 36 ------------------- .../test_keeper_force_recovery/test.py | 5 +-- .../configs/enable_keeper1.xml | 2 ++ .../configs/enable_keeper1_solo.xml | 2 ++ .../configs/enable_keeper2.xml | 2 ++ .../configs/enable_keeper3.xml | 2 ++ .../configs/use_keeper.xml | 16 --------- .../test.py | 5 +-- 17 files changed, 28 insertions(+), 60 deletions(-) delete mode 100644 tests/integration/test_keeper_force_recovery/configs/use_keeper.xml delete mode 100644 tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml index b7f9d1b058e..62109ee9092 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper1.xml @@ -1,5 +1,6 @@ + false 9181 1 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml index b773d59f259..2696c573180 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper2.xml @@ -1,5 +1,6 @@ + false 9181 2 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml index d4c2befc10f..fc0c0fd0300 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper3.xml @@ -1,5 +1,6 @@ + false 9181 3 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml index c039e709c9e..06f1c1d7195 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper4.xml @@ -1,5 +1,6 @@ + false 9181 4 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml index fb43b6524c8..5d3767ae969 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper5.xml @@ -1,5 +1,6 @@ + false 9181 5 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml index 430e662bf36..4d30822741a 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper6.xml @@ -1,5 +1,6 @@ + false 9181 6 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml index aa10774ad7d..b59141042ea 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper7.xml @@ -1,5 +1,6 @@ + false 9181 7 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml b/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml index 4f1c21a1084..711d70cb1ac 100644 --- a/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml +++ b/tests/integration/test_keeper_force_recovery/configs/enable_keeper8.xml @@ -1,5 +1,6 @@ + false 9181 8 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml b/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml index eaf0f01afc9..abd4ef85bf7 100644 --- a/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml +++ b/tests/integration/test_keeper_force_recovery/configs/recovered_keeper1.xml @@ -1,5 +1,6 @@ + false 9181 1 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery/configs/use_keeper.xml b/tests/integration/test_keeper_force_recovery/configs/use_keeper.xml deleted file mode 100644 index f41e8c6e49c..00000000000 --- a/tests/integration/test_keeper_force_recovery/configs/use_keeper.xml +++ /dev/null @@ -1,36 +0,0 @@ - - - - node1 - 9181 - - - node2 - 9181 - - - node3 - 9181 - - - node4 - 9181 - - - node5 - 9181 - - - node6 - 9181 - - - node7 - 9181 - - - node8 - 9181 - - - diff --git a/tests/integration/test_keeper_force_recovery/test.py b/tests/integration/test_keeper_force_recovery/test.py index f7c3787b4d8..f630e5a422b 100644 --- a/tests/integration/test_keeper_force_recovery/test.py +++ b/tests/integration/test_keeper_force_recovery/test.py @@ -22,10 +22,7 @@ def get_nodes(): nodes.append( cluster.add_instance( f"node{i+1}", - main_configs=[ - f"configs/enable_keeper{i+1}.xml", - f"configs/use_keeper.xml", - ], + main_configs=[f"configs/enable_keeper{i+1}.xml"], stay_alive=True, ) ) diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml index 441c1bc185d..94e59128bd3 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1.xml @@ -1,5 +1,6 @@ + false 9181 1 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml index f0cb887b062..6367b4b4c29 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper1_solo.xml @@ -1,5 +1,6 @@ + false 1 9181 1 @@ -11,6 +12,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml index e2e2c1fd7db..548d12c2e0a 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper2.xml @@ -1,5 +1,6 @@ + false 9181 2 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml index e2ac0400d88..65f9675cbd6 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml +++ b/tests/integration/test_keeper_force_recovery_single_node/configs/enable_keeper3.xml @@ -1,5 +1,6 @@ + false 9181 3 /var/lib/clickhouse/coordination/log @@ -10,6 +11,7 @@ 10000 75 trace + 200 diff --git a/tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml b/tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml deleted file mode 100644 index 384e984f210..00000000000 --- a/tests/integration/test_keeper_force_recovery_single_node/configs/use_keeper.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - node1 - 9181 - - - node2 - 9181 - - - node3 - 9181 - - - diff --git a/tests/integration/test_keeper_force_recovery_single_node/test.py b/tests/integration/test_keeper_force_recovery_single_node/test.py index 1c0d5e9a306..132c5488df6 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/test.py +++ b/tests/integration/test_keeper_force_recovery_single_node/test.py @@ -20,10 +20,7 @@ def get_nodes(): nodes.append( cluster.add_instance( f"node{i+1}", - main_configs=[ - f"configs/enable_keeper{i+1}.xml", - f"configs/use_keeper.xml", - ], + main_configs=[f"configs/enable_keeper{i+1}.xml"], stay_alive=True, ) ) From 8970a531453cbfd9916681bf658ba605db0129de Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 21 Jul 2023 11:38:46 +0000 Subject: [PATCH 0974/2047] Fix `countSubstrings()` hang with empty needle and a column haystack --- src/Functions/CountSubstringsImpl.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h index de00e9397d6..1501e50afcf 100644 --- a/src/Functions/CountSubstringsImpl.h +++ b/src/Functions/CountSubstringsImpl.h @@ -49,6 +49,9 @@ struct CountSubstringsImpl /// FIXME: suboptimal memset(&res[0], 0, res.size() * sizeof(res[0])); + if (needle.empty()) + return; // Return all zeros + /// Current index in the array of strings. size_t i = 0; From 66c8066cd59a685ded94af69d2cc37469cb29d77 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 21 Jul 2023 11:45:44 +0000 Subject: [PATCH 0975/2047] add test --- tests/queries/0_stateless/01590_countSubstrings.reference | 2 ++ tests/queries/0_stateless/01590_countSubstrings.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01590_countSubstrings.reference b/tests/queries/0_stateless/01590_countSubstrings.reference index 056cb4c53b6..95031cd3856 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.reference +++ b/tests/queries/0_stateless/01590_countSubstrings.reference @@ -7,6 +7,8 @@ empty 0 0 0 +0 +0 char 1 2 diff --git a/tests/queries/0_stateless/01590_countSubstrings.sql b/tests/queries/0_stateless/01590_countSubstrings.sql index dd2122ed6ff..6d2d87b1260 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.sql +++ b/tests/queries/0_stateless/01590_countSubstrings.sql @@ -12,6 +12,8 @@ select 'empty'; select countSubstrings('', '.'); select countSubstrings('', ''); select countSubstrings('.', ''); +select countSubstrings(toString(number), '') from numbers(1); +select countSubstrings('', toString(number)) from numbers(1); select 'char'; select countSubstrings('foobar.com', '.'); From 0f969923229375d72faac15257fc70bd7ece9095 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 16 Jul 2023 08:07:50 +0200 Subject: [PATCH 0976/2047] Fix possible EADDRINUSE ("Address already in use") in integration tests Here is one example [1]: minio1_1 | WARNING: Console endpoint is listening on a dynamic port (32911), please use --console-address ":PORT" to choose a static port. minio1_1 | ERROR Unable to initialize console server: Specified port is already in use minio1_1 | > Please ensure no other program uses the same address/port [1]: https://s3.amazonaws.com/clickhouse-test-reports/52103/7d510eac7c5f0dfb3361e269be30972e6022fada/integration_tests__tsan__[1_6].html And here is second [2]: java.net.BindException: Problem binding to [0.0.0.0:50020] java.net.BindException: Address already in use; For more details see: http://wiki.apache.org/hadoop/BindException [2]: https://s3.amazonaws.com/clickhouse-test-reports/51493/63e88b725d3d255a6534adce4d434ce5f95d2874/integration_tests__asan__[1_6].html v2: increase the limit from 5K to 10K Signed-off-by: Azat Khuzhin --- tests/integration/conftest.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 5933883f7b0..968571bfdde 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -12,6 +12,22 @@ from helpers.network import _NetworkManager logging.raiseExceptions = False +@pytest.fixture(autouse=True, scope="session") +def tune_local_port_range(): + # Lots of services uses non privileged ports: + # - hdfs -- 50020/50070/... + # - minio + # - mysql + # - psql + # + # So instead of tuning all these thirdparty services, let's simply + # prohibit using such ports for outgoing connections, this should fix + # possible "Address already in use" errors. + # + # NOTE: 5K is not enough, and sometimes leads to EADDRNOTAVAIL error. + run_and_check(["sysctl net.ipv4.ip_local_port_range='55000 65535'"], shell=True) + + @pytest.fixture(autouse=True, scope="session") def cleanup_environment(): try: From f2935e22ffb819bd5e22b0bb2688c2333cc386a3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 21 Jul 2023 13:54:10 +0200 Subject: [PATCH 0977/2047] Increase log level --- tests/config/install.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index efa5a9c086e..4d28c5c1650 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -42,7 +42,7 @@ ln -sf $SRC_PATH/config.d/transactions.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/encryption.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/CORS.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/zookeeper_log.xml $DEST_SERVER_PATH/config.d/ -ln -sf $SRC_PATH/config.d/logger_test.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/logger_trace.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/named_collection.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/ssl_certs.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/filesystem_cache_log.xml $DEST_SERVER_PATH/config.d/ From 1fe8076b9422e09b78887b44a724cf1cae5d7fd8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 20 Jul 2023 12:38:57 +0200 Subject: [PATCH 0978/2047] Fix capabilities installed via systemd service (fixes netlink/IO priorities) CapabilityBoundingSet that contained in systemd unit before is about allowing to set some capabilities, not about granting them. To grant them you need to use AmbientCapabilities. And if you do not use 'clickhouse install' then: - IO priorities was unavailable (since they requires CAP_SYS_NICE) - For taskstats the procfs was used instead of netlink Not a big deal, but still. Here how it had been tested: $ systemd-run -p CapabilityBoundingSet=CAP_NET_ADMIN --shell root:/etc (master)# capsh --print Current: cap_net_admin=ep Bounding set =cap_net_admin Ambient set = $ systemd-run -p User=azat -p CapabilityBoundingSet=CAP_NET_ADMIN --shell azat:/etc$ capsh --print Current: = Bounding set =cap_net_admin Ambient set = $ systemd-run -p User=azat -p AmbientCapabilities=CAP_NET_ADMIN -p CapabilityBoundingSet=CAP_NET_ADMIN --shell azat:/etc$ capsh --print Current: cap_net_admin=eip Bounding set =cap_net_admin Ambient set =cap_net_admin Note, if you are running it under root (without changing user) you don't need to specify AmbientCapabilities additionally, because root has all capabilities by default and they had been inherited. Signed-off-by: Azat Khuzhin --- packages/clickhouse-server.service | 1 + 1 file changed, 1 insertion(+) diff --git a/packages/clickhouse-server.service b/packages/clickhouse-server.service index 7742d8b278a..42dc5bd380d 100644 --- a/packages/clickhouse-server.service +++ b/packages/clickhouse-server.service @@ -29,6 +29,7 @@ EnvironmentFile=-/etc/default/clickhouse LimitCORE=infinity LimitNOFILE=500000 CapabilityBoundingSet=CAP_NET_ADMIN CAP_IPC_LOCK CAP_SYS_NICE CAP_NET_BIND_SERVICE +AmbientCapabilities=CAP_NET_ADMIN CAP_IPC_LOCK CAP_SYS_NICE CAP_NET_BIND_SERVICE [Install] # ClickHouse should not start from the rescue shell (rescue.target). From b45c2c939b974ea3306f9c2192f362d71a69c0e2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 21 Jul 2023 15:17:07 +0300 Subject: [PATCH 0979/2047] disable expression templates for time intervals (#52335) --- .../Impl/ConstantExpressionTemplate.cpp | 33 +++++++++++++++++++ .../Formats/Impl/ConstantExpressionTemplate.h | 2 ++ .../Formats/Impl/ValuesBlockInputFormat.cpp | 5 +++ ...2830_insert_values_time_interval.reference | 4 +++ .../02830_insert_values_time_interval.sql | 25 ++++++++++++++ 5 files changed, 69 insertions(+) create mode 100644 tests/queries/0_stateless/02830_insert_values_time_interval.reference create mode 100644 tests/queries/0_stateless/02830_insert_values_time_interval.sql diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 5d438d47de6..06efe0a20aa 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -177,6 +177,14 @@ private: if (function.name == "lambda") return; + /// Parsing of INTERVALs is quite hacky. Expressions are rewritten during parsing like this: + /// "now() + interval 1 day" -> "now() + toIntervalDay(1)" + /// "select now() + INTERVAL '1 day 1 hour 1 minute'" -> "now() + (toIntervalDay(1), toIntervalHour(1), toIntervalMinute(1))" + /// so the AST is completely different from the original expression . + /// Avoid extracting these literals and simply compare tokens. It makes the template less flexible but much simpler. + if (function.name.starts_with("toInterval")) + return; + FunctionOverloadResolverPtr builder = FunctionFactory::instance().get(function.name, context); /// Do not replace literals which must be constant ColumnNumbers dont_visit_children = builder->getArgumentsThatAreAlwaysConstant(); @@ -350,6 +358,31 @@ ConstantExpressionTemplate::TemplateStructure::TemplateStructure(LiteralsInfo & } +String ConstantExpressionTemplate::TemplateStructure::dumpTemplate() const +{ + WriteBufferFromOwnString res; + + size_t cur_column = 0; + size_t cur_token = 0; + size_t num_columns = literals.columns(); + while (cur_column < num_columns) + { + size_t skip_tokens_until = token_after_literal_idx[cur_column]; + while (cur_token < skip_tokens_until) + res << quote << tokens[cur_token++] << ", "; + + const DataTypePtr & type = literals.getByPosition(cur_column).type; + res << type->getName() << ", "; + ++cur_column; + } + + while (cur_token < tokens.size()) + res << quote << tokens[cur_token++] << ", "; + + res << "eof"; + return res.str(); +} + size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTPtr & expression, const LiteralsInfo & replaced_literals, const DataTypePtr & result_column_type, diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h index fbb3cbcd22a..71d0d0f7134 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h @@ -31,6 +31,8 @@ class ConstantExpressionTemplate : boost::noncopyable static size_t getTemplateHash(const ASTPtr & expression, const LiteralsInfo & replaced_literals, const DataTypePtr & result_column_type, bool null_as_default, const String & salt); + String dumpTemplate() const; + String result_column_name; std::vector tokens; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index d61e723fd75..3a65a6fe4ea 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -474,6 +475,10 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx context, &found_in_cache, delimiter); + + LOG_TEST(&Poco::Logger::get("ValuesBlockInputFormat"), "Will use an expression template to parse column {}: {}", + column_idx, structure->dumpTemplate()); + templates[column_idx].emplace(structure); if (found_in_cache) ++attempts_to_deduce_template_cached[column_idx]; diff --git a/tests/queries/0_stateless/02830_insert_values_time_interval.reference b/tests/queries/0_stateless/02830_insert_values_time_interval.reference new file mode 100644 index 00000000000..b5b57fbfbfe --- /dev/null +++ b/tests/queries/0_stateless/02830_insert_values_time_interval.reference @@ -0,0 +1,4 @@ +1 2023-07-21 22:54:02 +2 2023-07-21 21:53:01 +3 2023-07-21 21:53:01 +4 2023-07-20 21:54:02 diff --git a/tests/queries/0_stateless/02830_insert_values_time_interval.sql b/tests/queries/0_stateless/02830_insert_values_time_interval.sql new file mode 100644 index 00000000000..f5d5d8a4c04 --- /dev/null +++ b/tests/queries/0_stateless/02830_insert_values_time_interval.sql @@ -0,0 +1,25 @@ + +DROP TABLE IF EXISTS t1; + +CREATE TABLE t1 +( + c1 DateTime DEFAULT now() NOT NULL, + c2 DateTime DEFAULT now() NOT NULL, + c3 DateTime DEFAULT now() NOT NULL, + PRIMARY KEY(c1, c2, c3) +) ENGINE = MergeTree() +ORDER BY (c1, c2, c3); + +INSERT INTO t1 (c1,c2,c3) VALUES(now() + INTERVAL '1 day 1 hour 1 minute 1 second', now(), now()); + +DROP TABLE t1; + +CREATE TABLE t1 (n int, dt DateTime) ENGINE=Memory; + +SET input_format_values_interpret_expressions=0; +INSERT INTO t1 VALUES (1, toDateTime('2023-07-20 21:53:01') + INTERVAL '1 day 1 hour 1 minute 1 second'), (2, toDateTime('2023-07-20 21:53:01') + INTERVAL '1 day'); +INSERT INTO t1 VALUES (3, toDateTime('2023-07-20 21:53:01') + INTERVAL 1 DAY), (4, toDateTime('2023-07-20 21:53:01') + (toIntervalMinute(1), toIntervalSecond(1))); + +SELECT * FROM t1 ORDER BY n; + +DROP TABLE t1; From 3798bd6f509b7fc0591201c66c2e0d1b254835dd Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 12:52:07 +0000 Subject: [PATCH 0980/2047] Replace test by text_to_encrypt --- src/Common/examples/encrypt_decrypt.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/examples/encrypt_decrypt.cpp b/src/Common/examples/encrypt_decrypt.cpp index 2d8c5a5f61f..503802016cb 100644 --- a/src/Common/examples/encrypt_decrypt.cpp +++ b/src/Common/examples/encrypt_decrypt.cpp @@ -7,7 +7,7 @@ * Keys for codecs are loaded from section of configuration file. * * How to use: - * ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV test + * ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV text_to_encrypt */ int main(int argc, char ** argv) @@ -22,7 +22,7 @@ int main(int argc, char ** argv) << "action: -e for encryption and -d for decryption." << std::endl << "codec: AES_128_GCM_SIV or AES_256_GCM_SIV." << std::endl << std::endl << "Example:" << std::endl - << " ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV test"; + << " ./encrypt_decrypt /etc/clickhouse-server/config.xml -e AES_128_GCM_SIV text_to_encrypt"; return 3; } From 8ec8388a9ef063beb02b430ae4b89dfe5bab9ddd Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 21 Jul 2023 14:53:02 +0200 Subject: [PATCH 0981/2047] Update gtest_lru_file_cache.cpp --- src/Interpreters/tests/gtest_lru_file_cache.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index b9d12c8ed42..12e7d9372f7 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -470,6 +470,7 @@ TEST_F(FileCacheTest, get) auto & file_segment2 = get(holder2, 2); ASSERT_TRUE(file_segment2.getOrSetDownloader() != FileSegment::getCallerId()); + ASSERT_EQ(file_segment2.state(), State::DOWNLOADING); { std::lock_guard lock(mutex); @@ -478,8 +479,7 @@ TEST_F(FileCacheTest, get) cv.notify_one(); file_segment2.wait(file_segment2.range().right); - file_segment2.complete(); - ASSERT_TRUE(file_segment2.state() == State::DOWNLOADED); + ASSERT_EQ(file_segment2.getDownloadedSize(false), file_segment2.range().size()); }); { @@ -488,7 +488,8 @@ TEST_F(FileCacheTest, get) } download(file_segment); - ASSERT_TRUE(file_segment.state() == State::DOWNLOADED); + ASSERT_EQ(file_segment.state(), State::DOWNLOADED); + file_segment.completePartAndResetDownloader(); other_1.join(); From 4695ec6802c80d25e93a7b523821840c10a3b200 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 21 Jul 2023 14:56:29 +0200 Subject: [PATCH 0982/2047] Add an ability to specify allocations size for sampling memory profiler --- programs/server/Server.cpp | 21 ++++++++++----- src/Common/MemoryTracker.cpp | 11 ++++++-- src/Common/MemoryTracker.h | 18 +++++++++++++ src/Core/ServerSettings.h | 8 ++++-- src/Core/Settings.h | 4 ++- src/Interpreters/ProcessList.cpp | 3 +++ src/Interpreters/ThreadStatusExt.cpp | 2 ++ .../__init__.py | 1 + .../configs/max_untracked_memory.xml | 7 +++++ .../configs/memory_profiler.xml | 5 ++++ .../test.py | 27 +++++++++++++++++++ ...r_sample_min_max_allocation_size.reference | 1 + ...profiler_sample_min_max_allocation_size.sh | 18 +++++++++++++ 13 files changed, 115 insertions(+), 11 deletions(-) create mode 100644 tests/integration/test_memory_profiler_min_max_borders/__init__.py create mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml create mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml create mode 100644 tests/integration/test_memory_profiler_min_max_borders/test.py create mode 100644 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference create mode 100755 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 948824242fb..71bf8cc9e89 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1643,17 +1643,26 @@ try global_context->initializeTraceCollector(); /// Set up server-wide memory profiler (for total memory tracker). - UInt64 total_memory_profiler_step = config().getUInt64("total_memory_profiler_step", 0); - if (total_memory_profiler_step) + if (server_settings.total_memory_profiler_step) { - total_memory_tracker.setProfilerStep(total_memory_profiler_step); + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); } - double total_memory_tracker_sample_probability = config().getDouble("total_memory_tracker_sample_probability", 0); - if (total_memory_tracker_sample_probability > 0.0) + if (server_settings.total_memory_tracker_sample_probability > 0.0) { - total_memory_tracker.setSampleProbability(total_memory_tracker_sample_probability); + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); } + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + { + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + } + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + { + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + } #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 81cac2617c5..52cd9cc8073 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -229,7 +229,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size}); @@ -413,7 +413,7 @@ void MemoryTracker::free(Int64 size) } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size}); @@ -534,6 +534,13 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) ; } +bool MemoryTracker::isSizeOkForSampling(UInt64 size) const +{ + //LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "CHECKING SIZE {} IN BORDERS [{}; {}]", size, min_allocation_size_bytes, max_allocation_size_bytes); + /// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation + return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes); +} + bool canEnqueueBackgroundTask() { auto limit = background_memory_tracker.getSoftLimit(); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 4e29d40c953..768dc8a7404 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -67,6 +67,12 @@ private: /// To randomly sample allocations and deallocations in trace_log. double sample_probability = 0; + /// Randomly sample allocations only larger or equal to this size + UInt64 min_allocation_size_bytes = 0; + + /// Randomly sample allocations only smaller or equal to this size + UInt64 max_allocation_size_bytes = 0; + /// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy). /// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker. std::atomic parent {}; @@ -88,6 +94,8 @@ private: void setOrRaiseProfilerLimit(Int64 value); + bool isSizeOkForSampling(UInt64 size) const; + /// allocImpl(...) and free(...) should not be used directly friend struct CurrentMemoryTracker; void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); @@ -165,6 +173,16 @@ public: sample_probability = value; } + void setSampleMinAllocationSize(UInt64 value) + { + min_allocation_size_bytes = value; + } + + void setSampleMaxAllocationSize(UInt64 value) + { + max_allocation_size_bytes = value; + } + void setProfilerStep(Int64 value) { profiler_step = value; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 1a9f226041b..f7a6c9e950e 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -81,8 +81,12 @@ namespace DB M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \ M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ - M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) - + M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ + \ + M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \ + M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6fb26994d2f..bcfc179be5e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -426,7 +426,9 @@ class IColumn; M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ - M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(UInt64, memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \ \ M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \ diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 1503e396298..c299572ef41 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -223,7 +223,10 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q { /// Set up memory profiling thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step); + thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); + thread_group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); + thread_group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events); } diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 5acfe500b1d..49d9d3ccdf6 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -83,6 +83,8 @@ ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_contex const Settings & settings = storage_context->getSettingsRef(); group->memory_tracker.setProfilerStep(settings.memory_profiler_step); group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); + group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); + group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); group->memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator); group->memory_tracker.setParent(&background_memory_tracker); if (settings.memory_tracker_fault_probability > 0.0) diff --git a/tests/integration/test_memory_profiler_min_max_borders/__init__.py b/tests/integration/test_memory_profiler_min_max_borders/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml new file mode 100644 index 00000000000..56fc5ed34ca --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml new file mode 100644 index 00000000000..5b3e17d145f --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml @@ -0,0 +1,5 @@ + + 1 + 4096 + 8192 + diff --git a/tests/integration/test_memory_profiler_min_max_borders/test.py b/tests/integration/test_memory_profiler_min_max_borders/test.py new file mode 100644 index 00000000000..b768a442591 --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/test.py @@ -0,0 +1,27 @@ +from helpers.cluster import ClickHouseCluster +import pytest + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/memory_profiler.xml"], + user_configs=["configs/max_untracked_memory.xml"], +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_trace_boundaries_work(started_cluster): + node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null") + node.query("SYSTEM FLUSH LOGS") + + assert node.query("SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'") == "1\n" + assert node.query("SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)") == "0\n" diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh new file mode 100755 index 00000000000..b1fbea26da7 --- /dev/null +++ b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash +# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-cpu-aarch64, no-random-settings +# requires TraceCollector, does not available under sanitizers and aarch64 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +query_id="${CLICKHOUSE_DATABASE}_min_max_allocation_size_$RANDOM$RANDOM" +${CLICKHOUSE_CLIENT} --query_id="$query_id" --memory_profiler_sample_min_allocation_size=4096 --memory_profiler_sample_max_allocation_size=8192 --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --query "select randomPrintableASCII(number) from numbers(1000) FORMAT Null" + +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" + +# at least something allocated +${CLICKHOUSE_CLIENT} --query "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample'" + +# show wrong allocations +${CLICKHOUSE_CLIENT} --query "SELECT abs(size) FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" From 0aed62ec73b8de4614506f5b72a086d8d10db4aa Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 13:03:25 +0000 Subject: [PATCH 0983/2047] Add codec name into exception message --- src/Compression/CompressionCodecEncrypted.cpp | 2 +- ..._no_encryption_codecs.xml => config_no_encryption_key.xml} | 1 - .../integration/test_config_decryption/test_wrong_settings.py | 4 ++-- 3 files changed, 3 insertions(+), 4 deletions(-) rename tests/integration/test_config_decryption/configs/{config_no_encryption_codecs.xml => config_no_encryption_key.xml} (52%) diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index fb870ababa3..3f4e35a78a4 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -588,7 +588,7 @@ String CompressionCodecEncrypted::Configuration::getKey(EncryptionMethod method, if (current_params->keys_storage[method].contains(key_id)) key = current_params->keys_storage[method].at(key_id); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no key {} in config", key_id); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no key {} in config for {} encryption codec", key_id, getMethodName(method)); return key; } diff --git a/tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml b/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml similarity index 52% rename from tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml rename to tests/integration/test_config_decryption/configs/config_no_encryption_key.xml index 07bf69d17c8..5f7769f7403 100644 --- a/tests/integration/test_config_decryption/configs/config_no_encryption_codecs.xml +++ b/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml @@ -1,4 +1,3 @@ 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C - 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index e0fbd4b2948..62610964502 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -26,9 +26,9 @@ def test_invalid_chars(): ) -def test_no_encryption_codecs(): +def test_no_encryption_key(): start_clickhouse( - "configs/config_no_encryption_codecs.xml", "There is no key 0 in config" + "configs/config_no_encryption_key.xml", "There is no key 0 in config for AES_128_GCM_SIV encryption codec" ) From 8d4c840e2da0401787bafa2239907ff59160a003 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 21 Jul 2023 15:15:35 +0200 Subject: [PATCH 0984/2047] Some more cases --- tests/queries/0_stateless/01590_countSubstrings.reference | 3 +++ tests/queries/0_stateless/01590_countSubstrings.sql | 3 +++ 2 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/01590_countSubstrings.reference b/tests/queries/0_stateless/01590_countSubstrings.reference index 95031cd3856..367b910e569 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.reference +++ b/tests/queries/0_stateless/01590_countSubstrings.reference @@ -9,6 +9,9 @@ empty 0 0 0 +0 +0 +0 char 1 2 diff --git a/tests/queries/0_stateless/01590_countSubstrings.sql b/tests/queries/0_stateless/01590_countSubstrings.sql index 6d2d87b1260..b38cbb7d188 100644 --- a/tests/queries/0_stateless/01590_countSubstrings.sql +++ b/tests/queries/0_stateless/01590_countSubstrings.sql @@ -14,6 +14,9 @@ select countSubstrings('', ''); select countSubstrings('.', ''); select countSubstrings(toString(number), '') from numbers(1); select countSubstrings('', toString(number)) from numbers(1); +select countSubstrings('aaa', materialize('')); +select countSubstrings(materialize('aaa'), ''); +select countSubstrings(materialize('aaa'), materialize('')); select 'char'; select countSubstrings('foobar.com', '.'); From b5cf64466887e115656aab065848fb52784964ae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 21 Jul 2023 15:23:04 +0200 Subject: [PATCH 0985/2047] Check projection metadata the same way we check ordinary metadata. (#52361) * Check projection metadata the same way we check ordinary metadata. * Allow aggregate projection to have empty PK --------- Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++++--- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../02540_duplicate_primary_key.sql | 18 +----------------- .../02540_duplicate_primary_key2.reference | 1 - .../02540_duplicate_primary_key2.sql | 10 +--------- .../02816_check_projection_metadata.reference | 0 .../02816_check_projection_metadata.sql | 3 +++ 7 files changed, 13 insertions(+), 31 deletions(-) create mode 100644 tests/queries/0_stateless/02816_check_projection_metadata.reference create mode 100644 tests/queries/0_stateless/02816_check_projection_metadata.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a820bacf9a3..34be8156e71 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -465,9 +465,10 @@ void MergeTreeData::checkProperties( const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach, + bool allow_empty_sorting_key, ContextPtr local_context) const { - if (!new_metadata.sorting_key.definition_ast) + if (!new_metadata.sorting_key.definition_ast && !allow_empty_sorting_key) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORDER BY cannot be empty"); KeyDescription new_sorting_key = new_metadata.sorting_key; @@ -580,6 +581,9 @@ void MergeTreeData::checkProperties( if (projections_names.find(projection.name) != projections_names.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection with name {} already exists", backQuote(projection.name)); + /// We cannot alter a projection so far. So here we do not try to find a projection in old metadata. + bool is_aggregate = projection.type == ProjectionDescription::Type::Aggregate; + checkProperties(*projection.metadata, *projection.metadata, attach, is_aggregate, local_context); projections_names.insert(projection.name); } } @@ -593,7 +597,7 @@ void MergeTreeData::setProperties( bool attach, ContextPtr local_context) { - checkProperties(new_metadata, old_metadata, attach, local_context); + checkProperties(new_metadata, old_metadata, attach, false, local_context); setInMemoryMetadata(new_metadata); } @@ -3286,7 +3290,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } } - checkProperties(new_metadata, old_metadata, false, local_context); + checkProperties(new_metadata, old_metadata, false, false, local_context); checkTTLExpressions(new_metadata, old_metadata); if (!columns_to_check_conversion.empty()) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 8b5b50b1841..28611d09386 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1229,7 +1229,7 @@ protected: /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; - void checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false, ContextPtr local_context = nullptr) const; + void checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach, bool allow_empty_sorting_key, ContextPtr local_context) const; void setProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false, ContextPtr local_context = nullptr); diff --git a/tests/queries/0_stateless/02540_duplicate_primary_key.sql b/tests/queries/0_stateless/02540_duplicate_primary_key.sql index a084d76964b..6905c9d5133 100644 --- a/tests/queries/0_stateless/02540_duplicate_primary_key.sql +++ b/tests/queries/0_stateless/02540_duplicate_primary_key.sql @@ -86,20 +86,4 @@ CREATE TABLE test ) ENGINE = MergeTree PARTITION BY toYYYYMM(coverage) -ORDER BY (coverage, situation_name, NAME_toe, NAME_cockroach); - -insert into test select * from generateRandom() limit 10; - -with dissonance as ( - Select cast(toStartOfInterval(coverage, INTERVAL 1 day) as Date) as flour, count() as regulation - from test - group by flour having flour >= toDate(now())-100 - ), -cheetah as ( - Select flour, regulation from dissonance - union distinct - Select toDate(now())-1, ifnull((select regulation from dissonance where flour = toDate(now())-1),0) as regulation -) -Select flour, regulation from cheetah order by flour with fill step 1 limit 100 format Null; - -drop table test; +ORDER BY (coverage, situation_name, NAME_toe, NAME_cockroach); -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/02540_duplicate_primary_key2.reference b/tests/queries/0_stateless/02540_duplicate_primary_key2.reference index 08839f6bb29..e69de29bb2d 100644 --- a/tests/queries/0_stateless/02540_duplicate_primary_key2.reference +++ b/tests/queries/0_stateless/02540_duplicate_primary_key2.reference @@ -1 +0,0 @@ -200 diff --git a/tests/queries/0_stateless/02540_duplicate_primary_key2.sql b/tests/queries/0_stateless/02540_duplicate_primary_key2.sql index d0f02a894f2..53800c95550 100644 --- a/tests/queries/0_stateless/02540_duplicate_primary_key2.sql +++ b/tests/queries/0_stateless/02540_duplicate_primary_key2.sql @@ -88,12 +88,4 @@ CREATE TABLE test ) ENGINE = MergeTree PARTITION BY toYYYYMM(timestamp) -ORDER BY (xxxx17, xxxx14, xxxx16, toStartOfDay(timestamp), left(xxxx19, 10), timestamp); - -INSERT INTO test SELECT * replace 1 as xxxx16 replace 1 as xxxx1 replace '2022-02-02 01:00:00' as timestamp replace 'Airtel' as xxxx14 FROM generateRandom() LIMIT 100; -INSERT INTO test SELECT * replace 1 as xxxx16 replace 1 as xxxx1 replace '2022-02-02 01:00:00' as timestamp replace 'BSNL' as xxxx14 FROM generateRandom() LIMIT 100; -INSERT INTO test SELECT * replace 1 as xxxx16 replace 1 as xxxx1 replace '2022-02-02 01:00:00' as timestamp replace 'xxx' as xxxx14 FROM generateRandom() LIMIT 100; - -select sum(1) from test where toStartOfInterval(timestamp, INTERVAL 1 day) >= TIMESTAMP '2022-02-01 01:00:00' and xxxx14 in ('Airtel', 'BSNL') and xxxx1 = 1 GROUP BY xxxx16; - -drop table test; +ORDER BY (xxxx17, xxxx14, xxxx16, toStartOfDay(timestamp), left(xxxx19, 10), timestamp); -- { serverError BAD_ARGUMENTS} diff --git a/tests/queries/0_stateless/02816_check_projection_metadata.reference b/tests/queries/0_stateless/02816_check_projection_metadata.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02816_check_projection_metadata.sql b/tests/queries/0_stateless/02816_check_projection_metadata.sql new file mode 100644 index 00000000000..e7da043ad41 --- /dev/null +++ b/tests/queries/0_stateless/02816_check_projection_metadata.sql @@ -0,0 +1,3 @@ +create table kek (uuid FixedString(16), id int, ns String, dt DateTime64(6), projection null_pk (select * order by ns, 1, 4)) engine=MergeTree order by (id, dt, uuid); -- {serverError ILLEGAL_COLUMN } +-- this query could segfault or throw LOGICAL_ERROR previously, when we did not check projection PK +-- insert into kek select * from generageRandom(10000); From cb53d762eae54eef9411e2cf7548927f83fe187b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 21 Jul 2023 15:23:24 +0200 Subject: [PATCH 0986/2047] Fix one more case --- src/Functions/CountSubstringsImpl.h | 21 ++++++++++++--------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h index 1501e50afcf..8ba9ee99de8 100644 --- a/src/Functions/CountSubstringsImpl.h +++ b/src/Functions/CountSubstringsImpl.h @@ -226,16 +226,19 @@ struct CountSubstringsImpl const char * needle_beg = reinterpret_cast(&needle_data[prev_needle_offset]); size_t needle_size = needle_offsets[i] - prev_needle_offset - 1; - typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(needle_beg, needle_size); - - const UInt8 * end = reinterpret_cast(haystack.data() + haystack.size()); - const UInt8 * beg = reinterpret_cast(Impl::advancePos(haystack.data(), reinterpret_cast(end), start - 1)); - - const UInt8 * pos; - while ((pos = searcher.search(beg, end)) < end) + if (needle_size > 0) { - ++res[i]; - beg = pos + needle_size; + typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(needle_beg, needle_size); + + const UInt8 * end = reinterpret_cast(haystack.data() + haystack.size()); + const UInt8 * beg = reinterpret_cast(Impl::advancePos(haystack.data(), reinterpret_cast(end), start - 1)); + + const UInt8 * pos; + while ((pos = searcher.search(beg, end)) < end) + { + ++res[i]; + beg = pos + needle_size; + } } } From 10ec06917506c1a61caadf4c680bd0148520426f Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 13:29:40 +0000 Subject: [PATCH 0987/2047] Improve exception message text --- src/Common/Config/ConfigProcessor.cpp | 2 +- tests/integration/test_config_decryption/test_wrong_settings.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 6529e94a41d..73fc5c58b2f 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -232,7 +232,7 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) { const NodeListPtr children = element.childNodes(); if (children->length() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} should have only one text node", node->nodeName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} cannot contain nested elements", node->nodeName()); Node * text_node = node->firstChild(); if (text_node->nodeType() != Node::TEXT_NODE) diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index 62610964502..da32a8f0ac8 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -33,4 +33,4 @@ def test_no_encryption_key(): def test_subnodes(): - start_clickhouse("configs/config_subnodes.xml", "should have only one text node") + start_clickhouse("configs/config_subnodes.xml", "cannot contain nested elements") From 1daa26c74130003a4039dcc809b9d3d0a5bcba95 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 13:31:42 +0000 Subject: [PATCH 0988/2047] Fix black formatting --- .../integration/test_config_decryption/test_wrong_settings.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index da32a8f0ac8..b148f9a051a 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -28,7 +28,8 @@ def test_invalid_chars(): def test_no_encryption_key(): start_clickhouse( - "configs/config_no_encryption_key.xml", "There is no key 0 in config for AES_128_GCM_SIV encryption codec" + "configs/config_no_encryption_key.xml", + "There is no key 0 in config for AES_128_GCM_SIV encryption codec", ) From abd8bfed2b6e6c20b46ffbeb82699c8530523ffe Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 21 Jul 2023 15:44:49 +0200 Subject: [PATCH 0989/2047] Remove comment --- src/Common/MemoryTracker.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 52cd9cc8073..52cae0768dc 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -536,7 +536,6 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) bool MemoryTracker::isSizeOkForSampling(UInt64 size) const { - //LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "CHECKING SIZE {} IN BORDERS [{}; {}]", size, min_allocation_size_bytes, max_allocation_size_bytes); /// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes); } From c080e9b450faeaced13c149212456ab006648c3a Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 21 Jul 2023 21:48:49 +0800 Subject: [PATCH 0990/2047] Fix normal projection with merge table --- .../Optimizations/optimizeUseNormalProjection.cpp | 8 ++++++-- ..._projection_query_plan_optimization_misc.reference | 1 + .../01710_projection_query_plan_optimization_misc.sql | 11 +++++++++++ 3 files changed, 18 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.reference create mode 100644 tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.sql diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp index dd7a5d449bc..2a03a082d89 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseNormalProjection.cpp @@ -92,6 +92,10 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) break; } + /// Dangling query plan node. This might be generated by StorageMerge. + if (iter->node->step.get() == reading) + return false; + const auto metadata = reading->getStorageMetadata(); const auto & projections = metadata->projections; @@ -105,8 +109,8 @@ bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes) QueryDAG query; { - auto & clild = iter->node->children[iter->next_child - 1]; - if (!query.build(*clild)) + auto & child = iter->node->children[iter->next_child - 1]; + if (!query.build(*child)) return false; if (query.dag) diff --git a/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.reference b/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.reference new file mode 100644 index 00000000000..9874d6464ab --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.reference @@ -0,0 +1 @@ +1 2 diff --git a/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.sql b/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.sql new file mode 100644 index 00000000000..cb565313380 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_query_plan_optimization_misc.sql @@ -0,0 +1,11 @@ +drop table if exists t; + +create table t (x Int32, codectest Int32) engine = MergeTree order by x; + +alter table t add projection x (select * order by codectest); + +insert into t values (1, 2); + +select * from merge('', 't'); + +drop table t; From 2a6b96f9e339e602c59968741741e57b1675bf52 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 21 Jul 2023 13:51:40 +0000 Subject: [PATCH 0991/2047] Automatic style fix --- .../test_memory_profiler_min_max_borders/test.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_memory_profiler_min_max_borders/test.py b/tests/integration/test_memory_profiler_min_max_borders/test.py index b768a442591..6ab971fa9c4 100644 --- a/tests/integration/test_memory_profiler_min_max_borders/test.py +++ b/tests/integration/test_memory_profiler_min_max_borders/test.py @@ -23,5 +23,15 @@ def test_trace_boundaries_work(started_cluster): node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null") node.query("SYSTEM FLUSH LOGS") - assert node.query("SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'") == "1\n" - assert node.query("SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)") == "0\n" + assert ( + node.query( + "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'" + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" + ) + == "0\n" + ) From f82364d2c9bb9d9484e1f5c1648ef9f1284e3633 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 21 Jul 2023 17:03:30 +0200 Subject: [PATCH 0992/2047] Fix flaky test --- .../integration/test_storage_s3_queue/test.py | 61 +++++++++++-------- 1 file changed, 36 insertions(+), 25 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 798c5952682..f06772df9be 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -410,7 +410,7 @@ def test_streaming_to_view_(started_cluster, mode): DROP TABLE IF EXISTS test.s3_queue_persistent; DROP TABLE IF EXISTS test.s3_queue; DROP TABLE IF EXISTS test.persistent_s3_queue_mv; - + CREATE TABLE test.s3_queue_persistent ({table_format}) ENGINE = MergeTree() ORDER BY column1; @@ -461,15 +461,15 @@ def test_streaming_to_many_views(started_cluster, mode): DROP TABLE IF EXISTS test.persistent_s3_queue_mv_2; DROP TABLE IF EXISTS test.persistent_s3_queue_mv_3; - + CREATE TABLE test.s3_queue_persistent ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE TABLE test.s3_queue_persistent_2 ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE TABLE test.s3_queue_persistent_3 ({table_format}) ENGINE = MergeTree() ORDER BY column1; @@ -484,12 +484,12 @@ def test_streaming_to_many_views(started_cluster, mode): SELECT * FROM test.s3_queue; - + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_2 TO test.s3_queue_persistent_2 AS SELECT * FROM test.s3_queue; - + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_3 TO test.s3_queue_persistent_3 AS SELECT * @@ -530,7 +530,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): instance.query( f""" DROP TABLE IF EXISTS test.s3_queue; - + CREATE TABLE test.s3_queue ({table_format}) ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS @@ -615,7 +615,7 @@ def test_max_set_age(started_cluster): instance.query( f""" DROP TABLE IF EXISTS test.s3_queue; - + CREATE TABLE test.s3_queue ({table_format}) ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS @@ -672,14 +672,14 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', s3queue_polling_size = {poll_size}; - + CREATE TABLE test.s3_queue_copy ({table_format}) ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', s3queue_polling_size = {poll_size}; - + CREATE TABLE test.s3_queue_copy_2 ({table_format}) ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS @@ -690,20 +690,20 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): CREATE TABLE test.s3_queue_persistent ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE TABLE test.s3_queue_persistent_copy ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE TABLE test.s3_queue_persistent_copy_2 ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS SELECT * FROM test.s3_queue; - + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy TO test.s3_queue_persistent_copy AS SELECT * @@ -718,7 +718,18 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): total_values = generate_random_files( files_to_generate, prefix, started_cluster, bucket, row_num=1 ) - time.sleep((files_to_generate // poll_size) * 2) + + def get_count(table_name): + return int(run_query(instance, f"SELECT count() FROM {table_name}")) + + for _ in range(100): + if ( + get_count("test.s3_queue_persistent") + + get_count("test.s3_queue_persistent_copy") + + get_count("test.s3_queue_persistent_copy_2") + ) == files_to_generate: + break + time.sleep(1) get_query = f"SELECT * FROM test.s3_queue_persistent" res1 = [ @@ -734,18 +745,18 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): list(map(int, l.split())) for l in run_query(instance, get_query_copy_2).splitlines() ] - - # Checking that all engines have made progress - assert len(res1) > 0 - assert len(res2) > 0 - assert len(res3) > 0 - - # Checking that all files were processed only once - assert len(res1) + len(res2) + len(res3) == files_to_generate assert {tuple(v) for v in res1 + res2 + res3} == set( [tuple(i) for i in total_values] ) + # Checking that all files were processed only once + time.sleep(10) + assert ( + get_count("test.s3_queue_persistent") + + get_count("test.s3_queue_persistent_copy") + + get_count("test.s3_queue_persistent_copy_2") + ) == files_to_generate + @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): @@ -774,7 +785,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): CREATE TABLE test.s3_queue_persistent ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS SELECT * @@ -814,7 +825,7 @@ def test_max_set_size(started_cluster): instance.query( f""" DROP TABLE IF EXISTS test.s3_queue; - + CREATE TABLE test.s3_queue ({table_format}) ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS From 3acb6005f041051b7c00c48df5035843744a7e24 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 21 Jul 2023 17:08:01 +0200 Subject: [PATCH 0993/2047] Reduce the number of syscalls in FileCache::loadMetadata --- src/Interpreters/Cache/FileCache.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 91d1c63e832..42cc7b80a66 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -870,13 +870,12 @@ void FileCache::loadMetadata() } size_t total_size = 0; - for (auto key_prefix_it = fs::directory_iterator{metadata.getBaseDirectory()}; - key_prefix_it != fs::directory_iterator();) + for (auto key_prefix_it = fs::directory_iterator{metadata.getBaseDirectory()}; key_prefix_it != fs::directory_iterator(); + key_prefix_it++) { const fs::path key_prefix_directory = key_prefix_it->path(); - key_prefix_it++; - if (!fs::is_directory(key_prefix_directory)) + if (!key_prefix_it->is_directory()) { if (key_prefix_directory.filename() != "status") { @@ -887,19 +886,19 @@ void FileCache::loadMetadata() continue; } - if (fs::is_empty(key_prefix_directory)) + fs::directory_iterator key_it{key_prefix_directory}; + if (key_it == fs::directory_iterator{}) { LOG_DEBUG(log, "Removing empty key prefix directory: {}", key_prefix_directory.string()); fs::remove(key_prefix_directory); continue; } - for (fs::directory_iterator key_it{key_prefix_directory}; key_it != fs::directory_iterator();) + for (/* key_it already initialized to verify emptiness */; key_it != fs::directory_iterator(); key_it++) { const fs::path key_directory = key_it->path(); - ++key_it; - if (!fs::is_directory(key_directory)) + if (!key_it->is_directory()) { LOG_DEBUG( log, @@ -908,7 +907,7 @@ void FileCache::loadMetadata() continue; } - if (fs::is_empty(key_directory)) + if (fs::directory_iterator{key_directory} == fs::directory_iterator{}) { LOG_DEBUG(log, "Removing empty key directory: {}", key_directory.string()); fs::remove(key_directory); From 5fb5ba71edbaf664045871b0fc8d6d5d6f5f45e6 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 21 Jul 2023 15:40:53 +0000 Subject: [PATCH 0994/2047] Throw exception when several text nodes found in YAML for element node --- src/Common/Config/YAMLParser.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/Config/YAMLParser.cpp b/src/Common/Config/YAMLParser.cpp index a1de14afc13..72706cb98ba 100644 --- a/src/Common/Config/YAMLParser.cpp +++ b/src/Common/Config/YAMLParser.cpp @@ -112,6 +112,11 @@ namespace { if (key == "#text" && value_node.IsScalar()) { + for (Node * child_node = parent_xml_node.firstChild(); child_node; child_node = child_node->nextSibling()) + if (child_node->nodeType() == Node::TEXT_NODE) + throw Exception(ErrorCodes::CANNOT_PARSE_YAML, + "YAMLParser has encountered node with several text nodes " + "and cannot continue parsing of the file"); std::string value = value_node.as(); Poco::AutoPtr xml_value = xml_document->createTextNode(value); parent_xml_node.appendChild(xml_value); From a2b170a18e7db041eb41e631f693b3ddec8e79a7 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 21 Jul 2023 17:42:55 +0200 Subject: [PATCH 0995/2047] Avoid exception which I didn't understand --- src/Storages/StorageReplicatedMergeTree.cpp | 33 +++++++++++++-------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 841b646a126..bc8dbfa0e1f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4905,20 +4905,29 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() if (shutdown_prepared_called.exchange(true)) return; - auto settings_ptr = getSettings(); - /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP. - fetcher.blocker.cancelForever(); - merger_mutator.merges_blocker.cancelForever(); - parts_mover.moves_blocker.cancelForever(); - stopBeingLeader(); + try + { + auto settings_ptr = getSettings(); + /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP. + fetcher.blocker.cancelForever(); + merger_mutator.merges_blocker.cancelForever(); + parts_mover.moves_blocker.cancelForever(); + stopBeingLeader(); - if (attach_thread) - attach_thread->shutdown(); + if (attach_thread) + attach_thread->shutdown(); - restarting_thread.shutdown(/* part_of_full_shutdown */true); - /// Explicetly set the event, because the restarting thread will not set it again - startup_event.set(); - shutdown_deadline.emplace(std::chrono::system_clock::now() + std::chrono::milliseconds(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds())); + restarting_thread.shutdown(/* part_of_full_shutdown */true); + /// Explicetly set the event, because the restarting thread will not set it again + startup_event.set(); + shutdown_deadline.emplace(std::chrono::system_clock::now() + std::chrono::milliseconds(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds())); + } + catch (...) + { + /// Don't wait anything in case of inproper prepare for shutdown + shutdown_deadline.emplace(std::chrono::system_clock::now()); + throw; + } } void StorageReplicatedMergeTree::partialShutdown() From 538d1f7c173cae793c65e48dca039d2460de1e4a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 18 Jul 2023 18:55:34 +0200 Subject: [PATCH 0996/2047] Kill the runner process with all subprocesses --- tests/ci/worker/init_runner.sh | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 171dd743f90..5939ae8ced2 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -102,7 +102,8 @@ check_proceed_spot_termination() { runner_pid=$(pgrep Runner.Listener) if [ -n "$runner_pid" ]; then # Kill the runner to not allow it cancelling the job - kill -9 "$runner_pid" + # shellcheck disable=SC2046 + kill -9 $(list_children "$runner_pid") fi sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" terminate_and_exit @@ -234,6 +235,19 @@ is_job_assigned() { || return 1 } +list_children () { + local children + children=$(ps --ppid "$1" -o pid=) + if [ -z "$children" ]; then + return + fi + + for pid in $children; do + list_children "$pid" + done + echo "$children" +} + while true; do runner_pid=$(pgrep Runner.Listener) echo "Got runner pid '$runner_pid'" @@ -275,7 +289,8 @@ while true; do echo "During the metadata check the job was assigned, continue" continue fi - kill -9 "$runner_pid" + # shellcheck disable=SC2046 + kill -9 $(list_children "$runner_pid") sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" terminate_on_event fi From da37f32d4b4ed31eb740ed21ee910dcccba16fa0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 19 Jul 2023 15:11:14 +0200 Subject: [PATCH 0997/2047] Attempt to delete the runner before shutting down --- tests/ci/worker/init_runner.sh | 20 +++++++------------- 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 5939ae8ced2..4fbfa748d16 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -12,7 +12,8 @@ echo "Running init script" export DEBIAN_FRONTEND=noninteractive export RUNNER_HOME=/home/ubuntu/actions-runner -export RUNNER_URL="https://github.com/ClickHouse" +export RUNNER_ORG="ClickHouse" +export RUNNER_URL="https://github.com/${RUNNER_ORG}" # Funny fact, but metadata service has fixed IP INSTANCE_ID=$(ec2metadata --instance-id) export INSTANCE_ID @@ -282,18 +283,11 @@ while true; do RUNNER_AGE=$(( $(date +%s) - $(stat -c +%Y /proc/"$runner_pid" 2>/dev/null || date +%s) )) echo "The runner is launched $RUNNER_AGE seconds ago and still has hot received the job" if (( 60 < RUNNER_AGE )); then - echo "Check if the instance should tear down" - if ! no_terminating_metadata; then - # Another check if the worker still didn't start - if is_job_assigned; then - echo "During the metadata check the job was assigned, continue" - continue - fi - # shellcheck disable=SC2046 - kill -9 $(list_children "$runner_pid") - sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" - terminate_on_event - fi + echo "Attempt to delete the runner for a graceful shutdown" + sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" \ + || continue + echo "Runner didn't launch or have assigned jobs after ${RUNNER_AGE} seconds, shutting down" + terminate_and_exit fi fi sleep 5 From a117997d5b9b1386963b7f1d76e2d0a34155198c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 21 Jul 2023 16:12:04 +0200 Subject: [PATCH 0998/2047] Show the runner type in the job logs --- tests/ci/worker/init_runner.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 4fbfa748d16..d452c985407 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -173,6 +173,7 @@ set -uo pipefail echo "Runner's public DNS: $(ec2metadata --public-hostname)" echo "Runner's labels: ${LABELS}" +echo "Runner's instance type: $(ec2metadata --instance-type)" EOF # Create a post-run script that will restart docker daemon before the job started From 083be24a74d34b9e33ae5d3751cdbbb958d9f6fb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 21 Jul 2023 17:11:06 +0200 Subject: [PATCH 0999/2047] Improve logic of checking valid cached values in lambdas --- .../lambda_shared_package/lambda_shared/__init__.py | 9 +++++++++ .../ci/lambda_shared_package/lambda_shared/token.py | 11 +++++------ tests/ci/terminate_runner_lambda/app.py | 13 ++++--------- 3 files changed, 18 insertions(+), 15 deletions(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/__init__.py b/tests/ci/lambda_shared_package/lambda_shared/__init__.py index c56994cc86a..aa88342fcc3 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/__init__.py +++ b/tests/ci/lambda_shared_package/lambda_shared/__init__.py @@ -219,3 +219,12 @@ def list_runners(access_token: str) -> RunnerDescriptions: result.append(desc) return result + + +def cached_value_is_valid(updated_at: float, ttl: float) -> bool: + "a common function to identify if cachable value is still valid" + if updated_at == 0: + return False + if time.time() - ttl < updated_at: + return True + return False diff --git a/tests/ci/lambda_shared_package/lambda_shared/token.py b/tests/ci/lambda_shared_package/lambda_shared/token.py index d3bf15ab259..6d5653f6a58 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/token.py +++ b/tests/ci/lambda_shared_package/lambda_shared/token.py @@ -8,6 +8,8 @@ import boto3 # type: ignore import jwt import requests # type: ignore +from . import cached_value_is_valid + def get_key_and_app_from_aws() -> Tuple[str, int]: secret_name = "clickhouse_github_secret_key" @@ -68,7 +70,7 @@ def get_access_token_by_key_app(private_key: str, app_id: int) -> str: @dataclass class CachedToken: - time: int + time: float value: str updating: bool = False @@ -81,12 +83,9 @@ def get_cached_access_token() -> str: return _cached_token.value # Indicate that the value is updating now, so the cached value can be # used. The first setting and close-to-ttl are not counted as update - if _cached_token.time != 0 or time.time() - 590 < _cached_token.time: - _cached_token.updating = True - else: - _cached_token.updating = False + _cached_token.updating = cached_value_is_valid(_cached_token.time, 590) private_key, app_id = get_key_and_app_from_aws() - _cached_token.time = int(time.time()) + _cached_token.time = time.time() _cached_token.value = get_access_token_by_key_app(private_key, app_id) _cached_token.updating = False return _cached_token.value diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index ed198d855b9..01d1f80ad29 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -9,13 +9,13 @@ from typing import Any, Dict, List import boto3 # type: ignore -from lambda_shared import RunnerDescriptions, list_runners +from lambda_shared import RunnerDescriptions, list_runners, cached_value_is_valid from lambda_shared.token import get_access_token_by_key_app, get_cached_access_token @dataclass class CachedInstances: - time: int + time: float value: dict updating: bool = False @@ -27,17 +27,12 @@ def get_cached_instances() -> dict: """return cached instances description with updating it once per five minutes""" if time.time() - 250 < cached_instances.time or cached_instances.updating: return cached_instances.value - # Indicate that the value is updating now, so the cached value can be - # used. The first setting and close-to-ttl are not counted as update - if cached_instances.time != 0 or time.time() - 300 < cached_instances.time: - cached_instances.updating = True - else: - cached_instances.updating = False + cached_instances.updating = cached_value_is_valid(cached_instances.time, 300) ec2_client = boto3.client("ec2") instances_response = ec2_client.describe_instances( Filters=[{"Name": "instance-state-name", "Values": ["running"]}] ) - cached_instances.time = int(time.time()) + cached_instances.time = time.time() cached_instances.value = { instance["InstanceId"]: instance for reservation in instances_response["Reservations"] From 101214cb8f6b3f9d3e9eed9858436259fe411976 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 21 Jul 2023 17:11:36 +0200 Subject: [PATCH 1000/2047] Fix broken termination lambda by caching runners for at most 15 seconds --- tests/ci/terminate_runner_lambda/app.py | 24 +++++++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/tests/ci/terminate_runner_lambda/app.py b/tests/ci/terminate_runner_lambda/app.py index 01d1f80ad29..010f7dd6734 100644 --- a/tests/ci/terminate_runner_lambda/app.py +++ b/tests/ci/terminate_runner_lambda/app.py @@ -42,6 +42,28 @@ def get_cached_instances() -> dict: return cached_instances.value +@dataclass +class CachedRunners: + time: float + value: RunnerDescriptions + updating: bool = False + + +cached_runners = CachedRunners(0, []) + + +def get_cached_runners(access_token: str) -> RunnerDescriptions: + """From time to time request to GH api costs up to 3 seconds, and + it's a disaster from the termination lambda perspective""" + if time.time() - 5 < cached_runners.time or cached_instances.updating: + return cached_runners.value + cached_runners.updating = cached_value_is_valid(cached_runners.time, 15) + cached_runners.value = list_runners(access_token) + cached_runners.time = time.time() + cached_runners.updating = False + return cached_runners.value + + def how_many_instances_to_kill(event_data: dict) -> Dict[str, int]: data_array = event_data["CapacityToTerminate"] to_kill_by_zone = {} # type: Dict[str, int] @@ -99,7 +121,7 @@ def main(access_token: str, event: dict) -> Dict[str, List[str]]: ) print("Time spent on the requests to AWS: ", time.time() - start) - runners = list_runners(access_token) + runners = get_cached_runners(access_token) runner_ids = set(runner.name for runner in runners) # We used to delete potential hosts to terminate from GitHub runners pool, # but the documentation states: From ed97284bfae5b18f8dbc8841e8b296edd45cd286 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 21 Jul 2023 19:28:28 +0200 Subject: [PATCH 1001/2047] Update src/Storages/StorageReplicatedMergeTree.cpp --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bc8dbfa0e1f..f191440442d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4924,7 +4924,7 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() } catch (...) { - /// Don't wait anything in case of inproper prepare for shutdown + /// Don't wait anything in case of improper prepare for shutdown shutdown_deadline.emplace(std::chrono::system_clock::now()); throw; } From 6c8d5ca0a554ecc4fee32269858797d139f3c02a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 21 Jul 2023 21:33:51 +0000 Subject: [PATCH 1002/2047] Fix: remove redundant distinct with views --- src/Interpreters/ActionsDAG.cpp | 18 +++++++++++---- ...x_remove_dedundant_distinct_view.reference | 13 +++++++++++ ...810_fix_remove_dedundant_distinct_view.sql | 22 +++++++++++++++++++ 3 files changed, 49 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference create mode 100644 tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 906875dd314..ce273e78ff3 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2511,11 +2511,21 @@ FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr /// find input node which refers to the output node /// consider only aliases on the path const auto * node = output_node; - while (node && node->type == ActionsDAG::ActionType::ALIAS) + while (node) { - /// alias has only one child - chassert(node->children.size() == 1); - node = node->children.front(); + if (node->type == ActionsDAG::ActionType::ALIAS) + { + node = node->children.front(); + } + /// materiailze can occure when dealing with views, special case + /// TODO: not sure if it should be done here, looks too generic place + else if (node->type == ActionsDAG::ActionType::FUNCTION && node->function_base->getName() == "materialize") + { + chassert(node->children.size() == 1); + node = node->children.front(); + } + else + break; } if (node && node->type == ActionsDAG::ActionType::INPUT) index.emplace(output_node->result_name, node); diff --git a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference new file mode 100644 index 00000000000..01f14f82e94 --- /dev/null +++ b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference @@ -0,0 +1,13 @@ +-- { echoOn } +set query_plan_remove_redundant_distinct=1; +-- DISTINCT has to be removed since the view already has DISTINCT on the same column +SELECT count() +FROM +( + EXPLAIN SELECT DISTINCT x FROM tab_v +) +WHERE explain ILIKE '%distinct%'; +2 +SELECT DISTINCT x FROM tab_v; +2 +1 diff --git a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql new file mode 100644 index 00000000000..99fc24dae8b --- /dev/null +++ b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql @@ -0,0 +1,22 @@ +set allow_experimental_analyzer=1; + +drop table if exists tab_v; +drop table if exists tab; +create table tab (x UInt64, y UInt64) engine MergeTree() order by (x, y); +insert into tab values(1, 1); +insert into tab values(1, 2); +insert into tab values(2, 1); + +create view tab_v as select distinct(x) from tab; + +-- { echoOn } +set query_plan_remove_redundant_distinct=1; +-- DISTINCT has to be removed since the view already has DISTINCT on the same column +SELECT count() +FROM +( + EXPLAIN SELECT DISTINCT x FROM tab_v +) +WHERE explain ILIKE '%distinct%'; + +SELECT DISTINCT x FROM tab_v; From c6ffc9f266f1bb8a667a3d5beff9bd47a288ef74 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 01:05:39 +0300 Subject: [PATCH 1003/2047] Update 02815_fix_not_found_constants_col_in_block.sql --- .../0_stateless/02815_fix_not_found_constants_col_in_block.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql index c56d59c72d6..fa784cf12e3 100644 --- a/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql +++ b/tests/queries/0_stateless/02815_fix_not_found_constants_col_in_block.sql @@ -3,3 +3,4 @@ CREATE TABLE t0 (vkey UInt32, c0 Float32, primary key(c0)) engine = AggregatingM insert into t0 values (19000, 1); select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc, c_2_2 asc; select null as c_2_0, ref_2.c0 as c_2_1, ref_2.vkey as c_2_2 from t0 as ref_2 order by c_2_0 asc, c_2_1 asc; +DROP TABLE t0; From 687cbc57bba42a67b62b1b717e51a5be7e14b733 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 21 Jul 2023 22:15:02 +0000 Subject: [PATCH 1004/2047] Fix typo --- src/Interpreters/ActionsDAG.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index ce273e78ff3..284c42b658a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2517,7 +2517,7 @@ FindOriginalNodeForOutputName::FindOriginalNodeForOutputName(const ActionsDAGPtr { node = node->children.front(); } - /// materiailze can occure when dealing with views, special case + /// materiailze() function can occur when dealing with views /// TODO: not sure if it should be done here, looks too generic place else if (node->type == ActionsDAG::ActionType::FUNCTION && node->function_base->getName() == "materialize") { From 5ec63c782c6bdd62705f26cc9b09e8a640ca9da8 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sat, 22 Jul 2023 00:15:05 +0200 Subject: [PATCH 1005/2047] Fixed inserting into Buffer engine by not throwing exception from DatabaseCatalog::tryGetTable() when database name is empty --- src/Interpreters/DatabaseCatalog.cpp | 3 ++- ...rentDatabase_for_table_functions.reference | 17 +++++++++++++++++ ...14_currentDatabase_for_table_functions.sql | 19 +++++++++++++++++++ 3 files changed, 38 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference create mode 100644 tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index e0b6348ed3c..f9ed2c0d5ca 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -344,7 +344,8 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( DatabasePtr database; { std::lock_guard lock{databases_mutex}; - auto it = databases.find(table_id.getDatabaseName()); + // hasDatabase() to avod getDatabaseName() throwing exception if database is empty. + auto it = table_id.hasDatabase() ? databases.find(table_id.getDatabaseName()) : databases.end(); if (databases.end() == it) { if (exception) diff --git a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference new file mode 100644 index 00000000000..7ff95106d3d --- /dev/null +++ b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.reference @@ -0,0 +1,17 @@ +-- Based on https://github.com/ClickHouse/ClickHouse/issues/52436 +-- Test that inserts performed via Buffer table engine land into destination table. +-- { echoOn } + +DROP TABLE IF EXISTS null_table; +DROP TABLE IF EXISTS null_table_buffer; +DROP TABLE IF EXISTS null_mv; +DROP VIEW IF EXISTS number_view; +CREATE TABLE null_table (number UInt64) ENGINE = Null; +CREATE VIEW number_view as SELECT * FROM numbers(10) as tb; +CREATE MATERIALIZED VIEW null_mv Engine = Log AS SELECT * FROM null_table LEFT JOIN number_view as tb USING number; +CREATE TABLE null_table_buffer (number UInt64) ENGINE = Buffer(currentDatabase(), null_table, 1, 1, 1, 100, 200, 10000, 20000); +INSERT INTO null_table_buffer VALUES (1); +SELECT sleep(3) FORMAT Null; +-- Insert about should've landed into `null_mv` +SELECT count() FROM null_mv; +1 diff --git a/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql new file mode 100644 index 00000000000..74b5cf5f432 --- /dev/null +++ b/tests/queries/0_stateless/02814_currentDatabase_for_table_functions.sql @@ -0,0 +1,19 @@ +-- Based on https://github.com/ClickHouse/ClickHouse/issues/52436 +-- Test that inserts performed via Buffer table engine land into destination table. +-- { echoOn } + +DROP TABLE IF EXISTS null_table; +DROP TABLE IF EXISTS null_table_buffer; +DROP TABLE IF EXISTS null_mv; +DROP VIEW IF EXISTS number_view; + +CREATE TABLE null_table (number UInt64) ENGINE = Null; +CREATE VIEW number_view as SELECT * FROM numbers(10) as tb; +CREATE MATERIALIZED VIEW null_mv Engine = Log AS SELECT * FROM null_table LEFT JOIN number_view as tb USING number; + +CREATE TABLE null_table_buffer (number UInt64) ENGINE = Buffer(currentDatabase(), null_table, 1, 1, 1, 100, 200, 10000, 20000); +INSERT INTO null_table_buffer VALUES (1); +SELECT sleep(3) FORMAT Null; + +-- Insert about should've landed into `null_mv` +SELECT count() FROM null_mv; From 3cc7f137bd5f7623280a22b4a32d9a3169dcdfea Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sat, 22 Jul 2023 00:45:12 +0200 Subject: [PATCH 1006/2047] add tests with connection reset by peer --- .../integration/helpers/s3_mocks/broken_s3.py | 270 +++++++++++------- .../test_checking_s3_blobs_paranoid/test.py | 214 +++++++++++++- tests/integration/test_merge_tree_s3/test.py | 6 +- 3 files changed, 388 insertions(+), 102 deletions(-) diff --git a/tests/integration/helpers/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py index 8ff4f9e9203..0c794ae1ad4 100644 --- a/tests/integration/helpers/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -7,11 +7,18 @@ import urllib.parse import http.server import socketserver import string +import socket +import struct INF_COUNT = 100000000 +def _and_then(value, func): + assert callable(func) + return None if value is None else func(value) + + class MockControl: def __init__(self, cluster, container, port): self._cluster = cluster @@ -30,8 +37,8 @@ class MockControl: ) assert response == "OK", response - def setup_error_at_object_upload(self, count=None, after=None): - url = f"http://localhost:{self._port}/mock_settings/error_at_object_upload?nothing=1" + def setup_action(self, when, count=None, after=None, action="error_500", action_args=None): + url = f"http://localhost:{self._port}/mock_settings/{when}?nothing=1" if count is not None: url += f"&count={count}" @@ -39,25 +46,12 @@ class MockControl: if after is not None: url += f"&after={after}" - response = self._cluster.exec_in_container( - self._cluster.get_container_id(self._container), - [ - "curl", - "-s", - url, - ], - nothrow=True, - ) - assert response == "OK", response + if action is not None: + url += f"&action={action}" - def setup_error_at_part_upload(self, count=None, after=None): - url = f"http://localhost:{self._port}/mock_settings/error_at_part_upload?nothing=1" - - if count is not None: - url += f"&count={count}" - - if after is not None: - url += f"&after={after}" + if action_args is not None: + for x in action_args: + url += f"&action_args={x}" response = self._cluster.exec_in_container( self._cluster.get_container_id(self._container), @@ -70,22 +64,14 @@ class MockControl: ) assert response == "OK", response - def setup_error_at_create_multi_part_upload(self, count=None): - url = f"http://localhost:{self._port}/mock_settings/error_at_create_multi_part_upload" + def setup_at_object_upload(self, **kwargs): + self.setup_action("at_object_upload", **kwargs) - if count is not None: - url += f"?count={count}" + def setup_at_part_upload(self, **kwargs): + self.setup_action("at_part_upload", **kwargs) - response = self._cluster.exec_in_container( - self._cluster.get_container_id(self._container), - [ - "curl", - "-s", - url, - ], - nothrow=True, - ) - assert response == "OK", response + def setup_at_create_multi_part_upload(self, **kwargs): + self.setup_action("at_create_multi_part_upload", **kwargs) def setup_fake_puts(self, part_length): response = self._cluster.exec_in_container( @@ -166,13 +152,82 @@ class _ServerRuntime: return _runtime.slow_put.timeout return None + class Expected500ErrorAction: + def inject_error(self, request_handler): + data = ( + '' + "" + "ExpectedError" + "mock s3 injected error" + "txfbd566d03042474888193-00608d7537" + "" + ) + request_handler.write_error(data) + + class RedirectAction: + def __init__(self, host="localhost", port=1): + self.dst_host = _and_then(host, str) + self.dst_port = _and_then(port, int) + + def inject_error(self, request_handler): + request_handler.redirect(host=self.dst_host, port=self.dst_port) + + class ConnectionResetByPeerAction: + def __init__(self, with_partial_data=None): + self.partial_data = "" + if with_partial_data is not None: + self.partial_data = ( + '\n' + "\n" + ) + + def inject_error(self, request_handler): + request_handler.read_all_input() + + if self.partial_data: + request_handler.send_response(200) + request_handler.send_header("Content-Type", "text/xml") + request_handler.send_header("Content-Length", 10000) + request_handler.end_headers() + request_handler.wfile.write(bytes(self.partial_data, "UTF-8")) + + time.sleep(1) + request_handler.connection.setsockopt( + socket.SOL_SOCKET, + socket.SO_LINGER, + struct.pack('ii', 1, 0) + ) + request_handler.connection.close() + + class ConnectionRefusedAction(RedirectAction): + pass + class CountAfter: - def __init__(self, count_=None, after_=None): + def __init__(self, count_=None, after_=None, action_=None, action_args_=[]): self.count = count_ if count_ is not None else INF_COUNT self.after = after_ if after_ is not None else 0 + self.action = action_ + self.action_args = action_args_ + if self.action == "connection_refused": + self.error_handler = _ServerRuntime.ConnectionRefusedAction() + elif self.action == "connection_reset_by_peer": + self.error_handler = _ServerRuntime.ConnectionResetByPeerAction(*self.action_args) + elif self.action == "redirect_to": + self.error_handler = _ServerRuntime.RedirectAction(*self.action_args) + else: + self.error_handler = _ServerRuntime.Expected500ErrorAction() + + @staticmethod + def from_cgi_params(params): + return _ServerRuntime.CountAfter( + count_=_and_then(params.get("count", [None])[0], int), + after_=_and_then(params.get("after", [None])[0], int), + action_=params.get("action", [None])[0], + action_args_=params.get("action_args", []), + ) def __str__(self): - return f"count:{self.count} after:{self.after}" + return f"count:{self.count} after:{self.after} action:{self.action} action_args:{self.action_args}" def has_effect(self): if self.after: @@ -183,15 +238,18 @@ class _ServerRuntime: return True return False + def inject_error(self, request_handler): + self.error_handler.inject_error(request_handler) + def __init__(self): self.lock = threading.Lock() - self.error_at_part_upload = None - self.error_at_object_upload = None + self.at_part_upload = None + self.at_object_upload = None self.fake_put_when_length_bigger = None self.fake_uploads = dict() self.slow_put = None self.fake_multipart_upload = None - self.error_at_create_multi_part_upload = None + self.at_create_multi_part_upload = None def register_fake_upload(self, upload_id, key): with self.lock: @@ -205,23 +263,18 @@ class _ServerRuntime: def reset(self): with self.lock: - self.error_at_part_upload = None - self.error_at_object_upload = None + self.at_part_upload = None + self.at_object_upload = None self.fake_put_when_length_bigger = None self.fake_uploads = dict() self.slow_put = None self.fake_multipart_upload = None - self.error_at_create_multi_part_upload = None + self.at_create_multi_part_upload = None _runtime = _ServerRuntime() -def _and_then(value, func): - assert callable(func) - return None if value is None else func(value) - - def get_random_string(length): # choose from all lowercase letter letters = string.ascii_lowercase @@ -239,7 +292,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): def _ping(self): self._ok() - def _read_out(self): + def read_all_input(self): content_length = int(self.headers.get("Content-Length", 0)) to_read = content_length while to_read > 0: @@ -250,36 +303,38 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): str(self.rfile.read(size)) to_read -= size - def _redirect(self): - self._read_out() + def redirect(self, host=None, port=None): + if host is None and port is None: + host = self.server.upstream_host + port = self.server.upstream_port + + self.read_all_input() self.send_response(307) url = ( - f"http://{self.server.upstream_host}:{self.server.upstream_port}{self.path}" + f"http://{host}:{port}{self.path}" ) + self.log_message("redirect to %s", url) self.send_header("Location", url) self.end_headers() self.wfile.write(b"Redirected") - def _error(self, data): - self._read_out() + def write_error(self, data, content_length=None): + if content_length is None: + content_length = len(data) + self.log_message("write_error %s", data) + self.read_all_input() self.send_response(500) self.send_header("Content-Type", "text/xml") + self.send_header("Content-Length", str(content_length)) self.end_headers() - self.wfile.write(bytes(data, "UTF-8")) - - def _error_expected_500(self): - self._error( - '' - "" - "ExpectedError" - "mock s3 injected error" - "txfbd566d03042474888193-00608d7537" - "" - ) + if data: + self.wfile.write(bytes(data, "UTF-8")) def _fake_put_ok(self): - self._read_out() + self.log_message("fake put") + + self.read_all_input() self.send_response(200) self.send_header("Content-Type", "text/xml") @@ -288,7 +343,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.end_headers() def _fake_uploads(self, path, upload_id): - self._read_out() + self.read_all_input() parts = [x for x in path.split("/") if x] bucket = parts[0] @@ -310,7 +365,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.wfile.write(bytes(data, "UTF-8")) def _fake_post_ok(self, path): - self._read_out() + self.read_all_input() parts = [x for x in path.split("/") if x] bucket = parts[0] @@ -338,22 +393,18 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): path = [x for x in parts.path.split("/") if x] assert path[0] == "mock_settings", path if len(path) < 2: - return self._error("_mock_settings: wrong command") + return self.write_error("_mock_settings: wrong command") - if path[1] == "error_at_part_upload": + if path[1] == "at_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.error_at_part_upload = _ServerRuntime.CountAfter( - count_=_and_then(params.get("count", [None])[0], int), - after_=_and_then(params.get("after", [None])[0], int), - ) + _runtime.at_part_upload = _ServerRuntime.CountAfter.from_cgi_params(params) + self.log_message("set at_part_upload %s", _runtime.at_part_upload) return self._ok() - if path[1] == "error_at_object_upload": + if path[1] == "at_object_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.error_at_object_upload = _ServerRuntime.CountAfter( - count_=_and_then(params.get("count", [None])[0], int), - after_=_and_then(params.get("after", [None])[0], int), - ) + _runtime.at_object_upload = _ServerRuntime.CountAfter.from_cgi_params(params) + self.log_message("set at_object_upload %s", _runtime.at_object_upload) return self._ok() if path[1] == "fake_puts": @@ -361,6 +412,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): _runtime.fake_put_when_length_bigger = int( params.get("when_length_bigger", [1024 * 1024])[0] ) + self.log_message("set fake_puts %s", _runtime.fake_put_when_length_bigger) return self._ok() if path[1] == "slow_put": @@ -376,20 +428,21 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "setup_fake_multpartuploads": _runtime.fake_multipart_upload = True + self.log_message("set setup_fake_multpartuploads") return self._ok() - if path[1] == "error_at_create_multi_part_upload": + if path[1] == "at_create_multi_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.error_at_create_multi_part_upload = int( - params.get("count", [INF_COUNT])[0] - ) + _runtime.at_create_multi_part_upload = _ServerRuntime.CountAfter.from_cgi_params(params) + self.log_message("set at_create_multi_part_upload %s", _runtime.at_create_multi_part_upload) return self._ok() if path[1] == "reset": _runtime.reset() + self.log_message("reset") return self._ok() - return self._error("_mock_settings: wrong command") + return self.write_error("_mock_settings: wrong command") def do_GET(self): if self.path == "/": @@ -398,7 +451,8 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if self.path.startswith("/mock_settings"): return self._mock_settings() - return self._redirect() + self.log_message("get redirect") + return self.redirect() def do_PUT(self): content_length = int(self.headers.get("Content-Length", 0)) @@ -414,30 +468,52 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): upload_id = params.get("uploadId", [None])[0] if upload_id is not None: - if _runtime.error_at_part_upload is not None: - if _runtime.error_at_part_upload.has_effect(): - return self._error_expected_500() + if _runtime.at_part_upload is not None: + self.log_message( + "put error_at_object_upload %s, %s, %s", + _runtime.at_part_upload, + upload_id, + parts, + ) + + if _runtime.at_part_upload.has_effect(): + return _runtime.at_part_upload.inject_error(self) if _runtime.fake_multipart_upload: if _runtime.is_fake_upload(upload_id, parts.path): return self._fake_put_ok() else: - if _runtime.error_at_object_upload is not None: - if _runtime.error_at_object_upload.has_effect(): - return self._error_expected_500() + if _runtime.at_object_upload is not None: + if _runtime.at_object_upload.has_effect(): + self.log_message( + "put error_at_object_upload %s, %s, %s", + _runtime.at_object_upload, + parts, + ) + return _runtime.at_object_upload.inject_error(self) if _runtime.fake_put_when_length_bigger is not None: if content_length > _runtime.fake_put_when_length_bigger: + self.log_message( + "put fake_put_when_length_bigger %s, %s, %s", + _runtime.fake_put_when_length_bigger, + content_length, + parts, + ) return self._fake_put_ok() - return self._redirect() + self.log_message( + "put redirect %s", + parts, + ) + return self.redirect() def do_POST(self): parts = urllib.parse.urlsplit(self.path) params = urllib.parse.parse_qs(parts.query, keep_blank_values=True) uploads = params.get("uploads", [None])[0] if uploads is not None: - if _runtime.error_at_create_multi_part_upload: - _runtime.error_at_create_multi_part_upload -= 1 - return self._error_expected_500() + if _runtime.at_create_multi_part_upload is not None: + if _runtime.at_create_multi_part_upload.has_effect(): + return _runtime.at_create_multi_part_upload.inject_error(self) if _runtime.fake_multipart_upload: upload_id = get_random_string(5) @@ -448,13 +524,13 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if _runtime.is_fake_upload(upload_id, parts.path): return self._fake_post_ok(parts.path) - return self._redirect() + return self.redirect() def do_HEAD(self): - self._redirect() + self.redirect() def do_DELETE(self): - self._redirect() + self.redirect() class _ThreadedHTTPServer(socketserver.ThreadingMixIn, http.server.HTTPServer): diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index a80ad93d53d..b6b598c4557 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -41,6 +41,11 @@ def broken_s3(init_broken_s3): yield init_broken_s3 +@pytest.fixture(scope="module") +def init_connection_reset_by_peer(cluster): + yield start_s3_mock(cluster, "connection_reset_by_peer", "8084") + + def test_upload_after_check_works(cluster, broken_s3): node = cluster.instances["node"] @@ -91,7 +96,7 @@ def get_counters(node, query_id, log_type="ExceptionWhileProcessing"): def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression): node = cluster.instances["node"] - broken_s3.setup_error_at_create_multi_part_upload() + broken_s3.setup_at_create_multi_part_upload() insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_FAIL_CREATE_MPU_{compression}" error = node.query_and_get_error( @@ -134,7 +139,7 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( node = cluster.instances["node"] broken_s3.setup_fake_multpartuploads() - broken_s3.setup_error_at_part_upload(count=1, after=2) + broken_s3.setup_at_part_upload(count=1, after=2) insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_FAIL_UPLOAD_PART_{compression}" error = node.query_and_get_error( @@ -165,3 +170,208 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( assert count_create_multi_part_uploads == 1 assert count_upload_parts >= 2 assert count_s3_errors >= 2 + + +def test_when_s3_connection_refused_at_write_retried( + cluster, broken_s3 +): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_part_upload(count=3, after=2, action="connection_refused") + + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_REFUSED_RETRIED" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_refused_at_write_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id, log_type="QueryFinish" + ) + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 39 + assert count_s3_errors == 3 + + broken_s3.setup_at_part_upload(count=1000, after=2, action="connection_refused") + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_REFUSED_RETRIED_1" + error = node.query_and_get_error( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_refused_at_write_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + assert "Code: 499" in error, error + assert "Poco::Exception. Code: 1000, e.code() = 111, Connection refused" in error, error + + +def test_when_s3_connection_reset_by_peer_at_write_retried( + cluster, broken_s3 +): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_part_upload( + count=3, + after=2, + action="connection_reset_by_peer", + ) + + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_RETRIED" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_write_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id, log_type="QueryFinish" + ) + + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 39 + assert count_s3_errors == 3 + + broken_s3.setup_at_part_upload( + count=1000, + after=2, + action="connection_reset_by_peer", + ) + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_RETRIED_1" + error = node.query_and_get_error( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_write_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + assert "Code: 499" in error, error + assert "Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error, error + + +def test_when_s3_connection_reset_by_peer_at_read_retried( + cluster, broken_s3 +): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_create_multi_part_upload( + count=3, + after=0, + action="connection_reset_by_peer", + ) + + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_READ_RETRIED" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_read_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id, log_type="QueryFinish" + ) + + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 39 + assert count_s3_errors == 3 + + broken_s3.setup_at_create_multi_part_upload( + count=1000, + after=0, + action="connection_reset_by_peer", + ) + + insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_READ_RETRIED_1" + error = node.query_and_get_error( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_read_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + assert "Code: 499" in error, error + assert "Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error, error diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index f754bc905bf..d7c267eed50 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -783,9 +783,9 @@ def test_merge_canceled_by_s3_errors(cluster, broken_s3, node_name, storage_poli min_key = node.query("SELECT min(key) FROM test_merge_canceled_by_s3_errors") assert int(min_key) == 0, min_key - broken_s3.setup_error_at_object_upload() + broken_s3.setup_at_object_upload() broken_s3.setup_fake_multpartuploads() - broken_s3.setup_error_at_part_upload() + broken_s3.setup_at_part_upload() node.query("SYSTEM START MERGES test_merge_canceled_by_s3_errors") @@ -828,7 +828,7 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, broken_s3, node_name): settings={"materialize_ttl_after_modify": 0}, ) - broken_s3.setup_error_at_object_upload(count=1, after=1) + broken_s3.setup_at_object_upload(count=1, after=1) node.query("SYSTEM START MERGES merge_canceled_by_s3_errors_when_move") From 3398355e4649beaa2b0a3180aa687c364c245e81 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 02:04:30 +0200 Subject: [PATCH 1007/2047] Add something with unclear purpose --- docker/README.md | 2 +- tests/ci/docker_test.py | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/README.md b/docker/README.md index ec52ddd143e..76fdbf0f3f0 100644 --- a/docker/README.md +++ b/docker/README.md @@ -2,4 +2,4 @@ This directory contain Dockerfiles for `clickhouse-client` and `clickhouse-server`. They are updated in each release. -Also there is bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. +Also, there is a bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index d5d27f73694..61319041b9f 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -40,6 +40,7 @@ class TestDockerImageCheck(unittest.TestCase): [ di.DockerImage("docker/test/base", "clickhouse/test-base", False), di.DockerImage("docker/docs/builder", "clickhouse/docs-builder", True), + di.DockerImage("docker/test/sqltest", "clickhouse/sqltest", True), di.DockerImage( "docker/test/stateless", "clickhouse/stateless-test", From ae8f511ac5ffc6191394dd9fbfed9a0b082102e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 02:27:07 +0200 Subject: [PATCH 1008/2047] Fix a test --- tests/integration/test_zero_copy_fetch/configs/users.xml | 7 +++++++ tests/integration/test_zero_copy_fetch/test.py | 2 ++ 2 files changed, 9 insertions(+) create mode 100644 tests/integration/test_zero_copy_fetch/configs/users.xml diff --git a/tests/integration/test_zero_copy_fetch/configs/users.xml b/tests/integration/test_zero_copy_fetch/configs/users.xml new file mode 100644 index 00000000000..b0990ca3a60 --- /dev/null +++ b/tests/integration/test_zero_copy_fetch/configs/users.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/tests/integration/test_zero_copy_fetch/test.py b/tests/integration/test_zero_copy_fetch/test.py index 4f3d42096c3..dc79e5d8723 100644 --- a/tests/integration/test_zero_copy_fetch/test.py +++ b/tests/integration/test_zero_copy_fetch/test.py @@ -19,12 +19,14 @@ def started_cluster(): cluster.add_instance( "node1", main_configs=["configs/storage_conf.xml"], + user_configs=["configs/users.xml"], with_minio=True, with_zookeeper=True, ) cluster.add_instance( "node2", main_configs=["configs/storage_conf.xml"], + user_configs=["configs/users.xml"], with_minio=True, with_zookeeper=True, ) From 9a5aed35e24a9aa4d7de71971665449cf344f917 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 02:33:44 +0200 Subject: [PATCH 1009/2047] Add a note about potential caveats for the "session_timezone" setting --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cfcb56729d2..f267fa15276 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -775,7 +775,7 @@ class IColumn; M(Bool, allow_experimental_undrop_table_query, false, "Allow to use undrop query to restore dropped table in a limited time", 0) \ M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ - M(Timezone, session_timezone, "", "The default timezone for current session or query. The server default timezone if empty.", 0) \ + M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0)\ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. From e68234a231bf234d60ccfa262ca5a2374fb4f98a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:45:50 +0300 Subject: [PATCH 1010/2047] Revert "Re-add SipHash keyed functions" --- .../sql-reference/functions/hash-functions.md | 8 +- src/Functions/FunctionsHashing.h | 329 +++++------------- src/Functions/FunctionsHashingMisc.cpp | 5 - .../0_stateless/02534_keyed_siphash.reference | 37 -- .../0_stateless/02534_keyed_siphash.sql | 61 +--- .../02552_siphash128_reference.reference | 151 -------- .../02552_siphash128_reference.sql | 253 -------------- 7 files changed, 99 insertions(+), 745 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 556fe622c27..06097d92480 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -51,7 +51,7 @@ Calculates the MD5 from a string and returns the resulting set of bytes as Fixed If you do not need MD5 in particular, but you need a decent cryptographic 128-bit hash, use the ‘sipHash128’ function instead. If you want to get the same result as output by the md5sum utility, use lower(hex(MD5(s))). -## sipHash64 {#hash_functions-siphash64} +## sipHash64 (#hash_functions-siphash64) Produces a 64-bit [SipHash](https://en.wikipedia.org/wiki/SipHash) hash value. @@ -63,9 +63,9 @@ This is a cryptographic hash function. It works at least three times faster than The function [interprets](/docs/en/sql-reference/functions/type-conversion-functions.md/#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. It then combines the hashes by the following algorithm: -1. The first and the second hash value are concatenated to an array which is hashed. -2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. -3. This calculation is repeated for all remaining hash values of the original input. +1. The first and the second hash value are concatenated to an array which is hashed. +2. The previously calculated hash value and the hash of the third input parameter are hashed in a similar way. +3. This calculation is repeated for all remaining hash values of the original input. **Arguments** diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 82944630b10..279294b367c 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -79,51 +79,28 @@ namespace impl UInt64 key1 = 0; }; - struct SipHashKeyColumns + static SipHashKey parseSipHashKey(const ColumnWithTypeAndName & key) { - ColumnPtr key0; - ColumnPtr key1; - bool is_const; + SipHashKey ret{}; - size_t size() const - { - assert(key0 && key1); - assert(key0->size() == key1->size()); - return key0->size(); - } - SipHashKey getKey(size_t i) const - { - if (is_const) - i = 0; - const auto & key0data = assert_cast(*key0).getData(); - const auto & key1data = assert_cast(*key1).getData(); - return {key0data[i], key1data[i]}; - } - }; - - static SipHashKeyColumns parseSipHashKeyColumns(const ColumnWithTypeAndName & key) - { - const ColumnTuple * tuple = nullptr; - const auto * column = key.column.get(); - bool is_const = false; - if (isColumnConst(*column)) - { - is_const = true; - tuple = checkAndGetColumnConstData(column); - } - else - tuple = checkAndGetColumn(column); + const auto * tuple = checkAndGetColumn(key.column.get()); if (!tuple) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "key must be a tuple"); + if (tuple->tupleSize() != 2) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "wrong tuple size: key must be a tuple of 2 UInt64"); - SipHashKeyColumns ret{tuple->getColumnPtr(0), tuple->getColumnPtr(1), is_const}; - assert(ret.key0); - if (!checkColumn(*ret.key0)) + if (tuple->empty()) + return ret; + + if (const auto * key0col = checkAndGetColumn(&(tuple->getColumn(0)))) + ret.key0 = key0col->get64(0); + else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "first element of the key tuple is not UInt64"); - assert(ret.key1); - if (!checkColumn(*ret.key1)) + + if (const auto * key1col = checkAndGetColumn(&(tuple->getColumn(1)))) + ret.key1 = key1col->get64(0); + else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); return ret; @@ -352,10 +329,8 @@ struct SipHash64KeyedImpl static constexpr auto name = "sipHash64Keyed"; using ReturnType = UInt64; using Key = impl::SipHashKey; - using KeyColumns = impl::SipHashKeyColumns; - static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } - static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } + static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); } static UInt64 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash64Keyed(key.key0, key.key1, begin, size); } @@ -396,10 +371,8 @@ struct SipHash128KeyedImpl static constexpr auto name = "sipHash128Keyed"; using ReturnType = UInt128; using Key = impl::SipHashKey; - using KeyColumns = impl::SipHashKeyColumns; - static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } - static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } + static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); } static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) { return sipHash128Keyed(key.key0, key.key1, begin, size); } @@ -425,43 +398,13 @@ struct SipHash128ReferenceImpl using ReturnType = UInt128; - static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc(h1, h2); } + static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc(h1, h2); } static UInt128 apply(const char * data, const size_t size) { return sipHash128Reference(data, size); } static constexpr bool use_int_hash_for_pods = false; }; -struct SipHash128ReferenceKeyedImpl -{ - static constexpr auto name = "sipHash128ReferenceKeyed"; - using ReturnType = UInt128; - using Key = impl::SipHashKey; - using KeyColumns = impl::SipHashKeyColumns; - - static KeyColumns parseKeyColumns(const ColumnWithTypeAndName & key) { return impl::parseSipHashKeyColumns(key); } - static Key getKey(const KeyColumns & key, size_t i) { return key.getKey(i); } - - static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) - { - return sipHash128ReferenceKeyed(key.key0, key.key1, begin, size); - } - - static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2) - { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - UInt128 tmp; - reverseMemcpy(&tmp, &h1, sizeof(UInt128)); - h1 = tmp; - reverseMemcpy(&tmp, &h2, sizeof(UInt128)); - h2 = tmp; -#endif - UInt128 hashes[] = {h1, h2}; - return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt128)); - } - - static constexpr bool use_int_hash_for_pods = false; -}; /** Why we need MurmurHash2? * MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash. @@ -1080,7 +1023,7 @@ private: DECLARE_MULTITARGET_CODE( -template +template class FunctionAnyHash : public IFunction { public: @@ -1090,12 +1033,9 @@ private: using ToType = typename Impl::ReturnType; template - void executeIntType(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeIntType(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const { using ColVecType = ColumnVectorOrDecimal; - KeyType key{}; - if constexpr (Keyed) - key = Impl::getKey(key_cols, 0); if (const ColVecType * col_from = checkAndGetColumn(column)) { @@ -1104,9 +1044,6 @@ private: for (size_t i = 0; i < size; ++i) { ToType hash; - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); if constexpr (Impl::use_int_hash_for_pods) { @@ -1140,14 +1077,6 @@ private: } else if (auto col_from_const = checkAndGetColumnConst(column)) { - if constexpr (Keyed) - { - if (!key_cols.is_const) - { - ColumnPtr full_column = col_from_const->convertToFullColumn(); - return executeIntType(key_cols, full_column.get(), vec_to); - } - } auto value = col_from_const->template getValue(); ToType hash; @@ -1178,15 +1107,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) - { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); - } - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1194,12 +1116,9 @@ private: } template - void executeBigIntType(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeBigIntType(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const { using ColVecType = ColumnVectorOrDecimal; - KeyType key{}; - if constexpr (Keyed) - key = Impl::getKey(key_cols, 0); if (const ColVecType * col_from = checkAndGetColumn(column)) { @@ -1208,9 +1127,6 @@ private: for (size_t i = 0; i < size; ++i) { ToType hash; - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); if constexpr (std::endian::native == std::endian::little) hash = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); else @@ -1227,14 +1143,6 @@ private: } else if (auto col_from_const = checkAndGetColumnConst(column)) { - if constexpr (Keyed) - { - if (!key_cols.is_const) - { - ColumnPtr full_column = col_from_const->convertToFullColumn(); - return executeBigIntType(key_cols, full_column.get(), vec_to); - } - } auto value = col_from_const->template getValue(); ToType hash; @@ -1250,15 +1158,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) - { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); - } - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1266,16 +1167,10 @@ private: } template - void executeGeneric(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeGeneric(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const { - KeyType key{}; - if constexpr (Keyed) - key = Impl::getKey(key_cols, 0); for (size_t i = 0, size = column->size(); i < size; ++i) { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); StringRef bytes = column->getDataAt(i); const ToType hash = apply(key, bytes.data, bytes.size); if constexpr (first) @@ -1286,11 +1181,8 @@ private: } template - void executeString(const KeyColumnsType & key_cols, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeString(const KeyType & key, const IColumn * column, typename ColumnVector::Container & vec_to) const { - KeyType key{}; - if constexpr (Keyed) - key = Impl::getKey(key_cols, 0); if (const ColumnString * col_from = checkAndGetColumn(column)) { const typename ColumnString::Chars & data = col_from->getChars(); @@ -1300,9 +1192,6 @@ private: ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); const ToType hash = apply(key, reinterpret_cast(&data[current_offset]), offsets[i] - current_offset - 1); @@ -1323,9 +1212,6 @@ private: for (size_t i = 0; i < size; ++i) { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); const ToType hash = apply(key, reinterpret_cast(&data[i * n]), n); if constexpr (first) vec_to[i] = hash; @@ -1335,14 +1221,6 @@ private: } else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column)) { - if constexpr (Keyed) - { - if (!key_cols.is_const) - { - ColumnPtr full_column = col_from_const->convertToFullColumn(); - return executeString(key_cols, full_column.get(), vec_to); - } - } String value = col_from_const->getValue(); const ToType hash = apply(key, value.data(), value.size()); const size_t size = vec_to.size(); @@ -1350,15 +1228,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) - { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); vec_to[i] = combineHashes(key, vec_to[i], hash); - } - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", @@ -1366,7 +1237,7 @@ private: } template - void executeArray(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to) const + void executeArray(const KeyType & key, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to) const { const IDataType * nested_type = typeid_cast(*type).getNestedType().get(); @@ -1378,19 +1249,13 @@ private: typename ColumnVector::Container vec_temp(nested_size); bool nested_is_first = true; - executeForArgument(key_cols, nested_type, nested_column, vec_temp, nested_is_first); + executeForArgument(key, nested_type, nested_column, vec_temp, nested_is_first); const size_t size = offsets.size(); ColumnArray::Offset current_offset = 0; - KeyType key{}; - if constexpr (Keyed) - key = Impl::getKey(key_cols, 0); for (size_t i = 0; i < size; ++i) { - if constexpr (Keyed) - if (!key_cols.is_const && i != 0) - key = Impl::getKey(key_cols, i); ColumnArray::Offset next_offset = offsets[i]; ToType hash; @@ -1414,7 +1279,7 @@ private: { /// NOTE: here, of course, you can do without the materialization of the column. ColumnPtr full_column = col_from_const->convertToFullColumn(); - executeArray(key_cols, type, full_column.get(), vec_to); + executeArray(key, type, full_column.get(), vec_to); } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", @@ -1422,7 +1287,7 @@ private: } template - void executeAny(const KeyColumnsType & key_cols, const IDataType * from_type, const IColumn * icolumn, typename ColumnVector::Container & vec_to) const + void executeAny(const KeyType & key, const IDataType * from_type, const IColumn * icolumn, typename ColumnVector::Container & vec_to) const { WhichDataType which(from_type); @@ -1430,45 +1295,40 @@ private: throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument column '{}' size {} doesn't match result column size {} of function {}", icolumn->getName(), icolumn->size(), vec_to.size(), getName()); - if constexpr (Keyed) - if ((!key_cols.is_const && key_cols.size() != vec_to.size()) - || (key_cols.is_const && key_cols.size() != 1)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Key column size {} doesn't match result column size {} of function {}", key_cols.size(), vec_to.size(), getName()); - - if (which.isUInt8()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isUInt16()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isUInt32()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isUInt64()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isUInt128()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isUInt256()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isInt8()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isInt16()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isInt32()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isInt64()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isInt128()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isInt256()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isUUID()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isIPv4()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isIPv6()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isEnum8()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isEnum16()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isDate()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isDate32()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isDateTime()) executeIntType(key_cols, icolumn, vec_to); + if (which.isUInt8()) executeIntType(key, icolumn, vec_to); + else if (which.isUInt16()) executeIntType(key, icolumn, vec_to); + else if (which.isUInt32()) executeIntType(key, icolumn, vec_to); + else if (which.isUInt64()) executeIntType(key, icolumn, vec_to); + else if (which.isUInt128()) executeBigIntType(key, icolumn, vec_to); + else if (which.isUInt256()) executeBigIntType(key, icolumn, vec_to); + else if (which.isInt8()) executeIntType(key, icolumn, vec_to); + else if (which.isInt16()) executeIntType(key, icolumn, vec_to); + else if (which.isInt32()) executeIntType(key, icolumn, vec_to); + else if (which.isInt64()) executeIntType(key, icolumn, vec_to); + else if (which.isInt128()) executeBigIntType(key, icolumn, vec_to); + else if (which.isInt256()) executeBigIntType(key, icolumn, vec_to); + else if (which.isUUID()) executeBigIntType(key, icolumn, vec_to); + else if (which.isIPv4()) executeIntType(key, icolumn, vec_to); + else if (which.isIPv6()) executeBigIntType(key, icolumn, vec_to); + else if (which.isEnum8()) executeIntType(key, icolumn, vec_to); + else if (which.isEnum16()) executeIntType(key, icolumn, vec_to); + else if (which.isDate()) executeIntType(key, icolumn, vec_to); + else if (which.isDate32()) executeIntType(key, icolumn, vec_to); + else if (which.isDateTime()) executeIntType(key, icolumn, vec_to); /// TODO: executeIntType() for Decimal32/64 leads to incompatible result - else if (which.isDecimal32()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isDecimal64()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isDecimal128()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isDecimal256()) executeBigIntType(key_cols, icolumn, vec_to); - else if (which.isFloat32()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isFloat64()) executeIntType(key_cols, icolumn, vec_to); - else if (which.isString()) executeString(key_cols, icolumn, vec_to); - else if (which.isFixedString()) executeString(key_cols, icolumn, vec_to); - else if (which.isArray()) executeArray(key_cols, from_type, icolumn, vec_to); - else executeGeneric(key_cols, icolumn, vec_to); + else if (which.isDecimal32()) executeBigIntType(key, icolumn, vec_to); + else if (which.isDecimal64()) executeBigIntType(key, icolumn, vec_to); + else if (which.isDecimal128()) executeBigIntType(key, icolumn, vec_to); + else if (which.isDecimal256()) executeBigIntType(key, icolumn, vec_to); + else if (which.isFloat32()) executeIntType(key, icolumn, vec_to); + else if (which.isFloat64()) executeIntType(key, icolumn, vec_to); + else if (which.isString()) executeString(key, icolumn, vec_to); + else if (which.isFixedString()) executeString(key, icolumn, vec_to); + else if (which.isArray()) executeArray(key, from_type, icolumn, vec_to); + else executeGeneric(key, icolumn, vec_to); } - void executeForArgument(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) const + void executeForArgument(const KeyType & key, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) const { /// Flattening of tuples. if (const ColumnTuple * tuple = typeid_cast(column)) @@ -1477,7 +1337,7 @@ private: const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); for (size_t i = 0; i < tuple_size; ++i) - executeForArgument(key_cols, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); + executeForArgument(key, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); } else if (const ColumnTuple * tuple_const = checkAndGetColumnConstData(column)) { @@ -1487,24 +1347,24 @@ private: for (size_t i = 0; i < tuple_size; ++i) { auto tmp = ColumnConst::create(tuple_columns[i], column->size()); - executeForArgument(key_cols, tuple_types[i].get(), tmp.get(), vec_to, is_first); + executeForArgument(key, tuple_types[i].get(), tmp.get(), vec_to, is_first); } } else if (const auto * map = checkAndGetColumn(column)) { const auto & type_map = assert_cast(*type); - executeForArgument(key_cols, type_map.getNestedType().get(), map->getNestedColumnPtr().get(), vec_to, is_first); + executeForArgument(key, type_map.getNestedType().get(), map->getNestedColumnPtr().get(), vec_to, is_first); } else if (const auto * const_map = checkAndGetColumnConst(column)) { - executeForArgument(key_cols, type, const_map->convertToFullColumnIfConst().get(), vec_to, is_first); + executeForArgument(key, type, const_map->convertToFullColumnIfConst().get(), vec_to, is_first); } else { if (is_first) - executeAny(key_cols, type, column, vec_to); + executeAny(key, type, column, vec_to); else - executeAny(key_cols, type, column, vec_to); + executeAny(key, type, column, vec_to); } is_first = false; @@ -1535,33 +1395,30 @@ public: { auto col_to = ColumnVector::create(input_rows_count); - if (input_rows_count != 0) + typename ColumnVector::Container & vec_to = col_to->getData(); + + /// If using a "keyed" algorithm, the first argument is the key and + /// the data starts from the second argument. + /// Otherwise there is no key and all arguments are interpreted as data. + constexpr size_t first_data_argument = Keyed; + + if (arguments.size() <= first_data_argument) { - typename ColumnVector::Container & vec_to = col_to->getData(); + /// Return a fixed random-looking magic number when input is empty + vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); + } - /// If using a "keyed" algorithm, the first argument is the key and - /// the data starts from the second argument. - /// Otherwise there is no key and all arguments are interpreted as data. - constexpr size_t first_data_argument = Keyed; + KeyType key{}; + if constexpr (Keyed) + if (!arguments.empty()) + key = Impl::parseKey(arguments[0]); - if (arguments.size() <= first_data_argument) - { - /// Return a fixed random-looking magic number when input is empty - vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); - } - - KeyColumnsType key_cols{}; - if constexpr (Keyed) - if (!arguments.empty()) - key_cols = Impl::parseKeyColumns(arguments[0]); - - /// The function supports arbitrary number of arguments of arbitrary types. - bool is_first_argument = true; - for (size_t i = first_data_argument; i < arguments.size(); ++i) - { - const auto & col = arguments[i]; - executeForArgument(key_cols, col.type.get(), col.column.get(), vec_to, is_first_argument); - } + /// The function supports arbitrary number of arguments of arbitrary types. + bool is_first_argument = true; + for (size_t i = first_data_argument; i < arguments.size(); ++i) + { + const auto & col = arguments[i]; + executeForArgument(key, col.type.get(), col.column.get(), vec_to, is_first_argument); } if constexpr (std::is_same_v) /// backward-compatible @@ -1593,19 +1450,17 @@ public: ) // DECLARE_MULTITARGET_CODE -template -class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash +template +class FunctionAnyHash : public TargetSpecific::Default::FunctionAnyHash { public: explicit FunctionAnyHash(ContextPtr context) : selector(context) { - selector - .registerImplementation>(); + selector.registerImplementation>(); #if USE_MULTITARGET_CODE - selector.registerImplementation>(); - selector - .registerImplementation>(); + selector.registerImplementation>(); + selector.registerImplementation>(); #endif } @@ -1841,7 +1696,7 @@ struct NameIntHash32 { static constexpr auto name = "intHash32"; }; struct NameIntHash64 { static constexpr auto name = "intHash64"; }; using FunctionSipHash64 = FunctionAnyHash; -using FunctionSipHash64Keyed = FunctionAnyHash; +using FunctionSipHash64Keyed = FunctionAnyHash; using FunctionIntHash32 = FunctionIntHash; using FunctionIntHash64 = FunctionIntHash; #if USE_SSL @@ -1855,10 +1710,8 @@ using FunctionSHA384 = FunctionStringHashFixedString; using FunctionSHA512 = FunctionStringHashFixedString; #endif using FunctionSipHash128 = FunctionAnyHash; -using FunctionSipHash128Keyed = FunctionAnyHash; +using FunctionSipHash128Keyed = FunctionAnyHash; using FunctionSipHash128Reference = FunctionAnyHash; -using FunctionSipHash128ReferenceKeyed - = FunctionAnyHash; using FunctionCityHash64 = FunctionAnyHash; using FunctionFarmFingerprint64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; diff --git a/src/Functions/FunctionsHashingMisc.cpp b/src/Functions/FunctionsHashingMisc.cpp index f56568b2508..56c3c1ed00c 100644 --- a/src/Functions/FunctionsHashingMisc.cpp +++ b/src/Functions/FunctionsHashingMisc.cpp @@ -20,11 +20,6 @@ REGISTER_FUNCTION(Hashing) .examples{{"hash", "SELECT hex(sipHash128Reference('foo', '\\x01', 3))", ""}}, .categories{"Hash"} }); - factory.registerFunction(FunctionDocumentation{ - .description = "Same as [sipHash128Reference](#hash_functions-siphash128reference) but additionally takes an explicit key argument " - "instead of using a fixed key.", - .examples{{"hash", "SELECT hex(sipHash128ReferenceKeyed((506097522914230528, 1084818905618843912),'foo', '\\x01', 3));", ""}}, - .categories{"Hash"}}); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index a9f724365a8..ccc514e7ea2 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -197,40 +197,3 @@ E28DBDE7FE22E41C Check bug with hashing of const integer values 11862823756610506724 11862823756610506724 -86AE90BB6A238D3F6221457630142C9B -86AE90BB6A238D3F6221457630142C9B -Check memsan bug -18096612095653370192 -20AF99D3A87829E0 -12489502208762728797 -Check const columns -15080046610211022027 -15080046610211022027 -15080046610211022027 -15080046610211022027 -2E779C73D13981AA1AE19AFF9617EA49 -2E779C73D13981AA1AE19AFF9617EA49 -2E779C73D13981AA1AE19AFF9617EA49 -2E779C73D13981AA1AE19AFF9617EA49 -Check multiple keys as tuple from a table -11862823756610506724 -9357996107237883963 -86AE90BB6A238D3F6221457630142C9B -F6D93D8FEA6D7DECCDD95A7A0A2AA36D -Check multiple keys as separate ints from a table -11862823756610506724 -9357996107237883963 -86AE90BB6A238D3F6221457630142C9B -F6D93D8FEA6D7DECCDD95A7A0A2AA36D -Check constant key and data from a table -11862823756610506724 -11862823756610506724 -86AE90BB6A238D3F6221457630142C9B -86AE90BB6A238D3F6221457630142C9B -Check multiple keys as separate ints from a table with constant data -11862823756610506724 -9357996107237883963 -86AE90BB6A238D3F6221457630142C9B -F6D93D8FEA6D7DECCDD95A7A0A2AA36D -Check asan bug -0 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 4f3ae7d62bd..900b99f548a 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -263,10 +263,10 @@ select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); select sipHash128Keyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); -select sipHash64Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash128Keyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash64Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash128Keyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } +select sipHash64Keyed((0, 0), '1'); -- { serverError 48 } +select sipHash128Keyed((0, 0), '1'); -- { serverError 48 } +select sipHash64Keyed(toUInt64(0), '1'); -- { serverError 48 } +select sipHash128Keyed(toUInt64(0), '1'); -- { serverError 48 } select hex(sipHash64()); SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; @@ -280,57 +280,4 @@ INSERT INTO tab VALUES ((2, 2), 4); -- these two statements must produce the same result SELECT sipHash64Keyed(key, val) FROM tab; SELECT sipHash64Keyed(key, 4::UInt64) FROM tab; -SELECT hex(sipHash128Keyed(key, val)) FROM tab; -SELECT hex(sipHash128Keyed(key, 4::UInt64)) FROM tab; DROP TABLE tab; - -SELECT 'Check memsan bug'; -SELECT sipHash64Keyed((2::UInt64, toUInt64(2)), 4) GROUP BY toUInt64(2); -SELECT hex(sipHash64Keyed((toUInt64(9223372036854775806), toUInt64(-9223372036854775808)), char(2147483646, -2147483648, 1, 3, 4, 7, 2147483647))) GROUP BY toUInt64(257), (toUInt64(9223372036854775806), toUInt64(2147483646)); -SELECT sipHash64Keyed((toUInt64(9223372036854775806), 9223372036854775808::UInt64), char(2)) GROUP BY toUInt64(9223372036854775806); - -SELECT 'Check const columns'; -DROP TABLE IF EXISTS sipHashKeyed_test; -CREATE TABLE sipHashKeyed_test ENGINE = Memory() AS SELECT 1 a, 'test' b; -SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), 1, 'test'); -SELECT sipHash64(tuple(*)) FROM sipHashKeyed_test; -SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), tuple(*)) FROM sipHashKeyed_test; -SELECT sipHash64Keyed((toUInt64(0), toUInt64(0)), a, b) FROM sipHashKeyed_test; -SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), 1, 'test')); -SELECT hex(sipHash128(tuple(*))) FROM sipHashKeyed_test; -SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test; -SELECT hex(sipHash128Keyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test; -DROP TABLE sipHashKeyed_test; - -SELECT 'Check multiple keys as tuple from a table'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); -INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); -SELECT sipHash64Keyed(key, val) FROM sipHashKeyed_keys ORDER by key; -SELECT hex(sipHash128Keyed(key, val)) FROM sipHashKeyed_keys ORDER by key; -DROP TABLE sipHashKeyed_keys; - -SELECT 'Check multiple keys as separate ints from a table'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); -INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); -SELECT sipHash64Keyed((key0, key1), val) FROM sipHashKeyed_keys ORDER by key0; -SELECT hex(sipHash128Keyed((key0, key1), val)) FROM sipHashKeyed_keys ORDER by key0; -SELECT 'Check constant key and data from a table'; -SELECT sipHash64Keyed((2::UInt64, 2::UInt64), val) FROM sipHashKeyed_keys ORDER by val; -SELECT hex(sipHash128Keyed((2::UInt64, 2::UInt64), val)) FROM sipHashKeyed_keys ORDER by val; -DROP TABLE sipHashKeyed_keys; - -SELECT 'Check multiple keys as separate ints from a table with constant data'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES (2, 2); -INSERT INTO sipHashKeyed_keys VALUES (4, 4); -SELECT sipHash64Keyed((key0, key1), 4::UInt64) FROM sipHashKeyed_keys ORDER by key0; -SELECT hex(sipHash128Keyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0; -DROP TABLE sipHashKeyed_keys; - -SELECT 'Check asan bug'; -SELECT sipHash128((toUInt64(9223372036854775806), 1)) = sipHash128(1) GROUP BY sipHash128(1::UInt8), toUInt64(9223372036854775806); diff --git a/tests/queries/0_stateless/02552_siphash128_reference.reference b/tests/queries/0_stateless/02552_siphash128_reference.reference index ece9f6a4615..d00491fd7e5 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.reference +++ b/tests/queries/0_stateless/02552_siphash128_reference.reference @@ -1,152 +1 @@ -A3817F04BA25A8E66DF67214C7550293 -DA87C1D86B99AF44347659119B22FC45 -8177228DA4A45DC7FCA38BDEF60AFFE4 -9C70B60C5267A94E5F33B6B02985ED51 -F88164C12D9C8FAF7D0F6E7C7BCD5579 -1368875980776F8854527A07690E9627 -14EECA338B208613485EA0308FD7A15E -A1F1EBBED8DBC153C0B84AA61FF08239 -3B62A9BA6258F5610F83E264F31497B4 -264499060AD9BAABC47F8B02BB6D71ED -00110DC378146956C95447D3F3D0FBBA -0151C568386B6677A2B4DC6F81E5DC18 -D626B266905EF35882634DF68532C125 -9869E247E9C08B10D029934FC4B952F7 -31FCEFAC66D7DE9C7EC7485FE4494902 -5493E99933B0A8117E08EC0F97CFC3D9 -6EE2A4CA67B054BBFD3315BF85230577 -473D06E8738DB89854C066C47AE47740 -A426E5E423BF4885294DA481FEAEF723 -78017731CF65FAB074D5208952512EB1 -9E25FC833F2290733E9344A5E83839EB -568E495ABE525A218A2214CD3E071D12 -4A29B54552D16B9A469C10528EFF0AAE -C9D184DDD5A9F5E0CF8CE29A9ABF691C -2DB479AE78BD50D8882A8A178A6132AD -8ECE5F042D5E447B5051B9EACB8D8F6F -9C0B53B4B3C307E87EAEE08678141F66 -ABF248AF69A6EAE4BFD3EB2F129EEB94 -0664DA1668574B88B935F3027358AEF4 -AA4B9DC4BF337DE90CD4FD3C467C6AB7 -EA5C7F471FAF6BDE2B1AD7D4686D2287 -2939B0183223FAFC1723DE4F52C43D35 -7C3956CA5EEAFC3E363E9D556546EB68 -77C6077146F01C32B6B69D5F4EA9FFCF -37A6986CB8847EDF0925F0F1309B54DE -A705F0E69DA9A8F907241A2E923C8CC8 -3DC47D1F29C448461E9E76ED904F6711 -0D62BF01E6FC0E1A0D3C4751C5D3692B -8C03468BCA7C669EE4FD5E084BBEE7B5 -528A5BB93BAF2C9C4473CCE5D0D22BD9 -DF6A301E95C95DAD97AE0CC8C6913BD8 -801189902C857F39E73591285E70B6DB -E617346AC9C231BB3650AE34CCCA0C5B -27D93437EFB721AA401821DCEC5ADF89 -89237D9DED9C5E78D8B1C9B166CC7342 -4A6D8091BF5E7D651189FA94A250B14C -0E33F96055E7AE893FFC0E3DCF492902 -E61C432B720B19D18EC8D84BDC63151B -F7E5AEF549F782CF379055A608269B16 -438D030FD0B7A54FA837F2AD201A6403 -A590D3EE4FBF04E3247E0D27F286423F -5FE2C1A172FE93C4B15CD37CAEF9F538 -2C97325CBD06B36EB2133DD08B3A017C -92C814227A6BCA949FF0659F002AD39E -DCE850110BD8328CFBD50841D6911D87 -67F14984C7DA791248E32BB5922583DA -1938F2CF72D54EE97E94166FA91D2A36 -74481E9646ED49FE0F6224301604698E -57FCA5DE98A9D6D8006438D0583D8A1D -9FECDE1CEFDC1CBED4763674D9575359 -E3040C00EB28F15366CA73CBD872E740 -7697009A6A831DFECCA91C5993670F7A -5853542321F567A005D547A4F04759BD -5150D1772F50834A503E069A973FBD7C 1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -Check bug with hashing of const integer values -E940B12600C844966162FF8FE7A16AAE -E940B12600C844966162FF8FE7A16AAE -Check memsan bug -1CE422FEE7BD8DE20000000000000000 -Check const columns -B66B53476BDBEB8549A257E3B1766C30 -B66B53476BDBEB8549A257E3B1766C30 -B66B53476BDBEB8549A257E3B1766C30 -B66B53476BDBEB8549A257E3B1766C30 -Check multiple keys as tuple from a table -E940B12600C844966162FF8FE7A16AAE -EC58946A98A0D37F4E3FAC02FBBA9480 -Check multiple keys as separate ints from a table -E940B12600C844966162FF8FE7A16AAE -EC58946A98A0D37F4E3FAC02FBBA9480 -Check constant key and data from a table -E940B12600C844966162FF8FE7A16AAE -E940B12600C844966162FF8FE7A16AAE -Check multiple keys as separate ints from a table with constant data -E940B12600C844966162FF8FE7A16AAE -EC58946A98A0D37F4E3FAC02FBBA9480 diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql index f7324ed0ee4..200954c3b57 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.sql +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -1,254 +1 @@ --- Test Vectors from the SipHash reference C implementation: --- Written by --- Jean-Philippe Aumasson --- Daniel J. Bernstein --- Released under CC0 --- https://github.com/veorq/SipHash/blob/eee7d0d84dc7731df2359b243aa5e75d85f6eaef/vectors.h#L645 - -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - '')); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62))); - --- CH tests -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0)) == sipHash128Reference(char(0)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1)) == sipHash128Reference(char(0, 1)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2)) == sipHash128Reference(char(0, 1, 2)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3)) == sipHash128Reference(char(0, 1, 2, 3)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4)) == sipHash128Reference(char(0, 1, 2, 3, 4)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); - -select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError NOT_IMPLEMENTED } -select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError NOT_IMPLEMENTED } - SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; -SELECT hex(sipHash128ReferenceKeyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; - -SELECT 'Check bug with hashing of const integer values'; -DROP TABLE IF EXISTS tab; -CREATE TABLE tab (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; -INSERT INTO tab VALUES ((2, 2), 4); --- these two statements must produce the same result -SELECT hex(sipHash128ReferenceKeyed(key, val)) FROM tab; -SELECT hex(sipHash128ReferenceKeyed(key, 4::UInt64)) FROM tab; -DROP TABLE tab; - -SELECT 'Check memsan bug'; -SELECT hex(sipHash128ReferenceKeyed((toUInt64(2), toUInt64(-9223372036854775807)))) GROUP BY (toUInt64(506097522914230528), toUInt64(now64(2, NULL + NULL), 1084818905618843912)), toUInt64(2), NULL + NULL, char(-2147483649, 1); - -SELECT 'Check const columns'; -DROP TABLE IF EXISTS sipHashKeyed_test; -CREATE TABLE sipHashKeyed_test ENGINE = Memory() AS SELECT 1 a, 'test' b; -SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), 1, 'test')); -SELECT hex(sipHash128Reference(tuple(*))) FROM sipHashKeyed_test; -SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), tuple(*))) FROM sipHashKeyed_test; -SELECT hex(sipHash128ReferenceKeyed((toUInt64(0), toUInt64(0)), a, b)) FROM sipHashKeyed_test; -DROP TABLE sipHashKeyed_test; - -SELECT 'Check multiple keys as tuple from a table'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES ((2, 2), 4); -INSERT INTO sipHashKeyed_keys VALUES ((4, 4), 4); -SELECT hex(sipHash128ReferenceKeyed(key, val)) FROM sipHashKeyed_keys ORDER by key; -DROP TABLE sipHashKeyed_keys; - -SELECT 'Check multiple keys as separate ints from a table'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64, val UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES (2, 2, 4); -INSERT INTO sipHashKeyed_keys VALUES (4, 4, 4); -SELECT hex(sipHash128ReferenceKeyed((key0, key1), val)) FROM sipHashKeyed_keys ORDER by key0; -SELECT 'Check constant key and data from a table'; -SELECT hex(sipHash128ReferenceKeyed((2::UInt64, 2::UInt64), val)) FROM sipHashKeyed_keys ORDER by val; -DROP TABLE sipHashKeyed_keys; - -SELECT 'Check multiple keys as separate ints from a table with constant data'; -DROP TABLE IF EXISTS sipHashKeyed_keys; -CREATE TABLE sipHashKeyed_keys (key0 UInt64, key1 UInt64) ENGINE=Memory; -INSERT INTO sipHashKeyed_keys VALUES (2, 2); -INSERT INTO sipHashKeyed_keys VALUES (4, 4); -SELECT hex(sipHash128ReferenceKeyed((key0, key1), 4::UInt64)) FROM sipHashKeyed_keys ORDER by key0; -DROP TABLE sipHashKeyed_keys; From 7cc3372355d06dfc1184b3ebcd6d2164d179b7be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:30:09 +0200 Subject: [PATCH 1011/2047] Fix terrible trash --- src/Functions/FunctionsHashing.h | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 82944630b10..090d38fa73d 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1567,7 +1567,10 @@ public: if constexpr (std::is_same_v) /// backward-compatible { auto col_to_fixed_string = ColumnFixedString::create(sizeof(UInt128)); - col_to_fixed_string->getChars() = std::move(*reinterpret_cast(&col_to->getData())); + const auto & data = col_to->getData(); + auto & chars = col_to_fixed_string->getChars(); + chars.resize(data.size() * sizeof(UInt128)); + memcpy(chars.data(), data.data(), data.size() * sizeof(UInt128)); return col_to_fixed_string; } @@ -1601,12 +1604,12 @@ public: { selector .registerImplementation>(); - +/* #if USE_MULTITARGET_CODE selector.registerImplementation>(); selector .registerImplementation>(); -#endif +#endif*/ } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override From 4c2dabddb6d697ba3744e48e07e09aeaf8fc59d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:31:38 +0200 Subject: [PATCH 1012/2047] Add a test --- tests/queries/0_stateless/02831_trash.reference | 2 ++ tests/queries/0_stateless/02831_trash.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02831_trash.reference create mode 100644 tests/queries/0_stateless/02831_trash.sql diff --git a/tests/queries/0_stateless/02831_trash.reference b/tests/queries/0_stateless/02831_trash.reference new file mode 100644 index 00000000000..e25f2e9e23f --- /dev/null +++ b/tests/queries/0_stateless/02831_trash.reference @@ -0,0 +1,2 @@ +2761631236 +1210084689 diff --git a/tests/queries/0_stateless/02831_trash.sql b/tests/queries/0_stateless/02831_trash.sql new file mode 100644 index 00000000000..600e2ad0695 --- /dev/null +++ b/tests/queries/0_stateless/02831_trash.sql @@ -0,0 +1,2 @@ +SELECT CRC32IEEE(sipHash128()); +SELECT CRC32(murmurHash3_128()); From d2b178536e1e5b6d85c917d3d26bbe2cff7594ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:38:16 +0200 Subject: [PATCH 1013/2047] Fix terrible trash --- src/Functions/FunctionsHashing.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 090d38fa73d..8f8715ec3f1 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1604,12 +1604,12 @@ public: { selector .registerImplementation>(); -/* + #if USE_MULTITARGET_CODE selector.registerImplementation>(); selector .registerImplementation>(); -#endif*/ +#endif } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override From 4d0b75ebdd1bb69e155b237768c7db7a22cb09cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:42:16 +0200 Subject: [PATCH 1014/2047] Remove hashid --- .gitmodules | 3 - contrib/CMakeLists.txt | 1 - contrib/hashidsxx | 1 - contrib/hashidsxx-cmake/CMakeLists.txt | 14 -- src/Core/Settings.h | 2 +- src/Functions/CMakeLists.txt | 1 - src/Functions/FunctionHashID.cpp | 12 -- src/Functions/FunctionHashID.h | 170 ------------------ .../0_stateless/02293_hashid.reference | 15 -- tests/queries/0_stateless/02293_hashid.sql | 16 -- ...new_functions_must_be_documented.reference | 1 - 11 files changed, 1 insertion(+), 235 deletions(-) delete mode 160000 contrib/hashidsxx delete mode 100644 contrib/hashidsxx-cmake/CMakeLists.txt delete mode 100644 src/Functions/FunctionHashID.cpp delete mode 100644 src/Functions/FunctionHashID.h delete mode 100644 tests/queries/0_stateless/02293_hashid.reference delete mode 100644 tests/queries/0_stateless/02293_hashid.sql diff --git a/.gitmodules b/.gitmodules index 151dc28c55b..ba71a8ae3a7 100644 --- a/.gitmodules +++ b/.gitmodules @@ -258,9 +258,6 @@ [submodule "contrib/wyhash"] path = contrib/wyhash url = https://github.com/wangyi-fudan/wyhash -[submodule "contrib/hashidsxx"] - path = contrib/hashidsxx - url = https://github.com/schoentoon/hashidsxx [submodule "contrib/nats-io"] path = contrib/nats-io url = https://github.com/ClickHouse/nats.c diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 2af468970f1..0f68c0cbc7c 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -164,7 +164,6 @@ add_contrib (libpq-cmake libpq) add_contrib (nuraft-cmake NuRaft) add_contrib (fast_float-cmake fast_float) add_contrib (datasketches-cpp-cmake datasketches-cpp) -add_contrib (hashidsxx-cmake hashidsxx) option(ENABLE_NLP "Enable NLP functions support" ${ENABLE_LIBRARIES}) if (ENABLE_NLP) diff --git a/contrib/hashidsxx b/contrib/hashidsxx deleted file mode 160000 index 783f6911ccf..00000000000 --- a/contrib/hashidsxx +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 783f6911ccfdaca83e3cfac084c4aad888a80cee diff --git a/contrib/hashidsxx-cmake/CMakeLists.txt b/contrib/hashidsxx-cmake/CMakeLists.txt deleted file mode 100644 index 17f3888bd94..00000000000 --- a/contrib/hashidsxx-cmake/CMakeLists.txt +++ /dev/null @@ -1,14 +0,0 @@ -set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/hashidsxx") - -set (SRCS - "${LIBRARY_DIR}/hashids.cpp" -) - -set (HDRS - "${LIBRARY_DIR}/hashids.h" -) - -add_library(_hashidsxx ${SRCS} ${HDRS}) -target_include_directories(_hashidsxx SYSTEM PUBLIC "${LIBRARY_DIR}") - -add_library(ch_contrib::hashidsxx ALIAS _hashidsxx) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cfcb56729d2..bde51ae9971 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -761,7 +761,7 @@ class IColumn; /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ - M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions (hashid, etc)", 0) \ + M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \ diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 2f5c8a212f2..06436488050 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -21,7 +21,6 @@ list (APPEND PUBLIC_LIBS dbms ch_contrib::metrohash ch_contrib::murmurhash - ch_contrib::hashidsxx ch_contrib::morton_nd ) diff --git a/src/Functions/FunctionHashID.cpp b/src/Functions/FunctionHashID.cpp deleted file mode 100644 index 829b3d9d2f6..00000000000 --- a/src/Functions/FunctionHashID.cpp +++ /dev/null @@ -1,12 +0,0 @@ -#include "FunctionHashID.h" -#include - -namespace DB -{ - -REGISTER_FUNCTION(HashID) -{ - factory.registerFunction(); -} - -} diff --git a/src/Functions/FunctionHashID.h b/src/Functions/FunctionHashID.h deleted file mode 100644 index 680c3f6430b..00000000000 --- a/src/Functions/FunctionHashID.h +++ /dev/null @@ -1,170 +0,0 @@ -#pragma once - -#include "config.h" - -#include - -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int SUPPORT_IS_DISABLED; - extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; - extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; -} - -// hashid(string, salt) -class FunctionHashID : public IFunction -{ -public: - static constexpr auto name = "hashid"; - - static FunctionPtr create(ContextPtr context) - { - if (!context->getSettingsRef().allow_experimental_hash_functions) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Hashing function '{}' is experimental. Set `allow_experimental_hash_functions` setting to enable it", name); - - return std::make_shared(); - } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 0; } - - bool isVariadic() const override { return true; } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - if (arguments.empty()) - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expects at least one argument", getName()); - - const auto & id_col = arguments[0]; - if (!isUnsignedInteger(id_col.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "First argument of function {} must be unsigned integer, got {}", - getName(), - arguments[0].type->getName()); - - if (arguments.size() > 1) - { - const auto & hash_col = arguments[1]; - if (!isString(hash_col.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Second argument of function {} must be String, got {}", - getName(), - arguments[1].type->getName()); - } - - if (arguments.size() > 2) - { - const auto & min_length_col = arguments[2]; - if (!isUInt8(min_length_col.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Third argument of function {} must be UInt8, got {}", - getName(), - arguments[2].type->getName()); - } - - if (arguments.size() > 3) - { - const auto & alphabet_col = arguments[3]; - if (!isString(alphabet_col.type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Fourth argument of function {} must be String, got {}", - getName(), - arguments[3].type->getName()); - } - - if (arguments.size() > 4) - { - throw Exception( - ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, - "Function {} expect no more than four arguments (integer, salt, min_length, optional_alphabet), got {}", - getName(), - arguments.size()); - } - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override - { - const auto & numcolumn = arguments[0].column; - - if (checkAndGetColumn(numcolumn.get()) || checkAndGetColumn(numcolumn.get()) - || checkAndGetColumn(numcolumn.get()) || checkAndGetColumn(numcolumn.get())) - { - std::string salt; - UInt8 min_length = 0; - std::string alphabet; - - if (arguments.size() >= 4) - { - const auto & alphabetcolumn = arguments[3].column; - if (const auto * alpha_col = checkAndGetColumnConst(alphabetcolumn.get())) - { - alphabet = alpha_col->getValue(); - if (alphabet.find('\0') != std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Custom alphabet must not contain null character"); - } - } - else - alphabet.assign(DEFAULT_ALPHABET); - - if (arguments.size() >= 3) - { - const auto & minlengthcolumn = arguments[2].column; - if (const auto * min_length_col = checkAndGetColumnConst(minlengthcolumn.get())) - min_length = min_length_col->getValue(); - } - - if (arguments.size() >= 2) - { - const auto & saltcolumn = arguments[1].column; - if (const auto * salt_col = checkAndGetColumnConst(saltcolumn.get())) - salt = salt_col->getValue(); - } - - hashidsxx::Hashids hash(salt, min_length, alphabet); - - auto col_res = ColumnString::create(); - - for (size_t i = 0; i < input_rows_count; ++i) - { - col_res->insert(hash.encode({numcolumn->getUInt(i)})); - } - - return col_res; - } - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function hashid", - arguments[0].column->getName()); - } -}; - -} diff --git a/tests/queries/0_stateless/02293_hashid.reference b/tests/queries/0_stateless/02293_hashid.reference deleted file mode 100644 index dfc78349c05..00000000000 --- a/tests/queries/0_stateless/02293_hashid.reference +++ /dev/null @@ -1,15 +0,0 @@ -0 gY -1 jR -2 k5 -3 l5 -4 mO -0 pbgkmdljlpjoapne -1 akemglnjepjpodba -2 obmgndljgajpkeao -3 dldokmpjpgjgeanb -4 nkdlpgajngjnobme -YQrvD5XGvbx -Bm3zaOq7zbp -oV -oV -6b diff --git a/tests/queries/0_stateless/02293_hashid.sql b/tests/queries/0_stateless/02293_hashid.sql deleted file mode 100644 index 06af0b5e1d8..00000000000 --- a/tests/queries/0_stateless/02293_hashid.sql +++ /dev/null @@ -1,16 +0,0 @@ --- Tags: no-upgrade-check -SET allow_experimental_hash_functions = 1; - -select number, hashid(number) from system.numbers limit 5; -select number, hashid(number, 's3cr3t', 16, 'abcdefghijklmnop') from system.numbers limit 5; -select hashid(1234567890123456, 's3cr3t'); -select hashid(1234567890123456, 's3cr3t2'); - -SELECT hashid(1, hashid(2)); -SELECT hashid(1, 'k5'); -SELECT hashid(1, 'k5_othersalt'); - --- https://github.com/ClickHouse/ClickHouse/issues/39672 -SELECT - JSONExtractRaw(257, NULL), - hashid(1024, if(rand() % 10, 'truetruetruetrue', NULL), 's3\0r3t'); -- {serverError 43} diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index fc00bfdadca..595ebb483d5 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -346,7 +346,6 @@ hasAny hasColumnInTable hasSubstr hasThreadFuzzer -hashid hex hiveHash hop From 5f4756fb33f754913f4ab8ddfa84c39739920f19 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 04:54:58 +0200 Subject: [PATCH 1015/2047] Remove toDecimalString --- .../functions/type-conversion-functions.md | 38 --- .../functions/type-conversion-functions.md | 38 --- src/Functions/FunctionToDecimalString.cpp | 22 -- src/Functions/FunctionToDecimalString.h | 312 ------------------ src/IO/WriteHelpers.h | 44 +-- .../02676_to_decimal_string.reference | 21 -- .../0_stateless/02676_to_decimal_string.sql | 35 -- 7 files changed, 13 insertions(+), 497 deletions(-) delete mode 100644 src/Functions/FunctionToDecimalString.cpp delete mode 100644 src/Functions/FunctionToDecimalString.h delete mode 100644 tests/queries/0_stateless/02676_to_decimal_string.reference delete mode 100644 tests/queries/0_stateless/02676_to_decimal_string.sql diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 36f40b37238..c2bd525c483 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -945,44 +945,6 @@ Result: └────────────┴───────┘ ``` -## toDecimalString - -Converts a numeric value to String with the number of fractional digits in the output specified by the user. - -**Syntax** - -``` sql -toDecimalString(number, scale) -``` - -**Parameters** - -- `number` — Value to be represented as String, [Int, UInt](/docs/en/sql-reference/data-types/int-uint.md), [Float](/docs/en/sql-reference/data-types/float.md), [Decimal](/docs/en/sql-reference/data-types/decimal.md), -- `scale` — Number of fractional digits, [UInt8](/docs/en/sql-reference/data-types/int-uint.md). - * Maximum scale for [Decimal](/docs/en/sql-reference/data-types/decimal.md) and [Int, UInt](/docs/en/sql-reference/data-types/int-uint.md) types is 77 (it is the maximum possible number of significant digits for Decimal), - * Maximum scale for [Float](/docs/en/sql-reference/data-types/float.md) is 60. - -**Returned value** - -- Input value represented as [String](/docs/en/sql-reference/data-types/string.md) with given number of fractional digits (scale). - The number is rounded up or down according to common arithmetic in case requested scale is smaller than original number's scale. - -**Example** - -Query: - -``` sql -SELECT toDecimalString(CAST('64.32', 'Float64'), 5); -``` - -Result: - -```response -┌toDecimalString(CAST('64.32', 'Float64'), 5)─┐ -│ 64.32000 │ -└─────────────────────────────────────────────┘ -``` - ## reinterpretAsUInt(8\|16\|32\|64) ## reinterpretAsInt(8\|16\|32\|64) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index e53104d8d71..088b1a9a1f1 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -762,44 +762,6 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut; └────────────┴───────┘ ``` -## toDecimalString - -Принимает любой численный тип первым аргументом, возвращает строковое десятичное представление числа с точностью, заданной вторым аргументом. - -**Синтаксис** - -``` sql -toDecimalString(number, scale) -``` - -**Параметры** - -- `number` — Значение любого числового типа: [Int, UInt](/docs/ru/sql-reference/data-types/int-uint.md), [Float](/docs/ru/sql-reference/data-types/float.md), [Decimal](/docs/ru/sql-reference/data-types/decimal.md), -- `scale` — Требуемое количество десятичных знаков после запятой, [UInt8](/docs/ru/sql-reference/data-types/int-uint.md). - * Значение `scale` для типов [Decimal](/docs/ru/sql-reference/data-types/decimal.md) и [Int, UInt](/docs/ru/sql-reference/data-types/int-uint.md) должно не превышать 77 (так как это наибольшее количество значимых символов для этих типов), - * Значение `scale` для типа [Float](/docs/ru/sql-reference/data-types/float.md) не должно превышать 60. - -**Возвращаемое значение** - -- Строка ([String](/docs/en/sql-reference/data-types/string.md)), представляющая собой десятичное представление входного числа с заданной длиной дробной части. - При необходимости число округляется по стандартным правилам арифметики. - -**Пример использования** - -Запрос: - -``` sql -SELECT toDecimalString(CAST('64.32', 'Float64'), 5); -``` - -Результат: - -```response -┌─toDecimalString(CAST('64.32', 'Float64'), 5)┐ -│ 64.32000 │ -└─────────────────────────────────────────────┘ -``` - ## reinterpretAsUInt(8\|16\|32\|64) {#reinterpretasuint8163264} ## reinterpretAsInt(8\|16\|32\|64) {#reinterpretasint8163264} diff --git a/src/Functions/FunctionToDecimalString.cpp b/src/Functions/FunctionToDecimalString.cpp deleted file mode 100644 index fe417b19137..00000000000 --- a/src/Functions/FunctionToDecimalString.cpp +++ /dev/null @@ -1,22 +0,0 @@ -#include -#include -#include - -namespace DB -{ - -REGISTER_FUNCTION(ToDecimalString) -{ - factory.registerFunction( - FunctionDocumentation{ - .description=R"( -Returns string representation of a number. First argument is the number of any numeric type, -second argument is the desired number of digits in fractional part. Returns String. - - )", - .examples{{"toDecimalString", "SELECT toDecimalString(2.1456,2)", ""}}, - .categories{"String"} - }, FunctionFactory::CaseInsensitive); -} - -} diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h deleted file mode 100644 index 6ae007e6b66..00000000000 --- a/src/Functions/FunctionToDecimalString.h +++ /dev/null @@ -1,312 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_COLUMN; - extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER; -} - -class FunctionToDecimalString : public IFunction -{ -public: - static constexpr auto name = "toDecimalString"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override { return name; } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - size_t getNumberOfArguments() const override { return 2; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isNumber(*arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal first argument for formatDecimal function: got {}, expected numeric type", - arguments[0]->getName()); - - if (!isUInt8(*arguments[1])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal second argument for formatDecimal function: got {}, expected UInt8", - arguments[1]->getName()); - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - -private: - /// For operations with Integer/Float - template - void vectorConstant(const FromVectorType & vec_from, UInt8 precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const - { - size_t input_rows_count = vec_from.size(); - result_offsets.resize(input_rows_count); - - /// Buffer is used here and in functions below because resulting size cannot be precisely anticipated, - /// and buffer resizes on-the-go. Also, .count() provided by buffer is convenient in this case. - WriteBufferFromVector buf_to(vec_to); - - for (size_t i = 0; i < input_rows_count; ++i) - { - format(vec_from[i], buf_to, precision); - result_offsets[i] = buf_to.count(); - } - - buf_to.finalize(); - } - - template - void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const - { - size_t input_rows_count = vec_from.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested, shall not be more than {}", max_digits); - format(vec_from[i], buf_to, vec_precision[i]); - result_offsets[i] = buf_to.count(); - } - - buf_to.finalize(); - } - - template - void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const - { - size_t input_rows_count = vec_precision.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested, shall not be more than {}", max_digits); - format(value_from, buf_to, vec_precision[i]); - result_offsets[i] = buf_to.count(); - } - - buf_to.finalize(); - } - - /// For operations with Decimal - template - void vectorConstant(const FirstArgVectorType & vec_from, UInt8 precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const - { - /// There are no more than 77 meaning digits (as it is the max length of UInt256). So we can limit it with 77. - constexpr size_t max_digits = std::numeric_limits::digits10; - if (precision > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); - - WriteBufferFromVector buf_to(vec_to); - size_t input_rows_count = vec_from.size(); - result_offsets.resize(input_rows_count); - - for (size_t i = 0; i < input_rows_count; ++i) - { - writeText(vec_from[i], from_scale, buf_to, true, true, precision); - writeChar(0, buf_to); - result_offsets[i] = buf_to.count(); - } - buf_to.finalize(); - } - - template - void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const - { - size_t input_rows_count = vec_from.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); - writeText(vec_from[i], from_scale, buf_to, true, true, vec_precision[i]); - writeChar(0, buf_to); - result_offsets[i] = buf_to.count(); - } - buf_to.finalize(); - } - - template - void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const - { - size_t input_rows_count = vec_precision.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); - writeText(value_from, from_scale, buf_to, true, true, vec_precision[i]); - writeChar(0, buf_to); - result_offsets[i] = buf_to.count(); - } - buf_to.finalize(); - } - - template - static void format(T value, DB::WriteBuffer & out, UInt8 precision) - { - /// Maximum of 60 is hard-coded in 'double-conversion/double-conversion.h' for floating point values, - /// Catch this here to give user a more reasonable error. - if (precision > 60) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too high precision requested for Float, must not be more than 60, got {}", Int8(precision)); - - DB::DoubleConverter::BufferType buffer; - double_conversion::StringBuilder builder{buffer, sizeof(buffer)}; - - const auto result = DB::DoubleConverter::instance().ToFixed(value, precision, &builder); - - if (!result) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, "Error processing number: {}", value); - - out.write(buffer, builder.position()); - writeChar(0, out); - } - - template - static void format(T value, DB::WriteBuffer & out, UInt8 precision) - { - /// Fractional part for Integer is just trailing zeros. Let's limit it with 77 (like with Decimals). - constexpr size_t max_digits = std::numeric_limits::digits10; - if (precision > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested, shall not be more than {}", max_digits); - writeText(value, out); - if (precision > 0) [[likely]] - { - writeChar('.', out); - for (int i = 0; i < precision; ++i) - writeChar('0', out); - writeChar(0, out); - } - } - -public: - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override - { - switch (arguments[0].type->getTypeId()) - { - case TypeIndex::UInt8: return executeType(arguments); - case TypeIndex::UInt16: return executeType(arguments); - case TypeIndex::UInt32: return executeType(arguments); - case TypeIndex::UInt64: return executeType(arguments); - case TypeIndex::UInt128: return executeType(arguments); - case TypeIndex::UInt256: return executeType(arguments); - case TypeIndex::Int8: return executeType(arguments); - case TypeIndex::Int16: return executeType(arguments); - case TypeIndex::Int32: return executeType(arguments); - case TypeIndex::Int64: return executeType(arguments); - case TypeIndex::Int128: return executeType(arguments); - case TypeIndex::Int256: return executeType(arguments); - case TypeIndex::Float32: return executeType(arguments); - case TypeIndex::Float64: return executeType(arguments); - case TypeIndex::Decimal32: return executeType(arguments); - case TypeIndex::Decimal64: return executeType(arguments); - case TypeIndex::Decimal128: return executeType(arguments); - case TypeIndex::Decimal256: return executeType(arguments); - default: - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", - arguments[0].column->getName(), getName()); - } - } - -private: - template - ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const - { - const auto * from_col_const = typeid_cast(arguments[0].column.get()); - const auto * precision_col = checkAndGetColumn>(arguments[1].column.get()); - const auto * precision_col_const = typeid_cast(arguments[1].column.get()); - - auto result_col = ColumnString::create(); - auto * result_col_string = assert_cast(result_col.get()); - ColumnString::Chars & result_chars = result_col_string->getChars(); - ColumnString::Offsets & result_offsets = result_col_string->getOffsets(); - - if constexpr (is_decimal) - { - const auto * from_col = checkAndGetColumn>(arguments[0].column.get()); - UInt8 from_scale = from_col->getScale(); - - if (from_col) - { - if (precision_col_const) - vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets, from_scale); - else - vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale); - } - else if (from_col_const) - constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets, from_scale); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); - } - else - { - const auto * from_col = checkAndGetColumn>(arguments[0].column.get()); - if (from_col) - { - if (precision_col_const) - vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets); - else - vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets); - } - else if (from_col_const) - constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); - } - - return result_col; - } -}; - -} diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index aa4c9b17e48..0494cdf22e7 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -905,26 +905,26 @@ inline void writeText(const IPv4 & x, WriteBuffer & buf) { writeIPv4Text(x, buf) inline void writeText(const IPv6 & x, WriteBuffer & buf) { writeIPv6Text(x, buf); } template -void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros, - bool fixed_fractional_length, UInt32 fractional_length) +void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros) { /// If it's big integer, but the number of digits is small, /// use the implementation for smaller integers for more efficient arithmetic. + if constexpr (std::is_same_v) { if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); return; } } @@ -932,53 +932,35 @@ void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool { if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); return; } } constexpr size_t max_digits = std::numeric_limits::digits10; assert(scale <= max_digits); - assert(fractional_length <= max_digits); - char buf[max_digits]; - memset(buf, '0', std::max(scale, fractional_length)); + memset(buf, '0', scale); T value = x; Int32 last_nonzero_pos = 0; - - if (fixed_fractional_length && fractional_length < scale) - { - T new_value = value / DecimalUtils::scaleMultiplier(scale - fractional_length - 1); - auto round_carry = new_value % 10; - value = new_value / 10; - if (round_carry >= 5) - value += 1; - } - - for (Int32 pos = fixed_fractional_length ? std::min(scale - 1, fractional_length - 1) : scale - 1; pos >= 0; --pos) + for (Int32 pos = scale - 1; pos >= 0; --pos) { auto remainder = value % 10; value /= 10; - - if (remainder != 0 && last_nonzero_pos == 0) - last_nonzero_pos = pos; - - buf[pos] += static_cast(remainder); } writeChar('.', ostr); - ostr.write(buf, fixed_fractional_length ? fractional_length : (trailing_zeros ? scale : last_nonzero_pos + 1)); + ostr.write(buf, trailing_zeros ? scale : last_nonzero_pos + 1); } template -void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros, - bool fixed_fractional_length = false, UInt32 fractional_length = 0) +void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros) { T part = DecimalUtils::getWholePart(x, scale); @@ -989,7 +971,7 @@ void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer writeIntText(part, ostr); - if (scale || (fixed_fractional_length && fractional_length > 0)) + if (scale) { part = DecimalUtils::getFractionalPart(x, scale); if (part || trailing_zeros) @@ -997,7 +979,7 @@ void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer if (part < 0) part *= T(-1); - writeDecimalFractional(part, scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); + writeDecimalFractional(part, scale, ostr, trailing_zeros); } } } diff --git a/tests/queries/0_stateless/02676_to_decimal_string.reference b/tests/queries/0_stateless/02676_to_decimal_string.reference deleted file mode 100644 index 4c27ee5b528..00000000000 --- a/tests/queries/0_stateless/02676_to_decimal_string.reference +++ /dev/null @@ -1,21 +0,0 @@ -2.00000000000000000000000000000000000000000000000000000000000000000000000000000 -2.12 --2.00000000000000000000000000000000000000000000000000000000000000000000000000000 --2.12 -2.987600000000000033395508580724708735942840576171875000000000 -2.15 --2.987600000000000033395508580724708735942840576171875000000000 --2.15 -64.1230010986 -64.2340000000 --64.1230010986 --64.2340000000 --32.345 -32.34500000000000000000000000000000000000000000000000000000000000000000000000000 -32.46 --64.5671232345 -128.78932312332132985464 --128.78932312332132985464 -128.78932312332132985464000000000000000000000000000000000000000000000000000000000 -128.7893231233 --128.78932312332132985464123123789323123321329854600000000000000000000000000000000 diff --git a/tests/queries/0_stateless/02676_to_decimal_string.sql b/tests/queries/0_stateless/02676_to_decimal_string.sql deleted file mode 100644 index 563d60c62c7..00000000000 --- a/tests/queries/0_stateless/02676_to_decimal_string.sql +++ /dev/null @@ -1,35 +0,0 @@ --- Regular types -SELECT toDecimalString(2, 77); -- more digits required than exist -SELECT toDecimalString(2.123456, 2); -- rounding -SELECT toDecimalString(-2, 77); -- more digits required than exist -SELECT toDecimalString(-2.123456, 2); -- rounding - -SELECT toDecimalString(2.9876, 60); -- more digits required than exist (took 60 as it is float by default) -SELECT toDecimalString(2.1456, 2); -- rounding -SELECT toDecimalString(-2.9876, 60); -- more digits required than exist -SELECT toDecimalString(-2.1456, 2); -- rounding - --- Float32 and Float64 tests. No sense to test big float precision -- the result will be a mess anyway. -SELECT toDecimalString(64.123::Float32, 10); -SELECT toDecimalString(64.234::Float64, 10); -SELECT toDecimalString(-64.123::Float32, 10); -SELECT toDecimalString(-64.234::Float64, 10); - --- Decimals -SELECT toDecimalString(-32.345::Decimal32(3), 3); -SELECT toDecimalString(32.345::Decimal32(3), 77); -- more digits required than exist -SELECT toDecimalString(32.456::Decimal32(3), 2); -- rounding -SELECT toDecimalString('-64.5671232345'::Decimal64(10), 10); -SELECT toDecimalString('128.78932312332132985464'::Decimal128(20), 20); -SELECT toDecimalString('-128.78932312332132985464123123'::Decimal128(26), 20); -- rounding -SELECT toDecimalString('128.78932312332132985464'::Decimal128(20), 77); -- more digits required than exist -SELECT toDecimalString('128.789323123321329854641231237893231233213298546'::Decimal256(45), 10); -- rounding -SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 77); -- more digits required than exist - --- Max number of decimal fractional digits is defined as 77 for Int/UInt/Decimal and 60 for Float. --- These values shall work OK. -SELECT toDecimalString('32.32'::Float32, 61); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} -SELECT toDecimalString('64.64'::Float64, 61); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} -SELECT toDecimalString('88'::UInt8, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} -SELECT toDecimalString('646464'::Int256, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} -SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} From cda42e6dd4eaa56822ad64aad7aa09f632547d93 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 05:05:51 +0200 Subject: [PATCH 1016/2047] Add a test --- .../queries/0_stateless/02831_regexp_analyze_recursion.reference | 0 tests/queries/0_stateless/02831_regexp_analyze_recursion.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/02831_regexp_analyze_recursion.reference create mode 100644 tests/queries/0_stateless/02831_regexp_analyze_recursion.sql diff --git a/tests/queries/0_stateless/02831_regexp_analyze_recursion.reference b/tests/queries/0_stateless/02831_regexp_analyze_recursion.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql b/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql new file mode 100644 index 00000000000..018d1f031e6 --- /dev/null +++ b/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql @@ -0,0 +1 @@ +SELECT match('', repeat('(', 100000)); -- { serverError 306 } From 21ffce0ff20fc7f136d8d5b05369a1abcdc01be3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 05:06:02 +0200 Subject: [PATCH 1017/2047] Check regular expression depth --- src/Common/OptimizedRegularExpression.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index c542945c78d..0b80e2f3f97 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #define MIN_LENGTH_FOR_STRSTR 3 @@ -50,6 +51,8 @@ const char * analyzeImpl( bool & is_trivial, Literals & global_alternatives) { + checkStackSize(); + /** The expression is trivial if all the metacharacters in it are escaped. * The non-alternative string is * a string outside parentheses, From de2016261ef32878456de9efae5cfab748611853 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 05:08:27 +0200 Subject: [PATCH 1018/2047] Get rid of it --- docker/test/fasttest/run.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 828c73e6781..e25b5fdbfed 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -141,7 +141,6 @@ function clone_submodules contrib/jemalloc contrib/replxx contrib/wyhash - contrib/hashidsxx contrib/c-ares contrib/morton-nd contrib/xxHash From 20625d75ab52319b8e67e50d2df803d0e2dc0934 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 15 Jun 2023 14:08:43 +0200 Subject: [PATCH 1019/2047] Fix optimize_skip_unused_shards with JOINs In case of JOIN query may contains conditions for other tables, while optimize_skip_unused_shards was pretty dumb and failed to skip such columns. Fix this by removing JOIN before applying this optimization. v2: restriction for analyzer v3: ignore 01940_custom_tld_sharding_key under analyzer Signed-off-by: Azat Khuzhin Co-Authored-By: Alexey Milovidov --- src/Storages/StorageDistributed.cpp | 42 ++++++++------ src/Storages/StorageDistributed.h | 4 +- tests/analyzer_tech_debt.txt | 2 + ...optimize_skip_unused_shards_join.reference | 0 ...02790_optimize_skip_unused_shards_join.sql | 55 +++++++++++++++++++ 5 files changed, 84 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.reference create mode 100644 tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.sql diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 0727658160c..1a99d272cab 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -75,6 +75,7 @@ #include #include #include +#include #include #include @@ -434,7 +435,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( { /// 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.query); + 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): {}", @@ -1297,7 +1298,7 @@ ClusterPtr StorageDistributed::getCluster() const } ClusterPtr StorageDistributed::getOptimizedCluster( - ContextPtr local_context, const StorageSnapshotPtr & storage_snapshot, const ASTPtr & query_ptr) const + ContextPtr local_context, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info) const { ClusterPtr cluster = getCluster(); const Settings & settings = local_context->getSettingsRef(); @@ -1306,7 +1307,7 @@ ClusterPtr StorageDistributed::getOptimizedCluster( if (has_sharding_key && sharding_key_is_usable) { - ClusterPtr optimized = skipUnusedShards(cluster, query_ptr, storage_snapshot, local_context); + ClusterPtr optimized = skipUnusedShards(cluster, query_info, storage_snapshot, local_context); if (optimized) return optimized; } @@ -1355,25 +1356,34 @@ IColumn::Selector StorageDistributed::createSelector(const ClusterPtr cluster, c /// using constraints from "PREWHERE" and "WHERE" conditions, otherwise returns `nullptr` ClusterPtr StorageDistributed::skipUnusedShards( ClusterPtr cluster, - const ASTPtr & query_ptr, + const SelectQueryInfo & query_info, const StorageSnapshotPtr & storage_snapshot, ContextPtr local_context) const { - const auto & select = query_ptr->as(); - + const auto & select = query_info.query->as(); if (!select.prewhere() && !select.where()) - { return nullptr; - } + + /// FIXME: support analyzer + if (!query_info.syntax_analyzer_result) + return nullptr; ASTPtr condition_ast; - if (select.prewhere() && select.where()) + /// Remove JOIN from the query since it may contain a condition for other tables. + /// But only the conditions for the left table should be analyzed for shard skipping. { - condition_ast = makeASTFunction("and", select.prewhere()->clone(), select.where()->clone()); - } - else - { - condition_ast = select.prewhere() ? select.prewhere()->clone() : select.where()->clone(); + ASTPtr select_without_join_ptr = select.clone(); + ASTSelectQuery select_without_join = select_without_join_ptr->as(); + TreeRewriterResult analyzer_result_without_join = *query_info.syntax_analyzer_result; + + removeJoin(select_without_join, analyzer_result_without_join, local_context); + if (!select_without_join.prewhere() && !select_without_join.where()) + return nullptr; + + if (select_without_join.prewhere() && select_without_join.where()) + condition_ast = makeASTFunction("and", select_without_join.prewhere()->clone(), select_without_join.where()->clone()); + else + condition_ast = select_without_join.prewhere() ? select_without_join.prewhere()->clone() : select_without_join.where()->clone(); } replaceConstantExpressions(condition_ast, local_context, storage_snapshot->metadata->getColumns().getAll(), shared_from_this(), storage_snapshot); @@ -1396,11 +1406,9 @@ ClusterPtr StorageDistributed::skipUnusedShards( return nullptr; } - // Can't get definite answer if we can skip any shards + // Can't get a definite answer if we can skip any shards if (!blocks) - { return nullptr; - } std::set shards; diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index f45286341cf..615d6e337b6 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -182,10 +182,10 @@ private: /// Apply the following settings: /// - optimize_skip_unused_shards /// - force_optimize_skip_unused_shards - ClusterPtr getOptimizedCluster(ContextPtr, const StorageSnapshotPtr & storage_snapshot, const ASTPtr & query_ptr) const; + ClusterPtr getOptimizedCluster(ContextPtr, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info) const; ClusterPtr skipUnusedShards( - ClusterPtr cluster, const ASTPtr & query_ptr, const StorageSnapshotPtr & storage_snapshot, ContextPtr context) const; + ClusterPtr cluster, const SelectQueryInfo & query_info, const StorageSnapshotPtr & storage_snapshot, ContextPtr context) const; /// This method returns optimal query processing stage. /// diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e0f259306aa..8ffb94e17b8 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -130,3 +130,5 @@ 02581_share_big_sets_between_mutation_tasks_long 02581_share_big_sets_between_multiple_mutations_tasks_long 00992_system_parts_race_condition_zookeeper_long +02790_optimize_skip_unused_shards_join +01940_custom_tld_sharding_key diff --git a/tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.reference b/tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.sql b/tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.sql new file mode 100644 index 00000000000..0773e0a9a5e --- /dev/null +++ b/tests/queries/0_stateless/02790_optimize_skip_unused_shards_join.sql @@ -0,0 +1,55 @@ +-- Issue: https://github.com/ClickHouse/ClickHouse/issues/15995 + +DROP TABLE IF EXISTS outer; +DROP TABLE IF EXISTS inner; + +DROP TABLE IF EXISTS outer_distributed; +DROP TABLE IF EXISTS inner_distributed; + +CREATE TABLE IF NOT EXISTS outer +( + `id` UInt64, + `organization_id` UInt64, + `version` UInt64 +) +ENGINE = ReplacingMergeTree(version) +PARTITION BY organization_id % 8 +ORDER BY (organization_id, id); + +CREATE TABLE inner +( + `id` UInt64, + `outer_id` UInt64, + `organization_id` UInt64, + `version` UInt64, + `date` Date +) +ENGINE = ReplacingMergeTree(version) +PARTITION BY toYYYYMM(date) +ORDER BY (organization_id, outer_id); + +CREATE TABLE inner_distributed AS inner +ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), 'inner', intHash64(organization_id)); + +CREATE TABLE outer_distributed AS outer +ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), 'outer', intHash64(organization_id)); + +SELECT + sum(if(inner_distributed.id != 0, 1, 0)) AS total, + inner_distributed.date AS date +FROM outer_distributed AS outer_distributed +FINAL +LEFT JOIN +( + SELECT + inner_distributed.outer_id AS outer_id, + inner_distributed.id AS id, + inner_distributed.date AS date + FROM inner_distributed AS inner_distributed + FINAL + WHERE inner_distributed.organization_id = 15078 +) AS inner_distributed ON inner_distributed.outer_id = outer_distributed.id +WHERE (outer_distributed.organization_id = 15078) AND (date != toDate('1970-01-01')) +GROUP BY date +ORDER BY date DESC +SETTINGS distributed_product_mode = 'local', optimize_skip_unused_shards = 1; From c3c6560c9511163fc14eac2be8f89c30d3bcce1d Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 22 Jul 2023 08:44:19 +0000 Subject: [PATCH 1020/2047] Add 02815_range_dict_no_direct_join to analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index e0f259306aa..19b90a39800 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -130,3 +130,4 @@ 02581_share_big_sets_between_mutation_tasks_long 02581_share_big_sets_between_multiple_mutations_tasks_long 00992_system_parts_race_condition_zookeeper_long +02815_range_dict_no_direct_join From 5ca6c97832f786e6e3be085e3ec79829f9233cdd Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 22 Jul 2023 12:03:20 +0200 Subject: [PATCH 1021/2047] Update gtest_lru_file_cache.cpp --- src/Interpreters/tests/gtest_lru_file_cache.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index 12e7d9372f7..dab14a66ed7 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -489,7 +489,6 @@ TEST_F(FileCacheTest, get) download(file_segment); ASSERT_EQ(file_segment.state(), State::DOWNLOADED); - file_segment.completePartAndResetDownloader(); other_1.join(); From 8c0113e02bb19a4137871f43bbbe9b00702f8681 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 4 Jul 2023 15:54:52 +0200 Subject: [PATCH 1022/2047] Sanitize compose files in integration-tests-runner --- .../runner/compose/docker_compose_meili.yml | 5 ++-- .../runner/compose/docker_compose_mysql.yml | 10 +++---- .../compose/docker_compose_mysql_8_0.yml | 8 +++--- .../compose/docker_compose_mysql_cluster.yml | 26 +++++++++---------- .../compose/docker_compose_postgres.yml | 6 ++--- .../docker_compose_zookeeper_secure.yml | 6 ++--- 6 files changed, 30 insertions(+), 31 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_meili.yml b/docker/test/integration/runner/compose/docker_compose_meili.yml index c734c43b4c6..c1fad4aca87 100644 --- a/docker/test/integration/runner/compose/docker_compose_meili.yml +++ b/docker/test/integration/runner/compose/docker_compose_meili.yml @@ -1,16 +1,15 @@ version: '2.3' services: meili1: - image: getmeili/meilisearch:v0.27.0 + image: getmeili/meilisearch:v0.27.0 restart: always ports: - ${MEILI_EXTERNAL_PORT:-7700}:${MEILI_INTERNAL_PORT:-7700} meili_secure: - image: getmeili/meilisearch:v0.27.0 + image: getmeili/meilisearch:v0.27.0 restart: always ports: - ${MEILI_SECURE_EXTERNAL_PORT:-7700}:${MEILI_SECURE_INTERNAL_PORT:-7700} environment: MEILI_MASTER_KEY: "password" - diff --git a/docker/test/integration/runner/compose/docker_compose_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql.yml index 6b98a372bd0..103fe2769e9 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql.yml @@ -9,10 +9,10 @@ services: DATADIR: /mysql/ expose: - ${MYSQL_PORT:-3306} - command: --server_id=100 - --log-bin='mysql-bin-1.log' - --default-time-zone='+3:00' - --gtid-mode="ON" + command: --server_id=100 + --log-bin='mysql-bin-1.log' + --default-time-zone='+3:00' + --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 --log-error=/mysql/error.log @@ -21,4 +21,4 @@ services: volumes: - type: ${MYSQL_LOGS_FS:-tmpfs} source: ${MYSQL_LOGS:-} - target: /mysql/ \ No newline at end of file + target: /mysql/ diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml index d5fb5a53aaf..9c9c7430cec 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml @@ -9,9 +9,9 @@ services: DATADIR: /mysql/ expose: - ${MYSQL8_PORT:-3306} - command: --server_id=100 --log-bin='mysql-bin-1.log' - --default_authentication_plugin='mysql_native_password' - --default-time-zone='+3:00' --gtid-mode="ON" + command: --server_id=100 --log-bin='mysql-bin-1.log' + --default_authentication_plugin='mysql_native_password' + --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 --log-error=/mysql/error.log @@ -20,4 +20,4 @@ services: volumes: - type: ${MYSQL8_LOGS_FS:-tmpfs} source: ${MYSQL8_LOGS:-} - target: /mysql/ \ No newline at end of file + target: /mysql/ diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml index 8e145a3b408..73f9e39f0d6 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml @@ -9,10 +9,10 @@ services: DATADIR: /mysql/ expose: - ${MYSQL_CLUSTER_PORT:-3306} - command: --server_id=100 - --log-bin='mysql-bin-2.log' - --default-time-zone='+3:00' - --gtid-mode="ON" + command: --server_id=100 + --log-bin='mysql-bin-2.log' + --default-time-zone='+3:00' + --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 --log-error=/mysql/2_error.log @@ -31,10 +31,10 @@ services: DATADIR: /mysql/ expose: - ${MYSQL_CLUSTER_PORT:-3306} - command: --server_id=100 - --log-bin='mysql-bin-3.log' - --default-time-zone='+3:00' - --gtid-mode="ON" + command: --server_id=100 + --log-bin='mysql-bin-3.log' + --default-time-zone='+3:00' + --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 --log-error=/mysql/3_error.log @@ -53,10 +53,10 @@ services: DATADIR: /mysql/ expose: - ${MYSQL_CLUSTER_PORT:-3306} - command: --server_id=100 - --log-bin='mysql-bin-4.log' - --default-time-zone='+3:00' - --gtid-mode="ON" + command: --server_id=100 + --log-bin='mysql-bin-4.log' + --default-time-zone='+3:00' + --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 --log-error=/mysql/4_error.log @@ -65,4 +65,4 @@ services: volumes: - type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs} source: ${MYSQL_CLUSTER_LOGS:-} - target: /mysql/ \ No newline at end of file + target: /mysql/ diff --git a/docker/test/integration/runner/compose/docker_compose_postgres.yml b/docker/test/integration/runner/compose/docker_compose_postgres.yml index 1fb6b7a1410..2ef7eb17395 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres.yml @@ -12,9 +12,9 @@ services: timeout: 5s retries: 5 networks: - default: - aliases: - - postgre-sql.local + default: + aliases: + - postgre-sql.local environment: POSTGRES_HOST_AUTH_METHOD: "trust" POSTGRES_PASSWORD: mysecretpassword diff --git a/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml b/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml index 7a1c32e0023..b5dbae423b2 100644 --- a/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml +++ b/docker/test/integration/runner/compose/docker_compose_zookeeper_secure.yml @@ -12,7 +12,7 @@ services: command: ["zkServer.sh", "start-foreground"] entrypoint: /zookeeper-ssl-entrypoint.sh volumes: - - type: bind + - type: bind source: /misc/zookeeper-ssl-entrypoint.sh target: /zookeeper-ssl-entrypoint.sh - type: bind @@ -37,7 +37,7 @@ services: command: ["zkServer.sh", "start-foreground"] entrypoint: /zookeeper-ssl-entrypoint.sh volumes: - - type: bind + - type: bind source: /misc/zookeeper-ssl-entrypoint.sh target: /zookeeper-ssl-entrypoint.sh - type: bind @@ -61,7 +61,7 @@ services: command: ["zkServer.sh", "start-foreground"] entrypoint: /zookeeper-ssl-entrypoint.sh volumes: - - type: bind + - type: bind source: /misc/zookeeper-ssl-entrypoint.sh target: /zookeeper-ssl-entrypoint.sh - type: bind From c6e9f621f1cb19ac0ac0aa682339a3366efc89aa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 5 Jul 2023 15:36:27 +0200 Subject: [PATCH 1023/2047] Replace kssenii/nginx-test:1.1 by clickhouse/nginx-dav --- docker/images.json | 4 +++ docker/test/integration/nginx_dav/Dockerfile | 6 +++++ .../test/integration/nginx_dav/default.conf | 25 +++++++++++++++++++ .../runner/compose/docker_compose_nginx.yml | 2 +- .../integration/runner/dockerd-entrypoint.sh | 1 + tests/ci/integration_test_check.py | 13 +++++----- tests/integration/ci-runner.py | 1 + tests/integration/runner | 2 ++ 8 files changed, 47 insertions(+), 7 deletions(-) create mode 100644 docker/test/integration/nginx_dav/Dockerfile create mode 100644 docker/test/integration/nginx_dav/default.conf diff --git a/docker/images.json b/docker/images.json index e8fc329a640..9e4d19d30aa 100644 --- a/docker/images.json +++ b/docker/images.json @@ -161,5 +161,9 @@ "docker/test/sqllogic": { "name": "clickhouse/sqllogic-test", "dependent": [] + }, + "docker/test/integration/nginx_dav": { + "name": "clickhouse/nginx-dav", + "dependent": [] } } diff --git a/docker/test/integration/nginx_dav/Dockerfile b/docker/test/integration/nginx_dav/Dockerfile new file mode 100644 index 00000000000..42c1244f6dc --- /dev/null +++ b/docker/test/integration/nginx_dav/Dockerfile @@ -0,0 +1,6 @@ +FROM nginx:alpine-slim + +COPY default.conf /etc/nginx/conf.d/ + +RUN mkdir /usr/share/nginx/files/ \ + && chown nginx: /usr/share/nginx/files/ -R diff --git a/docker/test/integration/nginx_dav/default.conf b/docker/test/integration/nginx_dav/default.conf new file mode 100644 index 00000000000..466d0584a2d --- /dev/null +++ b/docker/test/integration/nginx_dav/default.conf @@ -0,0 +1,25 @@ +server { + listen 80; + + #root /usr/share/nginx/test.com; + index index.html index.htm; + + server_name test.com localhost; + + location / { + expires max; + root /usr/share/nginx/files; + client_max_body_size 20m; + client_body_temp_path /usr/share/nginx/tmp; + dav_methods PUT; # Allowed methods, only PUT is necessary + + create_full_put_path on; # nginx automatically creates nested directories + dav_access user:rw group:r all:r; # access permissions for files + + limit_except GET { + allow all; + } + } + + error_page 405 =200 $uri; +} diff --git a/docker/test/integration/runner/compose/docker_compose_nginx.yml b/docker/test/integration/runner/compose/docker_compose_nginx.yml index d0fb9fc1ff4..38d2a6d84c8 100644 --- a/docker/test/integration/runner/compose/docker_compose_nginx.yml +++ b/docker/test/integration/runner/compose/docker_compose_nginx.yml @@ -5,7 +5,7 @@ services: # Files will be put into /usr/share/nginx/files. nginx: - image: kssenii/nginx-test:1.1 + image: clickhouse/nginx-dav:${DOCKER_NGINX_DAV_TAG:-latest} restart: always ports: - 80:80 diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 3c4ff522b36..66750f2089a 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -73,6 +73,7 @@ export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} export DOCKER_KERBEROS_KDC_TAG=${DOCKER_KERBEROS_KDC_TAG:=latest} export DOCKER_KERBERIZED_HADOOP_TAG=${DOCKER_KERBERIZED_HADOOP_TAG:=latest} +export DOCKER_NGINX_DAV_TAG=${DOCKER_NGINX_DAV_TAG:=latest} cd /ClickHouse/tests/integration exec "$@" diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 222b2197117..d5b70720ee9 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -37,19 +37,20 @@ from upload_result_helper import upload_results # When update, update -# integration/ci-runner.py:ClickhouseIntegrationTestsRunner.get_images_names too +# tests/integration/ci-runner.py:ClickhouseIntegrationTestsRunner.get_images_names too IMAGES = [ + "clickhouse/dotnet-client", + "clickhouse/integration-helper", + "clickhouse/integration-test", "clickhouse/integration-tests-runner", + "clickhouse/kerberized-hadoop", + "clickhouse/kerberos-kdc", "clickhouse/mysql-golang-client", "clickhouse/mysql-java-client", "clickhouse/mysql-js-client", "clickhouse/mysql-php-client", + "clickhouse/nginx-dav", "clickhouse/postgresql-java-client", - "clickhouse/integration-test", - "clickhouse/kerberos-kdc", - "clickhouse/kerberized-hadoop", - "clickhouse/integration-helper", - "clickhouse/dotnet-client", ] diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index a6e9716dc20..8af86e37cce 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -313,6 +313,7 @@ class ClickhouseIntegrationTestsRunner: "clickhouse/mysql-java-client", "clickhouse/mysql-js-client", "clickhouse/mysql-php-client", + "clickhouse/nginx-dav", "clickhouse/postgresql-java-client", ] diff --git a/tests/integration/runner b/tests/integration/runner index 1b902803741..f12b62007a5 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -351,6 +351,8 @@ if __name__ == "__main__": env_tags += "-e {}={} ".format("DOCKER_KERBERIZED_HADOOP_TAG", tag) elif image == "clickhouse/kerberos-kdc": env_tags += "-e {}={} ".format("DOCKER_KERBEROS_KDC_TAG", tag) + elif image == "clickhouse/nginx-dav": + env_tags += "-e {}={} ".format("DOCKER_NGINX_DAV_TAG", tag) else: logging.info("Unknown image %s" % (image)) From cf01800a63c9f067be4ca398287abc12ba0066c3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 5 Jul 2023 15:39:26 +0200 Subject: [PATCH 1024/2047] Sort DOCKER_*_TAG stuff --- .../integration/runner/dockerd-entrypoint.sh | 10 ++++----- tests/integration/runner | 22 +++++++++---------- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 66750f2089a..b05aef76faf 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -64,16 +64,16 @@ export CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge export CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH=/clickhouse-library-bridge export DOCKER_BASE_TAG=${DOCKER_BASE_TAG:=latest} -export DOCKER_HELPER_TAG=${DOCKER_HELPER_TAG:=latest} -export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} export DOCKER_DOTNET_CLIENT_TAG=${DOCKER_DOTNET_CLIENT_TAG:=latest} +export DOCKER_HELPER_TAG=${DOCKER_HELPER_TAG:=latest} +export DOCKER_KERBERIZED_HADOOP_TAG=${DOCKER_KERBERIZED_HADOOP_TAG:=latest} +export DOCKER_KERBEROS_KDC_TAG=${DOCKER_KERBEROS_KDC_TAG:=latest} +export DOCKER_MYSQL_GOLANG_CLIENT_TAG=${DOCKER_MYSQL_GOLANG_CLIENT_TAG:=latest} export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} export DOCKER_MYSQL_JS_CLIENT_TAG=${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} -export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} -export DOCKER_KERBEROS_KDC_TAG=${DOCKER_KERBEROS_KDC_TAG:=latest} -export DOCKER_KERBERIZED_HADOOP_TAG=${DOCKER_KERBERIZED_HADOOP_TAG:=latest} export DOCKER_NGINX_DAV_TAG=${DOCKER_NGINX_DAV_TAG:=latest} +export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} cd /ClickHouse/tests/integration exec "$@" diff --git a/tests/integration/runner b/tests/integration/runner index f12b62007a5..4c2b1054538 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -331,18 +331,8 @@ if __name__ == "__main__": if args.docker_compose_images_tags is not None: for img_tag in args.docker_compose_images_tags: [image, tag] = img_tag.split(":") - if image == "clickhouse/mysql-golang-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) - elif image == "clickhouse/dotnet-client": + if image == "clickhouse/dotnet-client": env_tags += "-e {}={} ".format("DOCKER_DOTNET_CLIENT_TAG", tag) - elif image == "clickhouse/mysql-java-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) - elif image == "clickhouse/mysql-js-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) - elif image == "clickhouse/mysql-php-client": - env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) - elif image == "clickhouse/postgresql-java-client": - env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) elif image == "clickhouse/integration-helper": env_tags += "-e {}={} ".format("DOCKER_HELPER_TAG", tag) elif image == "clickhouse/integration-test": @@ -351,8 +341,18 @@ if __name__ == "__main__": env_tags += "-e {}={} ".format("DOCKER_KERBERIZED_HADOOP_TAG", tag) elif image == "clickhouse/kerberos-kdc": env_tags += "-e {}={} ".format("DOCKER_KERBEROS_KDC_TAG", tag) + elif image == "clickhouse/mysql-golang-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_GOLANG_CLIENT_TAG", tag) + elif image == "clickhouse/mysql-java-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JAVA_CLIENT_TAG", tag) + elif image == "clickhouse/mysql-js-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_JS_CLIENT_TAG", tag) + elif image == "clickhouse/mysql-php-client": + env_tags += "-e {}={} ".format("DOCKER_MYSQL_PHP_CLIENT_TAG", tag) elif image == "clickhouse/nginx-dav": env_tags += "-e {}={} ".format("DOCKER_NGINX_DAV_TAG", tag) + elif image == "clickhouse/postgresql-java-client": + env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) else: logging.info("Unknown image %s" % (image)) From 363201270c00f0ebfa61e80471b372f434370380 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 17:49:53 +0300 Subject: [PATCH 1025/2047] Update 01710_query_log_with_projection_info.sql --- .../0_stateless/01710_query_log_with_projection_info.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01710_query_log_with_projection_info.sql b/tests/queries/0_stateless/01710_query_log_with_projection_info.sql index 25e7e8fed60..cd84b392fe5 100644 --- a/tests/queries/0_stateless/01710_query_log_with_projection_info.sql +++ b/tests/queries/0_stateless/01710_query_log_with_projection_info.sql @@ -62,3 +62,5 @@ FROM system.query_log WHERE current_database=currentDatabase() and query = 'SELECT min(id) FROM t FORMAT Null;'; + +DROP TABLE t; From c3c8ea9c27669a548e2404e2ed6c2e8f3c734dc9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 17:30:55 +0200 Subject: [PATCH 1026/2047] Do something --- tests/ci/docker_test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 61319041b9f..8542de412d2 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -40,7 +40,12 @@ class TestDockerImageCheck(unittest.TestCase): [ di.DockerImage("docker/test/base", "clickhouse/test-base", False), di.DockerImage("docker/docs/builder", "clickhouse/docs-builder", True), - di.DockerImage("docker/test/sqltest", "clickhouse/sqltest", True), + di.DockerImage( + "docker/test/sqltest", + "clickhouse/sqltest", + True, + "clickhouse/test-base", # type: ignore + ), di.DockerImage( "docker/test/stateless", "clickhouse/stateless-test", From dab954a92d7893a7ebbef9cda0a3aedf63a96a50 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 22 Jul 2023 18:10:54 +0200 Subject: [PATCH 1027/2047] do not throw exception in OptimizedRegularExpressionImpl::analyze --- src/Common/OptimizedRegularExpression.cpp | 13 +++++++++++-- .../0_stateless/02831_regexp_analyze_recursion.sql | 2 +- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 0b80e2f3f97..918ebd75fc0 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -423,6 +423,7 @@ void OptimizedRegularExpressionImpl::analyze( bool & is_trivial, bool & required_substring_is_prefix, std::vector & alternatives) +try { Literals alternative_literals; Literal required_literal; @@ -432,12 +433,20 @@ void OptimizedRegularExpressionImpl::analyze( for (auto & lit : alternative_literals) alternatives.push_back(std::move(lit.literal)); } +catch(...) +{ + required_substring = ""; + is_trivial = false; + required_substring_is_prefix = false; + alternatives.clear(); + std::cerr << "Analyze RegularExpression failed, got error: {}" << DB::getCurrentExceptionMessage(false) << "\n"; +} template OptimizedRegularExpressionImpl::OptimizedRegularExpressionImpl(const std::string & regexp_, int options) { - std::vector alternativesDummy; /// this vector extracts patterns a,b,c from pattern (a|b|c). for now it's not used. - analyze(regexp_, required_substring, is_trivial, required_substring_is_prefix, alternativesDummy); + std::vector alternatives_dummy; /// this vector extracts patterns a,b,c from pattern (a|b|c). for now it's not used. + analyze(regexp_, required_substring, is_trivial, required_substring_is_prefix, alternatives_dummy); /// Just three following options are supported diff --git a/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql b/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql index 018d1f031e6..a2075ae903b 100644 --- a/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql +++ b/tests/queries/0_stateless/02831_regexp_analyze_recursion.sql @@ -1 +1 @@ -SELECT match('', repeat('(', 100000)); -- { serverError 306 } +SELECT match('', repeat('(', 100000)); -- { serverError 427 } From c60090ccbd30143d44ab715b8b7b5e0060a2095f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 22 Jul 2023 17:43:22 +0000 Subject: [PATCH 1028/2047] Add test with materialize() + fix --- ...2810_fix_remove_dedundant_distinct_view.reference | 12 ++++++++++-- .../02810_fix_remove_dedundant_distinct_view.sql | 10 +++++++++- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference index 01f14f82e94..ec714a5df07 100644 --- a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference +++ b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.reference @@ -8,6 +8,14 @@ FROM ) WHERE explain ILIKE '%distinct%'; 2 -SELECT DISTINCT x FROM tab_v; -2 +SELECT DISTINCT x FROM tab_v ORDER BY x; 1 +2 +-- explicitly checking that materialize() doesn't affect the result, - redundant DISTINCT is still removed +SELECT count() +FROM +( + EXPLAIN SELECT DISTINCT x FROM (SELECT materialize(x) as x FROM (select DISTINCT x from tab)) +) +WHERE explain ILIKE '%distinct%'; +2 diff --git a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql index 99fc24dae8b..ca0a2edd99d 100644 --- a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql +++ b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql @@ -19,4 +19,12 @@ FROM ) WHERE explain ILIKE '%distinct%'; -SELECT DISTINCT x FROM tab_v; +SELECT DISTINCT x FROM tab_v ORDER BY x; + +-- explicitly checking that materialize() doesn't affect the result, - redundant DISTINCT is still removed +SELECT count() +FROM +( + EXPLAIN SELECT DISTINCT x FROM (SELECT materialize(x) as x FROM (select DISTINCT x from tab)) +) +WHERE explain ILIKE '%distinct%'; From c4a86c148adf5f4830050222950ac2e65b9ac11b Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 22 Jul 2023 17:47:27 +0000 Subject: [PATCH 1029/2047] fix --- .../Passes/CustomizeFunctionsPass.cpp | 176 ------------------ ...th_count_distinct_implementation.reference | 6 - ..._if_with_count_distinct_implementation.sql | 10 - 3 files changed, 192 deletions(-) delete mode 100644 src/Analyzer/Passes/CustomizeFunctionsPass.cpp diff --git a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp b/src/Analyzer/Passes/CustomizeFunctionsPass.cpp deleted file mode 100644 index 983312728be..00000000000 --- a/src/Analyzer/Passes/CustomizeFunctionsPass.cpp +++ /dev/null @@ -1,176 +0,0 @@ -#include - -#include -#include - -#include - -#include - -#include -#include - -namespace DB -{ - -namespace -{ - -class CustomizeFunctionsVisitor : public InDepthQueryTreeVisitorWithContext -{ -public: - using Base = InDepthQueryTreeVisitorWithContext; - using Base::Base; - - void visitImpl(QueryTreeNodePtr & node) const - { - auto * function_node = node->as(); - if (!function_node) - return; - - const auto & settings = getSettings(); - - /// After successful function replacement function name and function name lowercase must be recalculated - auto function_name = function_node->getFunctionName(); - auto function_name_lowercase = Poco::toLower(function_name); - - if (function_node->isAggregateFunction() || function_node->isWindowFunction()) - { - auto count_distinct_implementation_function_name = String(settings.count_distinct_implementation); - - /// Replace countDistinct with countDistinct implementation - if (function_name_lowercase == "countdistinct") - { - resolveAggregateOrWindowFunctionNode(*function_node, count_distinct_implementation_function_name); - function_name = function_node->getFunctionName(); - function_name_lowercase = Poco::toLower(function_name); - } - - /// Replace countIfDistinct with countDistinctIf implementation - if (function_name_lowercase == "countifdistinct") - { - resolveAggregateOrWindowFunctionNode(*function_node, count_distinct_implementation_function_name + "If"); - function_name = function_node->getFunctionName(); - function_name_lowercase = Poco::toLower(function_name); - } - - if (settings.rewrite_count_distinct_if_with_count_distinct_implementation && function_name_lowercase == "countdistinctif") - { - resolveAggregateOrWindowFunctionNode(*function_node, count_distinct_implementation_function_name + "If"); - function_name = function_node->getFunctionName(); - function_name_lowercase = Poco::toLower(function_name); - } - - /// Replace aggregateFunctionIfDistinct into aggregateFunctionDistinctIf to make execution more optimal - if (function_name_lowercase.ends_with("ifdistinct")) - { - size_t prefix_length = function_name_lowercase.size() - strlen("ifdistinct"); - auto updated_function_name = function_name_lowercase.substr(0, prefix_length) + "DistinctIf"; - resolveAggregateOrWindowFunctionNode(*function_node, updated_function_name); - function_name = function_node->getFunctionName(); - function_name_lowercase = Poco::toLower(function_name); - } - - /// Rewrite all aggregate functions to add -OrNull suffix to them - if (settings.aggregate_functions_null_for_empty && !function_name.ends_with("OrNull")) - { - auto function_properies = AggregateFunctionFactory::instance().tryGetProperties(function_name); - if (function_properies && !function_properies->returns_default_when_only_null) - { - auto updated_function_name = function_name + "OrNull"; - resolveAggregateOrWindowFunctionNode(*function_node, updated_function_name); - function_name = function_node->getFunctionName(); - function_name_lowercase = Poco::toLower(function_name); - } - } - - /** Move -OrNull suffix ahead, this should execute after add -OrNull suffix. - * Used to rewrite aggregate functions with -OrNull suffix in some cases. - * Example: sumIfOrNull. - * Result: sumOrNullIf. - */ - if (function_name.ends_with("OrNull")) - { - auto function_properies = AggregateFunctionFactory::instance().tryGetProperties(function_name); - if (function_properies && !function_properies->returns_default_when_only_null) - { - size_t function_name_size = function_name.size(); - - static constexpr std::array suffixes_to_replace = {"MergeState", "Merge", "State", "If"}; - for (const auto & suffix : suffixes_to_replace) - { - auto suffix_string_value = String(suffix); - auto suffix_to_check = suffix_string_value + "OrNull"; - - if (!function_name.ends_with(suffix_to_check)) - continue; - - auto updated_function_name = function_name.substr(0, function_name_size - suffix_to_check.size()) + "OrNull" + suffix_string_value; - resolveAggregateOrWindowFunctionNode(*function_node, updated_function_name); - function_name = function_node->getFunctionName(); - function_name_lowercase = Poco::toLower(function_name); - break; - } - } - } - - return; - } - - if (settings.transform_null_in) - { - auto function_result_type = function_node->getResultType(); - - static constexpr std::array, 4> in_function_to_replace_null_in_function_map = - {{ - {"in", "nullIn"}, - {"notin", "notNullIn"}, - {"globalin", "globalNullIn"}, - {"globalnotin", "globalNotNullIn"}, - }}; - - for (const auto & [in_function_name, in_function_name_to_replace] : in_function_to_replace_null_in_function_map) - { - if (function_name_lowercase == in_function_name) - { - resolveOrdinaryFunctionNode(*function_node, String(in_function_name_to_replace)); - function_name = function_node->getFunctionName(); - function_name_lowercase = Poco::toLower(function_name); - break; - } - } - } - } - - static inline void resolveAggregateOrWindowFunctionNode(FunctionNode & function_node, const String & aggregate_function_name) - { - auto function_aggregate_function = function_node.getAggregateFunction(); - - AggregateFunctionProperties properties; - auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, - function_aggregate_function->getArgumentTypes(), - function_aggregate_function->getParameters(), - properties); - - if (function_node.isAggregateFunction()) - function_node.resolveAsAggregateFunction(std::move(aggregate_function)); - else if (function_node.isWindowFunction()) - function_node.resolveAsWindowFunction(std::move(aggregate_function)); - } - - inline void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const - { - auto function = FunctionFactory::instance().get(function_name, getContext()); - function_node.resolveAsFunction(function->build(function_node.getArgumentColumns())); - } -}; - -} - -void CustomizeFunctionsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) -{ - CustomizeFunctionsVisitor visitor(std::move(context)); - visitor.visit(query_tree_node); -} - -} diff --git a/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.reference b/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.reference index f187b72b8ea..efc3cd99060 100644 --- a/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.reference +++ b/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.reference @@ -4,9 +4,3 @@ FROM numbers(1000) 2 SELECT uniqExactIf(number % 10, (number % 5) = 2) FROM numbers(1000) -2 -SELECT countDistinctIf(number % 10, (number % 5) = 2) -FROM numbers(1000) -2 -SELECT uniqExactIf(number % 10, (number % 5) = 2) -FROM numbers(1000) diff --git a/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.sql b/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.sql index a06a22db55e..a81f53c1ef6 100644 --- a/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.sql +++ b/tests/queries/0_stateless/02554_rewrite_count_distinct_if_with_count_distinct_implementation.sql @@ -6,13 +6,3 @@ EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers( SET rewrite_count_distinct_if_with_count_distinct_implementation = 1; SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); - -SET allow_experimental_analyzer = 1; - -SET rewrite_count_distinct_if_with_count_distinct_implementation = 0; -SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); -EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); - -SET rewrite_count_distinct_if_with_count_distinct_implementation = 1; -SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); -EXPLAIN SYNTAX SELECT countDistinctIf(number % 10, number % 5 = 2) FROM numbers(1000); From afdda489bdfb27d1db2a7554223f5dfcb8cca7b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 19:53:16 +0200 Subject: [PATCH 1030/2047] Fix test --- .../02790_sql_standard_fetch.reference | 72 +++++++++---------- .../0_stateless/02790_sql_standard_fetch.sql | 12 ++-- 2 files changed, 42 insertions(+), 42 deletions(-) diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.reference b/tests/queries/0_stateless/02790_sql_standard_fetch.reference index 429eecbc936..270af6e5c17 100644 --- a/tests/queries/0_stateless/02790_sql_standard_fetch.reference +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.reference @@ -1,36 +1,36 @@ -┌─id─┬─name──┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry │ it │ 104 │ -│ 24 │ Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 32 │ Dave │ sales │ 96 │ -└────┴───────┴────────────┴────────┘ -┌─id─┬─name──┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry │ it │ 104 │ -│ 24 │ Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 32 │ Dave │ sales │ 96 │ -└────┴───────┴────────────┴────────┘ -┌─id─┬─name──┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry │ it │ 104 │ -│ 24 │ Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Cindy │ sales │ 96 │ -│ 32 │ Dave │ sales │ 96 │ -└────┴───────┴────────────┴────────┘ -┌─id─┬─name──┬─department─┬─salary─┐ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Cindy │ sales │ 96 │ -│ 32 │ Dave │ sales │ 96 │ -│ 22 │ Grace │ it │ 90 │ -│ 21 │ Emma │ it │ 84 │ -└────┴───────┴────────────┴────────┘ -┌─id─┬─name──┬─department─┬─salary─┐ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Cindy │ sales │ 96 │ -│ 32 │ Dave │ sales │ 96 │ -│ 22 │ Grace │ it │ 90 │ -│ 21 │ Emma │ it │ 84 │ -└────┴───────┴────────────┴────────┘ +┌─id─┬─name───────────┬─department─┬─salary─┐ +│ 25 │ Frank │ it │ 120 │ +│ 23 │ Henry or Irene │ it │ 104 │ +│ 24 │ Henry or Irene │ it │ 104 │ +│ 33 │ Alice │ sales │ 100 │ +│ 32 │ Dave or Cindy │ sales │ 96 │ +└────┴────────────────┴────────────┴────────┘ +┌─id─┬─name───────────┬─department─┬─salary─┐ +│ 25 │ Frank │ it │ 120 │ +│ 23 │ Henry or Irene │ it │ 104 │ +│ 24 │ Henry or Irene │ it │ 104 │ +│ 33 │ Alice │ sales │ 100 │ +│ 32 │ Dave or Cindy │ sales │ 96 │ +└────┴────────────────┴────────────┴────────┘ +┌─id─┬─name───────────┬─department─┬─salary─┐ +│ 25 │ Frank │ it │ 120 │ +│ 23 │ Henry or Irene │ it │ 104 │ +│ 24 │ Henry or Irene │ it │ 104 │ +│ 33 │ Alice │ sales │ 100 │ +│ 31 │ Dave or Cindy │ sales │ 96 │ +│ 32 │ Dave or Cindy │ sales │ 96 │ +└────┴────────────────┴────────────┴────────┘ +┌─id─┬─name──────────┬─department─┬─salary─┐ +│ 33 │ Alice │ sales │ 100 │ +│ 31 │ Dave or Cindy │ sales │ 96 │ +│ 32 │ Dave or Cindy │ sales │ 96 │ +│ 22 │ Grace │ it │ 90 │ +│ 21 │ Emma │ it │ 84 │ +└────┴───────────────┴────────────┴────────┘ +┌─id─┬─name──────────┬─department─┬─salary─┐ +│ 33 │ Alice │ sales │ 100 │ +│ 31 │ Dave or Cindy │ sales │ 96 │ +│ 32 │ Dave or Cindy │ sales │ 96 │ +│ 22 │ Grace │ it │ 90 │ +│ 21 │ Emma │ it │ 84 │ +└────┴───────────────┴────────────┴────────┘ diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.sql b/tests/queries/0_stateless/02790_sql_standard_fetch.sql index 4204279a746..07a806eddf9 100644 --- a/tests/queries/0_stateless/02790_sql_standard_fetch.sql +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.sql @@ -1,33 +1,33 @@ -- https://antonz.org/sql-fetch/ CREATE TEMPORARY TABLE employees (id UInt64, name String, department String, salary UInt64); -INSERT INTO employees VALUES (23, 'Henry', 'it', 104), (24, 'Irene', 'it', 104), (25, 'Frank', 'it', 120), (31, 'Cindy', 'sales', 96), (33, 'Alice', 'sales', 100), (32, 'Dave', 'sales', 96), (22, 'Grace', 'it', 90), (21, 'Emma', 'it', '84'); +INSERT INTO employees VALUES (23, 'Henry', 'it', 104), (24, 'Irene', 'it', 104), (25, 'Frank', 'it', 120), (31, 'Cindy', 'sales', 96), (33, 'Alice', 'sales', 100), (32, 'Dave', 'sales', 96), (22, 'Grace', 'it', 90), (21, 'Emma', 'it', 84); -- Determinism SET max_threads = 1, parallelize_output_from_storages = 0; -select * from (SELECT * FROM employees ORDER BY id, name, department, salary) +select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc limit 5 format PrettyCompactNoEscapes; -select * from (SELECT * FROM employees ORDER BY id, name, department, salary) +select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc fetch first 5 rows only format PrettyCompactNoEscapes; -select * from (SELECT * FROM employees ORDER BY id, name, department, salary) +select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc fetch first 5 rows with ties format PrettyCompactNoEscapes; -select * from (SELECT * FROM employees ORDER BY id, name, department, salary) +select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc offset 3 rows fetch next 5 rows only format PrettyCompactNoEscapes; -select * from (SELECT * FROM employees ORDER BY id, name, department, salary) +select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc offset 3 rows fetch first 5 rows only From 7a24de801d93957cd87e8a1d2f726b934912b038 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 19:53:45 +0200 Subject: [PATCH 1031/2047] Fix test --- .../02790_sql_standard_fetch.reference | 72 +++++++++---------- .../0_stateless/02790_sql_standard_fetch.sql | 10 +-- 2 files changed, 41 insertions(+), 41 deletions(-) diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.reference b/tests/queries/0_stateless/02790_sql_standard_fetch.reference index 270af6e5c17..ef578b526da 100644 --- a/tests/queries/0_stateless/02790_sql_standard_fetch.reference +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.reference @@ -1,36 +1,36 @@ -┌─id─┬─name───────────┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry or Irene │ it │ 104 │ -│ 24 │ Henry or Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 32 │ Dave or Cindy │ sales │ 96 │ -└────┴────────────────┴────────────┴────────┘ -┌─id─┬─name───────────┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry or Irene │ it │ 104 │ -│ 24 │ Henry or Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 32 │ Dave or Cindy │ sales │ 96 │ -└────┴────────────────┴────────────┴────────┘ -┌─id─┬─name───────────┬─department─┬─salary─┐ -│ 25 │ Frank │ it │ 120 │ -│ 23 │ Henry or Irene │ it │ 104 │ -│ 24 │ Henry or Irene │ it │ 104 │ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Dave or Cindy │ sales │ 96 │ -│ 32 │ Dave or Cindy │ sales │ 96 │ -└────┴────────────────┴────────────┴────────┘ -┌─id─┬─name──────────┬─department─┬─salary─┐ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Dave or Cindy │ sales │ 96 │ -│ 32 │ Dave or Cindy │ sales │ 96 │ -│ 22 │ Grace │ it │ 90 │ -│ 21 │ Emma │ it │ 84 │ -└────┴───────────────┴────────────┴────────┘ -┌─id─┬─name──────────┬─department─┬─salary─┐ -│ 33 │ Alice │ sales │ 100 │ -│ 31 │ Dave or Cindy │ sales │ 96 │ -│ 32 │ Dave or Cindy │ sales │ 96 │ -│ 22 │ Grace │ it │ 90 │ -│ 21 │ Emma │ it │ 84 │ -└────┴───────────────┴────────────┴────────┘ +┌─name───────────┬─department─┬─salary─┐ +│ Frank │ it │ 120 │ +│ Henry or Irene │ it │ 104 │ +│ Henry or Irene │ it │ 104 │ +│ Alice │ sales │ 100 │ +│ Dave or Cindy │ sales │ 96 │ +└────────────────┴────────────┴────────┘ +┌─name───────────┬─department─┬─salary─┐ +│ Frank │ it │ 120 │ +│ Henry or Irene │ it │ 104 │ +│ Henry or Irene │ it │ 104 │ +│ Alice │ sales │ 100 │ +│ Dave or Cindy │ sales │ 96 │ +└────────────────┴────────────┴────────┘ +┌─name───────────┬─department─┬─salary─┐ +│ Frank │ it │ 120 │ +│ Henry or Irene │ it │ 104 │ +│ Henry or Irene │ it │ 104 │ +│ Alice │ sales │ 100 │ +│ Dave or Cindy │ sales │ 96 │ +│ Dave or Cindy │ sales │ 96 │ +└────────────────┴────────────┴────────┘ +┌─name──────────┬─department─┬─salary─┐ +│ Alice │ sales │ 100 │ +│ Dave or Cindy │ sales │ 96 │ +│ Dave or Cindy │ sales │ 96 │ +│ Grace │ it │ 90 │ +│ Emma │ it │ 84 │ +└───────────────┴────────────┴────────┘ +┌─name──────────┬─department─┬─salary─┐ +│ Alice │ sales │ 100 │ +│ Dave or Cindy │ sales │ 96 │ +│ Dave or Cindy │ sales │ 96 │ +│ Grace │ it │ 90 │ +│ Emma │ it │ 84 │ +└───────────────┴────────────┴────────┘ diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.sql b/tests/queries/0_stateless/02790_sql_standard_fetch.sql index 07a806eddf9..638cc66682d 100644 --- a/tests/queries/0_stateless/02790_sql_standard_fetch.sql +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.sql @@ -6,28 +6,28 @@ INSERT INTO employees VALUES (23, 'Henry', 'it', 104), (24, 'Irene', 'it', 104), -- Determinism SET max_threads = 1, parallelize_output_from_storages = 0; -select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) +select transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc limit 5 format PrettyCompactNoEscapes; -select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) +select transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc fetch first 5 rows only format PrettyCompactNoEscapes; -select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) +select transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc fetch first 5 rows with ties format PrettyCompactNoEscapes; -select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) +select transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc offset 3 rows fetch next 5 rows only format PrettyCompactNoEscapes; -select id, transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) +select transform(name, ['Henry', 'Irene', 'Dave', 'Cindy'], ['Henry or Irene', 'Henry or Irene', 'Dave or Cindy', 'Dave or Cindy']) AS name, department, salary from (SELECT * FROM employees ORDER BY id, name, department, salary) order by salary desc offset 3 rows fetch first 5 rows only From 2c6bc318476ce98b916cd2ffb6a9a44f5a5488f8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 22 Jul 2023 18:07:22 +0000 Subject: [PATCH 1032/2047] Test is not dependent on new analyzer --- .../0_stateless/02810_fix_remove_dedundant_distinct_view.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql index ca0a2edd99d..10a68721c51 100644 --- a/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql +++ b/tests/queries/0_stateless/02810_fix_remove_dedundant_distinct_view.sql @@ -1,5 +1,3 @@ -set allow_experimental_analyzer=1; - drop table if exists tab_v; drop table if exists tab; create table tab (x UInt64, y UInt64) engine MergeTree() order by (x, y); From d25cd0d0b635196b1a4cb2178d93b7060bf02819 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 20:21:33 +0200 Subject: [PATCH 1033/2047] Partial revert --- tests/queries/0_stateless/01187_set_profile_as_setting.sh | 2 +- .../0_stateless/02360_rename_table_along_with_log_name.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01187_set_profile_as_setting.sh b/tests/queries/0_stateless/01187_set_profile_as_setting.sh index fccac57aea8..dacb609d790 100755 --- a/tests/queries/0_stateless/01187_set_profile_as_setting.sh +++ b/tests/queries/0_stateless/01187_set_profile_as_setting.sh @@ -4,13 +4,13 @@ unset CLICKHOUSE_LOG_COMMENT CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -n -m -q "select value, changed from system.settings where name='readonly';" $CLICKHOUSE_CLIENT -n -m -q "set profile='default'; select value, changed from system.settings where name='readonly';" $CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" 2>&1| grep -Fa "Cannot modify 'send_logs_level' setting in readonly mode" > /dev/null && echo "OK" +CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=fatal/g') $CLICKHOUSE_CLIENT -n -m -q "set profile='readonly'; select value, changed from system.settings where name='readonly';" ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query=select+value,changed+from+system.settings+where+name='readonly'" diff --git a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh index c07dcdd549b..e8c7f844b5c 100755 --- a/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh +++ b/tests/queries/0_stateless/02360_rename_table_along_with_log_name.sh @@ -1,7 +1,6 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh @@ -12,6 +11,7 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS y;" $CLICKHOUSE_CLIENT -q "CREATE TABLE x(i int) ENGINE MergeTree ORDER BY i;" $CLICKHOUSE_CLIENT -q "RENAME TABLE x TO y;" +CLICKHOUSE_CLIENT_WITH_LOG=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') regexp="${CLICKHOUSE_DATABASE}\\.x" # Check if there are still log entries with old table name $CLICKHOUSE_CLIENT_WITH_LOG --send_logs_source_regexp "$regexp" -q "INSERT INTO y VALUES(1);" From 0b258dda4ee618a4d002e2b5246d68bbd2c77c7e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 21 Jul 2023 08:31:45 +0200 Subject: [PATCH 1034/2047] Reproducible builds for Rust From now on cargo will not download anything from the internet during builds. This step had been moved for docker image builds (via cargo vendor). And now cargo inside docker.io/clickhouse/binary-builder will not use any crates from the internet, so we don't need to add --offline for cargo commands in cmake (corrosion_import_crate()). Also the docker build command had been adjusted to allow following symlinks inside build context, by using tar, this is required for Rust packages. Note, that to make proper Cargo.lock that could be vendored I did the following: - per-project locks had been removed (since there is no automatic way to sync the workspace Cargo.lock with per-project Cargo.lock, since cargo update/generate-lockfile will use only per-project Cargo.toml files apparently, -Z minimal-versions does not helps either) - and to generate Cargo.lock with less changes I've pinned version in the Cargo.toml strictly, i.e. not 'foo = "0.1"' but 'foo = "=0.1"' then the Cargo.lock for workspace had been generated and afterwards I've reverted this part. Plus I have to update the dependencies afterwards, since otherwise there are conflicts with dependencies for std library. Non trivial. Signed-off-by: Azat Khuzhin --- .gitignore | 2 + docker/packager/binary/Dockerfile | 27 ++ docker/packager/binary/rust | 1 + rust/.dockerignore | 4 + rust/.gitignore | 4 + rust/BLAKE3/Cargo.lock | 92 ----- rust/CMakeLists.txt | 2 + rust/{skim => }/Cargo.lock | 519 +++++++++++++++++++++++++-- rust/Cargo.toml | 12 + rust/prql/Cargo.lock | 569 ------------------------------ tests/ci/docker_images_check.py | 33 +- tests/ci/docker_test.py | 12 +- 12 files changed, 582 insertions(+), 695 deletions(-) create mode 120000 docker/packager/binary/rust create mode 100644 rust/.dockerignore create mode 100644 rust/.gitignore delete mode 100644 rust/BLAKE3/Cargo.lock rename rust/{skim => }/Cargo.lock (66%) create mode 100644 rust/Cargo.toml delete mode 100644 rust/prql/Cargo.lock diff --git a/.gitignore b/.gitignore index 39d6f3f9fc8..5341f23a94f 100644 --- a/.gitignore +++ b/.gitignore @@ -69,6 +69,7 @@ cmake-build-* *.pyc __pycache__ *.pytest_cache +.mypy_cache test.cpp CPackConfig.cmake @@ -167,3 +168,4 @@ tests/integration/**/_gen /rust/**/target # It is autogenerated from *.in /rust/**/.cargo/config.toml +/rust/**/vendor diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 897bcd24d04..99e748c41d4 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -58,6 +58,33 @@ RUN curl https://sh.rustup.rs -sSf | bash -s -- -y && \ rustup target add aarch64-apple-darwin && \ rustup target add powerpc64le-unknown-linux-gnu +# Create vendor cache for cargo. +# +# Note, that the config.toml for the root is used, you will not be able to +# install any other crates, except those which had been vendored (since if +# there is "replace-with" for some source, then cargo will not look to other +# remotes except this). +# +# Notes for the command itself: +# - --chown is required to preserve the rights +# - unstable-options for -C +# - chmod is required to fix the permissions, since builds are running from a different user +# - copy of the Cargo.lock is required for proper dependencies versions +# - cargo vendor --sync is requried to overcome [1] bug. +# +# [1]: https://github.com/rust-lang/wg-cargo-std-aware/issues/23 +COPY --chown=root:root /rust /rust/packages +RUN cargo -Z unstable-options -C /rust/packages vendor > $CARGO_HOME/config.toml && \ + cp "$(rustc --print=sysroot)"/lib/rustlib/src/rust/Cargo.lock "$(rustc --print=sysroot)"/lib/rustlib/src/rust/library/test/ && \ + cargo -Z unstable-options -C /rust/packages vendor --sync "$(rustc --print=sysroot)"/lib/rustlib/src/rust/library/test/Cargo.toml && \ + rm "$(rustc --print=sysroot)"/lib/rustlib/src/rust/library/test/Cargo.lock && \ + sed -i "s#\"vendor\"#\"/rust/vendor\"#" $CARGO_HOME/config.toml && \ + cat $CARGO_HOME/config.toml && \ + mv /rust/packages/vendor /rust/vendor && \ + chmod -R o=r+X /rust/vendor && \ + ls -R -l /rust/packages && \ + rm -r /rust/packages + # NOTE: Seems like gcc-11 is too new for ubuntu20 repository # A cross-linker for RISC-V 64 (we need it, because LLVM's LLD does not work): RUN add-apt-repository ppa:ubuntu-toolchain-r/test --yes \ diff --git a/docker/packager/binary/rust b/docker/packager/binary/rust new file mode 120000 index 00000000000..742dc49e9ac --- /dev/null +++ b/docker/packager/binary/rust @@ -0,0 +1 @@ +../../../rust \ No newline at end of file diff --git a/rust/.dockerignore b/rust/.dockerignore new file mode 100644 index 00000000000..6b761aa401c --- /dev/null +++ b/rust/.dockerignore @@ -0,0 +1,4 @@ +# Just in case ignore any cargo stuff (and just in case someone will run this +# docker build locally with build context using folder root): +target +vendor diff --git a/rust/.gitignore b/rust/.gitignore new file mode 100644 index 00000000000..f850cd563c9 --- /dev/null +++ b/rust/.gitignore @@ -0,0 +1,4 @@ +# This is for tar --exclude-vcs-ignores (and just in case someone will run +# docker build locally with build context created via tar): +target +vendor diff --git a/rust/BLAKE3/Cargo.lock b/rust/BLAKE3/Cargo.lock deleted file mode 100644 index 9ac60773732..00000000000 --- a/rust/BLAKE3/Cargo.lock +++ /dev/null @@ -1,92 +0,0 @@ -# This file is automatically @generated by Cargo. -# It is not intended for manual editing. -version = 3 - -[[package]] -name = "_ch_rust_blake3" -version = "0.1.0" -dependencies = [ - "blake3", - "libc", -] - -[[package]] -name = "arrayref" -version = "0.3.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4c527152e37cf757a3f78aae5a06fbeefdb07ccc535c980a3208ee3060dd544" - -[[package]] -name = "arrayvec" -version = "0.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8da52d66c7071e2e3fa2a1e5c6d088fec47b593032b254f5e980de8ea54454d6" - -[[package]] -name = "blake3" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "526c210b4520e416420759af363083471656e819a75e831b8d2c9d5a584f2413" -dependencies = [ - "arrayref", - "arrayvec", - "cc", - "cfg-if", - "constant_time_eq", - "digest", -] - -[[package]] -name = "cc" -version = "1.0.73" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fff2a6927b3bb87f9595d67196a70493f627687a71d87a0d692242c33f58c11" - -[[package]] -name = "cfg-if" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" - -[[package]] -name = "constant_time_eq" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "245097e9a4535ee1e3e3931fcfcd55a796a44c643e8596ff6566d68f09b87bbc" - -[[package]] -name = "digest" -version = "0.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3dd60d1080a57a05ab032377049e0591415d2b31afd7028356dbf3cc6dcb066" -dependencies = [ - "generic-array", -] - -[[package]] -name = "generic-array" -version = "0.14.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bff49e947297f3312447abdca79f45f4738097cc82b06e72054d2223f601f1b9" -dependencies = [ - "typenum", - "version_check", -] - -[[package]] -name = "libc" -version = "0.2.132" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8371e4e5341c3a96db127eb2465ac681ced4c433e01dd0e938adbef26ba93ba5" - -[[package]] -name = "typenum" -version = "1.15.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dcf81ac59edc17cc8697ff311e8f5ef2d99fcbd9817b34cec66f90b6c3dfd987" - -[[package]] -name = "version_check" -version = "0.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" diff --git a/rust/CMakeLists.txt b/rust/CMakeLists.txt index 41451fe0a1e..ca0886cb300 100644 --- a/rust/CMakeLists.txt +++ b/rust/CMakeLists.txt @@ -55,6 +55,8 @@ function(clickhouse_import_crate) endif() endif() + # Note, here --offline is not used, since on CI vendor archive is used, and + # passing --offline here will be inconvenient for local development. corrosion_import_crate(NO_STD ${ARGN} PROFILE ${profile}) endfunction() diff --git a/rust/skim/Cargo.lock b/rust/Cargo.lock similarity index 66% rename from rust/skim/Cargo.lock rename to rust/Cargo.lock index f55ea8a84b0..07bbf8ba27e 100644 --- a/rust/skim/Cargo.lock +++ b/rust/Cargo.lock @@ -2,6 +2,22 @@ # It is not intended for manual editing. version = 3 +[[package]] +name = "_ch_rust_blake3" +version = "0.1.0" +dependencies = [ + "blake3", + "libc", +] + +[[package]] +name = "_ch_rust_prql" +version = "0.1.0" +dependencies = [ + "prql-compiler", + "serde_json", +] + [[package]] name = "_ch_rust_skim_rust" version = "0.1.0" @@ -12,6 +28,32 @@ dependencies = [ "term", ] +[[package]] +name = "addr2line" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4fa78e18c64fce05e902adecd7a5eed15a5e0a3439f7b0e169f0252214865e3" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "ahash" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" +dependencies = [ + "getrandom", + "once_cell", + "version_check", +] + [[package]] name = "aho-corasick" version = "1.0.2" @@ -36,6 +78,31 @@ dependencies = [ "libc", ] +[[package]] +name = "anyhow" +version = "1.0.72" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b13c32d80ecc7ab747b80c3784bce54ee8a7a0cc4fbda9bf4cda2cf6fe90854" +dependencies = [ + "backtrace", +] + +[[package]] +name = "ariadne" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "367fd0ad87307588d087544707bc5fbf4805ded96c7db922b70d368fa1cb5702" +dependencies = [ + "unicode-width", + "yansi", +] + +[[package]] +name = "arrayref" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b4930d2cb77ce62f89ee5d5289b4ac049559b1c45539271f5ed4fdc7db34545" + [[package]] name = "arrayvec" version = "0.7.4" @@ -48,6 +115,21 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" +[[package]] +name = "backtrace" +version = "0.3.68" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4319208da049c43661739c5fade2ba182f09d1dc2299b32298d3a31692b17e12" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", +] + [[package]] name = "beef" version = "0.5.2" @@ -60,6 +142,29 @@ version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" +[[package]] +name = "blake3" +version = "1.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "199c42ab6972d92c9f8995f086273d25c42fc0f7b2a1fcefba465c1352d25ba5" +dependencies = [ + "arrayref", + "arrayvec", + "cc", + "cfg-if", + "constant_time_eq", + "digest", +] + +[[package]] +name = "block-buffer" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" +dependencies = [ + "generic-array", +] + [[package]] name = "bumpalo" version = "3.13.0" @@ -93,6 +198,16 @@ dependencies = [ "winapi", ] +[[package]] +name = "chumsky" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23170228b96236b5a7299057ac284a321457700bc8c41a4476052f0f4ba5349d" +dependencies = [ + "hashbrown 0.12.3", + "stacker", +] + [[package]] name = "codespan-reporting" version = "0.11.1" @@ -103,6 +218,12 @@ dependencies = [ "unicode-width", ] +[[package]] +name = "constant_time_eq" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" + [[package]] name = "core-foundation-sys" version = "0.8.4" @@ -177,10 +298,41 @@ dependencies = [ ] [[package]] -name = "cxx" -version = "1.0.101" +name = "crypto-common" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5032837c1384de3708043de9d4e97bb91290faca6c16529a28aa340592a78166" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +dependencies = [ + "generic-array", + "typenum", +] + +[[package]] +name = "csv" +version = "1.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "626ae34994d3d8d668f4269922248239db4ae42d538b14c398b74a52208e8086" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "csv-core" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b2466559f260f48ad25fe6317b3c8dac77b5bdb5763ac7d9d6103530663bc90" +dependencies = [ + "memchr", +] + +[[package]] +name = "cxx" +version = "1.0.102" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f68e12e817cb19eaab81aaec582b4052d07debd3c3c6b083b9d361db47c7dc9d" dependencies = [ "cc", "cxxbridge-flags", @@ -190,9 +342,9 @@ dependencies = [ [[package]] name = "cxx-build" -version = "1.0.101" +version = "1.0.102" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "51368b3d0dbf356e10fcbfd455a038503a105ee556f7ee79b6bb8c53a7247456" +checksum = "e789217e4ab7cf8cc9ce82253180a9fe331f35f5d339f0ccfe0270b39433f397" dependencies = [ "cc", "codespan-reporting", @@ -200,24 +352,24 @@ dependencies = [ "proc-macro2", "quote", "scratch", - "syn 2.0.26", + "syn 2.0.27", ] [[package]] name = "cxxbridge-flags" -version = "1.0.101" +version = "1.0.102" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d9062157072e4aafc8e56ceaf8325ce850c5ae37578c852a0d4de2cecdded13" +checksum = "78a19f4c80fd9ab6c882286fa865e92e07688f4387370a209508014ead8751d0" [[package]] name = "cxxbridge-macro" -version = "1.0.101" +version = "1.0.102" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf01e8a540f5a4e0f284595834f81cf88572f244b768f051724537afa99a2545" +checksum = "b8fcfa71f66c8563c4fa9dd2bb68368d50267856f831ac5d85367e0805f9606c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.26", + "syn 2.0.27", ] [[package]] @@ -296,6 +448,17 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +dependencies = [ + "block-buffer", + "crypto-common", + "subtle", +] + [[package]] name = "dirs-next" version = "2.0.0" @@ -319,9 +482,27 @@ dependencies = [ [[package]] name = "either" -version = "1.8.1" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" +checksum = "a26ae43d7bcc3b814de94796a5e736d4029efb0ee900c12e2d54c993ad1a1e07" + +[[package]] +name = "enum-as-inner" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "equivalent" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "fnv" @@ -338,6 +519,16 @@ dependencies = [ "thread_local", ] +[[package]] +name = "generic-array" +version = "0.14.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +dependencies = [ + "typenum", + "version_check", +] + [[package]] name = "getrandom" version = "0.2.10" @@ -349,6 +540,33 @@ dependencies = [ "wasi 0.11.0+wasi-snapshot-preview1", ] +[[package]] +name = "gimli" +version = "0.27.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c80984affa11d98d1b88b66ac8853f143217b399d3c74116778ff8fdb4ed2e" + +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" +dependencies = [ + "ahash", +] + +[[package]] +name = "hashbrown" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" + +[[package]] +name = "heck" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" + [[package]] name = "hermit-abi" version = "0.3.2" @@ -384,6 +602,31 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" +[[package]] +name = "indexmap" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d" +dependencies = [ + "equivalent", + "hashbrown 0.14.0", +] + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" + [[package]] name = "js-sys" version = "0.3.64" @@ -444,6 +687,21 @@ dependencies = [ "autocfg", ] +[[package]] +name = "minimal-lexical" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" + +[[package]] +name = "miniz_oxide" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +dependencies = [ + "adler", +] + [[package]] name = "nix" version = "0.24.3" @@ -470,10 +728,20 @@ dependencies = [ ] [[package]] -name = "num-traits" -version = "0.2.15" +name = "nom" +version = "7.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "578ede34cf02f8924ab9447f50c28075b4d3e5b269972345e7e0372b38c6cdcd" +checksum = "d273983c5a657a70a3e8f2a01329822f3b8c8172b73826411a55751e404a0a4a" +dependencies = [ + "memchr", + "minimal-lexical", +] + +[[package]] +name = "num-traits" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f30b0abd723be7e2ffca1272140fac1a2f084c77ec3e123c192b66af1ee9e6c2" dependencies = [ "autocfg", ] @@ -488,6 +756,15 @@ dependencies = [ "libc", ] +[[package]] +name = "object" +version = "0.31.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bda667d9f2b5051b8833f59f3bf748b28ef54f850f4fcb389a252aa383866d1" +dependencies = [ + "memchr", +] + [[package]] name = "once_cell" version = "1.18.0" @@ -509,6 +786,41 @@ dependencies = [ "unicode-ident", ] +[[package]] +name = "prql-compiler" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c99b52154002ac7f286dd2293c2f8d4e30526c1d396b14deef5ada1deef3c9ff" +dependencies = [ + "anyhow", + "ariadne", + "chumsky", + "csv", + "enum-as-inner", + "itertools", + "lazy_static", + "log", + "once_cell", + "regex", + "semver", + "serde", + "serde_json", + "serde_yaml", + "sqlformat", + "sqlparser", + "strum", + "strum_macros", +] + +[[package]] +name = "psm" +version = "0.1.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5787f7cda34e3033a72192c018bc5883100330f362ef279a8cbccfce8bb4e874" +dependencies = [ + "cc", +] + [[package]] name = "quote" version = "1.0.31" @@ -589,12 +901,24 @@ version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5ea92a5b6195c6ef2a0295ea818b312502c6fc94dde986c5553242e18fd4ce2" +[[package]] +name = "rustc-demangle" +version = "0.1.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" + [[package]] name = "rustversion" version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" +[[package]] +name = "ryu" +version = "1.0.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ad4cc8da4ef723ed60bced201181d83791ad433213d8c24efffda1eec85d741" + [[package]] name = "scopeguard" version = "1.2.0" @@ -608,10 +932,57 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a3cf7c11c38cb994f3d40e8a8cde3bbd1f72a435e4c49e85d6553d8312306152" [[package]] -name = "serde" -version = "1.0.171" +name = "semver" +version = "1.0.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30e27d1e4fd7659406c492fd6cfaf2066ba8773de45ca75e855590f856dc34a9" +checksum = "b0293b4b29daaf487284529cc2f5675b8e57c61f70167ba415a463651fd6a918" +dependencies = [ + "serde", +] + +[[package]] +name = "serde" +version = "1.0.174" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b88756493a5bd5e5395d53baa70b194b05764ab85b59e43e4b8f4e1192fa9b1" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde_derive" +version = "1.0.174" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e5c3a298c7f978e53536f95a63bdc4c4a64550582f31a0359a9afda6aede62e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.27", +] + +[[package]] +name = "serde_json" +version = "1.0.103" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d03b412469450d4404fe8499a268edd7f8b79fecb074b0d812ad64ca21f4031b" +dependencies = [ + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "serde_yaml" +version = "0.9.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a49e178e4452f45cb61d0cd8cebc1b0fafd3e41929e996cef79aa3aca91f574" +dependencies = [ + "indexmap", + "itoa", + "ryu", + "serde", + "unsafe-libyaml", +] [[package]] name = "skim" @@ -638,12 +1009,74 @@ dependencies = [ "vte", ] +[[package]] +name = "sqlformat" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c12bc9199d1db8234678b7051747c07f517cdcf019262d1847b94ec8b1aee3e" +dependencies = [ + "itertools", + "nom", + "unicode_categories", +] + +[[package]] +name = "sqlparser" +version = "0.33.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "355dc4d4b6207ca8a3434fc587db0a8016130a574dbcdbfb93d7f7b5bc5b211a" +dependencies = [ + "log", + "serde", +] + +[[package]] +name = "stacker" +version = "0.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c886bd4480155fd3ef527d45e9ac8dd7118a898a46530b7b94c3e21866259fce" +dependencies = [ + "cc", + "cfg-if", + "libc", + "psm", + "winapi", +] + [[package]] name = "strsim" version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" +[[package]] +name = "strum" +version = "0.24.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" +dependencies = [ + "strum_macros", +] + +[[package]] +name = "strum_macros" +version = "0.24.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "rustversion", + "syn 1.0.109", +] + +[[package]] +name = "subtle" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" + [[package]] name = "syn" version = "1.0.109" @@ -657,9 +1090,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.26" +version = "2.0.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45c3457aacde3c65315de5031ec191ce46604304d2446e803d71ade03308d970" +checksum = "b60f673f44a8255b9c8c657daf66a596d435f2da81a555b06dc644d080ba45e0" dependencies = [ "proc-macro2", "quote", @@ -688,22 +1121,22 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.43" +version = "1.0.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a35fc5b8971143ca348fa6df4f024d4d55264f3468c71ad1c2f365b0a4d58c42" +checksum = "611040a08a0439f8248d1990b111c95baa9c704c805fa1f62104b39655fd7f90" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.43" +version = "1.0.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "463fe12d7993d3b327787537ce8dd4dfa058de32fc2b195ef3cde03dc4771e8f" +checksum = "090198534930841fab3a5d1bb637cde49e339654e606195f8d9c76eeb081dc96" dependencies = [ "proc-macro2", "quote", - "syn 2.0.26", + "syn 2.0.27", ] [[package]] @@ -766,6 +1199,12 @@ dependencies = [ "unicode-width", ] +[[package]] +name = "typenum" +version = "1.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "497961ef93d974e23eb6f433eb5fe1b7930b659f06d12dec6fc44a8f554c0bba" + [[package]] name = "unicode-ident" version = "1.0.11" @@ -778,12 +1217,30 @@ version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" +[[package]] +name = "unicode_categories" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39ec24b3121d976906ece63c9daad25b85969647682eee313cb5779fdd69e14e" + +[[package]] +name = "unsafe-libyaml" +version = "0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f28467d3e1d3c6586d8f25fa243f544f5800fec42d97032474e17222c2b75cfa" + [[package]] name = "utf8parse" version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" +[[package]] +name = "version_check" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" + [[package]] name = "vte" version = "0.11.1" @@ -838,7 +1295,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.26", + "syn 2.0.27", "wasm-bindgen-shared", ] @@ -860,7 +1317,7 @@ checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.26", + "syn 2.0.27", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -967,3 +1424,9 @@ name = "windows_x86_64_msvc" version = "0.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1a515f5799fe4961cb532f983ce2b23082366b898e52ffbce459c86f67c8378a" + +[[package]] +name = "yansi" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" diff --git a/rust/Cargo.toml b/rust/Cargo.toml new file mode 100644 index 00000000000..2a2b582cea8 --- /dev/null +++ b/rust/Cargo.toml @@ -0,0 +1,12 @@ +# workspace is required to vendor crates for all packages. +[workspace] +members = [ + "BLAKE3", + "skim", + "prql", +] +resolver = "2" + +# FIXME: even though the profiles should be defined in the main cargo config we +# cannot do this yet, since we compile each package separatelly, so you should +# ignore warning from cargo about this. diff --git a/rust/prql/Cargo.lock b/rust/prql/Cargo.lock deleted file mode 100644 index da94e4ca852..00000000000 --- a/rust/prql/Cargo.lock +++ /dev/null @@ -1,569 +0,0 @@ -# This file is automatically @generated by Cargo. -# It is not intended for manual editing. -version = 3 - -[[package]] -name = "_ch_rust_prql" -version = "0.1.0" -dependencies = [ - "prql-compiler", - "serde_json", -] - -[[package]] -name = "addr2line" -version = "0.20.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4fa78e18c64fce05e902adecd7a5eed15a5e0a3439f7b0e169f0252214865e3" -dependencies = [ - "gimli", -] - -[[package]] -name = "adler" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" - -[[package]] -name = "ahash" -version = "0.7.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" -dependencies = [ - "getrandom", - "once_cell", - "version_check", -] - -[[package]] -name = "aho-corasick" -version = "1.0.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43f6cb1bf222025340178f382c426f13757b2960e89779dfcb319c32542a5a41" -dependencies = [ - "memchr", -] - -[[package]] -name = "anyhow" -version = "1.0.71" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c7d0618f0e0b7e8ff11427422b64564d5fb0be1940354bfe2e0529b18a9d9b8" -dependencies = [ - "backtrace", -] - -[[package]] -name = "ariadne" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "367fd0ad87307588d087544707bc5fbf4805ded96c7db922b70d368fa1cb5702" -dependencies = [ - "unicode-width", - "yansi", -] - -[[package]] -name = "backtrace" -version = "0.3.68" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4319208da049c43661739c5fade2ba182f09d1dc2299b32298d3a31692b17e12" -dependencies = [ - "addr2line", - "cc", - "cfg-if", - "libc", - "miniz_oxide", - "object", - "rustc-demangle", -] - -[[package]] -name = "cc" -version = "1.0.79" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50d30906286121d95be3d479533b458f87493b30a4b5f79a607db8f5d11aa91f" - -[[package]] -name = "cfg-if" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" - -[[package]] -name = "chumsky" -version = "0.9.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23170228b96236b5a7299057ac284a321457700bc8c41a4476052f0f4ba5349d" -dependencies = [ - "hashbrown 0.12.3", - "stacker", -] - -[[package]] -name = "csv" -version = "1.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "626ae34994d3d8d668f4269922248239db4ae42d538b14c398b74a52208e8086" -dependencies = [ - "csv-core", - "itoa", - "ryu", - "serde", -] - -[[package]] -name = "csv-core" -version = "0.1.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b2466559f260f48ad25fe6317b3c8dac77b5bdb5763ac7d9d6103530663bc90" -dependencies = [ - "memchr", -] - -[[package]] -name = "either" -version = "1.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fcaabb2fef8c910e7f4c7ce9f67a1283a1715879a7c230ca9d6d1ae31f16d91" - -[[package]] -name = "enum-as-inner" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" -dependencies = [ - "heck", - "proc-macro2", - "quote", - "syn 1.0.109", -] - -[[package]] -name = "equivalent" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88bffebc5d80432c9b140ee17875ff173a8ab62faad5b257da912bd2f6c1c0a1" - -[[package]] -name = "getrandom" -version = "0.2.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" -dependencies = [ - "cfg-if", - "libc", - "wasi", -] - -[[package]] -name = "gimli" -version = "0.27.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6c80984affa11d98d1b88b66ac8853f143217b399d3c74116778ff8fdb4ed2e" - -[[package]] -name = "hashbrown" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" -dependencies = [ - "ahash", -] - -[[package]] -name = "hashbrown" -version = "0.14.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" - -[[package]] -name = "heck" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" - -[[package]] -name = "indexmap" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d" -dependencies = [ - "equivalent", - "hashbrown 0.14.0", -] - -[[package]] -name = "itertools" -version = "0.10.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" -dependencies = [ - "either", -] - -[[package]] -name = "itoa" -version = "1.0.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62b02a5381cc465bd3041d84623d0fa3b66738b52b8e2fc3bab8ad63ab032f4a" - -[[package]] -name = "lazy_static" -version = "1.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" - -[[package]] -name = "libc" -version = "0.2.147" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" - -[[package]] -name = "log" -version = "0.4.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b06a4cde4c0f271a446782e3eff8de789548ce57dbc8eca9292c27f4a42004b4" - -[[package]] -name = "memchr" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" - -[[package]] -name = "minimal-lexical" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" - -[[package]] -name = "miniz_oxide" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" -dependencies = [ - "adler", -] - -[[package]] -name = "nom" -version = "7.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d273983c5a657a70a3e8f2a01329822f3b8c8172b73826411a55751e404a0a4a" -dependencies = [ - "memchr", - "minimal-lexical", -] - -[[package]] -name = "object" -version = "0.31.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bda667d9f2b5051b8833f59f3bf748b28ef54f850f4fcb389a252aa383866d1" -dependencies = [ - "memchr", -] - -[[package]] -name = "once_cell" -version = "1.18.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" - -[[package]] -name = "proc-macro2" -version = "1.0.63" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b368fba921b0dce7e60f5e04ec15e565b3303972b42bcfde1d0713b881959eb" -dependencies = [ - "unicode-ident", -] - -[[package]] -name = "prql-compiler" -version = "0.8.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c99b52154002ac7f286dd2293c2f8d4e30526c1d396b14deef5ada1deef3c9ff" -dependencies = [ - "anyhow", - "ariadne", - "chumsky", - "csv", - "enum-as-inner", - "itertools", - "lazy_static", - "log", - "once_cell", - "regex", - "semver", - "serde", - "serde_json", - "serde_yaml", - "sqlformat", - "sqlparser", - "strum", - "strum_macros", -] - -[[package]] -name = "psm" -version = "0.1.21" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5787f7cda34e3033a72192c018bc5883100330f362ef279a8cbccfce8bb4e874" -dependencies = [ - "cc", -] - -[[package]] -name = "quote" -version = "1.0.29" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "573015e8ab27661678357f27dc26460738fd2b6c86e46f386fde94cb5d913105" -dependencies = [ - "proc-macro2", -] - -[[package]] -name = "regex" -version = "1.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89089e897c013b3deb627116ae56a6955a72b8bed395c9526af31c9fe528b484" -dependencies = [ - "aho-corasick", - "memchr", - "regex-automata", - "regex-syntax", -] - -[[package]] -name = "regex-automata" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa250384981ea14565685dea16a9ccc4d1c541a13f82b9c168572264d1df8c56" -dependencies = [ - "aho-corasick", - "memchr", - "regex-syntax", -] - -[[package]] -name = "regex-syntax" -version = "0.7.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ab07dc67230e4a4718e70fd5c20055a4334b121f1f9db8fe63ef39ce9b8c846" - -[[package]] -name = "rustc-demangle" -version = "0.1.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" - -[[package]] -name = "rustversion" -version = "1.0.13" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc31bd9b61a32c31f9650d18add92aa83a49ba979c143eefd27fe7177b05bd5f" - -[[package]] -name = "ryu" -version = "1.0.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe232bdf6be8c8de797b22184ee71118d63780ea42ac85b61d1baa6d3b782ae9" - -[[package]] -name = "semver" -version = "1.0.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bebd363326d05ec3e2f532ab7660680f3b02130d780c299bca73469d521bc0ed" -dependencies = [ - "serde", -] - -[[package]] -name = "serde" -version = "1.0.166" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d01b7404f9d441d3ad40e6a636a7782c377d2abdbe4fa2440e2edcc2f4f10db8" -dependencies = [ - "serde_derive", -] - -[[package]] -name = "serde_derive" -version = "1.0.166" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5dd83d6dde2b6b2d466e14d9d1acce8816dedee94f735eac6395808b3483c6d6" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.23", -] - -[[package]] -name = "serde_json" -version = "1.0.100" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f1e14e89be7aa4c4b78bdbdc9eb5bf8517829a600ae8eaa39a6e1d960b5185c" -dependencies = [ - "itoa", - "ryu", - "serde", -] - -[[package]] -name = "serde_yaml" -version = "0.9.22" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "452e67b9c20c37fa79df53201dc03839651086ed9bbe92b3ca585ca9fdaa7d85" -dependencies = [ - "indexmap", - "itoa", - "ryu", - "serde", - "unsafe-libyaml", -] - -[[package]] -name = "sqlformat" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c12bc9199d1db8234678b7051747c07f517cdcf019262d1847b94ec8b1aee3e" -dependencies = [ - "itertools", - "nom", - "unicode_categories", -] - -[[package]] -name = "sqlparser" -version = "0.33.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "355dc4d4b6207ca8a3434fc587db0a8016130a574dbcdbfb93d7f7b5bc5b211a" -dependencies = [ - "log", - "serde", -] - -[[package]] -name = "stacker" -version = "0.1.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c886bd4480155fd3ef527d45e9ac8dd7118a898a46530b7b94c3e21866259fce" -dependencies = [ - "cc", - "cfg-if", - "libc", - "psm", - "winapi", -] - -[[package]] -name = "strum" -version = "0.24.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" -dependencies = [ - "strum_macros", -] - -[[package]] -name = "strum_macros" -version = "0.24.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" -dependencies = [ - "heck", - "proc-macro2", - "quote", - "rustversion", - "syn 1.0.109", -] - -[[package]] -name = "syn" -version = "1.0.109" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" -dependencies = [ - "proc-macro2", - "quote", - "unicode-ident", -] - -[[package]] -name = "syn" -version = "2.0.23" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59fb7d6d8281a51045d62b8eb3a7d1ce347b76f312af50cd3dc0af39c87c1737" -dependencies = [ - "proc-macro2", - "quote", - "unicode-ident", -] - -[[package]] -name = "unicode-ident" -version = "1.0.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22049a19f4a68748a168c0fc439f9516686aa045927ff767eca0a85101fb6e73" - -[[package]] -name = "unicode-width" -version = "0.1.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" - -[[package]] -name = "unicode_categories" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39ec24b3121d976906ece63c9daad25b85969647682eee313cb5779fdd69e14e" - -[[package]] -name = "unsafe-libyaml" -version = "0.2.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1865806a559042e51ab5414598446a5871b561d21b6764f2eabb0dd481d880a6" - -[[package]] -name = "version_check" -version = "0.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" - -[[package]] -name = "wasi" -version = "0.11.0+wasi-snapshot-preview1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" - -[[package]] -name = "winapi" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" -dependencies = [ - "winapi-i686-pc-windows-gnu", - "winapi-x86_64-pc-windows-gnu", -] - -[[package]] -name = "winapi-i686-pc-windows-gnu" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" - -[[package]] -name = "winapi-x86_64-pc-windows-gnu" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" - -[[package]] -name = "yansi" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 16a58a90dcf..fff2975cea4 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -8,6 +8,7 @@ import shutil import subprocess import time import sys +from glob import glob from pathlib import Path from typing import Any, Dict, List, Optional, Set, Tuple, Union @@ -31,6 +32,17 @@ TEMP_PATH = os.path.join(RUNNER_TEMP, "docker_images_check") ImagesDict = Dict[str, dict] +# workaround for mypy issue [1]: +# +# "Argument 1 to "map" has incompatible type overloaded function" [1] +# +# [1]: https://github.com/python/mypy/issues/9864 +# +# NOTE: simply lambda will do the trick as well, but pylint will not like it +def realpath(*args, **kwargs): + return os.path.realpath(*args, **kwargs) + + class DockerImage: def __init__( self, @@ -111,8 +123,23 @@ def get_changed_docker_images( changed_images = [] for dockerfile_dir, image_description in images_dict.items(): + source_dir = GITHUB_WORKSPACE.rstrip("/") + "/" + dockerfile_files = glob(f"{source_dir}/{dockerfile_dir}/**", recursive=True) + # resolve symlinks + dockerfile_files = list(map(realpath, dockerfile_files)) + # trim prefix to get relative path again, to match with files_changed + dockerfile_files = list(map(lambda x: x[len(source_dir) :], dockerfile_files)) + logging.info( + "Docker %s (source_dir=%s) build context for PR %s @ %s: %s", + dockerfile_dir, + source_dir, + pr_info.number, + pr_info.sha, + str(dockerfile_files), + ) + for f in files_changed: - if f.startswith(dockerfile_dir): + if f in dockerfile_files: name = image_description["name"] only_amd64 = image_description.get("only_amd64", False) logging.info( @@ -245,6 +272,8 @@ def build_and_push_one_image( cache_from = f"{cache_from} --cache-from type=registry,ref={image.repo}:{tag}" cmd = ( + # tar is requried to follow symlinks, since docker-build cannot do this + f"tar -v --exclude-vcs-ignores --show-transformed-names --transform 's#{image.full_path.lstrip('/')}#./#' --dereference --create {image.full_path} | " "docker buildx build --builder default " f"--label build-url={GITHUB_RUN_URL} " f"{from_tag_arg}" @@ -254,7 +283,7 @@ def build_and_push_one_image( f"{cache_from} " f"--cache-to type=inline,mode=max " f"{push_arg}" - f"--progress plain {image.full_path}" + f"--progress plain -" ) logging.info("Docker command to run: %s", cmd) with TeePopen(cmd, build_log) as proc: diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index d5d27f73694..c679ab984ee 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -126,12 +126,13 @@ class TestDockerImageCheck(unittest.TestCase): mock_popen.assert_called_once() mock_machine.assert_not_called() self.assertIn( + "tar -v --exclude-vcs-ignores --show-transformed-names --transform 's#path#./#' --dereference --create path | " f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " "--build-arg FROM_TAG=version " f"--build-arg CACHE_INVALIDATOR={GITHUB_RUN_URL} " "--tag name:version --cache-from type=registry,ref=name:version " "--cache-from type=registry,ref=name:latest " - "--cache-to type=inline,mode=max --push --progress plain path", + "--cache-to type=inline,mode=max --push --progress plain -", mock_popen.call_args.args, ) self.assertTrue(result) @@ -143,12 +144,13 @@ class TestDockerImageCheck(unittest.TestCase): mock_popen.assert_called_once() mock_machine.assert_not_called() self.assertIn( + "tar -v --exclude-vcs-ignores --show-transformed-names --transform 's#path#./#' --dereference --create path | " f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " "--build-arg FROM_TAG=version2 " f"--build-arg CACHE_INVALIDATOR={GITHUB_RUN_URL} " "--tag name:version2 --cache-from type=registry,ref=name:version2 " "--cache-from type=registry,ref=name:latest " - "--cache-to type=inline,mode=max --progress plain path", + "--cache-to type=inline,mode=max --progress plain -", mock_popen.call_args.args, ) self.assertTrue(result) @@ -160,11 +162,12 @@ class TestDockerImageCheck(unittest.TestCase): mock_popen.assert_called_once() mock_machine.assert_not_called() self.assertIn( + "tar -v --exclude-vcs-ignores --show-transformed-names --transform 's#path#./#' --dereference --create path | " f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " f"--build-arg CACHE_INVALIDATOR={GITHUB_RUN_URL} " "--tag name:version2 --cache-from type=registry,ref=name:version2 " "--cache-from type=registry,ref=name:latest " - "--cache-to type=inline,mode=max --progress plain path", + "--cache-to type=inline,mode=max --progress plain -", mock_popen.call_args.args, ) self.assertFalse(result) @@ -178,13 +181,14 @@ class TestDockerImageCheck(unittest.TestCase): mock_popen.assert_called_once() mock_machine.assert_not_called() self.assertIn( + "tar -v --exclude-vcs-ignores --show-transformed-names --transform 's#path#./#' --dereference --create path | " f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " f"--build-arg CACHE_INVALIDATOR={GITHUB_RUN_URL} " "--tag name:version2 --cache-from type=registry,ref=name:version2 " "--cache-from type=registry,ref=name:latest " "--cache-from type=registry,ref=name:cached-version " "--cache-from type=registry,ref=name:another-cached " - "--cache-to type=inline,mode=max --progress plain path", + "--cache-to type=inline,mode=max --progress plain -", mock_popen.call_args.args, ) self.assertFalse(result) From dc7c66396223329021641372c9156261edce5f99 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sat, 22 Jul 2023 23:44:11 +0200 Subject: [PATCH 1035/2047] Update comment in DatabaseCatalog.cpp --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f9ed2c0d5ca..0d74e86a26d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -344,7 +344,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( DatabasePtr database; { std::lock_guard lock{databases_mutex}; - // hasDatabase() to avod getDatabaseName() throwing exception if database is empty. + // Callers assume that this method doesn't throw the exceptions, but getDatabaseName() can if there is no database part. auto it = table_id.hasDatabase() ? databases.find(table_id.getDatabaseName()) : databases.end(); if (databases.end() == it) { From 12065d94c5e35c51e3a94c1919f1a38f4723d272 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sat, 22 Jul 2023 23:59:02 +0200 Subject: [PATCH 1036/2047] Update comment DatabaseCatalog.cpp --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 0d74e86a26d..c8f332ae76d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -344,7 +344,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( DatabasePtr database; { std::lock_guard lock{databases_mutex}; - // Callers assume that this method doesn't throw the exceptions, but getDatabaseName() can if there is no database part. + // Callers assume that this method doesn't throw exceptions, but getDatabaseName() will throw if there is no database part. auto it = table_id.hasDatabase() ? databases.find(table_id.getDatabaseName()) : databases.end(); if (databases.end() == it) { From ef0dca626142322fa5420eea8fab491bb53c4ac2 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sun, 23 Jul 2023 00:37:34 +0200 Subject: [PATCH 1037/2047] fix style --- src/Common/OptimizedRegularExpression.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 918ebd75fc0..e636b0b987d 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -433,7 +433,7 @@ try for (auto & lit : alternative_literals) alternatives.push_back(std::move(lit.literal)); } -catch(...) +catch (...) { required_substring = ""; is_trivial = false; From 9bd8bdca98d21605f10d172b76c80951f990d965 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sun, 23 Jul 2023 01:14:26 +0200 Subject: [PATCH 1038/2047] Better error message in case of empty database name --- src/Interpreters/DatabaseCatalog.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index c8f332ae76d..f54b0e0ab3a 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -343,9 +343,17 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( DatabasePtr database; { - std::lock_guard lock{databases_mutex}; // Callers assume that this method doesn't throw exceptions, but getDatabaseName() will throw if there is no database part. - auto it = table_id.hasDatabase() ? databases.find(table_id.getDatabaseName()) : databases.end(); + // So, fail early and gracefully... + if (!table_id.hasDatabase()) + { + if (exception) + exception->emplace(Exception(ErrorCodes::UNKNOWN_DATABASE, "Empty database name")); + return {}; + } + + std::lock_guard lock{databases_mutex}; + auto it = databases.find(table_id.getDatabaseName()); if (databases.end() == it) { if (exception) From 2e61e92c865f1080eb4f246ee8ac0d66e6a68a5d Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 23 Jul 2023 04:07:18 +0000 Subject: [PATCH 1039/2047] update doc --- docs/en/operations/settings/settings.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 580b51a984d..4b6dc033f60 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4572,3 +4572,13 @@ Type: Int64 Default: 0 +## rewrite_count_distinct_if_with_count_distinct_implementation + +Allows you to rewrite `countDistcintIf` with [count_distinct_implementation](#settings-count_distinct_implementation) setting. + +Possible values: + +- true — Allow. +- false — Disallow. + +Default value: `false`. From 00d6f2ee08a3e442363a078b322adab7b6988f91 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 04:56:47 +0200 Subject: [PATCH 1040/2047] Use incbin for resources, part 1 --- .gitmodules | 3 + contrib/CMakeLists.txt | 2 +- contrib/incbin | 1 + contrib/incbin-cmake/CMakeLists.txt | 4 + contrib/nlp-data-cmake/CMakeLists.txt | 15 -- programs/install/CMakeLists.txt | 3 + programs/install/Install.cpp | 13 +- programs/keeper/CMakeLists.txt | 15 -- programs/keeper/Keeper.cpp | 6 +- programs/server/CMakeLists.txt | 12 +- programs/server/Server.cpp | 11 +- programs/server/resources.cpp | 0 src/CMakeLists.txt | 6 +- src/Common/CMakeLists.txt | 2 +- src/Common/Config/ConfigProcessor.cpp | 33 ++-- src/Common/Config/ConfigProcessor.h | 3 + src/Common/FrequencyHolder.cpp | 181 ++++++++++++++++++ src/Common/FrequencyHolder.h | 170 +--------------- src/Daemon/BaseDaemon.cpp | 1 - src/Server/WebUIRequestHandler.cpp | 14 +- src/Storages/System/CMakeLists.txt | 12 +- .../System/attachInformationSchemaTables.cpp | 24 ++- 22 files changed, 268 insertions(+), 263 deletions(-) create mode 160000 contrib/incbin create mode 100644 contrib/incbin-cmake/CMakeLists.txt delete mode 100644 contrib/nlp-data-cmake/CMakeLists.txt create mode 100644 programs/server/resources.cpp create mode 100644 src/Common/FrequencyHolder.cpp diff --git a/.gitmodules b/.gitmodules index ba71a8ae3a7..30085fb8dd4 100644 --- a/.gitmodules +++ b/.gitmodules @@ -340,3 +340,6 @@ [submodule "contrib/c-ares"] path = contrib/c-ares url = https://github.com/c-ares/c-ares.git +[submodule "contrib/incbin"] + path = contrib/incbin + url = https://github.com/graphitemaster/incbin.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 0f68c0cbc7c..fdf6e60e58f 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -164,13 +164,13 @@ add_contrib (libpq-cmake libpq) add_contrib (nuraft-cmake NuRaft) add_contrib (fast_float-cmake fast_float) add_contrib (datasketches-cpp-cmake datasketches-cpp) +add_contrib (incbin-cmake incbin) option(ENABLE_NLP "Enable NLP functions support" ${ENABLE_LIBRARIES}) if (ENABLE_NLP) add_contrib (libstemmer-c-cmake libstemmer_c) add_contrib (wordnet-blast-cmake wordnet-blast) add_contrib (lemmagen-c-cmake lemmagen-c) - add_contrib (nlp-data-cmake nlp-data) add_contrib (cld2-cmake cld2) endif() diff --git a/contrib/incbin b/contrib/incbin new file mode 160000 index 00000000000..6e576cae5ab --- /dev/null +++ b/contrib/incbin @@ -0,0 +1 @@ +Subproject commit 6e576cae5ab5810f25e2631f2e0b80cbe7dc8cbf diff --git a/contrib/incbin-cmake/CMakeLists.txt b/contrib/incbin-cmake/CMakeLists.txt new file mode 100644 index 00000000000..e64ebc99c73 --- /dev/null +++ b/contrib/incbin-cmake/CMakeLists.txt @@ -0,0 +1,4 @@ +set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/incbin") +add_library(_incbin INTERFACE) +target_include_directories(_incbin SYSTEM INTERFACE ${LIBRARY_DIR}) +add_library(ch_contrib::incbin ALIAS _incbin) diff --git a/contrib/nlp-data-cmake/CMakeLists.txt b/contrib/nlp-data-cmake/CMakeLists.txt deleted file mode 100644 index 5380269c479..00000000000 --- a/contrib/nlp-data-cmake/CMakeLists.txt +++ /dev/null @@ -1,15 +0,0 @@ -include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake) - -set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/nlp-data") - -add_library (_nlp_data INTERFACE) - -clickhouse_embed_binaries( - TARGET nlp_dictionaries - RESOURCE_DIR "${LIBRARY_DIR}" - RESOURCES charset.zst tonality_ru.zst programming.zst -) - -add_dependencies(_nlp_data nlp_dictionaries) -target_link_libraries(_nlp_data INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") -add_library(ch_contrib::nlp_data ALIAS _nlp_data) diff --git a/programs/install/CMakeLists.txt b/programs/install/CMakeLists.txt index c3f4d96d631..f3f562bab7c 100644 --- a/programs/install/CMakeLists.txt +++ b/programs/install/CMakeLists.txt @@ -10,3 +10,6 @@ set (CLICKHOUSE_INSTALL_LINK ) clickhouse_program_add_library(install) + +# For incbin +target_include_directories(clickhouse-install-lib PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/../server") diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index d83e189f7ef..da2c95af62c 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -20,10 +20,7 @@ #include #include #include -#include -#include #include -#include #include #include #include @@ -35,6 +32,12 @@ #include +#include + +/// Embedded configuration files used inside the install program +INCBIN(resource_config_xml, "config.xml"); +INCBIN(resource_users_xml, "users.xml"); + /** This tool can be used to install ClickHouse without a deb/rpm/tgz package, having only "clickhouse" binary. * It also allows to avoid dependency on systemd, upstart, SysV init. @@ -560,7 +563,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (!fs::exists(main_config_file)) { - std::string_view main_config_content = getResource("config.xml"); + std::string_view main_config_content(reinterpret_cast(gresource_config_xmlData), gresource_config_xmlSize); if (main_config_content.empty()) { fmt::print("There is no default config.xml, you have to download it and place to {}.\n", main_config_file.string()); @@ -672,7 +675,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (!fs::exists(users_config_file)) { - std::string_view users_config_content = getResource("users.xml"); + std::string_view users_config_content(reinterpret_cast(gresource_users_xmlData), gresource_users_xmlSize); if (users_config_content.empty()) { fmt::print("There is no default users.xml, you have to download it and place to {}.\n", users_config_file.string()); diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 940e6848597..317e35959aa 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -1,16 +1,3 @@ -include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake) - -if (OS_LINUX) - set (LINK_RESOURCE_LIB INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") - # for some reason INTERFACE linkage doesn't work for standalone binary - set (LINK_RESOURCE_LIB_STANDALONE_KEEPER "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") -endif () - -clickhouse_embed_binaries( - TARGET clickhouse_keeper_configs - RESOURCES keeper_config.xml keeper_embedded.xml -) - set(CLICKHOUSE_KEEPER_SOURCES Keeper.cpp ) @@ -29,7 +16,6 @@ set (CLICKHOUSE_KEEPER_LINK clickhouse_program_add(keeper) install(FILES keeper_config.xml DESTINATION "${CLICKHOUSE_ETC_DIR}/clickhouse-keeper" COMPONENT clickhouse-keeper) -add_dependencies(clickhouse-keeper-lib clickhouse_keeper_configs) if (BUILD_STANDALONE_KEEPER) # Straight list of all required sources @@ -215,7 +201,6 @@ if (BUILD_STANDALONE_KEEPER) ${LINK_RESOURCE_LIB_STANDALONE_KEEPER} ) - add_dependencies(clickhouse-keeper clickhouse_keeper_configs) set_target_properties(clickhouse-keeper PROPERTIES RUNTIME_OUTPUT_DIRECTORY ../) if (SPLIT_DEBUG_SYMBOLS) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 6034d63a016..a38467c3369 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -457,8 +457,10 @@ try const std::string key_path = config().getString("openSSL.server.privateKeyFile", ""); std::vector extra_paths = {include_from_path}; - if (!cert_path.empty()) extra_paths.emplace_back(cert_path); - if (!key_path.empty()) extra_paths.emplace_back(key_path); + if (!cert_path.empty()) + extra_paths.emplace_back(cert_path); + if (!key_path.empty()) + extra_paths.emplace_back(key_path); /// ConfigReloader have to strict parameters which are redundant in our case auto main_config_reloader = std::make_unique( diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 855973d10e1..e008e65acf6 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -1,12 +1,8 @@ -include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake) - set(CLICKHOUSE_SERVER_SOURCES MetricsTransmitter.cpp Server.cpp ) -set (LINK_RESOURCE_LIB INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") - set (CLICKHOUSE_SERVER_LINK PRIVATE clickhouse_aggregate_functions @@ -33,10 +29,4 @@ endif() clickhouse_program_add(server) -install(FILES config.xml users.xml DESTINATION "${CLICKHOUSE_ETC_DIR}/clickhouse-server" COMPONENT clickhouse) - -clickhouse_embed_binaries( - TARGET clickhouse_server_configs - RESOURCES config.xml users.xml embedded.xml play.html dashboard.html js/uplot.js -) -add_dependencies(clickhouse-server-lib clickhouse_server_configs) +target_include_directories(clickhouse-server-lib PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 33fdcc9c1a8..229a169dc1e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -128,6 +128,10 @@ # include #endif +#include +/// A minimal file used when the server is run without installation +INCBIN(resource_embedded_xml, "embedded.xml"); + namespace CurrentMetrics { extern const Metric Revision; @@ -393,6 +397,7 @@ int Server::run() void Server::initialize(Poco::Util::Application & self) { + ConfigProcessor::registerEmbeddedConfig("config.xml", std::string_view(reinterpret_cast(gresource_embedded_xmlData), gresource_embedded_xmlSize)); BaseDaemon::initialize(self); logger().information("starting up"); @@ -1105,8 +1110,10 @@ try const std::string key_path = config().getString("openSSL.server.privateKeyFile", ""); std::vector extra_paths = {include_from_path}; - if (!cert_path.empty()) extra_paths.emplace_back(cert_path); - if (!key_path.empty()) extra_paths.emplace_back(key_path); + if (!cert_path.empty()) + extra_paths.emplace_back(cert_path); + if (!key_path.empty()) + extra_paths.emplace_back(key_path); auto main_config_reloader = std::make_unique( config_path, diff --git a/programs/server/resources.cpp b/programs/server/resources.cpp new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index f870993f080..fda8bafde59 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -210,7 +210,7 @@ if (TARGET ch_contrib::jemalloc) target_link_libraries (clickhouse_storages_system PRIVATE ch_contrib::jemalloc) endif() -target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::sparsehash) +target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::sparsehash ch_contrib::incbin) add_subdirectory(Access/Common) add_subdirectory(Common/ZooKeeper) @@ -296,7 +296,7 @@ macro (dbms_target_include_directories) endforeach () endmacro () -dbms_target_include_directories (PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") +dbms_target_include_directories (PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src" "${ClickHouse_SOURCE_DIR}/programs/server") target_include_directories (clickhouse_common_io PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") if (TARGET ch_contrib::llvm) @@ -561,7 +561,7 @@ if (ENABLE_NLP) dbms_target_link_libraries (PUBLIC ch_contrib::stemmer) dbms_target_link_libraries (PUBLIC ch_contrib::wnb) dbms_target_link_libraries (PUBLIC ch_contrib::lemmagen) - dbms_target_link_libraries (PUBLIC ch_contrib::nlp_data) + target_include_directories(clickhouse_common_io PUBLIC ${CMAKE_SOURCE_DIR}/contrib/nlp-data) endif() if (TARGET ch_contrib::ulid) diff --git a/src/Common/CMakeLists.txt b/src/Common/CMakeLists.txt index e527b3dec43..b83c8431f0a 100644 --- a/src/Common/CMakeLists.txt +++ b/src/Common/CMakeLists.txt @@ -9,5 +9,5 @@ if (ENABLE_EXAMPLES) endif() if (ENABLE_MYSQL) - add_subdirectory (mysqlxx) + add_subdirectory(mysqlxx) endif () diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 5bbc8eae0de..c3a8f69cf3f 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -83,6 +83,13 @@ ConfigProcessor::~ConfigProcessor() Poco::Logger::destroy("ConfigProcessor"); } +static std::unordered_map embedded_configs; + +void ConfigProcessor::registerEmbeddedConfig(std::string name, std::string_view content) +{ + embedded_configs[name] = content; +} + /// Vector containing the name of the element and a sorted list of attribute names and values /// (except "remove" and "replace" attributes). @@ -281,15 +288,15 @@ void ConfigProcessor::doIncludesRecursive( { std::string value = node->nodeValue(); - bool replace_occured = false; + bool replace_occurred = false; size_t pos; while ((pos = value.find(substitution.first)) != std::string::npos) { value.replace(pos, substitution.first.length(), substitution.second); - replace_occured = true; + replace_occurred = true; } - if (replace_occured) + if (replace_occurred) node->setNodeValue(value); } } @@ -528,26 +535,14 @@ XMLDocumentPtr ConfigProcessor::processConfig( } else { - /// These embedded files added during build with some cmake magic. - /// Look at the end of programs/server/CMakeLists.txt. - std::string embedded_name; - if (path == "config.xml") - embedded_name = "embedded.xml"; - - if (path == "keeper_config.xml") - embedded_name = "keeper_embedded.xml"; - - /// When we can use config embedded in binary. - if (!embedded_name.empty()) + /// When we can use a config embedded in the binary. + if (auto it = embedded_configs.find(path); it != embedded_configs.end()) { - auto resource = getResource(embedded_name); - if (resource.empty()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Configuration file {} doesn't exist and there is no embedded config", path); LOG_DEBUG(log, "There is no file '{}', will use embedded config.", path); - config = dom_parser.parseMemory(resource.data(), resource.size()); + config = dom_parser.parseMemory(it->second.data(), it->second.size()); } else - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Configuration file {} doesn't exist", path); + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Configuration file {} doesn't exist and there is no embedded config", path); } std::vector contributing_files; diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 0ca3e46db88..eefe65ef06c 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -65,6 +65,9 @@ public: zkutil::ZooKeeperNodeCache * zk_node_cache = nullptr, const zkutil::EventPtr & zk_changed_event = nullptr); + /// These configurations will be used if there is no configuration file. + static void registerEmbeddedConfig(std::string name, std::string_view content); + /// loadConfig* functions apply processConfig and create Poco::Util::XMLConfiguration. /// The resulting XML document is saved into a file with the name diff --git a/src/Common/FrequencyHolder.cpp b/src/Common/FrequencyHolder.cpp new file mode 100644 index 00000000000..3b755cacacb --- /dev/null +++ b/src/Common/FrequencyHolder.cpp @@ -0,0 +1,181 @@ +#include + +#include + +/// Embedded SQL definitions +INCBIN(resource_charset_zst, "charset.zst"); +INCBIN(resource_tonality_ru_zst, "tonality_ru.zst"); +INCBIN(resource_programming_zst, "programming.zst"); + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FILE_DOESNT_EXIST; +} + + +FrequencyHolder & FrequencyHolder::getInstance() +{ + static FrequencyHolder instance; + return instance; +} + +FrequencyHolder::FrequencyHolder() +{ + loadEmotionalDict(); + loadEncodingsFrequency(); + loadProgrammingFrequency(); +} + +void FrequencyHolder::loadEncodingsFrequency() +{ + Poco::Logger * log = &Poco::Logger::get("EncodingsFrequency"); + + LOG_TRACE(log, "Loading embedded charset frequencies"); + + std::string_view resource(reinterpret_cast(gresource_charset_zstData), gresource_charset_zstSize); + if (resource.empty()) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded charset frequencies"); + + String line; + UInt16 bigram; + Float64 frequency; + String charset_name; + + auto buf = std::make_unique(resource.data(), resource.size()); + ZstdInflatingReadBuffer in(std::move(buf)); + + while (!in.eof()) + { + readString(line, in); + in.ignore(); + + if (line.empty()) + continue; + + ReadBufferFromString buf_line(line); + + // Start loading a new charset + if (line.starts_with("// ")) + { + // Skip "// " + buf_line.ignore(3); + readString(charset_name, buf_line); + + /* In our dictionary we have lines with form: _ + * If we need to find language of data, we return + * If we need to find charset of data, we return . + */ + size_t sep = charset_name.find('_'); + + Encoding enc; + enc.lang = charset_name.substr(0, sep); + enc.name = charset_name.substr(sep + 1); + encodings_freq.push_back(std::move(enc)); + } + else + { + readIntText(bigram, buf_line); + buf_line.ignore(); + readFloatText(frequency, buf_line); + + encodings_freq.back().map[bigram] = frequency; + } + } + LOG_TRACE(log, "Charset frequencies was added, charsets count: {}", encodings_freq.size()); +} + +void FrequencyHolder::loadEmotionalDict() +{ + Poco::Logger * log = &Poco::Logger::get("EmotionalDict"); + LOG_TRACE(log, "Loading embedded emotional dictionary"); + + std::string_view resource(reinterpret_cast(gresource_tonality_ru_zstData), gresource_tonality_ru_zstSize); + if (resource.empty()) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded emotional dictionary"); + + String line; + String word; + Float64 tonality; + size_t count = 0; + + auto buf = std::make_unique(resource.data(), resource.size()); + ZstdInflatingReadBuffer in(std::move(buf)); + + while (!in.eof()) + { + readString(line, in); + in.ignore(); + + if (line.empty()) + continue; + + ReadBufferFromString buf_line(line); + + readStringUntilWhitespace(word, buf_line); + buf_line.ignore(); + readFloatText(tonality, buf_line); + + StringRef ref{string_pool.insert(word.data(), word.size()), word.size()}; + emotional_dict[ref] = tonality; + ++count; + } + LOG_TRACE(log, "Emotional dictionary was added. Word count: {}", std::to_string(count)); +} + +void FrequencyHolder::loadProgrammingFrequency() +{ + Poco::Logger * log = &Poco::Logger::get("ProgrammingFrequency"); + + LOG_TRACE(log, "Loading embedded programming languages frequencies loading"); + + std::string_view resource(reinterpret_cast(gresource_programming_zstData), gresource_programming_zstSize); + if (resource.empty()) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded programming languages frequencies"); + + String line; + String bigram; + Float64 frequency; + String programming_language; + + auto buf = std::make_unique(resource.data(), resource.size()); + ZstdInflatingReadBuffer in(std::move(buf)); + + while (!in.eof()) + { + readString(line, in); + in.ignore(); + + if (line.empty()) + continue; + + ReadBufferFromString buf_line(line); + + // Start loading a new language + if (line.starts_with("// ")) + { + // Skip "// " + buf_line.ignore(3); + readString(programming_language, buf_line); + + Language lang; + lang.name = programming_language; + programming_freq.push_back(std::move(lang)); + } + else + { + readStringUntilWhitespace(bigram, buf_line); + buf_line.ignore(); + readFloatText(frequency, buf_line); + + StringRef ref{string_pool.insert(bigram.data(), bigram.size()), bigram.size()}; + programming_freq.back().map[ref] = frequency; + } + } + LOG_TRACE(log, "Programming languages frequencies was added"); +} + +} diff --git a/src/Common/FrequencyHolder.h b/src/Common/FrequencyHolder.h index 74098598441..270e4dbbd2a 100644 --- a/src/Common/FrequencyHolder.h +++ b/src/Common/FrequencyHolder.h @@ -7,7 +7,6 @@ #include #include -#include #include #include #include @@ -20,11 +19,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int FILE_DOESNT_EXIST; -} - /// FrequencyHolder class is responsible for storing and loading dictionaries /// needed for text classification functions: /// @@ -56,11 +50,7 @@ public: using EncodingMap = HashMap; using EncodingContainer = std::vector; - static FrequencyHolder & getInstance() - { - static FrequencyHolder instance; - return instance; - } + static FrequencyHolder & getInstance(); const Map & getEmotionalDict() const { @@ -78,161 +68,11 @@ public: } private: + FrequencyHolder(); - FrequencyHolder() - { - loadEmotionalDict(); - loadEncodingsFrequency(); - loadProgrammingFrequency(); - } - - void loadEncodingsFrequency() - { - Poco::Logger * log = &Poco::Logger::get("EncodingsFrequency"); - - LOG_TRACE(log, "Loading embedded charset frequencies"); - - auto resource = getResource("charset.zst"); - if (resource.empty()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded charset frequencies"); - - String line; - UInt16 bigram; - Float64 frequency; - String charset_name; - - auto buf = std::make_unique(resource.data(), resource.size()); - ZstdInflatingReadBuffer in(std::move(buf)); - - while (!in.eof()) - { - readString(line, in); - in.ignore(); - - if (line.empty()) - continue; - - ReadBufferFromString buf_line(line); - - // Start loading a new charset - if (line.starts_with("// ")) - { - // Skip "// " - buf_line.ignore(3); - readString(charset_name, buf_line); - - /* In our dictionary we have lines with form: _ - * If we need to find language of data, we return - * If we need to find charset of data, we return . - */ - size_t sep = charset_name.find('_'); - - Encoding enc; - enc.lang = charset_name.substr(0, sep); - enc.name = charset_name.substr(sep + 1); - encodings_freq.push_back(std::move(enc)); - } - else - { - readIntText(bigram, buf_line); - buf_line.ignore(); - readFloatText(frequency, buf_line); - - encodings_freq.back().map[bigram] = frequency; - } - } - LOG_TRACE(log, "Charset frequencies was added, charsets count: {}", encodings_freq.size()); - } - - void loadEmotionalDict() - { - Poco::Logger * log = &Poco::Logger::get("EmotionalDict"); - LOG_TRACE(log, "Loading embedded emotional dictionary"); - - auto resource = getResource("tonality_ru.zst"); - if (resource.empty()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded emotional dictionary"); - - String line; - String word; - Float64 tonality; - size_t count = 0; - - auto buf = std::make_unique(resource.data(), resource.size()); - ZstdInflatingReadBuffer in(std::move(buf)); - - while (!in.eof()) - { - readString(line, in); - in.ignore(); - - if (line.empty()) - continue; - - ReadBufferFromString buf_line(line); - - readStringUntilWhitespace(word, buf_line); - buf_line.ignore(); - readFloatText(tonality, buf_line); - - StringRef ref{string_pool.insert(word.data(), word.size()), word.size()}; - emotional_dict[ref] = tonality; - ++count; - } - LOG_TRACE(log, "Emotional dictionary was added. Word count: {}", std::to_string(count)); - } - - void loadProgrammingFrequency() - { - Poco::Logger * log = &Poco::Logger::get("ProgrammingFrequency"); - - LOG_TRACE(log, "Loading embedded programming languages frequencies loading"); - - auto resource = getResource("programming.zst"); - if (resource.empty()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "There is no embedded programming languages frequencies"); - - String line; - String bigram; - Float64 frequency; - String programming_language; - - auto buf = std::make_unique(resource.data(), resource.size()); - ZstdInflatingReadBuffer in(std::move(buf)); - - while (!in.eof()) - { - readString(line, in); - in.ignore(); - - if (line.empty()) - continue; - - ReadBufferFromString buf_line(line); - - // Start loading a new language - if (line.starts_with("// ")) - { - // Skip "// " - buf_line.ignore(3); - readString(programming_language, buf_line); - - Language lang; - lang.name = programming_language; - programming_freq.push_back(std::move(lang)); - } - else - { - readStringUntilWhitespace(bigram, buf_line); - buf_line.ignore(); - readFloatText(frequency, buf_line); - - StringRef ref{string_pool.insert(bigram.data(), bigram.size()), bigram.size()}; - programming_freq.back().map[ref] = frequency; - } - } - LOG_TRACE(log, "Programming languages frequencies was added"); - } + void loadEncodingsFrequency(); + void loadEmotionalDict(); + void loadProgrammingFrequency(); Arena string_pool; diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 3852ec5ada5..f61ca054b2a 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -38,7 +38,6 @@ #include #include -#include #include #include #include diff --git a/src/Server/WebUIRequestHandler.cpp b/src/Server/WebUIRequestHandler.cpp index 3997e0f19b6..cb9e8935d8c 100644 --- a/src/Server/WebUIRequestHandler.cpp +++ b/src/Server/WebUIRequestHandler.cpp @@ -6,10 +6,16 @@ #include #include -#include #include +#include + +/// Embedded HTML pages +INCBIN(resource_play_html, "play.html"); +INCBIN(resource_dashboard_html, "dashboard.html"); +INCBIN(resource_uplot_js, "js/uplot.js"); + namespace DB { @@ -34,13 +40,13 @@ void WebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerR if (request.getURI().starts_with("/play")) { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_OK); - *response.send() << getResource("play.html"); + *response.send() << std::string_view(reinterpret_cast(gresource_play_htmlData), gresource_play_htmlSize); } else if (request.getURI().starts_with("/dashboard")) { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_OK); - std::string html(getResource("dashboard.html")); + std::string html(reinterpret_cast(gresource_dashboard_htmlData), gresource_dashboard_htmlSize); /// Replace a link to external JavaScript file to embedded file. /// This allows to open the HTML without running a server and to host it on server. @@ -55,7 +61,7 @@ void WebUIRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerR else if (request.getURI() == "/js/uplot.js") { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_OK); - *response.send() << getResource("js/uplot.js"); + *response.send() << std::string_view(reinterpret_cast(gresource_uplot_jsData), gresource_uplot_jsSize); } else { diff --git a/src/Storages/System/CMakeLists.txt b/src/Storages/System/CMakeLists.txt index 1d2a3de5101..6b7d1739e33 100644 --- a/src/Storages/System/CMakeLists.txt +++ b/src/Storages/System/CMakeLists.txt @@ -43,18 +43,9 @@ list (APPEND storages_system_sources ${GENERATED_TIMEZONES_SRC}) # Overlength strings set_source_files_properties(${GENERATED_LICENSES_SRC} PROPERTIES COMPILE_FLAGS -w) -include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake) -clickhouse_embed_binaries( - TARGET information_schema_metadata - RESOURCE_DIR "${CMAKE_CURRENT_SOURCE_DIR}/InformationSchema/" - RESOURCES schemata.sql tables.sql views.sql columns.sql -) - list (SORT storages_system_sources) # Reproducible build add_library(clickhouse_storages_system ${storages_system_sources}) -add_dependencies(clickhouse_storages_system information_schema_metadata) - target_link_libraries(clickhouse_storages_system PRIVATE dbms common @@ -62,5 +53,6 @@ target_link_libraries(clickhouse_storages_system PRIVATE clickhouse_common_zookeeper clickhouse_parsers Poco::JSON - INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}" ) + +target_include_directories(clickhouse_storages_system PRIVATE InformationSchema) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 61a91685324..bfc5c8c64e2 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -3,14 +3,21 @@ #include #include #include -#include +#include + +/// Embedded SQL definitions +INCBIN(resource_schemata_sql, "schemata.sql"); +INCBIN(resource_tables_sql, "tables.sql"); +INCBIN(resource_views_sql, "views.sql"); +INCBIN(resource_columns_sql, "columns.sql"); + namespace DB { /// View structures are taken from http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt -static void createInformationSchemaView(ContextMutablePtr context, IDatabase & database, const String & view_name) +static void createInformationSchemaView(ContextMutablePtr context, IDatabase & database, const String & view_name, std::string_view query) { try { @@ -21,12 +28,11 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d bool is_uppercase = database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE; String metadata_resource_name = view_name + ".sql"; - auto attach_query = getResource(metadata_resource_name); - if (attach_query.empty()) + if (query.empty()) return; ParserCreateQuery parser; - ASTPtr ast = parseQuery(parser, attach_query.data(), attach_query.data() + attach_query.size(), + ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "Attach query from embedded resource " + metadata_resource_name, DBMS_DEFAULT_MAX_QUERY_SIZE, DBMS_DEFAULT_MAX_PARSER_DEPTH); @@ -50,10 +56,10 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d void attachInformationSchema(ContextMutablePtr context, IDatabase & information_schema_database) { - createInformationSchemaView(context, information_schema_database, "schemata"); - createInformationSchemaView(context, information_schema_database, "tables"); - createInformationSchemaView(context, information_schema_database, "views"); - createInformationSchemaView(context, information_schema_database, "columns"); + createInformationSchemaView(context, information_schema_database, "schemata", std::string_view(reinterpret_cast(gresource_schemata_sqlData), gresource_schemata_sqlSize)); + createInformationSchemaView(context, information_schema_database, "tables", std::string_view(reinterpret_cast(gresource_tables_sqlData), gresource_tables_sqlSize)); + createInformationSchemaView(context, information_schema_database, "views", std::string_view(reinterpret_cast(gresource_views_sqlData), gresource_views_sqlSize)); + createInformationSchemaView(context, information_schema_database, "columns", std::string_view(reinterpret_cast(gresource_columns_sqlData), gresource_columns_sqlSize)); } } From 4170d1458bdbccafe2f8cb2c671ee044b3efe9ba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 05:25:14 +0200 Subject: [PATCH 1041/2047] Use incbin for resources, part 2 --- cmake/embed_binary.cmake | 58 ------------------ contrib/cctz-cmake/CMakeLists.txt | 45 ++++++++------ src/Common/Config/ConfigProcessor.cpp | 1 - src/Common/DateLUTImpl.cpp | 17 ++++-- src/Common/SymbolIndex.cpp | 61 +++---------------- src/Common/SymbolIndex.h | 30 +-------- src/Common/getResource.cpp | 52 ---------------- src/Common/getResource.h | 7 --- src/Common/tests/gtest_DateLUTImpl.cpp | 14 ++--- .../System/StorageSystemTimeZones.cpp | 7 ++- 10 files changed, 58 insertions(+), 234 deletions(-) delete mode 100644 cmake/embed_binary.cmake delete mode 100644 src/Common/getResource.cpp delete mode 100644 src/Common/getResource.h diff --git a/cmake/embed_binary.cmake b/cmake/embed_binary.cmake deleted file mode 100644 index e5428c24939..00000000000 --- a/cmake/embed_binary.cmake +++ /dev/null @@ -1,58 +0,0 @@ -# Embed a set of resource files into a resulting object file. -# -# Signature: `clickhouse_embed_binaries(TARGET RESOURCE_DIR RESOURCES ...) -# -# This will generate a static library target named ``, which contains the contents of -# each `` file. The files should be located in ``. defaults to -# ${CMAKE_CURRENT_SOURCE_DIR}, and the resources may not be empty. -# -# Each resource will result in three symbols in the final archive, based on the name ``. -# These are: -# 1. `_binary__start`: Points to the start of the binary data from ``. -# 2. `_binary__end`: Points to the end of the binary data from ``. -# 2. `_binary__size`: Points to the size of the binary data from ``. -# -# `` is a normalized name derived from ``, by replacing the characters "./-" with -# the character "_", and the character "+" with "_PLUS_". This scheme is similar to those generated -# by `ld -r -b binary`, and matches the expectations in `./base/common/getResource.cpp`. -macro(clickhouse_embed_binaries) - set(one_value_args TARGET RESOURCE_DIR) - set(resources RESOURCES) - cmake_parse_arguments(EMBED "" "${one_value_args}" ${resources} ${ARGN}) - - if (NOT DEFINED EMBED_TARGET) - message(FATAL_ERROR "A target name must be provided for embedding binary resources into") - endif() - - if (NOT DEFINED EMBED_RESOURCE_DIR) - set(EMBED_RESOURCE_DIR "${CMAKE_CURRENT_SOURCE_DIR}") - endif() - - list(LENGTH EMBED_RESOURCES N_RESOURCES) - if (N_RESOURCES LESS 1) - message(FATAL_ERROR "The list of binary resources to embed may not be empty") - endif() - - add_library("${EMBED_TARGET}" STATIC) - set_target_properties("${EMBED_TARGET}" PROPERTIES LINKER_LANGUAGE C) - - set(EMBED_TEMPLATE_FILE "${PROJECT_SOURCE_DIR}/programs/embed_binary.S.in") - - foreach(RESOURCE_FILE ${EMBED_RESOURCES}) - set(ASSEMBLY_FILE_NAME "${RESOURCE_FILE}.S") - set(BINARY_FILE_NAME "${RESOURCE_FILE}") - - # Normalize the name of the resource. - string(REGEX REPLACE "[\./-]" "_" SYMBOL_NAME "${RESOURCE_FILE}") # - must be last in regex - string(REPLACE "+" "_PLUS_" SYMBOL_NAME "${SYMBOL_NAME}") - - # Generate the configured assembly file in the output directory. - configure_file("${EMBED_TEMPLATE_FILE}" "${CMAKE_CURRENT_BINARY_DIR}/${ASSEMBLY_FILE_NAME}" @ONLY) - - # Set the include directory for relative paths specified for `.incbin` directive. - set_property(SOURCE "${CMAKE_CURRENT_BINARY_DIR}/${ASSEMBLY_FILE_NAME}" APPEND PROPERTY INCLUDE_DIRECTORIES "${EMBED_RESOURCE_DIR}") - - target_sources("${EMBED_TARGET}" PRIVATE "${CMAKE_CURRENT_BINARY_DIR}/${ASSEMBLY_FILE_NAME}") - set_target_properties("${EMBED_TARGET}" PROPERTIES OBJECT_DEPENDS "${RESOURCE_FILE}") - endforeach() -endmacro() diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 10070fbd949..8aa3c7886db 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -1,4 +1,3 @@ -include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake) set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/cctz") set (SRCS @@ -23,12 +22,10 @@ if (OS_FREEBSD) endif () # Related to time_zones table: -# StorageSystemTimeZones.generated.cpp is autogenerated each time during a build -# data in this file will be used to populate the system.time_zones table, this is specific to OS_LINUX -# as the library that's built using embedded tzdata is also specific to OS_LINUX -set(SYSTEM_STORAGE_TZ_FILE "${PROJECT_BINARY_DIR}/src/Storages/System/StorageSystemTimeZones.generated.cpp") +# TimeZones.generated.cpp is autogenerated each time during a build +set(TIMEZONES_FILE "${CMAKE_CURRENT_BINARY_DIR}/TimeZones.generated.cpp") # remove existing copies so that its generated fresh on each build. -file(REMOVE ${SYSTEM_STORAGE_TZ_FILE}) +file(REMOVE ${TIMEZONES_FILE}) # get the list of timezones from tzdata shipped with cctz set(TZDIR "${LIBRARY_DIR}/testdata/zoneinfo") @@ -36,28 +33,36 @@ file(STRINGS "${LIBRARY_DIR}/testdata/version" TZDATA_VERSION) set_property(GLOBAL PROPERTY TZDATA_VERSION_PROP "${TZDATA_VERSION}") message(STATUS "Packaging with tzdata version: ${TZDATA_VERSION}") -set(TIMEZONE_RESOURCE_FILES) - # each file in that dir (except of tab and localtime) store the info about timezone execute_process(COMMAND bash -c "cd ${TZDIR} && find * -type f -and ! -name '*.tab' -and ! -name 'localtime' | LC_ALL=C sort | paste -sd ';' -" OUTPUT_STRIP_TRAILING_WHITESPACE OUTPUT_VARIABLE TIMEZONES) -file(APPEND ${SYSTEM_STORAGE_TZ_FILE} "// autogenerated by ClickHouse/contrib/cctz-cmake/CMakeLists.txt\n") -file(APPEND ${SYSTEM_STORAGE_TZ_FILE} "const char * auto_time_zones[] {\n" ) +file(APPEND ${TIMEZONES_FILE} "// autogenerated by ClickHouse/contrib/cctz-cmake/CMakeLists.txt\n") +file(APPEND ${TIMEZONES_FILE} "#include \n") +set (COUNTER 1) foreach(TIMEZONE ${TIMEZONES}) - file(APPEND ${SYSTEM_STORAGE_TZ_FILE} " \"${TIMEZONE}\",\n") - list(APPEND TIMEZONE_RESOURCE_FILES "${TIMEZONE}") + file(APPEND ${TIMEZONES_FILE} "INCBIN(resource_timezone${COUNTER}, \"${TIMEZONE}\");\n") + MATH(EXPR COUNTER "${COUNTER}+1") endforeach(TIMEZONE) -file(APPEND ${SYSTEM_STORAGE_TZ_FILE} " nullptr};\n") -clickhouse_embed_binaries( - TARGET tzdata - RESOURCE_DIR "${TZDIR}" - RESOURCES ${TIMEZONE_RESOURCE_FILES} -) -add_dependencies(_cctz tzdata) -target_link_libraries(_cctz INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") + +file(APPEND ${TIMEZONES_FILE} "#include \n") +file(APPEND ${TIMEZONES_FILE} "struct TimeZone { const char * name; const unsigned char * data; size_t size; };\n") +file(APPEND ${TIMEZONES_FILE} "TimeZone auto_time_zones[] {\n" ) + +set (COUNTER 1) +foreach(TIMEZONE ${TIMEZONES}) + file(APPEND ${TIMEZONES_FILE} " {\"${TIMEZONE}\", gresource_timezone${COUNTER}Data, gresource_timezone${COUNTER}Size},\n") + MATH(EXPR COUNTER "${COUNTER}+1") +endforeach(TIMEZONE) + +file(APPEND ${TIMEZONES_FILE} " {nullptr, nullptr, 0}};\n") + +add_library (tzdata ${TIMEZONES_FILE}) +target_link_libraries(tzdata ch_contrib::incbin) +target_include_directories(tzdata PRIVATE ${TZDIR}) +target_link_libraries(_cctz tzdata) add_library(ch_contrib::cctz ALIAS _cctz) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index c3a8f69cf3f..bda181eceeb 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index 8146b35cc5f..3619462e79b 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -13,6 +12,10 @@ #include +/// Embedded timezones. +struct TimeZone { const char * name; const unsigned char * data; size_t size; }; +extern TimeZone auto_time_zones[]; + namespace { @@ -249,9 +252,15 @@ namespace cctz_extension const std::string & name, const std::function(const std::string & name)> & fallback) { - std::string_view resource = getResource(name); - if (!resource.empty()) - return std::make_unique(resource.data(), resource.size()); + const TimeZone * timezone = auto_time_zones; + while (timezone->name != nullptr) + { + if (timezone->name == name) + break; + ++timezone; + } + if (timezone->size) + return std::make_unique(reinterpret_cast(timezone->data), timezone->size); return fallback(name); } diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index cb02bb3ff75..ac406538033 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -87,50 +87,13 @@ namespace /// https://stackoverflow.com/questions/32088140/multiple-string-tables-in-elf-object -void updateResources(ElfW(Addr) base_address, std::string_view object_name, std::string_view name, const void * address, SymbolIndex::Resources & resources) -{ - const char * char_address = static_cast(address); - - if (name.starts_with("_binary_") || name.starts_with("binary_")) - { - if (name.ends_with("_start")) - { - name = name.substr((name[0] == '_') + strlen("binary_")); - name = name.substr(0, name.size() - strlen("_start")); - - auto & resource = resources[name]; - if (!resource.base_address || resource.base_address == base_address) - { - resource.base_address = base_address; - resource.start = std::string_view{char_address, 0}; // NOLINT(bugprone-string-constructor) - resource.object_name = object_name; - } - } - if (name.ends_with("_end")) - { - name = name.substr((name[0] == '_') + strlen("binary_")); - name = name.substr(0, name.size() - strlen("_end")); - - auto & resource = resources[name]; - if (!resource.base_address || resource.base_address == base_address) - { - resource.base_address = base_address; - resource.end = std::string_view{char_address, 0}; // NOLINT(bugprone-string-constructor) - resource.object_name = object_name; - } - } - } -} - - /// Based on the code of musl-libc and the answer of Kanalpiroge on /// https://stackoverflow.com/questions/15779185/list-all-the-functions-symbols-on-the-fly-in-c-code-on-a-linux-architecture /// It does not extract all the symbols (but only public - exported and used for dynamic linking), /// but will work if we cannot find or parse ELF files. void collectSymbolsFromProgramHeaders( dl_phdr_info * info, - std::vector & symbols, - SymbolIndex::Resources & resources) + std::vector & symbols) { /* Iterate over all headers of the current shared lib * (first call is for the executable itself) @@ -248,9 +211,6 @@ void collectSymbolsFromProgramHeaders( /// We are not interested in empty symbols. if (elf_sym[sym_index].st_size) symbols.push_back(symbol); - - /// But resources can be represented by a pair of empty symbols (indicating their boundaries). - updateResources(base_address, info->dlpi_name, symbol.name, symbol.address_begin, resources); } break; @@ -281,8 +241,7 @@ void collectSymbolsFromELFSymbolTable( const Elf & elf, const Elf::Section & symbol_table, const Elf::Section & string_table, - std::vector & symbols, - SymbolIndex::Resources & resources) + std::vector & symbols) { /// Iterate symbol table. const ElfSym * symbol_table_entry = reinterpret_cast(symbol_table.begin()); @@ -312,8 +271,6 @@ void collectSymbolsFromELFSymbolTable( if (symbol_table_entry->st_size) symbols.push_back(symbol); - - updateResources(info->dlpi_addr, info->dlpi_name, symbol.name, symbol.address_begin, resources); } } @@ -323,8 +280,7 @@ bool searchAndCollectSymbolsFromELFSymbolTable( const Elf & elf, unsigned section_header_type, const char * string_table_name, - std::vector & symbols, - SymbolIndex::Resources & resources) + std::vector & symbols) { std::optional symbol_table; std::optional string_table; @@ -342,7 +298,7 @@ bool searchAndCollectSymbolsFromELFSymbolTable( return false; } - collectSymbolsFromELFSymbolTable(info, elf, *symbol_table, *string_table, symbols, resources); + collectSymbolsFromELFSymbolTable(info, elf, *symbol_table, *string_table, symbols); return true; } @@ -351,7 +307,6 @@ void collectSymbolsFromELF( dl_phdr_info * info, std::vector & symbols, std::vector & objects, - SymbolIndex::Resources & resources, String & build_id) { String object_name; @@ -462,11 +417,11 @@ void collectSymbolsFromELF( object.name = object_name; objects.push_back(std::move(object)); - searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_SYMTAB, ".strtab", symbols, resources); + searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_SYMTAB, ".strtab", symbols); /// Unneeded if they were parsed from "program headers" of loaded objects. #if defined USE_MUSL - searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_DYNSYM, ".dynstr", symbols, resources); + searchAndCollectSymbolsFromELFSymbolTable(info, *objects.back().elf, SHT_DYNSYM, ".dynstr", symbols); #endif } @@ -479,8 +434,8 @@ int collectSymbols(dl_phdr_info * info, size_t, void * data_ptr) { SymbolIndex::Data & data = *reinterpret_cast(data_ptr); - collectSymbolsFromProgramHeaders(info, data.symbols, data.resources); - collectSymbolsFromELF(info, data.symbols, data.objects, data.resources, data.build_id); + collectSymbolsFromProgramHeaders(info, data.symbols); + collectSymbolsFromELF(info, data.symbols, data.objects, data.build_id); /* Continue iterations */ return 0; diff --git a/src/Common/SymbolIndex.h b/src/Common/SymbolIndex.h index 4fd108434d5..8c7b8971805 100644 --- a/src/Common/SymbolIndex.h +++ b/src/Common/SymbolIndex.h @@ -8,6 +8,7 @@ #include #include + namespace DB { @@ -45,44 +46,15 @@ public: const std::vector & symbols() const { return data.symbols; } const std::vector & objects() const { return data.objects; } - std::string_view getResource(String name) const - { - if (auto it = data.resources.find(name); it != data.resources.end()) - return it->second.data(); - return {}; - } - /// The BuildID that is generated by compiler. String getBuildID() const { return data.build_id; } String getBuildIDHex() const; - struct ResourcesBlob - { - /// Symbol can be presented in multiple shared objects, - /// base_address will be used to compare only symbols from the same SO. - ElfW(Addr) base_address = 0; - /// Just a human name of the SO. - std::string_view object_name; - /// Data blob. - std::string_view start; - std::string_view end; - - std::string_view data() const - { - assert(end.data() >= start.data()); - return std::string_view{start.data(), static_cast(end.data() - start.data())}; - } - }; - using Resources = std::unordered_map; - struct Data { std::vector symbols; std::vector objects; String build_id; - - /// Resources (embedded binary data) are located by symbols in form of _binary_name_start and _binary_name_end. - Resources resources; }; private: Data data; diff --git a/src/Common/getResource.cpp b/src/Common/getResource.cpp deleted file mode 100644 index 72ba24c2f44..00000000000 --- a/src/Common/getResource.cpp +++ /dev/null @@ -1,52 +0,0 @@ -#include "getResource.h" -#include -#include -#include -#include - - -std::string_view getResource(std::string_view name) -{ - // Convert the resource file name into the form generated by `ld -r -b binary`. - std::string name_replaced(name); - std::replace(name_replaced.begin(), name_replaced.end(), '/', '_'); - std::replace(name_replaced.begin(), name_replaced.end(), '-', '_'); - std::replace(name_replaced.begin(), name_replaced.end(), '.', '_'); - boost::replace_all(name_replaced, "+", "_PLUS_"); - -#if defined USE_MUSL - /// If static linking is used, we cannot use dlsym and have to parse ELF symbol table by ourself. - return DB::SymbolIndex::instance().getResource(name_replaced); - -#else - // In most `dlsym(3)` APIs, one passes the symbol name as it appears via - // something like `nm` or `objdump -t`. For example, a symbol `_foo` would be - // looked up with the string `"_foo"`. - // - // Apple's linker is confusingly different. The NOTES on the man page for - // `dlsym(3)` claim that one looks up the symbol with "the name used in C - // source code". In this example, that would mean using the string `"foo"`. - // This apparently applies even in the case where the symbol did not originate - // from C source, such as the embedded binary resource files used here. So - // the symbol name must not have a leading `_` on Apple platforms. It's not - // clear how this applies to other symbols, such as those which _have_ a leading - // underscore in them by design, many leading underscores, etc. -#if defined OS_DARWIN - std::string prefix = "binary_"; -#else - std::string prefix = "_binary_"; -#endif - std::string symbol_name_start = prefix + name_replaced + "_start"; - std::string symbol_name_end = prefix + name_replaced + "_end"; - - const char * sym_start = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_start.c_str())); - const char * sym_end = reinterpret_cast(dlsym(RTLD_DEFAULT, symbol_name_end.c_str())); - - if (sym_start && sym_end) - { - auto resource_size = static_cast(std::distance(sym_start, sym_end)); - return { sym_start, resource_size }; - } - return {}; -#endif -} diff --git a/src/Common/getResource.h b/src/Common/getResource.h deleted file mode 100644 index 8975cc7841e..00000000000 --- a/src/Common/getResource.h +++ /dev/null @@ -1,7 +0,0 @@ -#pragma once - -#include - -/// Get resource from binary if exists. Otherwise return empty string view. -/// Resources are data that is embedded into executable at link time. -std::string_view getResource(std::string_view name); diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index 04f63403ec2..b09319c78d6 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -15,7 +15,8 @@ #endif // All timezones present at build time and embedded into ClickHouse binary. -extern const char * auto_time_zones[]; +struct TimeZone { const char * name; const unsigned char * data; size_t size; }; +extern TimeZone auto_time_zones[]; namespace { @@ -32,14 +33,14 @@ std::vector allTimezones(bool with_weird_offsets = true) { std::vector result; - const auto * timezone_name = auto_time_zones; - while (*timezone_name) + const TimeZone * timezone = auto_time_zones; + while (timezone->name) { - bool weird_offsets = (std::string_view(*timezone_name) == "Africa/Monrovia"); + bool weird_offsets = (std::string_view(timezone->name) == "Africa/Monrovia"); if (!weird_offsets || with_weird_offsets) - result.push_back(*timezone_name); - ++timezone_name; + result.push_back(timezone->name); + ++timezone; } return result; @@ -548,4 +549,3 @@ INSTANTIATE_TEST_SUITE_P(AllTimezones_Year1970, // {0, 0 + 11 * 3600 * 24 + 12, 11}, })) ); - diff --git a/src/Storages/System/StorageSystemTimeZones.cpp b/src/Storages/System/StorageSystemTimeZones.cpp index dc3711812a6..41227ab7780 100644 --- a/src/Storages/System/StorageSystemTimeZones.cpp +++ b/src/Storages/System/StorageSystemTimeZones.cpp @@ -4,7 +4,8 @@ #include -extern const char * auto_time_zones[]; +struct TimeZone { const char * name; const unsigned char * data; size_t size; }; +extern TimeZone auto_time_zones[]; namespace DB { @@ -17,7 +18,7 @@ NamesAndTypesList StorageSystemTimeZones::getNamesAndTypes() void StorageSystemTimeZones::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const { - for (auto * it = auto_time_zones; *it; ++it) - res_columns[0]->insert(String(*it)); + for (auto * it = auto_time_zones; it->name != nullptr; ++it) + res_columns[0]->insert(String(it->name)); } } From c8f8a23c71dc88ab53318be369ca17b528047b05 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 06:09:15 +0200 Subject: [PATCH 1042/2047] Fix errors --- contrib/cctz-cmake/CMakeLists.txt | 21 +++++++++++++------ src/Common/DateLUTImpl.cpp | 18 +++++++--------- src/Common/tests/gtest_DateLUTImpl.cpp | 13 ++++++------ src/Storages/System/CMakeLists.txt | 2 -- .../System/StorageSystemTimeZones.cpp | 7 +++---- 5 files changed, 31 insertions(+), 30 deletions(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 8aa3c7886db..7edeada6e59 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -48,17 +48,26 @@ foreach(TIMEZONE ${TIMEZONES}) MATH(EXPR COUNTER "${COUNTER}+1") endforeach(TIMEZONE) -file(APPEND ${TIMEZONES_FILE} "#include \n") -file(APPEND ${TIMEZONES_FILE} "struct TimeZone { const char * name; const unsigned char * data; size_t size; };\n") -file(APPEND ${TIMEZONES_FILE} "TimeZone auto_time_zones[] {\n" ) +file(APPEND ${TIMEZONES_FILE} "const char * auto_time_zones[] {\n" ) -set (COUNTER 1) foreach(TIMEZONE ${TIMEZONES}) - file(APPEND ${TIMEZONES_FILE} " {\"${TIMEZONE}\", gresource_timezone${COUNTER}Data, gresource_timezone${COUNTER}Size},\n") + file(APPEND ${TIMEZONES_FILE} " \"${TIMEZONE}\",\n") MATH(EXPR COUNTER "${COUNTER}+1") endforeach(TIMEZONE) -file(APPEND ${TIMEZONES_FILE} " {nullptr, nullptr, 0}};\n") +file(APPEND ${TIMEZONES_FILE} "};\n\n") + +file(APPEND ${TIMEZONES_FILE} "#include \n\n") +file(APPEND ${TIMEZONES_FILE} "std::string_view getTimeZone(const char * name)\n{\n" ) + +set (COUNTER 1) +foreach(TIMEZONE ${TIMEZONES}) + file(APPEND ${TIMEZONES_FILE} " if (std::string_view(\"${TIMEZONE}\") == name) return { reinterpret_cast(gresource_timezone${COUNTER}Data), gresource_timezone${COUNTER}Size };\n") + MATH(EXPR COUNTER "${COUNTER}+1") +endforeach(TIMEZONE) + +file(APPEND ${TIMEZONES_FILE} " return {};\n") +file(APPEND ${TIMEZONES_FILE} "}\n") add_library (tzdata ${TIMEZONES_FILE}) target_link_libraries(tzdata ch_contrib::incbin) diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index 3619462e79b..d5e04238ef9 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -10,11 +10,12 @@ #include #include #include +#include /// Embedded timezones. -struct TimeZone { const char * name; const unsigned char * data; size_t size; }; -extern TimeZone auto_time_zones[]; +std::string_view getTimeZone(const char * name); + namespace { @@ -252,15 +253,10 @@ namespace cctz_extension const std::string & name, const std::function(const std::string & name)> & fallback) { - const TimeZone * timezone = auto_time_zones; - while (timezone->name != nullptr) - { - if (timezone->name == name) - break; - ++timezone; - } - if (timezone->size) - return std::make_unique(reinterpret_cast(timezone->data), timezone->size); + std::string_view tz_file = getTimeZone(name.data()); + + if (!tz_file.empty()) + return std::make_unique(tz_file.data(), tz_file.size()); return fallback(name); } diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index b09319c78d6..3d3a3f04941 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -15,8 +15,7 @@ #endif // All timezones present at build time and embedded into ClickHouse binary. -struct TimeZone { const char * name; const unsigned char * data; size_t size; }; -extern TimeZone auto_time_zones[]; +extern const char * auto_time_zones[]; namespace { @@ -33,14 +32,14 @@ std::vector allTimezones(bool with_weird_offsets = true) { std::vector result; - const TimeZone * timezone = auto_time_zones; - while (timezone->name) + const auto * timezone_name = auto_time_zones; + while (*timezone_name) { - bool weird_offsets = (std::string_view(timezone->name) == "Africa/Monrovia"); + bool weird_offsets = (std::string_view(*timezone_name) == "Africa/Monrovia"); if (!weird_offsets || with_weird_offsets) - result.push_back(timezone->name); - ++timezone; + result.push_back(*timezone_name); + ++timezone_name; } return result; diff --git a/src/Storages/System/CMakeLists.txt b/src/Storages/System/CMakeLists.txt index 6b7d1739e33..c3a2e726365 100644 --- a/src/Storages/System/CMakeLists.txt +++ b/src/Storages/System/CMakeLists.txt @@ -30,7 +30,6 @@ endif() add_dependencies(generate-source generate-contributors) set(GENERATED_LICENSES_SRC "${CMAKE_CURRENT_BINARY_DIR}/StorageSystemLicenses.generated.cpp") -set(GENERATED_TIMEZONES_SRC "${CMAKE_CURRENT_BINARY_DIR}/StorageSystemTimeZones.generated.cpp") add_custom_command( OUTPUT StorageSystemLicenses.generated.cpp @@ -38,7 +37,6 @@ add_custom_command( WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR}) list (APPEND storages_system_sources ${GENERATED_LICENSES_SRC}) -list (APPEND storages_system_sources ${GENERATED_TIMEZONES_SRC}) # Overlength strings set_source_files_properties(${GENERATED_LICENSES_SRC} PROPERTIES COMPILE_FLAGS -w) diff --git a/src/Storages/System/StorageSystemTimeZones.cpp b/src/Storages/System/StorageSystemTimeZones.cpp index 41227ab7780..dc3711812a6 100644 --- a/src/Storages/System/StorageSystemTimeZones.cpp +++ b/src/Storages/System/StorageSystemTimeZones.cpp @@ -4,8 +4,7 @@ #include -struct TimeZone { const char * name; const unsigned char * data; size_t size; }; -extern TimeZone auto_time_zones[]; +extern const char * auto_time_zones[]; namespace DB { @@ -18,7 +17,7 @@ NamesAndTypesList StorageSystemTimeZones::getNamesAndTypes() void StorageSystemTimeZones::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const { - for (auto * it = auto_time_zones; it->name != nullptr; ++it) - res_columns[0]->insert(String(it->name)); + for (auto * it = auto_time_zones; *it; ++it) + res_columns[0]->insert(String(*it)); } } From 8013cb1f784f6324b3c7b227499751dc7e666009 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 23 Jul 2023 08:46:44 +0200 Subject: [PATCH 1043/2047] Remove skip_startup_tables from IDatabase::loadStoredObjects() Signed-off-by: Azat Khuzhin --- src/Databases/DatabaseAtomic.cpp | 5 ++--- src/Databases/DatabaseAtomic.h | 2 +- src/Databases/DatabaseLazy.cpp | 3 +-- src/Databases/DatabaseLazy.h | 2 +- src/Databases/DatabaseOrdinary.cpp | 9 +-------- src/Databases/DatabaseOrdinary.h | 2 +- src/Databases/DatabaseReplicated.cpp | 5 ++--- src/Databases/DatabaseReplicated.h | 2 +- src/Databases/IDatabase.h | 3 +-- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- src/Databases/MySQL/DatabaseMySQL.h | 2 +- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 2 +- src/Databases/TablesLoader.cpp | 2 +- 14 files changed, 16 insertions(+), 27 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 7e20b6f6535..0f65069db35 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -441,11 +441,10 @@ void DatabaseAtomic::beforeLoadingMetadata(ContextMutablePtr /*context*/, Loadin } } -void DatabaseAtomic::loadStoredObjects( - ContextMutablePtr local_context, LoadingStrictnessLevel mode, bool skip_startup_tables) +void DatabaseAtomic::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel mode) { beforeLoadingMetadata(local_context, mode); - DatabaseOrdinary::loadStoredObjects(local_context, mode, skip_startup_tables); + DatabaseOrdinary::loadStoredObjects(local_context, mode); } void DatabaseAtomic::startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index cb275812098..70553b2d5c2 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -48,7 +48,7 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; - void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode) override; void beforeLoadingMetadata(ContextMutablePtr context, LoadingStrictnessLevel mode) override; diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index f27c6c0c3ee..896ae99656f 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -37,8 +37,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, } -void DatabaseLazy::loadStoredObjects( - ContextMutablePtr local_context, LoadingStrictnessLevel /*mode*/, bool /* skip_startup_tables */) +void DatabaseLazy::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel /*mode*/) { iterateMetadataFiles(local_context, [this, &local_context](const String & file_name) { diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index b01038073ef..2b1b119754d 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -26,7 +26,7 @@ public: bool canContainDistributedTables() const override { return false; } - void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel /*mode*/, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel /*mode*/) override; void createTable( ContextPtr context, diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 8c92b8064ca..51d37b84e14 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -89,8 +89,7 @@ DatabaseOrdinary::DatabaseOrdinary( { } -void DatabaseOrdinary::loadStoredObjects( - ContextMutablePtr local_context, LoadingStrictnessLevel mode, bool skip_startup_tables) +void DatabaseOrdinary::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel mode) { /** Tables load faster if they are loaded in sorted (by name) order. * Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order, @@ -159,12 +158,6 @@ void DatabaseOrdinary::loadStoredObjects( } pool.wait(); - - if (!skip_startup_tables) - { - /// After all tables was basically initialized, startup them. - startupTables(pool, mode); - } } void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTablesMetadata & metadata, bool is_startup) diff --git a/src/Databases/DatabaseOrdinary.h b/src/Databases/DatabaseOrdinary.h index f9aa3214ef5..cabc8f9c55b 100644 --- a/src/Databases/DatabaseOrdinary.h +++ b/src/Databases/DatabaseOrdinary.h @@ -21,7 +21,7 @@ public: String getEngineName() const override { return "Ordinary"; } - void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode) override; bool supportsLoadingInTopologicalOrder() const override { return true; } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 25c23e2be17..d3b3d4b545f 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -495,11 +495,10 @@ void DatabaseReplicated::beforeLoadingMetadata(ContextMutablePtr /*context*/, Lo tryConnectToZooKeeperAndInitDatabase(mode); } -void DatabaseReplicated::loadStoredObjects( - ContextMutablePtr local_context, LoadingStrictnessLevel mode, bool skip_startup_tables) +void DatabaseReplicated::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel mode) { beforeLoadingMetadata(local_context, mode); - DatabaseAtomic::loadStoredObjects(local_context, mode, skip_startup_tables); + DatabaseAtomic::loadStoredObjects(local_context, mode); } UInt64 DatabaseReplicated::getMetadataHash(const String & table_name) const diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index ff1a4aba41c..8e33f482ac1 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -67,7 +67,7 @@ public: void drop(ContextPtr /*context*/) override; - void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr context, LoadingStrictnessLevel mode) override; void beforeLoadingMetadata(ContextMutablePtr context, LoadingStrictnessLevel mode) override; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index a9577dfc84a..9bed3c4bfc5 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -134,8 +134,7 @@ public: /// You can call only once, right after the object is created. virtual void loadStoredObjects( /// NOLINT ContextMutablePtr /*context*/, - LoadingStrictnessLevel /*mode*/, - bool /* skip_startup_tables */) + LoadingStrictnessLevel /*mode*/) { } diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 70bd32efed9..94e5ba1773e 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -402,7 +402,7 @@ String DatabaseMySQL::getMetadataPath() const return metadata_path; } -void DatabaseMySQL::loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/, bool /* skip_startup_tables */) +void DatabaseMySQL::loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/) { std::lock_guard lock{mutex}; diff --git a/src/Databases/MySQL/DatabaseMySQL.h b/src/Databases/MySQL/DatabaseMySQL.h index f34a2fff4f7..e5b1f434d2f 100644 --- a/src/Databases/MySQL/DatabaseMySQL.h +++ b/src/Databases/MySQL/DatabaseMySQL.h @@ -76,7 +76,7 @@ public: void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; - void loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/) override; StoragePtr detachTable(ContextPtr context, const String & table_name) override; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index f4d750f85d4..812a0d8717e 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -296,7 +296,7 @@ void DatabasePostgreSQL::drop(ContextPtr /*context*/) } -void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, LoadingStrictnessLevel /*mode*/, bool /* skip_startup_tables */) +void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, LoadingStrictnessLevel /*mode*/) { { std::lock_guard lock{mutex}; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 31fa036c0ee..d731e06649b 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -44,7 +44,7 @@ public: bool empty() const override; - void loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/, bool skip_startup_tables) override; + void loadStoredObjects(ContextMutablePtr, LoadingStrictnessLevel /*mode*/) override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index ea0f2072430..f8b4e7fe33b 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -49,7 +49,7 @@ void TablesLoader::loadTables() if (need_resolve_dependencies && database.second->supportsLoadingInTopologicalOrder()) databases_to_load.push_back(database.first); else - database.second->loadStoredObjects(global_context, strictness_mode, /* skip_startup_tables */ true); + database.second->loadStoredObjects(global_context, strictness_mode); } if (databases_to_load.empty()) From 282258a855cfed40e0b2cd7c0ada3ec1defe8e06 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sun, 23 Jul 2023 11:29:29 +0200 Subject: [PATCH 1044/2047] fix style --- src/Common/OptimizedRegularExpression.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index e636b0b987d..05e6aefbb5e 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -439,7 +440,7 @@ catch (...) is_trivial = false; required_substring_is_prefix = false; alternatives.clear(); - std::cerr << "Analyze RegularExpression failed, got error: {}" << DB::getCurrentExceptionMessage(false) << "\n"; + LOG_ERROR(&Poco::Logger::get("OptimizeRegularExpression"), "Analyze RegularExpression failed, got error: {}", DB::getCurrentExceptionMessage(false)); } template From 4c1f8f38cd4073b24064e076a677082db546c680 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 17:30:23 +0200 Subject: [PATCH 1045/2047] Fix CI --- docker/test/fasttest/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index e25b5fdbfed..60e6199aaa4 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -147,6 +147,7 @@ function clone_submodules contrib/simdjson contrib/liburing contrib/libfiu + contrib/incbin ) git submodule sync From 8902bbdb60b466498ab2825000502195d5d35c91 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 17:59:49 +0200 Subject: [PATCH 1046/2047] Fix fasttest --- src/Common/FrequencyHolder.cpp | 4 ++++ src/Common/FrequencyHolder.h | 6 ++++++ src/Functions/FunctionsCharsetClassification.cpp | 2 +- src/Functions/FunctionsLanguageClassification.cpp | 4 +--- 4 files changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Common/FrequencyHolder.cpp b/src/Common/FrequencyHolder.cpp index 3b755cacacb..fe03e6a1b44 100644 --- a/src/Common/FrequencyHolder.cpp +++ b/src/Common/FrequencyHolder.cpp @@ -1,5 +1,7 @@ #include +#if USE_NLP + #include /// Embedded SQL definitions @@ -179,3 +181,5 @@ void FrequencyHolder::loadProgrammingFrequency() } } + +#endif diff --git a/src/Common/FrequencyHolder.h b/src/Common/FrequencyHolder.h index 270e4dbbd2a..73675ed9814 100644 --- a/src/Common/FrequencyHolder.h +++ b/src/Common/FrequencyHolder.h @@ -1,5 +1,9 @@ #pragma once +#include "config.h" + +#if USE_NLP + #include #include @@ -81,3 +85,5 @@ private: EncodingContainer encodings_freq; }; } + +#endif diff --git a/src/Functions/FunctionsCharsetClassification.cpp b/src/Functions/FunctionsCharsetClassification.cpp index a25da8f6c13..237d4c37fa2 100644 --- a/src/Functions/FunctionsCharsetClassification.cpp +++ b/src/Functions/FunctionsCharsetClassification.cpp @@ -3,7 +3,7 @@ #include #include -#include + namespace DB { diff --git a/src/Functions/FunctionsLanguageClassification.cpp b/src/Functions/FunctionsLanguageClassification.cpp index 6088fd52efa..55485d41ce0 100644 --- a/src/Functions/FunctionsLanguageClassification.cpp +++ b/src/Functions/FunctionsLanguageClassification.cpp @@ -5,19 +5,17 @@ #include #include #include -#include #include #include #include -#include #include #include #include #include -#include #include + namespace DB { /* Determine language of Unicode UTF-8 text. From 43bd6d1b8336f282cc4548c0f61b52516f49ac13 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 19:00:49 +0300 Subject: [PATCH 1047/2047] Revert "Add an ability to specify allocations size for sampling memory profiler" --- programs/server/Server.cpp | 21 +++-------- src/Common/MemoryTracker.cpp | 10 +---- src/Common/MemoryTracker.h | 18 --------- src/Core/ServerSettings.h | 8 +--- src/Core/Settings.h | 4 +- src/Interpreters/ProcessList.cpp | 3 -- src/Interpreters/ThreadStatusExt.cpp | 2 - .../__init__.py | 1 - .../configs/max_untracked_memory.xml | 7 ---- .../configs/memory_profiler.xml | 5 --- .../test.py | 37 ------------------- ...r_sample_min_max_allocation_size.reference | 1 - ...profiler_sample_min_max_allocation_size.sh | 18 --------- 13 files changed, 11 insertions(+), 124 deletions(-) delete mode 100644 tests/integration/test_memory_profiler_min_max_borders/__init__.py delete mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml delete mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml delete mode 100644 tests/integration/test_memory_profiler_min_max_borders/test.py delete mode 100644 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference delete mode 100755 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 33fdcc9c1a8..9202d4b32c1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1637,26 +1637,17 @@ try global_context->initializeTraceCollector(); /// Set up server-wide memory profiler (for total memory tracker). - if (server_settings.total_memory_profiler_step) + UInt64 total_memory_profiler_step = config().getUInt64("total_memory_profiler_step", 0); + if (total_memory_profiler_step) { - total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); + total_memory_tracker.setProfilerStep(total_memory_profiler_step); } - if (server_settings.total_memory_tracker_sample_probability > 0.0) + double total_memory_tracker_sample_probability = config().getDouble("total_memory_tracker_sample_probability", 0); + if (total_memory_tracker_sample_probability > 0.0) { - total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); + total_memory_tracker.setSampleProbability(total_memory_tracker_sample_probability); } - - if (server_settings.total_memory_profiler_sample_min_allocation_size) - { - total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); - } - - if (server_settings.total_memory_profiler_sample_max_allocation_size) - { - total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); - } - } #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 52cae0768dc..81cac2617c5 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -229,7 +229,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size}); @@ -413,7 +413,7 @@ void MemoryTracker::free(Int64 size) } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size}); @@ -534,12 +534,6 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) ; } -bool MemoryTracker::isSizeOkForSampling(UInt64 size) const -{ - /// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation - return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes); -} - bool canEnqueueBackgroundTask() { auto limit = background_memory_tracker.getSoftLimit(); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 768dc8a7404..4e29d40c953 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -67,12 +67,6 @@ private: /// To randomly sample allocations and deallocations in trace_log. double sample_probability = 0; - /// Randomly sample allocations only larger or equal to this size - UInt64 min_allocation_size_bytes = 0; - - /// Randomly sample allocations only smaller or equal to this size - UInt64 max_allocation_size_bytes = 0; - /// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy). /// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker. std::atomic parent {}; @@ -94,8 +88,6 @@ private: void setOrRaiseProfilerLimit(Int64 value); - bool isSizeOkForSampling(UInt64 size) const; - /// allocImpl(...) and free(...) should not be used directly friend struct CurrentMemoryTracker; void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); @@ -173,16 +165,6 @@ public: sample_probability = value; } - void setSampleMinAllocationSize(UInt64 value) - { - min_allocation_size_bytes = value; - } - - void setSampleMaxAllocationSize(UInt64 value) - { - max_allocation_size_bytes = value; - } - void setProfilerStep(Int64 value) { profiler_step = value; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index f7a6c9e950e..1a9f226041b 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -81,12 +81,8 @@ namespace DB M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \ M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ - M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ - \ - M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \ - M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ - M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ - M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) + M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) + DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4fc93500910..24be644ee55 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -427,9 +427,7 @@ class IColumn; M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ - M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ - M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ - M(UInt64, memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \ \ M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \ diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index c299572ef41..1503e396298 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -223,10 +223,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q { /// Set up memory profiling thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step); - thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); - thread_group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); - thread_group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events); } diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index bac16c05533..398bea26b87 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -83,8 +83,6 @@ ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_contex const Settings & settings = storage_context->getSettingsRef(); group->memory_tracker.setProfilerStep(settings.memory_profiler_step); group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); - group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); - group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); group->memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator); group->memory_tracker.setParent(&background_memory_tracker); if (settings.memory_tracker_fault_probability > 0.0) diff --git a/tests/integration/test_memory_profiler_min_max_borders/__init__.py b/tests/integration/test_memory_profiler_min_max_borders/__init__.py deleted file mode 100644 index e5a0d9b4834..00000000000 --- a/tests/integration/test_memory_profiler_min_max_borders/__init__.py +++ /dev/null @@ -1 +0,0 @@ -#!/usr/bin/env python3 diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml deleted file mode 100644 index 56fc5ed34ca..00000000000 --- a/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - - 1 - - - diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml deleted file mode 100644 index 5b3e17d145f..00000000000 --- a/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml +++ /dev/null @@ -1,5 +0,0 @@ - - 1 - 4096 - 8192 - diff --git a/tests/integration/test_memory_profiler_min_max_borders/test.py b/tests/integration/test_memory_profiler_min_max_borders/test.py deleted file mode 100644 index 6ab971fa9c4..00000000000 --- a/tests/integration/test_memory_profiler_min_max_borders/test.py +++ /dev/null @@ -1,37 +0,0 @@ -from helpers.cluster import ClickHouseCluster -import pytest - -cluster = ClickHouseCluster(__file__) -node = cluster.add_instance( - "node", - main_configs=["configs/memory_profiler.xml"], - user_configs=["configs/max_untracked_memory.xml"], -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def test_trace_boundaries_work(started_cluster): - node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null") - node.query("SYSTEM FLUSH LOGS") - - assert ( - node.query( - "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'" - ) - == "1\n" - ) - assert ( - node.query( - "SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" - ) - == "0\n" - ) diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh deleted file mode 100755 index b1fbea26da7..00000000000 --- a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh +++ /dev/null @@ -1,18 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-cpu-aarch64, no-random-settings -# requires TraceCollector, does not available under sanitizers and aarch64 - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -query_id="${CLICKHOUSE_DATABASE}_min_max_allocation_size_$RANDOM$RANDOM" -${CLICKHOUSE_CLIENT} --query_id="$query_id" --memory_profiler_sample_min_allocation_size=4096 --memory_profiler_sample_max_allocation_size=8192 --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --query "select randomPrintableASCII(number) from numbers(1000) FORMAT Null" - -${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" - -# at least something allocated -${CLICKHOUSE_CLIENT} --query "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample'" - -# show wrong allocations -${CLICKHOUSE_CLIENT} --query "SELECT abs(size) FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" From e56e1ebd5d8fbb808867c1f98e421383acf38b1f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 18:29:07 +0200 Subject: [PATCH 1048/2047] Fix fasttest --- src/Functions/FunctionsCharsetClassification.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Functions/FunctionsCharsetClassification.cpp b/src/Functions/FunctionsCharsetClassification.cpp index 237d4c37fa2..7704e3eafc0 100644 --- a/src/Functions/FunctionsCharsetClassification.cpp +++ b/src/Functions/FunctionsCharsetClassification.cpp @@ -1,4 +1,9 @@ #include + +#include "config.h" + +#if USE_NLP + #include #include @@ -150,3 +155,5 @@ REGISTER_FUNCTION(DetectCharset) } } + +#endif From 039cac69cf6d30cc58c8531b1efac4d9847cb599 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 23 Jul 2023 18:35:37 +0200 Subject: [PATCH 1049/2047] Fix test_insert_same_partition_and_merge by increasing wait time --- tests/integration/test_merge_tree_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 761b5257a34..86b70f8db70 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -215,7 +215,7 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical): if attempt == 59: assert parts_count == "(1)" - time.sleep(1) + time.sleep(10) assert azure_query(node, f"SELECT sum(id) FROM {TABLE_NAME} FORMAT Values") == "(0)" assert ( From 311b3adf89b9d54c4b3bf40feb4179d967ed3d2e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 20:10:41 +0200 Subject: [PATCH 1050/2047] Fix fasttest --- src/Functions/FunctionsCharsetClassification.cpp | 2 -- src/Functions/FunctionsProgrammingClassification.cpp | 5 +++++ src/Functions/FunctionsTonalityClassification.cpp | 5 +++++ 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsCharsetClassification.cpp b/src/Functions/FunctionsCharsetClassification.cpp index 7704e3eafc0..05b173e3d95 100644 --- a/src/Functions/FunctionsCharsetClassification.cpp +++ b/src/Functions/FunctionsCharsetClassification.cpp @@ -1,7 +1,5 @@ #include -#include "config.h" - #if USE_NLP #include diff --git a/src/Functions/FunctionsProgrammingClassification.cpp b/src/Functions/FunctionsProgrammingClassification.cpp index 8a552a30e65..a93e1d9a87d 100644 --- a/src/Functions/FunctionsProgrammingClassification.cpp +++ b/src/Functions/FunctionsProgrammingClassification.cpp @@ -1,4 +1,7 @@ #include + +#if USE_NLP + #include #include #include @@ -118,3 +121,5 @@ REGISTER_FUNCTION(DetectProgrammingLanguage) } } + +#endif diff --git a/src/Functions/FunctionsTonalityClassification.cpp b/src/Functions/FunctionsTonalityClassification.cpp index e39f9c63758..3de38d99c88 100644 --- a/src/Functions/FunctionsTonalityClassification.cpp +++ b/src/Functions/FunctionsTonalityClassification.cpp @@ -1,4 +1,7 @@ #include + +#if USE_NLP + #include #include #include @@ -87,3 +90,5 @@ REGISTER_FUNCTION(DetectTonality) } } + +#endif From 49f4ef6ffb9264d8b4a31c8e4ab683f01afd4268 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 20:11:24 +0200 Subject: [PATCH 1051/2047] Fix typo --- src/Functions/FunctionsCharsetClassification.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsCharsetClassification.cpp b/src/Functions/FunctionsCharsetClassification.cpp index 05b173e3d95..0a332ab70a9 100644 --- a/src/Functions/FunctionsCharsetClassification.cpp +++ b/src/Functions/FunctionsCharsetClassification.cpp @@ -49,7 +49,7 @@ namespace return res; } - /// Сount how many times each bigram occurs in the text. + /// Count how many times each bigram occurs in the text. template ALWAYS_INLINE inline void calculateStats( const UInt8 * data, From e21a4c4c9a3f50436b8e708b6a38cdf8eee3c6be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 20:57:16 +0200 Subject: [PATCH 1052/2047] Fix the test --- .../02415_all_new_functions_must_be_documented.reference | 4 ---- .../02415_all_new_functions_must_be_documented.sql | 4 +++- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 595ebb483d5..b7097ad329b 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -238,10 +238,6 @@ defaultValueOfArgumentType defaultValueOfTypeName degrees demangle -detectCharset -detectLanguageUnknown -detectProgrammingLanguage -detectTonality divide dotProduct dumpColumnStructure diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql index ed95c06d016..4f40da6c626 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.sql @@ -15,5 +15,7 @@ AND name NOT IN ( 'h3ToGeoBoundary', 'h3ToParent', 'h3ToString', 'h3UnidirectionalEdgeIsValid', 'h3kRing', 'stringToH3', 'geoToS2', 's2CapContains', 's2CapUnion', 's2CellsIntersect', 's2GetNeighbors', 's2RectAdd', 's2RectContains', 's2RectIntersection', 's2RectUnion', 's2ToGeo', 'normalizeUTF8NFC', 'normalizeUTF8NFD', 'normalizeUTF8NFKC', 'normalizeUTF8NFKD', - 'lemmatize', 'tokenize', 'stem', 'synonyms' -- these functions are not enabled in fast test + 'lemmatize', 'tokenize', 'stem', 'synonyms', + 'detectCharset', 'detectLanguageUnknown', 'detectProgrammingLanguage', 'detectTonality' + -- these functions are not enabled in fast test ) ORDER BY name; From 67f643f27e5930765d0b6881c415ffacf369c14f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jul 2023 21:00:28 +0200 Subject: [PATCH 1053/2047] Fix error --- contrib/cctz-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index 7edeada6e59..fde31dd469d 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -55,7 +55,7 @@ foreach(TIMEZONE ${TIMEZONES}) MATH(EXPR COUNTER "${COUNTER}+1") endforeach(TIMEZONE) -file(APPEND ${TIMEZONES_FILE} "};\n\n") +file(APPEND ${TIMEZONES_FILE} " nullptr\n};\n\n") file(APPEND ${TIMEZONES_FILE} "#include \n\n") file(APPEND ${TIMEZONES_FILE} "std::string_view getTimeZone(const char * name)\n{\n" ) From e02948580b31c61e32860da04f966a21231e14c7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 23 Jul 2023 22:38:59 +0200 Subject: [PATCH 1054/2047] Don't shutdown interserver before tables --- programs/server/Server.cpp | 91 +++++++++++++++++++++++++++++--------- programs/server/Server.h | 11 ++++- 2 files changed, 79 insertions(+), 23 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c7a7ba71e83..8c6e41d28c6 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -739,11 +739,12 @@ try [&]() -> std::vector { std::vector metrics; - metrics.reserve(servers_to_start_before_tables.size()); + + std::lock_guard lock(servers_lock); + metrics.reserve(servers_to_start_before_tables.size() + servers.size()); for (const auto & server : servers_to_start_before_tables) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); - std::lock_guard lock(servers_lock); for (const auto & server : servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); return metrics; @@ -1302,7 +1303,7 @@ try global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config); std::lock_guard lock(servers_lock); - updateServers(*config, server_pool, async_metrics, servers); + updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables); } global_context->updateStorageConfiguration(*config); @@ -1404,10 +1405,27 @@ try } - for (auto & server : servers_to_start_before_tables) { - server.start(); - LOG_INFO(log, "Listening for {}", server.getDescription()); + std::lock_guard lock(servers_lock); + /// We should start interserver communications before (and more imporant shutdown after) tables. + /// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down. + /// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can + /// communicate with zookeeper, execute merges, etc. + createInterserverServers( + config(), + interserver_listen_hosts, + listen_try, + server_pool, + async_metrics, + servers_to_start_before_tables, + /* start_servers= */ false); + + + for (auto & server : servers_to_start_before_tables) + { + server.start(); + LOG_INFO(log, "Listening for {}", server.getDescription()); + } } /// Initialize access storages. @@ -1527,10 +1545,13 @@ try { LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish."); size_t current_connections = 0; - for (auto & server : servers_to_start_before_tables) { - server.stop(); - current_connections += server.currentConnections(); + std::lock_guard lock(servers_lock); + for (auto & server : servers_to_start_before_tables) + { + server.stop(); + current_connections += server.currentConnections(); + } } if (current_connections) @@ -1709,7 +1730,7 @@ try { std::lock_guard lock(servers_lock); - createServers(config(), listen_hosts, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers); + createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers); if (servers.empty()) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No servers started (add valid listen_host and 'tcp_port' or 'http_port' " @@ -1967,7 +1988,6 @@ HTTPContextPtr Server::httpContext() const void Server::createServers( Poco::Util::AbstractConfiguration & config, const Strings & listen_hosts, - const Strings & interserver_listen_hosts, bool listen_try, Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, @@ -2189,6 +2209,23 @@ void Server::createServers( httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); }); } +} + +void Server::createInterserverServers( + Poco::Util::AbstractConfiguration & config, + const Strings & interserver_listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers) +{ + const Settings & settings = global_context->getSettingsRef(); + + Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0); + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(settings.http_receive_timeout); + http_params->setKeepAliveTimeout(keep_alive_timeout); /// Now iterate over interserver_listen_hosts for (const auto & interserver_listen_host : interserver_listen_hosts) @@ -2237,14 +2274,14 @@ void Server::createServers( #endif }); } - } void Server::updateServers( Poco::Util::AbstractConfiguration & config, Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, - std::vector & servers) + std::vector & servers, + std::vector & servers_to_start_before_tables) { Poco::Logger * log = &logger(); @@ -2270,11 +2307,19 @@ void Server::updateServers( Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : this->config(); + std::vector all_servers; + all_servers.reserve(servers.size() + servers_to_start_before_tables.size()); for (auto & server : servers) + all_servers.push_back(&server); + + for (auto & server : servers_to_start_before_tables) + all_servers.push_back(&server); + + for (auto * server : all_servers) { - if (!server.isStopping()) + if (!server->isStopping()) { - std::string port_name = server.getPortName(); + std::string port_name = server->getPortName(); bool has_host = false; bool is_http = false; if (port_name.starts_with("protocols.")) @@ -2312,27 +2357,29 @@ void Server::updateServers( /// NOTE: better to compare using getPortName() over using /// dynamic_cast<> since HTTPServer is also used for prometheus and /// internal replication communications. - is_http = server.getPortName() == "http_port" || server.getPortName() == "https_port"; + is_http = server->getPortName() == "http_port" || server->getPortName() == "https_port"; } if (!has_host) - has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server.getListenHost()) != listen_hosts.end(); + has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server->getListenHost()) != listen_hosts.end(); bool has_port = !config.getString(port_name, "").empty(); bool force_restart = is_http && !isSameConfiguration(previous_config, config, "http_handlers"); if (force_restart) - LOG_TRACE(log, " had been changed, will reload {}", server.getDescription()); + LOG_TRACE(log, " had been changed, will reload {}", server->getDescription()); - if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber() || force_restart) + if (!has_host || !has_port || config.getInt(server->getPortName()) != server->portNumber() || force_restart) { - server.stop(); - LOG_INFO(log, "Stopped listening for {}", server.getDescription()); + server->stop(); + LOG_INFO(log, "Stopped listening for {}", server->getDescription()); } } } - createServers(config, listen_hosts, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); + createServers(config, listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); + createInterserverServers(config, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ true); std::erase_if(servers, std::bind_front(check_server, "")); + std::erase_if(servers_to_start_before_tables, std::bind_front(check_server, "")); } } diff --git a/programs/server/Server.h b/programs/server/Server.h index e9ae6d8d937..d13378dcd65 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -102,6 +102,14 @@ private: void createServers( Poco::Util::AbstractConfiguration & config, const Strings & listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers = false); + + void createInterserverServers( + Poco::Util::AbstractConfiguration & config, const Strings & interserver_listen_hosts, bool listen_try, Poco::ThreadPool & server_pool, @@ -113,7 +121,8 @@ private: Poco::Util::AbstractConfiguration & config, Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, - std::vector & servers); + std::vector & servers, + std::vector & servers_to_start_before_tables); }; } From c0f16dcf031b62e2eebdef249c132e9351203bc0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 23 Jul 2023 21:10:12 +0000 Subject: [PATCH 1055/2047] Test from fuzzer --- .../02831_ast_fuzz_asan_join.reference | 0 .../0_stateless/02831_ast_fuzz_asan_join.sql | 22 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02831_ast_fuzz_asan_join.reference create mode 100644 tests/queries/0_stateless/02831_ast_fuzz_asan_join.sql diff --git a/tests/queries/0_stateless/02831_ast_fuzz_asan_join.reference b/tests/queries/0_stateless/02831_ast_fuzz_asan_join.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02831_ast_fuzz_asan_join.sql b/tests/queries/0_stateless/02831_ast_fuzz_asan_join.sql new file mode 100644 index 00000000000..7c7bfd2df88 --- /dev/null +++ b/tests/queries/0_stateless/02831_ast_fuzz_asan_join.sql @@ -0,0 +1,22 @@ +SELECT + '0', + toTypeName(materialize(js2.s)) +FROM +( + SELECT number AS k + FROM numbers(100) +) AS js1 +FULL OUTER JOIN +( + SELECT + toLowCardinality(2147483647 + 256) AS k, + '-0.0000000001', + 1024, + toString(number + 10) AS s + FROM numbers(1024) +) AS js2 ON js1.k = js2.k +ORDER BY + inf DESC NULLS FIRST, + js1.k ASC NULLS LAST, + js2.k ASC +FORMAT `Null` From 1e467867e68c2c382f26291753bab45e2bc87a60 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 00:03:40 +0200 Subject: [PATCH 1056/2047] Attempt to fix LTO --- contrib/cctz-cmake/CMakeLists.txt | 3 +-- programs/install/Install.cpp | 6 ++++-- programs/server/Server.cpp | 2 +- src/Common/FrequencyHolder.cpp | 6 +++--- src/Common/config.h.in | 4 ++++ src/Server/WebUIRequestHandler.cpp | 8 +++++--- src/Storages/System/attachInformationSchemaTables.cpp | 10 ++++++---- src/configure_config.cmake | 2 ++ 8 files changed, 26 insertions(+), 15 deletions(-) diff --git a/contrib/cctz-cmake/CMakeLists.txt b/contrib/cctz-cmake/CMakeLists.txt index fde31dd469d..7161f743de1 100644 --- a/contrib/cctz-cmake/CMakeLists.txt +++ b/contrib/cctz-cmake/CMakeLists.txt @@ -44,7 +44,7 @@ file(APPEND ${TIMEZONES_FILE} "#include \n") set (COUNTER 1) foreach(TIMEZONE ${TIMEZONES}) - file(APPEND ${TIMEZONES_FILE} "INCBIN(resource_timezone${COUNTER}, \"${TIMEZONE}\");\n") + file(APPEND ${TIMEZONES_FILE} "INCBIN(resource_timezone${COUNTER}, \"${TZDIR}/${TIMEZONE}\");\n") MATH(EXPR COUNTER "${COUNTER}+1") endforeach(TIMEZONE) @@ -71,7 +71,6 @@ file(APPEND ${TIMEZONES_FILE} "}\n") add_library (tzdata ${TIMEZONES_FILE}) target_link_libraries(tzdata ch_contrib::incbin) -target_include_directories(tzdata PRIVATE ${TZDIR}) target_link_libraries(_cctz tzdata) add_library(ch_contrib::cctz ALIAS _cctz) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index da2c95af62c..d7086c95beb 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -34,9 +34,11 @@ #include +#include "config.h" + /// Embedded configuration files used inside the install program -INCBIN(resource_config_xml, "config.xml"); -INCBIN(resource_users_xml, "users.xml"); +INCBIN(resource_config_xml, SOURCE_DIR "/programs/server/config.xml"); +INCBIN(resource_users_xml, SOURCE_DIR "/programs/server/users.xml"); /** This tool can be used to install ClickHouse without a deb/rpm/tgz package, having only "clickhouse" binary. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 229a169dc1e..2ab89ad048a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -130,7 +130,7 @@ #include /// A minimal file used when the server is run without installation -INCBIN(resource_embedded_xml, "embedded.xml"); +INCBIN(resource_embedded_xml, SOURCE_DIR "/programs/server/embedded.xml"); namespace CurrentMetrics { diff --git a/src/Common/FrequencyHolder.cpp b/src/Common/FrequencyHolder.cpp index fe03e6a1b44..7dc1f622aeb 100644 --- a/src/Common/FrequencyHolder.cpp +++ b/src/Common/FrequencyHolder.cpp @@ -5,9 +5,9 @@ #include /// Embedded SQL definitions -INCBIN(resource_charset_zst, "charset.zst"); -INCBIN(resource_tonality_ru_zst, "tonality_ru.zst"); -INCBIN(resource_programming_zst, "programming.zst"); +INCBIN(resource_charset_zst, SOURCE_DIR "/contrib/nlp-data/charset.zst"); +INCBIN(resource_tonality_ru_zst, SOURCE_DIR "/contrib/nlp-data/tonality_ru.zst"); +INCBIN(resource_programming_zst, SOURCE_DIR "/contrib/nlp-data/programming.zst"); namespace DB diff --git a/src/Common/config.h.in b/src/Common/config.h.in index a2c18fc330f..628f0847d65 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -59,3 +59,7 @@ #cmakedefine01 USE_ULID #cmakedefine01 FIU_ENABLE #cmakedefine01 USE_BCRYPT + +/// This is needed for .incbin in assembly. For some reason, include paths don't work there in presence of LTO. +/// That's why we use absolute paths. +#cmakedefine SOURCE_DIR "@SOURCE_DIR@" diff --git a/src/Server/WebUIRequestHandler.cpp b/src/Server/WebUIRequestHandler.cpp index cb9e8935d8c..6fa1d65de42 100644 --- a/src/Server/WebUIRequestHandler.cpp +++ b/src/Server/WebUIRequestHandler.cpp @@ -11,10 +11,12 @@ #include +#include "config.h" + /// Embedded HTML pages -INCBIN(resource_play_html, "play.html"); -INCBIN(resource_dashboard_html, "dashboard.html"); -INCBIN(resource_uplot_js, "js/uplot.js"); +INCBIN(resource_play_html, SOURCE_DIR "/programs/server/play.html"); +INCBIN(resource_dashboard_html, SOURCE_DIR "/programs/server/dashboard.html"); +INCBIN(resource_uplot_js, SOURCE_DIR "/programs/server/js/uplot.js"); namespace DB diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index bfc5c8c64e2..12cef89b553 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -5,11 +5,13 @@ #include #include +#include "config.h" + /// Embedded SQL definitions -INCBIN(resource_schemata_sql, "schemata.sql"); -INCBIN(resource_tables_sql, "tables.sql"); -INCBIN(resource_views_sql, "views.sql"); -INCBIN(resource_columns_sql, "columns.sql"); +INCBIN(resource_schemata_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/schemata.sql"); +INCBIN(resource_tables_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/tables.sql"); +INCBIN(resource_views_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/views.sql"); +INCBIN(resource_columns_sql, SOURCE_DIR "/src/Storages/System/InformationSchema/columns.sql"); namespace DB diff --git a/src/configure_config.cmake b/src/configure_config.cmake index ae6305705c2..5529e2f2f39 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -162,3 +162,5 @@ endif () if (TARGET ch_contrib::fiu) set(FIU_ENABLE 1) endif() + +set(SOURCE_DIR ${CMAKE_SOURCE_DIR}) From cfed3589b03961252adcfef746576ca717c7246f Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Sun, 23 Jul 2023 20:25:44 +0300 Subject: [PATCH 1057/2047] fix style --- .../integration/helpers/s3_mocks/broken_s3.py | 31 ++++++----- .../test_checking_s3_blobs_paranoid/test.py | 54 ++++++++++++------- 2 files changed, 53 insertions(+), 32 deletions(-) diff --git a/tests/integration/helpers/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py index 0c794ae1ad4..6e1572af262 100644 --- a/tests/integration/helpers/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -37,7 +37,9 @@ class MockControl: ) assert response == "OK", response - def setup_action(self, when, count=None, after=None, action="error_500", action_args=None): + def setup_action( + self, when, count=None, after=None, action="error_500", action_args=None + ): url = f"http://localhost:{self._port}/mock_settings/{when}?nothing=1" if count is not None: @@ -175,7 +177,7 @@ class _ServerRuntime: class ConnectionResetByPeerAction: def __init__(self, with_partial_data=None): self.partial_data = "" - if with_partial_data is not None: + if with_partial_data is not None and with_partial_data == "1": self.partial_data = ( '\n' "\n" @@ -193,9 +195,7 @@ class _ServerRuntime: time.sleep(1) request_handler.connection.setsockopt( - socket.SOL_SOCKET, - socket.SO_LINGER, - struct.pack('ii', 1, 0) + socket.SOL_SOCKET, socket.SO_LINGER, struct.pack("ii", 1, 0) ) request_handler.connection.close() @@ -211,7 +211,9 @@ class _ServerRuntime: if self.action == "connection_refused": self.error_handler = _ServerRuntime.ConnectionRefusedAction() elif self.action == "connection_reset_by_peer": - self.error_handler = _ServerRuntime.ConnectionResetByPeerAction(*self.action_args) + self.error_handler = _ServerRuntime.ConnectionResetByPeerAction( + *self.action_args + ) elif self.action == "redirect_to": self.error_handler = _ServerRuntime.RedirectAction(*self.action_args) else: @@ -311,9 +313,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.read_all_input() self.send_response(307) - url = ( - f"http://{host}:{port}{self.path}" - ) + url = f"http://{host}:{port}{self.path}" self.log_message("redirect to %s", url) self.send_header("Location", url) self.end_headers() @@ -403,7 +403,9 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "at_object_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.at_object_upload = _ServerRuntime.CountAfter.from_cgi_params(params) + _runtime.at_object_upload = _ServerRuntime.CountAfter.from_cgi_params( + params + ) self.log_message("set at_object_upload %s", _runtime.at_object_upload) return self._ok() @@ -433,8 +435,13 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "at_create_multi_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.at_create_multi_part_upload = _ServerRuntime.CountAfter.from_cgi_params(params) - self.log_message("set at_create_multi_part_upload %s", _runtime.at_create_multi_part_upload) + _runtime.at_create_multi_part_upload = ( + _ServerRuntime.CountAfter.from_cgi_params(params) + ) + self.log_message( + "set at_create_multi_part_upload %s", + _runtime.at_create_multi_part_upload, + ) return self._ok() if path[1] == "reset": diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index b6b598c4557..5f7a2a6cc7f 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -172,9 +172,7 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( assert count_s3_errors >= 2 -def test_when_s3_connection_refused_at_write_retried( - cluster, broken_s3 -): +def test_when_s3_connection_refused_is_retried(cluster, broken_s3): node = cluster.instances["node"] broken_s3.setup_fake_multpartuploads() @@ -231,11 +229,14 @@ def test_when_s3_connection_refused_at_write_retried( ) assert "Code: 499" in error, error - assert "Poco::Exception. Code: 1000, e.code() = 111, Connection refused" in error, error + assert ( + "Poco::Exception. Code: 1000, e.code() = 111, Connection refused" in error + ), error -def test_when_s3_connection_reset_by_peer_at_write_retried( - cluster, broken_s3 +@pytest.mark.parametrize("send_something", [True, False]) +def test_when_s3_connection_reset_by_peer_at_upload_is_retried( + cluster, broken_s3, send_something ): node = cluster.instances["node"] @@ -244,14 +245,15 @@ def test_when_s3_connection_reset_by_peer_at_write_retried( count=3, after=2, action="connection_reset_by_peer", + action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_RETRIED" + insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD" node.query( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_write_retried', + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_upload_is_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -279,13 +281,14 @@ def test_when_s3_connection_reset_by_peer_at_write_retried( count=1000, after=2, action="connection_reset_by_peer", + action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_RETRIED_1" + insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_1" error = node.query_and_get_error( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_write_retried', + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_upload_is_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -301,12 +304,17 @@ def test_when_s3_connection_reset_by_peer_at_write_retried( query_id=insert_query_id, ) - assert "Code: 499" in error, error - assert "Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error, error + assert "Code: 1000" in error, error + assert ( + "DB::Exception: Connection reset by peer." in error + or + "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + ), error -def test_when_s3_connection_reset_by_peer_at_read_retried( - cluster, broken_s3 +@pytest.mark.parametrize("send_something", [True, False]) +def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( + cluster, broken_s3, send_something ): node = cluster.instances["node"] @@ -315,14 +323,15 @@ def test_when_s3_connection_reset_by_peer_at_read_retried( count=3, after=0, action="connection_reset_by_peer", + action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_READ_RETRIED" + insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD" node.query( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_read_retried', + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_create_mpu_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -350,14 +359,15 @@ def test_when_s3_connection_reset_by_peer_at_read_retried( count=1000, after=0, action="connection_reset_by_peer", + action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_RESET_BY_PEER_READ_RETRIED_1" + insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_1" error = node.query_and_get_error( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_read_retried', + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_create_mpu_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -373,5 +383,9 @@ def test_when_s3_connection_reset_by_peer_at_read_retried( query_id=insert_query_id, ) - assert "Code: 499" in error, error - assert "Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error, error + assert "Code: 1000" in error, error + assert ( + "DB::Exception: Connection reset by peer." in error + or + "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + ), error From f207e5bf1184e75c2769b687b9f5c08f85104384 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 24 Jul 2023 00:31:37 +0200 Subject: [PATCH 1058/2047] Add TODO --- src/Storages/S3Queue/S3QueueHolder.cpp | 3 +++ src/Storages/S3Queue/S3QueueSource.cpp | 2 ++ 2 files changed, 5 insertions(+) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 0900c84a2e8..7997fd2e051 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -49,6 +49,7 @@ void S3QueueHolder::S3QueueCollection::write(WriteBuffer & out) const out << processed_file.timestamp << "\n"; out << processed_file.retries_count << "\n"; } + /// todo(kssenii): use a more flexible format? } String S3QueueHolder::S3QueueCollection::toString() const @@ -104,6 +105,7 @@ void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) { files.erase(files.begin(), files.begin() + (files.size() - max_size)); } + /// todo(kssenii): use deque here } @@ -177,6 +179,7 @@ S3QueueHolder::S3QueueHolder( zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const { + /// todo(kssenii): current_zookeeper is not updated at all apart from in constructor, remove the lock? std::lock_guard lock(current_zookeeper_mutex); return current_zookeeper; } diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 151b63b8051..31eda8db31f 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -81,6 +81,7 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( , glob_iterator(std::make_unique( client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) { + /// todo(kssenii): remove this loop, it should not be here while (true) { KeyWithInfo val = glob_iterator->next(); @@ -321,6 +322,7 @@ void StorageS3QueueSource::deleteProcessedObject(const String & file_path) { LOG_WARNING(log, "Delete processed file {} from bucket {}", file_path, bucket); S3::DeleteObjectRequest request; + /// todo(kssenii): looks incorrect String delete_key = file_path.substr(bucket.length() + 1); request.WithKey(delete_key).WithBucket(bucket); From 7b4d0cf9d5b261eb68bd1db4021fcc350b907fc1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 00:51:20 +0200 Subject: [PATCH 1059/2047] Fix Darwin --- contrib/incbin-cmake/CMakeLists.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/contrib/incbin-cmake/CMakeLists.txt b/contrib/incbin-cmake/CMakeLists.txt index e64ebc99c73..8f4dad7e0d9 100644 --- a/contrib/incbin-cmake/CMakeLists.txt +++ b/contrib/incbin-cmake/CMakeLists.txt @@ -2,3 +2,7 @@ set(LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/incbin") add_library(_incbin INTERFACE) target_include_directories(_incbin SYSTEM INTERFACE ${LIBRARY_DIR}) add_library(ch_contrib::incbin ALIAS _incbin) + +# Warning "incbin is incompatible with bitcode. Using the library will break upload to App Store if you have bitcode enabled. +# Add `#define INCBIN_SILENCE_BITCODE_WARNING` before including this header to silence this warning." +target_compile_definitions(_inclin PUBLIC INCBIN_SILENCE_BITCODE_WARNING) From 641c086dbd771c14cc7db089e265ec508da9ccff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 00:53:11 +0200 Subject: [PATCH 1060/2047] Fix Darwin --- contrib/incbin-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/incbin-cmake/CMakeLists.txt b/contrib/incbin-cmake/CMakeLists.txt index 8f4dad7e0d9..5778cf83c22 100644 --- a/contrib/incbin-cmake/CMakeLists.txt +++ b/contrib/incbin-cmake/CMakeLists.txt @@ -5,4 +5,4 @@ add_library(ch_contrib::incbin ALIAS _incbin) # Warning "incbin is incompatible with bitcode. Using the library will break upload to App Store if you have bitcode enabled. # Add `#define INCBIN_SILENCE_BITCODE_WARNING` before including this header to silence this warning." -target_compile_definitions(_inclin PUBLIC INCBIN_SILENCE_BITCODE_WARNING) +target_compile_definitions(_incbin INTERFACE INCBIN_SILENCE_BITCODE_WARNING) From 40f5649811bb579b3cf8d634281f862675934773 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 01:02:11 +0200 Subject: [PATCH 1061/2047] Fix test --- .../no_allow_vertical_merges_from_compact_to_wide_parts.xml | 5 +++++ .../test_vertical_merges_from_compact_parts.py | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_backward_compatibility/configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml diff --git a/tests/integration/test_backward_compatibility/configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml b/tests/integration/test_backward_compatibility/configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml new file mode 100644 index 00000000000..c69be846c46 --- /dev/null +++ b/tests/integration/test_backward_compatibility/configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml @@ -0,0 +1,5 @@ + + + 0 + + diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index 82ffcc20b60..481621cacfe 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -15,7 +15,7 @@ node_old = cluster.add_instance( ) node_new = cluster.add_instance( "node2", - main_configs=["configs/no_compress_marks.xml"], + main_configs=["configs/no_compress_marks.xml", "configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml"], with_zookeeper=True, stay_alive=True, allow_analyzer=False, From dba7a0dffc4927a88c04cb7b9ec93faeeba40b3c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 23 Jul 2023 23:18:19 +0000 Subject: [PATCH 1062/2047] Automatic style fix --- .../test_vertical_merges_from_compact_parts.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py index 481621cacfe..9c9d1a4d312 100644 --- a/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py +++ b/tests/integration/test_backward_compatibility/test_vertical_merges_from_compact_parts.py @@ -15,7 +15,10 @@ node_old = cluster.add_instance( ) node_new = cluster.add_instance( "node2", - main_configs=["configs/no_compress_marks.xml", "configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml"], + main_configs=[ + "configs/no_compress_marks.xml", + "configs/no_allow_vertical_merges_from_compact_to_wide_parts.xml", + ], with_zookeeper=True, stay_alive=True, allow_analyzer=False, From f46568929dcecf5851ae6290bb3be4fc64e52308 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 24 Jul 2023 07:35:48 +0800 Subject: [PATCH 1063/2047] add doc --- docs/en/operations/system-tables/events.md | 1 + docs/en/operations/system-tables/metrics.md | 1 + 2 files changed, 2 insertions(+) diff --git a/docs/en/operations/system-tables/events.md b/docs/en/operations/system-tables/events.md index ba5602ee292..b53628a8384 100644 --- a/docs/en/operations/system-tables/events.md +++ b/docs/en/operations/system-tables/events.md @@ -10,6 +10,7 @@ Columns: - `event` ([String](../../sql-reference/data-types/string.md)) — Event name. - `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of events occurred. - `description` ([String](../../sql-reference/data-types/string.md)) — Event description. +- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for event. **Example** diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index 5a7dfd03eb4..ea36411cf3d 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -10,6 +10,7 @@ Columns: - `metric` ([String](../../sql-reference/data-types/string.md)) — Metric name. - `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — Metric value. - `description` ([String](../../sql-reference/data-types/string.md)) — Metric description. +- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for metric. The list of supported metrics you can find in the [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) source file of ClickHouse. From d7cdfb47d3795a3a09c2a204789c95e9726dc2b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 01:56:04 +0200 Subject: [PATCH 1064/2047] Fix merge --- src/IO/WriteHelpers.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 0494cdf22e7..76778543bd0 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -953,6 +953,11 @@ void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool { auto remainder = value % 10; value /= 10; + + if (remainder != 0 && last_nonzero_pos == 0) + last_nonzero_pos = pos; + + buf[pos] += static_cast(remainder); } writeChar('.', ostr); From 75efee9675f277fc3405ca5b256296aa406baca4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 05:34:00 +0200 Subject: [PATCH 1065/2047] Fix errors --- programs/install/CMakeLists.txt | 3 --- programs/server/CMakeLists.txt | 2 +- src/CMakeLists.txt | 3 +-- 3 files changed, 2 insertions(+), 6 deletions(-) diff --git a/programs/install/CMakeLists.txt b/programs/install/CMakeLists.txt index f3f562bab7c..c3f4d96d631 100644 --- a/programs/install/CMakeLists.txt +++ b/programs/install/CMakeLists.txt @@ -10,6 +10,3 @@ set (CLICKHOUSE_INSTALL_LINK ) clickhouse_program_add_library(install) - -# For incbin -target_include_directories(clickhouse-install-lib PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/../server") diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index e008e65acf6..b8241afa1eb 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -29,4 +29,4 @@ endif() clickhouse_program_add(server) -target_include_directories(clickhouse-server-lib PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}) +install(FILES config.xml users.xml DESTINATION "${CLICKHOUSE_ETC_DIR}/clickhouse-server" COMPONENT clickhouse) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index fda8bafde59..975bf9bb618 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -296,7 +296,7 @@ macro (dbms_target_include_directories) endforeach () endmacro () -dbms_target_include_directories (PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src" "${ClickHouse_SOURCE_DIR}/programs/server") +dbms_target_include_directories (PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") target_include_directories (clickhouse_common_io PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src") if (TARGET ch_contrib::llvm) @@ -561,7 +561,6 @@ if (ENABLE_NLP) dbms_target_link_libraries (PUBLIC ch_contrib::stemmer) dbms_target_link_libraries (PUBLIC ch_contrib::wnb) dbms_target_link_libraries (PUBLIC ch_contrib::lemmagen) - target_include_directories(clickhouse_common_io PUBLIC ${CMAKE_SOURCE_DIR}/contrib/nlp-data) endif() if (TARGET ch_contrib::ulid) From 169b9d5cc0c8dc54d31bc7229204b195f294c877 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 05:49:06 +0200 Subject: [PATCH 1066/2047] Fix tidy --- src/Functions/GregorianDate.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Functions/GregorianDate.cpp b/src/Functions/GregorianDate.cpp index da1172c8916..aaaeeb7339d 100644 --- a/src/Functions/GregorianDate.cpp +++ b/src/Functions/GregorianDate.cpp @@ -20,12 +20,12 @@ namespace ErrorCodes namespace { - static inline constexpr bool is_leap_year(int32_t year) + inline constexpr bool is_leap_year(int32_t year) { return (year % 4 == 0) && ((year % 400 == 0) || (year % 100 != 0)); } - static inline constexpr uint8_t monthLength(bool is_leap_year, uint8_t month) + inline constexpr uint8_t monthLength(bool is_leap_year, uint8_t month) { switch (month) { @@ -49,7 +49,7 @@ namespace /** Integer division truncated toward negative infinity. */ template - static inline constexpr I div(I x, J y) + inline constexpr I div(I x, J y) { const auto y_cast = static_cast(y); if (x > 0 && y_cast < 0) @@ -63,7 +63,7 @@ namespace /** Integer modulus, satisfying div(x, y)*y + mod(x, y) == x. */ template - static inline constexpr I mod(I x, J y) + inline constexpr I mod(I x, J y) { const auto y_cast = static_cast(y); const auto r = x % y_cast; @@ -76,13 +76,13 @@ namespace /** Like std::min(), but the type of operands may differ. */ template - static inline constexpr I min(I x, J y) + inline constexpr I min(I x, J y) { const auto y_cast = static_cast(y); return x < y_cast ? x : y_cast; } - static inline char readDigit(ReadBuffer & in) + inline char readDigit(ReadBuffer & in) { char c; if (!in.read(c)) @@ -93,7 +93,7 @@ namespace return c - '0'; } - static inline bool tryReadDigit(ReadBuffer & in, char & c) + inline bool tryReadDigit(ReadBuffer & in, char & c) { if (in.read(c) && c >= '0' && c <= '9') { From d7f7f16fbcfa8063e295708b4feb3b0079ad05f0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 10:44:13 +0200 Subject: [PATCH 1067/2047] Introduce IStorage::supportsTrivialCountOptimization() Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterSelectQuery.cpp | 3 +-- src/Planner/PlannerJoinTree.cpp | 3 +++ src/Storages/IStorage.h | 3 +++ src/Storages/MergeTree/MergeTreeData.h | 2 ++ src/Storages/StorageMaterializedMySQL.h | 2 ++ 5 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d07a6521544..fc3ea3a13ca 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2274,8 +2274,7 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle && !settings.allow_experimental_query_deduplication && !settings.empty_result_for_aggregation_by_empty_set && storage - && storage->getName() != "MaterializedMySQL" - && !storage->hasLightweightDeletedMask() + && storage->supportsTrivialCountOptimization() && query_info.filter_asts.empty() && query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 5d8f8ca8741..c118fccded4 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -182,6 +182,9 @@ bool applyTrivialCountIfPossible( return false; const auto & storage = table_node.getStorage(); + if (!storage->supportsTrivialCountOptimization()) + return false; + auto storage_id = storage->getStorageID(); auto row_policy_filter = query_context->getRowPolicyFilter(storage_id.getDatabaseName(), storage_id.getTableName(), diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 76641b656a2..701e02a85ac 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -254,6 +254,9 @@ public: /// because those are internally translated into 'ALTER UDPATE' mutations. virtual bool supportsDelete() const { return false; } + /// Return true if the trivial count query could be optimized without reading the data at all. + virtual bool supportsTrivialCountOptimization() const { return false; } + private: StorageID storage_id; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 41fc4657854..5e6b043c31c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -434,6 +434,8 @@ public: bool areAsynchronousInsertsEnabled() const override { return getSettings()->async_insert; } + bool supportsTrivialCountOptimization() const override { return !hasLightweightDeletedMask(); } + NamesAndTypesList getVirtuals() const override; bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, ContextPtr, const StorageMetadataPtr & metadata_snapshot) const override; diff --git a/src/Storages/StorageMaterializedMySQL.h b/src/Storages/StorageMaterializedMySQL.h index 08fbb61960f..e6fcbc203e6 100644 --- a/src/Storages/StorageMaterializedMySQL.h +++ b/src/Storages/StorageMaterializedMySQL.h @@ -41,6 +41,8 @@ public: void drop() override { nested_storage->drop(); } + bool supportsTrivialCountOptimization() const override { return false; } + private: [[noreturn]] static void throwNotAllowed() { From a0070eda02736903b984518daf3d1c79bfe5fd94 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 10:48:21 +0200 Subject: [PATCH 1068/2047] Slightly optimize code in ClusterProxy::executeQuery() Signed-off-by: Azat Khuzhin --- src/Interpreters/ClusterProxy/executeQuery.cpp | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 3dea52faf46..5efba383e4b 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -176,11 +176,9 @@ void executeQuery( size_t shards = query_info.getCluster()->getShardCount(); for (const auto & shard_info : query_info.getCluster()->getShardsInfo()) { - ASTPtr query_ast_for_shard; - if (query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1) + ASTPtr query_ast_for_shard = query_ast->clone(); + if (sharding_key_expr && query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1) { - query_ast_for_shard = query_ast->clone(); - OptimizeShardingKeyRewriteInVisitor::Data visitor_data{ sharding_key_expr, sharding_key_expr->getSampleBlock().getByPosition(0).type, @@ -191,8 +189,6 @@ void executeQuery( OptimizeShardingKeyRewriteInVisitor visitor(visitor_data); visitor.visit(query_ast_for_shard); } - else - query_ast_for_shard = query_ast->clone(); if (shard_filter_generator) { From 67095d2150cafc91c0eebea4a17a8dc5f17b307c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 10:48:58 +0200 Subject: [PATCH 1069/2047] Fix comment for function argument in TableFunctionRemote Signed-off-by: Azat Khuzhin --- src/TableFunctions/TableFunctionRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 4143014a7b3..e6d72ddf17b 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -264,7 +264,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr secure, /* priority= */ Priority{1}, /* cluster_name= */ "", - /* password= */ "" + /* cluster_secret= */ "" }; cluster = std::make_shared(context->getSettingsRef(), names, params); } From b22247609036020e9bc4da64f1a297e49c29edfa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 14:19:08 +0200 Subject: [PATCH 1070/2047] Add ability to pass table for connections checks per-shard to ReadFromRemote Signed-off-by: Azat Khuzhin --- src/Interpreters/ClusterProxy/SelectStreamFactory.cpp | 1 + src/Interpreters/ClusterProxy/SelectStreamFactory.h | 2 ++ src/Processors/QueryPlan/ReadFromRemote.cpp | 6 ++++-- src/Processors/QueryPlan/ReadFromRemote.h | 1 + 4 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 0cf3f360994..953e38d56cd 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -124,6 +124,7 @@ void SelectStreamFactory::createForShard( { remote_shards.emplace_back(Shard{ .query = query_ast, + .main_table = main_table, .header = header, .shard_info = shard_info, .lazy = lazy, diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 030c0b77dd5..1cc5a3b1a77 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -50,6 +50,8 @@ public: { /// Query and header may be changed depending on shard. ASTPtr query; + /// Used to check the table existence on remote node + StorageID main_table; Block header; Cluster::ShardInfo shard_info; diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 5cc13f45df4..7a99c363232 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -162,7 +162,9 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream if (my_table_func_ptr) try_results = my_shard.shard_info.pool->getManyForTableFunction(timeouts, ¤t_settings, PoolMode::GET_MANY); else - try_results = my_shard.shard_info.pool->getManyChecked(timeouts, ¤t_settings, PoolMode::GET_MANY, my_main_table.getQualifiedName()); + try_results = my_shard.shard_info.pool->getManyChecked( + timeouts, ¤t_settings, PoolMode::GET_MANY, + my_shard.main_table ? my_shard.main_table.getQualifiedName() : my_main_table.getQualifiedName()); } catch (const Exception & ex) { @@ -241,7 +243,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact remote_query_executor->setPoolMode(PoolMode::GET_MANY); if (!table_func_ptr) - remote_query_executor->setMainTable(main_table); + remote_query_executor->setMainTable(shard.main_table ? shard.main_table : main_table); pipes.emplace_back(createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending)); addConvertingActions(pipes.back(), output_stream->header); diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index d4005d81f1b..ac869cd89f9 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -22,6 +22,7 @@ using ThrottlerPtr = std::shared_ptr; class ReadFromRemote final : public ISourceStep { public: + /// @param main_table_ if Shards contains main_table then this parameter will be ignored ReadFromRemote( ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, From 83c0f03b98d6b3cbd10f9690256aed2fada47177 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 14:21:53 +0200 Subject: [PATCH 1071/2047] Change signature of the updateSettingsForCluster() to avoid cluster requirement Signed-off-by: Azat Khuzhin --- src/Interpreters/ClusterProxy/executeQuery.cpp | 11 ++++++++--- src/Interpreters/ClusterProxy/executeQuery.h | 8 ++++++-- src/Storages/getStructureOfRemoteTable.cpp | 4 ++-- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 5efba383e4b..2fed626ffb7 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -35,7 +35,12 @@ namespace ErrorCodes namespace ClusterProxy { -ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table, const SelectQueryInfo * query_info, Poco::Logger * log) +ContextMutablePtr updateSettingsForCluster(bool interserver_mode, + ContextPtr context, + const Settings & settings, + const StorageID & main_table, + const SelectQueryInfo * query_info, + Poco::Logger * log) { Settings new_settings = settings; new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time); @@ -43,7 +48,7 @@ ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr c /// If "secret" (in remote_servers) is not in use, /// user on the shard is not the same as the user on the initiator, /// hence per-user limits should not be applied. - if (cluster.getSecret().empty()) + if (!interserver_mode) { /// Does not matter on remote servers, because queries are sent under different user. new_settings.max_concurrent_queries_for_user = 0; @@ -170,7 +175,7 @@ void executeQuery( std::vector plans; SelectStreamFactory::Shards remote_shards; - auto new_context = updateSettingsForCluster(*query_info.getCluster(), context, settings, main_table, &query_info, log); + auto new_context = updateSettingsForCluster(!query_info.getCluster()->getSecret().empty(), context, settings, main_table, &query_info, log); new_context->increaseDistributedDepth(); size_t shards = query_info.getCluster()->getShardCount(); diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 41f6da55686..511914e99e4 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -34,8 +34,12 @@ class SelectStreamFactory; /// - optimize_skip_unused_shards_nesting /// /// @return new Context with adjusted settings -ContextMutablePtr updateSettingsForCluster( - const Cluster & cluster, ContextPtr context, const Settings & settings, const StorageID & main_table, const SelectQueryInfo * query_info = nullptr, Poco::Logger * log = nullptr); +ContextMutablePtr updateSettingsForCluster(bool interserver_mode, + ContextPtr context, + const Settings & settings, + const StorageID & main_table, + const SelectQueryInfo * query_info = nullptr, + Poco::Logger * log = nullptr); using AdditionalShardFilterGenerator = std::function; /// Execute a distributed query, creating a query plan, from which the query pipeline can be built. diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index e5fc01be9f4..cbed05e30ed 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -58,7 +58,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( } ColumnsDescription res; - auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context->getSettingsRef(), table_id); + auto new_context = ClusterProxy::updateSettingsForCluster(!cluster.getSecret().empty(), context, context->getSettingsRef(), table_id); /// Ignore limit for result number of rows (that could be set during handling CSE/CTE), /// since this is a service query and should not lead to query failure. @@ -177,7 +177,7 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables( const auto & shards_info = cluster.getShardsInfo(); auto query = "DESC TABLE " + remote_table_id.getFullTableName(); - auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context->getSettingsRef(), remote_table_id); + auto new_context = ClusterProxy::updateSettingsForCluster(!cluster.getSecret().empty(), context, context->getSettingsRef(), remote_table_id); new_context->setSetting("describe_extend_object_types", true); /// Expect only needed columns from the result of DESC TABLE. From 323128df6f3c779f3b2fe4a751fa98372a54fbbb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Jun 2023 15:02:32 +0200 Subject: [PATCH 1072/2047] Remove non existing ctor of Cluster::Address Signed-off-by: Azat Khuzhin --- src/Interpreters/Cluster.h | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index de10a445d01..b90acd1d576 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -144,12 +144,6 @@ public: UInt32 shard_index_ = 0, UInt32 replica_index_ = 0); - Address( - const String & host_port_, - const ClusterConnectionParameters & params, - UInt32 shard_index_, - UInt32 replica_index_); - Address( const DatabaseReplicaInfo & info, const ClusterConnectionParameters & params, From 4a33e027c518f51d120c60b21ccd962264e1356a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 20 Jun 2023 17:31:45 +0200 Subject: [PATCH 1073/2047] Split StorageReplicatedMergeTree reading methods Signed-off-by: Azat Khuzhin --- src/Storages/StorageReplicatedMergeTree.cpp | 141 ++++++++++++-------- src/Storages/StorageReplicatedMergeTree.h | 32 ++++- 2 files changed, 119 insertions(+), 54 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 13c0fb3f7c2..4e053c4598c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4902,67 +4902,102 @@ void StorageReplicatedMergeTree::read( snapshot_data.alter_conversions = {}; }); - /** The `select_sequential_consistency` setting has two meanings: - * 1. To throw an exception if on a replica there are not all parts which have been written down on quorum of remaining replicas. - * 2. Do not read parts that have not yet been written to the quorum of the replicas. - * For this you have to synchronously go to ZooKeeper. - */ - if (local_context->getSettingsRef().select_sequential_consistency) - { - auto max_added_blocks = std::make_shared(getMaxAddedBlocks()); - if (auto plan = reader.read( - column_names, storage_snapshot, query_info, local_context, - max_block_size, num_streams, processed_stage, std::move(max_added_blocks), /*enable_parallel_reading*/false)) - query_plan = std::move(*plan); - return; - } + const auto & settings = local_context->getSettingsRef(); + + /// The `select_sequential_consistency` setting has two meanings: + /// 1. To throw an exception if on a replica there are not all parts which have been written down on quorum of remaining replicas. + /// 2. Do not read parts that have not yet been written to the quorum of the replicas. + /// For this you have to synchronously go to ZooKeeper. + if (settings.select_sequential_consistency) + return readLocalSequentialConsistencyImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); if (local_context->canUseParallelReplicasOnInitiator()) + return readParallelReplicasImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); + + readLocalImpl(query_plan, column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); +} + +void StorageReplicatedMergeTree::readLocalSequentialConsistencyImpl( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) +{ + auto max_added_blocks = std::make_shared(getMaxAddedBlocks()); + auto plan = reader.read(column_names, storage_snapshot, query_info, local_context, + max_block_size, num_streams, processed_stage, std::move(max_added_blocks), + /* enable_parallel_reading= */false); + if (plan) + query_plan = std::move(*plan); +} + +void StorageReplicatedMergeTree::readParallelReplicasImpl( + QueryPlan & query_plan, + const Names & /*column_names*/, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + const size_t /*max_block_size*/, + const size_t /*num_streams*/) +{ + auto table_id = getStorageID(); + + auto parallel_replicas_cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); + + ASTPtr modified_query_ast; + Block header; + if (local_context->getSettingsRef().allow_experimental_analyzer) { - auto table_id = getStorageID(); + auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree); - ASTPtr modified_query_ast; - - Block header; - - if (local_context->getSettingsRef().allow_experimental_analyzer) - { - auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree); - - header = InterpreterSelectQueryAnalyzer::getSampleBlock( - modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); - modified_query_ast = queryNodeToSelectQuery(modified_query_tree); - } - else - { - modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, - table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - header - = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); - } - - auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); - - ClusterProxy::SelectStreamFactory select_stream_factory = - ClusterProxy::SelectStreamFactory( - header, - {}, - storage_snapshot, - processed_stage); - - ClusterProxy::executeQueryWithParallelReplicas( - query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, - select_stream_factory, modified_query_ast, - local_context, query_info, cluster); + header = InterpreterSelectQueryAnalyzer::getSampleBlock( + modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); + modified_query_ast = queryNodeToSelectQuery(modified_query_tree); } else { - if (auto plan = reader.read( - column_names, storage_snapshot, query_info, - local_context, max_block_size, num_streams, - processed_stage, nullptr, /*enable_parallel_reading*/local_context->canUseParallelReplicasOnFollower())) - query_plan = std::move(*plan); + modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, + table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); + header + = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); } + + ClusterProxy::SelectStreamFactory select_stream_factory = ClusterProxy::SelectStreamFactory( + header, + {}, + storage_snapshot, + processed_stage); + + ClusterProxy::executeQueryWithParallelReplicas( + query_plan, getStorageID(), + /* table_func_ptr= */ nullptr, + select_stream_factory, modified_query_ast, + local_context, query_info, parallel_replicas_cluster); +} + +void StorageReplicatedMergeTree::readLocalImpl( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + const size_t max_block_size, + const size_t num_streams) +{ + auto plan = reader.read( + column_names, storage_snapshot, query_info, + local_context, max_block_size, num_streams, + processed_stage, + /* max_block_numbers_to_read= */ nullptr, + /* enable_parallel_reading= */ local_context->canUseParallelReplicasOnFollower()); + if (plan) + query_plan = std::move(*plan); } template diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 1a1b3c3b10c..ded940bc1d2 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -130,7 +130,7 @@ public: const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, + ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams) override; @@ -513,6 +513,36 @@ private: static std::optional distributedWriteFromClusterStorage(const std::shared_ptr & src_storage_cluster, const ASTInsertQuery & query, ContextPtr context); + void readLocalImpl( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams); + + void readLocalSequentialConsistencyImpl( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams); + + void readParallelReplicasImpl( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams); + template void foreachActiveParts(Func && func, bool select_sequential_consistency) const; From b22313ef2d721ec0f8687515de58f4e2ba785d1d Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 24 Jul 2023 03:54:34 +0000 Subject: [PATCH 1074/2047] Replace with three way comparison --- src/Common/IntervalTree.h | 34 ++-------------------------------- 1 file changed, 2 insertions(+), 32 deletions(-) diff --git a/src/Common/IntervalTree.h b/src/Common/IntervalTree.h index 2214a4e842d..ad079a312f2 100644 --- a/src/Common/IntervalTree.h +++ b/src/Common/IntervalTree.h @@ -27,39 +27,9 @@ struct Interval }; template -bool operator<(const Interval & lhs, const Interval & rhs) +auto operator<=>(const Interval & lhs, const Interval & rhs) { - return std::tie(lhs.left, lhs.right) < std::tie(rhs.left, rhs.right); -} - -template -bool operator<=(const Interval & lhs, const Interval & rhs) -{ - return std::tie(lhs.left, lhs.right) <= std::tie(rhs.left, rhs.right); -} - -template -bool operator==(const Interval & lhs, const Interval & rhs) -{ - return std::tie(lhs.left, lhs.right) == std::tie(rhs.left, rhs.right); -} - -template -bool operator!=(const Interval & lhs, const Interval & rhs) -{ - return std::tie(lhs.left, lhs.right) != std::tie(rhs.left, rhs.right); -} - -template -bool operator>(const Interval & lhs, const Interval & rhs) -{ - return std::tie(lhs.left, lhs.right) > std::tie(rhs.left, rhs.right); -} - -template -bool operator>=(const Interval & lhs, const Interval & rhs) -{ - return std::tie(lhs.left, lhs.right) >= std::tie(rhs.left, rhs.right); + return std::tie(lhs.left, lhs.right) <=> std::tie(rhs.left, rhs.right); } struct IntervalTreeVoidValue From ac54be9652414e10a1b79ec4f92439db5155310b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 05:56:18 +0200 Subject: [PATCH 1075/2047] Fix a test --- tests/integration/test_backward_compatibility/test_functions.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index fa24b146fec..c86c3ba0ab2 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -143,6 +143,7 @@ def test_string_functions(start_cluster): "position", "substring", "CAST", + "getTypeSerializationStreams", # NOTE: no need to ignore now()/now64() since they will fail because they don't accept any argument # 22.8 Backward Incompatible Change: Extended range of Date32 "toDate32OrZero", From 2389e0f0b68d03ecbb117745ed00c54979715ea7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 09:54:51 +0200 Subject: [PATCH 1076/2047] Randomize timezone in tests across non-deterministic around 1970 and default There was some cases when some patches to the datetime code leads to flaky tests, due to the tests itself had been runned using regular timezone (TZ). But if you will this tests with something "specific" (that is not strictly defined around 1970 year), those tests will fail. So to catch such issues in the PRs itself, let's randomize session_timezone as well. Signed-off-by: Azat Khuzhin --- docker/test/stateless/run.sh | 3 +++ tests/clickhouse-test | 19 +++++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index fe53925ecc8..3694fb7c2f6 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -4,6 +4,9 @@ set -e -x -a # Choose random timezone for this test run. +# +# NOTE: that clickhouse-test will randomize session_timezone by itself as well +# (it will choose between default server timezone and something specific). TZ="$(rg -v '#' /usr/share/zoneinfo/zone.tab | awk '{print $3}' | shuf | head -n1)" echo "Choosen random timezone $TZ" ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone diff --git a/tests/clickhouse-test b/tests/clickhouse-test index abd109d00b2..185e3003c95 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -529,6 +529,12 @@ def threshold_generator(always_on_prob, always_off_prob, min_val, max_val): return gen +# To keep dependency list as short as possible, tzdata is not used here (to +# avoid try/except block for import) +def get_localzone(): + return os.getenv("TZ", "/".join(os.readlink("/etc/localtime").split("/")[-2:])) + + class SettingsRandomizer: settings = { "max_insert_threads": lambda: 0 @@ -602,6 +608,19 @@ class SettingsRandomizer: "enable_memory_bound_merging_of_aggregation_results": lambda: random.randint( 0, 1 ), + "session_timezone": lambda: random.choice( + [ + # special non-deterministic around 1970 timezone, see [1]. + # + # [1]: https://github.com/ClickHouse/ClickHouse/issues/42653 + "America/Mazatlan", + "America/Hermosillo", + "Mexico/BajaSur", + # server default that is randomized across all timezones + # NOTE: due to lots of trickery we cannot use empty timezone here, but this should be the same. + get_localzone(), + ] + ), } @staticmethod From bc167dfde81c44bb93ee7dd0c634ff3428ea3c33 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 18 Jul 2023 06:20:05 +0200 Subject: [PATCH 1077/2047] clickhouse-test: add proper escaping for HTTP parameters The problem is that old versions of cURL (7.81.0 at least) handle additional parameters incorrectly if in previous parameter was "/": $ docker run --rm curlimages/curl:8.1.2 --http1.1 --get -vvv 'http://kernel.org/?bar=foo/baz' --data-urlencode "query=select 1 format Null"; echo > GET /?bar=foo/baz&query=select+1+format+Null HTTP/1.1 > User-Agent: curl/8.1.2 $ docker run --rm curlimages/curl:7.81.0 --http1.1 --get -vvv 'http://kernel.org/?bar=foo/baz' --data-urlencode "query=select 1 format Null"; echo > GET /?bar=foo/baz?query=select+1+format+Null HTTP/1.1 > User-Agent: curl/7.81.0-DEV Note, that I thought about making the same for cli, but it is not that easy, even after getting rid of sh -c and string contantenation, it still cannot be done for CLICKHOUSE_CLIENT_OPT. Signed-off-by: Azat Khuzhin --- tests/clickhouse-test | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 185e3003c95..c63e1e3ae52 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -625,16 +625,16 @@ class SettingsRandomizer: @staticmethod def get_random_settings(args): - random_settings = [] + random_settings = {} is_debug = BuildFlags.DEBUG in args.build_flags for setting, generator in SettingsRandomizer.settings.items(): if ( is_debug and setting == "allow_prefetched_read_pool_for_remote_filesystem" ): - random_settings.append(f"{setting}=0") + random_settings[setting] = 0 else: - random_settings.append(f"{setting}={generator()}") + random_settings[setting] = generator() return random_settings @@ -670,10 +670,10 @@ class MergeTreeSettingsRandomizer: @staticmethod def get_random_settings(args): - random_settings = [] + random_settings = {} for setting, generator in MergeTreeSettingsRandomizer.settings.items(): if setting not in args.changed_merge_tree_settings: - random_settings.append(f"{setting}={generator()}") + random_settings[setting] = generator() return random_settings @@ -785,7 +785,14 @@ class TestCase: @staticmethod def cli_format_settings(settings_list) -> str: - return " ".join([f"--{setting}" for setting in settings_list]) + out = [] + for k, v in settings_list.items(): + out.extend([f"--{k}", str(v)]) + return " ".join(out) + + @staticmethod + def http_format_settings(settings_list) -> str: + return urllib.parse.urlencode(settings_list) def has_show_create_table_in_test(self): return not subprocess.call(["grep", "-iq", "show create", self.case_file]) @@ -793,11 +800,12 @@ class TestCase: def add_random_settings(self, client_options): new_options = "" if self.randomize_settings: + http_params = self.http_format_settings(self.random_settings) if len(self.base_url_params) == 0: - os.environ["CLICKHOUSE_URL_PARAMS"] = "&".join(self.random_settings) + os.environ["CLICKHOUSE_URL_PARAMS"] = http_params else: os.environ["CLICKHOUSE_URL_PARAMS"] = ( - self.base_url_params + "&" + "&".join(self.random_settings) + self.base_url_params + "&" + http_params ) new_options += f" {self.cli_format_settings(self.random_settings)}" From 6ae4d291800c7d9b32622f1d520f1ab27b9f90b7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 19 Jul 2023 13:22:31 +0200 Subject: [PATCH 1078/2047] Fix tests after session_timezone randomization Signed-off-by: Azat Khuzhin --- .../0_stateless/00387_use_client_time_zone.sh | 3 ++- tests/queries/0_stateless/00427_alter_primary_key.sh | 11 ++++++----- tests/queries/0_stateless/00933_ttl_simple.sql | 12 ++++++++++++ ...42_system_reload_dictionary_reloads_completely.sh | 4 ++-- .../0_stateless/01070_modify_ttl_recalc_only.sql | 3 +++ .../0_stateless/02530_dictionaries_update_field.sh | 3 ++- 6 files changed, 27 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00387_use_client_time_zone.sh b/tests/queries/0_stateless/00387_use_client_time_zone.sh index 2a6d81eebfe..e54d5244eef 100755 --- a/tests/queries/0_stateless/00387_use_client_time_zone.sh +++ b/tests/queries/0_stateless/00387_use_client_time_zone.sh @@ -5,4 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -env TZ=UTC ${CLICKHOUSE_CLIENT} --use_client_time_zone=1 --query="SELECT toDateTime(1000000000)" +# NOTE: session_timezone overrides use_client_time_zone, disable it randomization +env TZ=UTC ${CLICKHOUSE_CLIENT} --session_timezone '' --use_client_time_zone=1 --query="SELECT toDateTime(1000000000)" diff --git a/tests/queries/0_stateless/00427_alter_primary_key.sh b/tests/queries/0_stateless/00427_alter_primary_key.sh index 1269e2ad6e3..f9984384d79 100755 --- a/tests/queries/0_stateless/00427_alter_primary_key.sh +++ b/tests/queries/0_stateless/00427_alter_primary_key.sh @@ -7,11 +7,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function perform() { local query=$1 - TZ=UTC $CLICKHOUSE_CLIENT \ - --allow_deprecated_syntax_for_merge_tree=1 \ - --use_client_time_zone=1 \ - --input_format_values_interpret_expressions=0 \ - --query "$query" 2>/dev/null + local settings=( + --allow_deprecated_syntax_for_merge_tree 1 + --session_timezone UTC + --input_format_values_interpret_expressions 0 + ) + TZ=UTC $CLICKHOUSE_CLIENT "${settings[@]}" --query "$query" 2>/dev/null if [ "$?" -ne 0 ]; then echo "query failed" fi diff --git a/tests/queries/0_stateless/00933_ttl_simple.sql b/tests/queries/0_stateless/00933_ttl_simple.sql index 2bf686822d5..ad40e7c7e47 100644 --- a/tests/queries/0_stateless/00933_ttl_simple.sql +++ b/tests/queries/0_stateless/00933_ttl_simple.sql @@ -1,3 +1,15 @@ +-- disable timezone randomization since otherwise TTL may fail at particular datetime, i.e.: +-- +-- SELECT +-- now(), +-- toDate(toTimeZone(now(), 'America/Mazatlan')), +-- today() +-- +-- ┌───────────────now()─┬─toDate(toTimeZone(now(), 'America/Mazatlan'))─┬────today()─┐ +-- │ 2023-07-24 06:24:06 │ 2023-07-23 │ 2023-07-24 │ +-- └─────────────────────┴───────────────────────────────────────────────┴────────────┘ +set session_timezone = ''; + drop table if exists ttl_00933_1; -- Column TTL works only with wide parts, because it's very expensive to apply it for compact parts diff --git a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh index f2b30e05040..9d34470c38d 100755 --- a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh +++ b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh @@ -7,8 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -o pipefail -# Run the client. -$CLICKHOUSE_CLIENT --multiquery <<'EOF' +# NOTE: dictionaries TTLs works with server timezone, so session_timeout cannot be used +$CLICKHOUSE_CLIENT --session_timezone '' --multiquery <<'EOF' DROP DATABASE IF EXISTS dictdb_01042; CREATE DATABASE dictdb_01042; CREATE TABLE dictdb_01042.table(x Int64, y Int64, insert_time DateTime) ENGINE = MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql index 247e412484f..7ac70d41871 100644 --- a/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql +++ b/tests/queries/0_stateless/01070_modify_ttl_recalc_only.sql @@ -2,6 +2,9 @@ set mutations_sync = 2; +-- system.parts has server default, timezone cannot be randomized +set session_timezone = ''; + drop table if exists ttl; create table ttl (d Date, a Int) engine = MergeTree order by a partition by toDayOfMonth(d) diff --git a/tests/queries/0_stateless/02530_dictionaries_update_field.sh b/tests/queries/0_stateless/02530_dictionaries_update_field.sh index 569466fe606..6ac10ea2308 100755 --- a/tests/queries/0_stateless/02530_dictionaries_update_field.sh +++ b/tests/queries/0_stateless/02530_dictionaries_update_field.sh @@ -5,7 +5,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q " +# NOTE: dictionaries will be updated according to server TZ, not session, so prohibit it's randomization +$CLICKHOUSE_CLIENT --session_timezone '' -q " CREATE TABLE table_for_update_field_dictionary ( key UInt64, From eb8e7f49f7579016069ea1b4ca37c78eef1fb831 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Thu, 13 Jul 2023 22:44:31 +0800 Subject: [PATCH 1079/2047] Rename: extend the tests to QueryTree validation --- ...date_predicate_optimizations_ast_query_tree_rewrite.reference} | 0 ...02785_date_predicate_optimizations_ast_query_tree_rewrite.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02785_date_predicate_optimizations_ast_rewrite.reference => 02785_date_predicate_optimizations_ast_query_tree_rewrite.reference} (100%) rename tests/queries/0_stateless/{02785_date_predicate_optimizations_ast_rewrite.sql => 02785_date_predicate_optimizations_ast_query_tree_rewrite.sql} (100%) diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference similarity index 100% rename from tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference rename to tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.sql similarity index 100% rename from tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql rename to tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.sql From b88fc4277a0a85b1b6d590d26e1b796cc5d6c138 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Thu, 13 Jul 2023 22:48:05 +0800 Subject: [PATCH 1080/2047] Add a pass in Analyzer for time filter optimization with preimage The preimage has been identified as the general solution to the optimization of predicates with time converters (#15257). PR #50951 implemented this solution via the AST rewrite. As a follow-up, this commit extends the optimization to the experi- mental analyzer by replacing the sub-QueryTree of the time filter with its preimage-transformed one. The optimization is implemented as a new pass in the Analyzer. --- ...ateOrDateTimeConverterWithPreimagePass.cpp | 218 ++++ ...eDateOrDateTimeConverterWithPreimagePass.h | 24 + src/Analyzer/QueryTreePassManager.cpp | 2 + ...783_date_predicate_optimizations.reference | 50 + .../02783_date_predicate_optimizations.sql | 50 + ...mizations_ast_query_tree_rewrite.reference | 1035 +++++++++++++++++ ...e_optimizations_ast_query_tree_rewrite.sql | 28 + 7 files changed, 1407 insertions(+) create mode 100644 src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp create mode 100644 src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.h diff --git a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp new file mode 100644 index 00000000000..f7f69598c4b --- /dev/null +++ b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp @@ -0,0 +1,218 @@ +#include + +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ + +class OptimizeDateOrDateTimeConverterWithPreimageVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + + explicit OptimizeDateOrDateTimeConverterWithPreimageVisitor(ContextPtr context) + : Base(std::move(context)) + {} + + static bool needChildVisit(QueryTreeNodePtr & node, QueryTreeNodePtr & /*child*/) + { + const static std::unordered_set relations = { + "equals", + "notEquals", + "less", + "greater", + "lessOrEquals", + "greaterOrEquals", + }; + + if (const auto * function = node->as()) + { + return !relations.contains(function->getFunctionName()); + } + + return true; + } + + void visitImpl(QueryTreeNodePtr & node) const + { + const static std::unordered_map swap_relations = { + {"equals", "equals"}, + {"notEquals", "notEquals"}, + {"less", "greater"}, + {"greater", "less"}, + {"lessOrEquals", "greaterOrEquals"}, + {"greaterOrEquals", "lessOrEquals"}, + }; + + const auto * function = node->as(); + + if (!function || !swap_relations.contains(function->getFunctionName())) return; + + if (function->getArguments().getNodes().size() != 2) return; + + size_t func_id = function->getArguments().getNodes().size(); + + for (size_t i = 0; i < function->getArguments().getNodes().size(); i++) + { + if (const auto * func = function->getArguments().getNodes()[i]->as()) + { + func_id = i; + } + } + + if (func_id == function->getArguments().getNodes().size()) return; + + size_t literal_id = 1 - func_id; + const auto * literal = function->getArguments().getNodes()[literal_id]->as(); + + if (!literal || literal->getValue().getType() != Field::Types::UInt64) return; + + String comparator = literal_id > func_id ? function->getFunctionName(): swap_relations.at(function->getFunctionName()); + + const auto * func_node = function->getArguments().getNodes()[func_id]->as(); + /// Currently we only handle single-argument functions. + if (!func_node || func_node->getArguments().getNodes().size() != 1) return; + + const auto * column_id = func_node->getArguments().getNodes()[0]->as(); + if (!column_id) return; + + const auto & converter = FunctionFactory::instance().tryGet(func_node->getFunctionName(), getContext()); + if (!converter) return; + + ColumnsWithTypeAndName args; + args.emplace_back(column_id->getColumnType(), "tmp"); + auto converter_base = converter->build(args); + if (!converter_base || !converter_base->hasInformationAboutPreimage()) return; + + auto preimage_range = converter_base->getPreimage(*(column_id->getColumnType()), literal->getValue()); + if (!preimage_range) return; + + const auto new_node = generateOptimizedDateFilter(comparator, *column_id, *preimage_range); + + if (!new_node) return; + + node = new_node; + } + +private: + QueryTreeNodePtr generateOptimizedDateFilter(const String & comparator, const ColumnNode & column_node, const std::pair& range) const + { + const DateLUTImpl & date_lut = DateLUT::instance("UTC"); + + String start_date_or_date_time; + String end_date_or_date_time; + + if (isDateOrDate32(column_node.getColumnType().get())) + { + start_date_or_date_time = date_lut.dateToString(range.first.get()); + end_date_or_date_time = date_lut.dateToString(range.second.get()); + } + else if (isDateTime(column_node.getColumnType().get()) || isDateTime64(column_node.getColumnType().get())) + { + start_date_or_date_time = date_lut.timeToString(range.first.get()); + end_date_or_date_time = date_lut.timeToString(range.second.get()); + } + else [[unlikely]] return {}; + + if (comparator == "equals") + { + const auto lhs = std::make_shared("greaterOrEquals"); + lhs->getArguments().getNodes().push_back(std::make_shared(column_node.getColumn(), column_node.getColumnSource())); + lhs->getArguments().getNodes().push_back(std::make_shared(start_date_or_date_time)); + resolveOrdinaryFunctionNode(*lhs, lhs->getFunctionName()); + + const auto rhs = std::make_shared("less"); + rhs->getArguments().getNodes().push_back(std::make_shared(column_node.getColumn(), column_node.getColumnSource())); + rhs->getArguments().getNodes().push_back(std::make_shared(end_date_or_date_time)); + resolveOrdinaryFunctionNode(*rhs, rhs->getFunctionName()); + + const auto new_date_filter = std::make_shared("and"); + new_date_filter->getArguments().getNodes() = {lhs, rhs}; + resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName()); + + return new_date_filter; + } + else if (comparator == "notEquals") + { + const auto lhs = std::make_shared("less"); + lhs->getArguments().getNodes().push_back(std::make_shared(column_node.getColumn(), column_node.getColumnSource())); + lhs->getArguments().getNodes().push_back(std::make_shared(start_date_or_date_time)); + resolveOrdinaryFunctionNode(*lhs, lhs->getFunctionName()); + + const auto rhs = std::make_shared("greaterOrEquals"); + rhs->getArguments().getNodes().push_back(std::make_shared(column_node.getColumn(), column_node.getColumnSource())); + rhs->getArguments().getNodes().push_back(std::make_shared(end_date_or_date_time)); + resolveOrdinaryFunctionNode(*rhs, rhs->getFunctionName()); + + const auto new_date_filter = std::make_shared("or"); + new_date_filter->getArguments().getNodes() = {lhs, rhs}; + resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName()); + + return new_date_filter; + } + else if (comparator == "greater") + { + const auto new_date_filter = std::make_shared("greaterOrEquals"); + new_date_filter->getArguments().getNodes().push_back(std::make_shared(column_node.getColumn(), column_node.getColumnSource())); + new_date_filter->getArguments().getNodes().push_back(std::make_shared(end_date_or_date_time)); + resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName()); + + return new_date_filter; + } + else if (comparator == "lessOrEquals") + { + const auto new_date_filter = std::make_shared("less"); + new_date_filter->getArguments().getNodes().push_back(std::make_shared(column_node.getColumn(), column_node.getColumnSource())); + new_date_filter->getArguments().getNodes().push_back(std::make_shared(end_date_or_date_time)); + resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName()); + + return new_date_filter; + } + else if (comparator == "less" || comparator == "greaterOrEquals") + { + const auto new_date_filter = std::make_shared(comparator); + new_date_filter->getArguments().getNodes().push_back(std::make_shared(column_node.getColumn(), column_node.getColumnSource())); + new_date_filter->getArguments().getNodes().push_back(std::make_shared(start_date_or_date_time)); + resolveOrdinaryFunctionNode(*new_date_filter, new_date_filter->getFunctionName()); + + return new_date_filter; + } + else [[unlikely]] + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected equals, notEquals, less, lessOrEquals, greater, greaterOrEquals. Actual {}", + comparator); + } + } + + void resolveOrdinaryFunctionNode(FunctionNode & function_node, const String & function_name) const + { + auto function = FunctionFactory::instance().get(function_name, getContext()); + function_node.resolveAsFunction(function->build(function_node.getArgumentColumns())); + } +}; + +} + +void OptimizeDateOrDateTimeConverterWithPreimagePass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) +{ + OptimizeDateOrDateTimeConverterWithPreimageVisitor visitor(std::move(context)); + visitor.visit(query_tree_node); +} + +} diff --git a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.h b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.h new file mode 100644 index 00000000000..c29f18e330b --- /dev/null +++ b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.h @@ -0,0 +1,24 @@ +#pragma once + +#include + +namespace DB +{ + +/** Replace predicate having Date/DateTime converters with their preimages to improve performance. + * Given a Date column c, toYear(c) = 2023 -> c >= '2023-01-01' AND c < '2024-01-01' + * Or if c is a DateTime column, toYear(c) = 2023 -> c >= '2023-01-01 00:00:00' AND c < '2024-01-01 00:00:00'. + * The similar optimization also applies to other converters. + */ +class OptimizeDateOrDateTimeConverterWithPreimagePass final : public IQueryTreePass +{ +public: + String getName() override { return "OptimizeDateOrDateTimeConverterWithPreimagePass"; } + + String getDescription() override { return "Replace predicate having Date/DateTime converters with their preimages"; } + + void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; + +}; + +} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index a6da2a66615..cd3abd9593e 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -42,6 +42,7 @@ #include #include #include +#include namespace DB { @@ -278,6 +279,7 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); } } diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.reference b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference index 872a5dd1d7d..5c3cab9bf4a 100644 --- a/tests/queries/0_stateless/02783_date_predicate_optimizations.reference +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference @@ -1,5 +1,7 @@ 2021-12-31 23:00:00 0 2021-12-31 23:00:00 0 +2021-12-31 23:00:00 0 +2021-12-31 23:00:00 0 Date 2 3 @@ -13,6 +15,18 @@ Date 4 1 4 +2 +3 +2 +4 +1 +3 +3 +2 +1 +4 +1 +4 DateTime 2 3 @@ -26,6 +40,18 @@ DateTime 4 1 4 +2 +3 +2 +4 +1 +3 +3 +2 +1 +4 +1 +4 Date32 2 3 @@ -39,6 +65,18 @@ Date32 4 1 4 +2 +3 +2 +4 +1 +3 +3 +2 +1 +4 +1 +4 DateTime64 2 3 @@ -52,3 +90,15 @@ DateTime64 4 1 4 +2 +3 +2 +4 +1 +3 +3 +2 +1 +4 +1 +4 diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql index 0a2fa6cc93b..4da8cebff1c 100644 --- a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql @@ -11,6 +11,8 @@ INSERT INTO source values ('2021-12-31 23:00:00', 0); SELECT * FROM source WHERE toYYYYMM(ts) = 202112; SELECT * FROM source WHERE toYear(ts) = 2021; +SELECT * FROM source WHERE toYYYYMM(ts) = 202112 SETTINGS allow_experimental_analyzer=1; +SELECT * FROM source WHERE toYear(ts) = 2021 SETTINGS allow_experimental_analyzer=1; DROP TABLE IF EXISTS source; CREATE TABLE source @@ -44,6 +46,18 @@ SELECT count(*) FROM source WHERE toYear(dt) < 2023; SELECT count(*) FROM source WHERE toYear(dt) <= 2023; SELECT count(*) FROM source WHERE toYear(dt) > 2023; SELECT count(*) FROM source WHERE toYear(dt) >= 2023; +SELECT count(*) FROM source WHERE toYYYYMM(dt) = 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt) <> 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt) < 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt) <= 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt) > 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt) >= 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) = 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) <> 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) < 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) <= 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) > 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt) >= 2023 SETTINGS allow_experimental_analyzer=1; SELECT 'DateTime'; SELECT count(*) FROM source WHERE toYYYYMM(ts) = 202312; @@ -58,6 +72,18 @@ SELECT count(*) FROM source WHERE toYear(ts) < 2023; SELECT count(*) FROM source WHERE toYear(ts) <= 2023; SELECT count(*) FROM source WHERE toYear(ts) > 2023; SELECT count(*) FROM source WHERE toYear(ts) >= 2023; +SELECT count(*) FROM source WHERE toYYYYMM(ts) = 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts) <> 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts) < 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts) <= 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts) > 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts) >= 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) = 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) <> 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) < 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) <= 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) > 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts) >= 2023 SETTINGS allow_experimental_analyzer=1; SELECT 'Date32'; SELECT count(*) FROM source WHERE toYYYYMM(dt_32) = 202312; @@ -72,6 +98,18 @@ SELECT count(*) FROM source WHERE toYear(dt_32) < 2023; SELECT count(*) FROM source WHERE toYear(dt_32) <= 2023; SELECT count(*) FROM source WHERE toYear(dt_32) > 2023; SELECT count(*) FROM source WHERE toYear(dt_32) >= 2023; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) = 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <> 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) < 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <= 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) > 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(dt_32) >= 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) = 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) <> 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) < 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) <= 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) > 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(dt_32) >= 2023 SETTINGS allow_experimental_analyzer=1; SELECT 'DateTime64'; SELECT count(*) FROM source WHERE toYYYYMM(ts_64) = 202312; @@ -86,4 +124,16 @@ SELECT count(*) FROM source WHERE toYear(ts_64) < 2023; SELECT count(*) FROM source WHERE toYear(ts_64) <= 2023; SELECT count(*) FROM source WHERE toYear(ts_64) > 2023; SELECT count(*) FROM source WHERE toYear(ts_64) >= 2023; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) = 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <> 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) < 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <= 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) > 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYYYYMM(ts_64) >= 202312 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) = 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) <> 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) < 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) <= 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) > 2023 SETTINGS allow_experimental_analyzer=1; +SELECT count(*) FROM source WHERE toYear(ts_64) >= 2023 SETTINGS allow_experimental_analyzer=1; DROP TABLE source; diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference index 9235e7e106a..0fd2f694aeb 100644 --- a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference +++ b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.reference @@ -1,87 +1,1122 @@ SELECT value1 FROM date_t WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 11, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 15, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 < \'1993-01-01\') OR (date1 >= \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 11, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 15, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 9, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 10, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (date1 >= \'1994-01-01\') AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 9, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 10, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (date1 < \'1994-01-01\') AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 9, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 10, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (date1 >= \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 9, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 10, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1998-01-01\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 11, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 15, constant_value: \'1998-01-01\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) OR ((date1 >= \'1994-01-01\') AND (date1 < \'1995-01-01\'))) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + FUNCTION id: 10, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 12, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 13, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 14, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 15, nodes: 2 + COLUMN id: 16, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 17, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 18, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + FUNCTION id: 20, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 21, nodes: 2 + COLUMN id: 22, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 23, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 24, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 25, nodes: 2 + COLUMN id: 26, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 27, constant_value: \'1995-01-01\', constant_value_type: String + FUNCTION id: 28, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 29, nodes: 2 + FUNCTION id: 30, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 31, nodes: 2 + COLUMN id: 32, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 33, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 34, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 35, nodes: 2 + COLUMN id: 32, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 36, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1, toYear(date1) AS year1 FROM date_t WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + year1 UInt16 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + FUNCTION id: 4, function_name: toYear, function_type: ordinary, result_type: UInt16 + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: date1, result_type: Date, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 7, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 8, nodes: 2 + FUNCTION id: 9, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 10, nodes: 2 + FUNCTION id: 11, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 12, nodes: 2 + COLUMN id: 13, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 14, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 15, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 16, nodes: 2 + COLUMN id: 17, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 18, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 19, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 20, nodes: 2 + FUNCTION id: 21, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 22, nodes: 2 + COLUMN id: 23, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 25, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 26, nodes: 2 + COLUMN id: 23, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 27, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 9, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 10, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t PREWHERE (date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\') WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + PREWHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 9, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 10, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 12, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 13, constant_value: \'1994-01-01\', constant_value_type: String + WHERE + FUNCTION id: 14, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 15, nodes: 2 + FUNCTION id: 16, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 18, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 19, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 20, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 21, nodes: 2 + COLUMN id: 18, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 22, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 9, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 10, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + COLUMN id: 8, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 12, constant_value: UInt64_3, constant_value_type: UInt8 + HAVING + FUNCTION id: 13, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 14, nodes: 2 + FUNCTION id: 15, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 16, nodes: 2 + COLUMN id: 17, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 18, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 19, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 20, nodes: 2 + COLUMN id: 21, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 22, constant_value: \'1994-01-01\', constant_value_type: String + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (toYYYYMM(date1) = 199300) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: toYYYYMM, function_type: ordinary, result_type: UInt32 + ARGUMENTS + LIST id: 9, nodes: 1 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 11, constant_value: UInt64_199300, constant_value_type: UInt32 + FUNCTION id: 12, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + FUNCTION id: 14, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 15, nodes: 2 + COLUMN id: 16, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 17, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 18, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 16, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (toYYYYMM(date1) = 199313) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: toYYYYMM, function_type: ordinary, result_type: UInt32 + ARGUMENTS + LIST id: 9, nodes: 1 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 11, constant_value: UInt64_199313, constant_value_type: UInt32 + FUNCTION id: 12, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + FUNCTION id: 14, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 15, nodes: 2 + COLUMN id: 16, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 17, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 18, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 16, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 20, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 >= \'1993-12-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 11, constant_value: \'1993-12-01\', constant_value_type: String + FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 15, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 >= \'1992-03-01\') AND (date1 < \'1992-04-01\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 11, constant_value: \'1992-03-01\', constant_value_type: String + FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 15, constant_value: \'1992-04-01\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 < \'1992-03-01\') OR (date1 >= \'1992-04-01\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 11, constant_value: \'1992-03-01\', constant_value_type: String + FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 15, constant_value: \'1992-04-01\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (date1 < \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 9, constant_value: \'1992-03-01\', constant_value_type: String + FUNCTION id: 10, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (date1 >= \'1992-04-01\') AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 9, constant_value: \'1992-04-01\', constant_value_type: String + FUNCTION id: 10, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (date1 < \'1992-04-01\') AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 9, constant_value: \'1992-04-01\', constant_value_type: String + FUNCTION id: 10, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE (date1 >= \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + COLUMN id: 8, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 9, constant_value: \'1992-03-01\', constant_value_type: String + FUNCTION id: 10, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 11, nodes: 2 + FUNCTION id: 12, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 15, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 16, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + COLUMN id: 14, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 18, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date_t WHERE ((date1 >= \'1992-03-01\') OR ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\'))) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: or, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 11, constant_value: \'1992-03-01\', constant_value_type: String + FUNCTION id: 12, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + FUNCTION id: 14, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 15, nodes: 2 + COLUMN id: 16, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 17, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 18, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: date1, result_type: Date, source_id: 3 + CONSTANT id: 21, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 22, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + FUNCTION id: 24, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 25, nodes: 2 + COLUMN id: 26, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 27, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 28, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 29, nodes: 2 + COLUMN id: 26, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 30, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM datetime_t WHERE ((date1 >= \'1993-01-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.datetime_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: DateTime, source_id: 3 + CONSTANT id: 11, constant_value: \'1993-01-01 00:00:00\', constant_value_type: String + FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: DateTime, source_id: 3 + CONSTANT id: 15, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM datetime_t WHERE ((date1 >= \'1993-12-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.datetime_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: DateTime, source_id: 3 + CONSTANT id: 11, constant_value: \'1993-12-01 00:00:00\', constant_value_type: String + FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: DateTime, source_id: 3 + CONSTANT id: 15, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date32_t WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date32_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: Date32, source_id: 3 + CONSTANT id: 11, constant_value: \'1993-01-01\', constant_value_type: String + FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: Date32, source_id: 3 + CONSTANT id: 15, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM date32_t WHERE ((date1 >= \'1993-12-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.date32_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: Date32, source_id: 3 + CONSTANT id: 11, constant_value: \'1993-12-01\', constant_value_type: String + FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: Date32, source_id: 3 + CONSTANT id: 15, constant_value: \'1994-01-01\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM datetime64_t WHERE ((date1 >= \'1993-01-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.datetime64_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: DateTime64(3), source_id: 3 + CONSTANT id: 11, constant_value: \'1993-01-01 00:00:00\', constant_value_type: String + FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: DateTime64(3), source_id: 3 + CONSTANT id: 15, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 SELECT value1 FROM datetime64_t WHERE ((date1 >= \'1993-12-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) +QUERY id: 0 + PROJECTION COLUMNS + value1 String + PROJECTION + LIST id: 1, nodes: 1 + COLUMN id: 2, column_name: value1, result_type: String, source_id: 3 + JOIN TREE + TABLE id: 3, table_name: default.datetime64_t + WHERE + FUNCTION id: 4, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 5, nodes: 2 + FUNCTION id: 6, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 7, nodes: 2 + FUNCTION id: 8, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 9, nodes: 2 + COLUMN id: 10, column_name: date1, result_type: DateTime64(3), source_id: 3 + CONSTANT id: 11, constant_value: \'1993-12-01 00:00:00\', constant_value_type: String + FUNCTION id: 12, function_name: less, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 13, nodes: 2 + COLUMN id: 14, column_name: date1, result_type: DateTime64(3), source_id: 3 + CONSTANT id: 15, constant_value: \'1994-01-01 00:00:00\', constant_value_type: String + FUNCTION id: 16, function_name: and, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: greaterOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 19, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 21, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 22, function_name: lessOrEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 23, nodes: 2 + COLUMN id: 20, column_name: id, result_type: UInt32, source_id: 3 + CONSTANT id: 24, constant_value: UInt64_3, constant_value_type: UInt8 + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.sql b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.sql index 266be59b0a3..9cc8dd74e5d 100644 --- a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.sql +++ b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_query_tree_rewrite.sql @@ -2,46 +2,74 @@ DROP TABLE IF EXISTS date_t; CREATE TABLE date_t (id UInt32, value1 String, date1 Date) ENGINE ReplacingMergeTree() ORDER BY id; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM date_t WHERE year1 = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1, toYear(date1) as year1 FROM date_t WHERE year1 = 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199203 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date_t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; DROP TABLE date_t; DROP TABLE IF EXISTS datetime_t; CREATE TABLE datetime_t (id UInt32, value1 String, date1 Datetime) ENGINE ReplacingMergeTree() ORDER BY id; EXPLAIN SYNTAX SELECT value1 FROM datetime_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM datetime_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; DROP TABLE datetime_t; DROP TABLE IF EXISTS date32_t; CREATE TABLE date32_t (id UInt32, value1 String, date1 Date32) ENGINE ReplacingMergeTree() ORDER BY id; EXPLAIN SYNTAX SELECT value1 FROM date32_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date32_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM date32_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM date32_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; DROP TABLE date32_t; DROP TABLE IF EXISTS datetime64_t; CREATE TABLE datetime64_t (id UInt32, value1 String, date1 Datetime64) ENGINE ReplacingMergeTree() ORDER BY id; EXPLAIN SYNTAX SELECT value1 FROM datetime64_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime64_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; EXPLAIN SYNTAX SELECT value1 FROM datetime64_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; +EXPLAIN QUERY TREE run_passes=1 SELECT value1 FROM datetime64_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3 SETTINGS allow_experimental_analyzer=1; DROP TABLE datetime64_t; From 810137e57a53467e9fea668769749c559af12bc1 Mon Sep 17 00:00:00 2001 From: Dmitry Kardymon Date: Mon, 24 Jul 2023 05:59:07 +0000 Subject: [PATCH 1081/2047] Add new peak_memory_usage to docs --- docs/en/interfaces/http.md | 18 +++++++++--------- docs/ru/interfaces/http.md | 16 ++++++++-------- docs/zh/interfaces/http.md | 18 +++++++++--------- 3 files changed, 26 insertions(+), 26 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 3a7f6d4d854..37821f0fee1 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -56,7 +56,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} 1 ``` @@ -286,9 +286,9 @@ Similarly, you can use ClickHouse sessions in the HTTP protocol. To do this, you You can receive information about the progress of a query in `X-ClickHouse-Progress` response headers. To do this, enable [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Example of the header sequence: ``` text -X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128"} +X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","peak_memory_usage":"4371480"} +X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","peak_memory_usage":"13621616"} +X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","peak_memory_usage":"23155600"} ``` Possible header fields: @@ -416,7 +416,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -581,7 +581,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -621,7 +621,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact
% @@ -673,7 +673,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -692,7 +692,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Relative Path File * Connection #0 to host localhost left intact diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index b8c5ee77f0c..981f1c7b5a2 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -50,7 +50,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} 1 ``` @@ -266,9 +266,9 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812 Прогресс выполнения запроса можно отслеживать с помощью заголовков ответа `X-ClickHouse-Progress`. Для этого включите [send_progress_in_http_headers](../operations/settings/settings.md#settings-send_progress_in_http_headers). Пример последовательности заголовков: ``` text -X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128"} +X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","peak_memory_usage":"4371480"} +X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","peak_memory_usage":"13621616"} +X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","peak_memory_usage":"23155600"} ``` Возможные поля заголовка: @@ -529,7 +529,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -569,7 +569,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact
% @@ -621,7 +621,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -640,7 +640,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Relative Path File * Connection #0 to host localhost left intact diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index c7a0f355a92..f84768beccc 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -53,7 +53,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} 1 ``` @@ -262,9 +262,9 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812 您可以在`X-ClickHouse-Progress`响应头中收到查询进度的信息。为此,启用[Http Header携带进度](../operations/settings/settings.md#settings-send_progress_in_http_headers)。示例: ``` text -X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128"} -X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128"} +X-ClickHouse-Progress: {"read_rows":"2752512","read_bytes":"240570816","total_rows_to_read":"8880128","peak_memory_usage":"4371480"} +X-ClickHouse-Progress: {"read_rows":"5439488","read_bytes":"482285394","total_rows_to_read":"8880128","peak_memory_usage":"13621616"} +X-ClickHouse-Progress: {"read_rows":"8783786","read_bytes":"819092887","total_rows_to_read":"8880128","peak_memory_usage":"23155600"} ``` 显示字段信息: @@ -363,7 +363,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -521,7 +521,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -561,7 +561,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < * Connection #0 to host localhost left intact
% @@ -613,7 +613,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -632,7 +632,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=3 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","peak_memory_usage":"0"} < Relative Path File * Connection #0 to host localhost left intact From ceaaa78fdcfac2243bcf28624336217bd44898f0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 08:04:38 +0200 Subject: [PATCH 1082/2047] Fix transform --- src/Functions/transform.cpp | 37 ++++++++++++++++++------------------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 1fc0e3adf96..a48d8d47489 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -156,15 +156,15 @@ namespace { initialize(arguments, result_type); - const auto * in = arguments.front().column.get(); - - if (isColumnConst(*in)) + if (isColumnConst(*arguments[0].column)) return executeConst(arguments, result_type, input_rows_count); ColumnPtr default_non_const; if (!cache.default_column && arguments.size() == 4) default_non_const = castColumn(arguments[3], result_type); + ColumnPtr in = cache.default_column ? arguments[0].column : castColumn(arguments[0], result_type); + auto column_result = result_type->createColumn(); if (cache.is_empty) { @@ -174,30 +174,30 @@ namespace } else if (cache.table_num_to_idx) { - if (!executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const)) + if (!executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const) + && !executeNum>(in.get(), *column_result, default_non_const)) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName()); } } else if (cache.table_string_to_idx) { - if (!executeString(in, *column_result, default_non_const)) - executeContiguous(in, *column_result, default_non_const); + if (!executeString(in.get(), *column_result, default_non_const)) + executeContiguous(in.get(), *column_result, default_non_const); } else if (cache.table_anything_to_idx) { - executeAnything(in, *column_result, default_non_const); + executeAnything(in.get(), *column_result, default_non_const); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized"); @@ -810,7 +810,6 @@ namespace cache.initialized = true; } }; - } REGISTER_FUNCTION(Transform) From aaa0bf64fd888332bfa59c284508d4e7a84d372c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 08:05:55 +0200 Subject: [PATCH 1083/2047] Add a test --- .../02832_transform_fixed_string_no_default.reference | 1 + .../0_stateless/02832_transform_fixed_string_no_default.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference create mode 100644 tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql diff --git a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference new file mode 100644 index 00000000000..9daeafb9864 --- /dev/null +++ b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference @@ -0,0 +1 @@ +test diff --git a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql new file mode 100644 index 00000000000..8d316d3413f --- /dev/null +++ b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql @@ -0,0 +1 @@ +SELECT transform(name, ['a', 'b'], ['', NULL]) AS name FROM (SELECT 'test'::Nullable(FixedString(4)) AS name); From 890a3754a6a093545122e42bcab066a27c72ed5e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 08:19:46 +0200 Subject: [PATCH 1084/2047] Fix error --- src/Functions/transform.cpp | 55 ++++++++++++++++++++----------------- 1 file changed, 30 insertions(+), 25 deletions(-) diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index a48d8d47489..79168d82c54 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -156,14 +156,18 @@ namespace { initialize(arguments, result_type); - if (isColumnConst(*arguments[0].column)) + const auto * in = arguments[0].column.get(); + + if (isColumnConst(*in)) return executeConst(arguments, result_type, input_rows_count); ColumnPtr default_non_const; if (!cache.default_column && arguments.size() == 4) default_non_const = castColumn(arguments[3], result_type); - ColumnPtr in = cache.default_column ? arguments[0].column : castColumn(arguments[0], result_type); + ColumnPtr in_casted = arguments[0].column; + if (arguments.size() == 3) + in_casted = castColumn(arguments[0], result_type); auto column_result = result_type->createColumn(); if (cache.is_empty) @@ -174,30 +178,30 @@ namespace } else if (cache.table_num_to_idx) { - if (!executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const) - && !executeNum>(in.get(), *column_result, default_non_const)) + if (!executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted) + && !executeNum>(in, *column_result, default_non_const, *in_casted)) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName()); } } else if (cache.table_string_to_idx) { - if (!executeString(in.get(), *column_result, default_non_const)) - executeContiguous(in.get(), *column_result, default_non_const); + if (!executeString(in, *column_result, default_non_const, *in_casted)) + executeContiguous(in, *column_result, default_non_const, *in_casted); } else if (cache.table_anything_to_idx) { - executeAnything(in.get(), *column_result, default_non_const); + executeAnything(in, *column_result, default_non_const, *in_casted); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized"); @@ -218,7 +222,7 @@ namespace return impl->execute(args, result_type, input_rows_count); } - void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const + void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const { const size_t size = in->size(); const auto & table = *cache.table_anything_to_idx; @@ -236,11 +240,11 @@ namespace else if (default_non_const) column_result.insertFrom(*default_non_const, i); else - column_result.insertFrom(*in, i); + column_result.insertFrom(in_casted, i); } } - void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const + void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const { const size_t size = in->size(); const auto & table = *cache.table_string_to_idx; @@ -255,12 +259,12 @@ namespace else if (default_non_const) column_result.insertFrom(*default_non_const, i); else - column_result.insertFrom(*in, i); + column_result.insertFrom(in_casted, i); } } template - bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const) const + bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const { const auto * const in = checkAndGetColumn(in_untyped); if (!in) @@ -297,7 +301,7 @@ namespace else if (default_non_const) column_result.insertFrom(*default_non_const, i); else - column_result.insertFrom(*in, i); + column_result.insertFrom(in_casted, i); } } return true; @@ -451,7 +455,7 @@ namespace } } - bool executeString(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const) const + bool executeString(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted) const { const auto * const in = checkAndGetColumn(in_untyped); if (!in) @@ -488,7 +492,7 @@ namespace else if (default_non_const) column_result.insertFrom(*default_non_const, 0); else - column_result.insertFrom(*in, i); + column_result.insertFrom(in_casted, i); } } return true; @@ -810,6 +814,7 @@ namespace cache.initialized = true; } }; + } REGISTER_FUNCTION(Transform) From c79492240194f0d5dd9053c70a967c39a7536cb3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 08:20:30 +0200 Subject: [PATCH 1085/2047] More tests --- .../02832_transform_fixed_string_no_default.reference | 2 ++ .../0_stateless/02832_transform_fixed_string_no_default.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference index 9daeafb9864..ea545c90391 100644 --- a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference +++ b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.reference @@ -1 +1,3 @@ test + +\N diff --git a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql index 8d316d3413f..0e58c716c9f 100644 --- a/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql +++ b/tests/queries/0_stateless/02832_transform_fixed_string_no_default.sql @@ -1 +1,3 @@ SELECT transform(name, ['a', 'b'], ['', NULL]) AS name FROM (SELECT 'test'::Nullable(FixedString(4)) AS name); +SELECT transform(name, ['test', 'b'], ['', NULL]) AS name FROM (SELECT 'test'::Nullable(FixedString(4)) AS name); +SELECT transform(name, ['a', 'test'], ['', NULL]) AS name FROM (SELECT 'test'::Nullable(FixedString(4)) AS name); From 0e46cf86b772e1513d837d6019181a6d291b7219 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 24 Jul 2023 08:52:19 +0200 Subject: [PATCH 1086/2047] Added try-except to check cases when second backup/restore is picked up first --- .../test_disallow_concurrency.py | 69 +++++++++++++++---- 1 file changed, 57 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index d0ce2e03016..a863a6e2047 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -133,9 +133,21 @@ def test_concurrent_backups_on_same_node(): ) assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] - error = nodes[0].query_and_get_error( - f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" - ) + try: + error = nodes[0].query_and_get_error( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" + ) + except Exception as e: + status = ( + nodes[0] + .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .rstrip("\n") + ) + # It is possible that the second backup was picked up first, and then the async backup + if status == "CREATING_BACKUP" or status == "BACKUP_FAILED": + return + else: + raise e expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", @@ -179,9 +191,20 @@ def test_concurrent_backups_on_different_nodes(): ) assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] - error = nodes[0].query_and_get_error( - f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" - ) + try: + error = nodes[0].query_and_get_error( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" + ) + except Exception as e: + status = ( + nodes[1] + .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .rstrip("\n") + ) + if status == "CREATING_BACKUP" or status == "BACKUP_FAILED": + return + else: + raise e expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", @@ -224,9 +247,20 @@ def test_concurrent_restores_on_same_node(): ) assert status in ["RESTORING", "RESTORED"] - error = nodes[0].query_and_get_error( - f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" - ) + try: + error = nodes[0].query_and_get_error( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" + ) + except Exception as e: + status = ( + nodes[0] + .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .rstrip("\n") + ) + if status == "RESTORING" or status == "RESTORE_FAILED": + return + else: + raise e expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", @@ -269,9 +303,20 @@ def test_concurrent_restores_on_different_node(): ) assert status in ["RESTORING", "RESTORED"] - error = nodes[1].query_and_get_error( - f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" - ) + try: + error = nodes[1].query_and_get_error( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" + ) + except Exception as e: + status = ( + nodes[0] + .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .rstrip("\n") + ) + if status == "RESTORING" or status == "RESTORE_FAILED": + return + else: + raise e expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", From 96d40ff3c4dd34a9396c625b8a1d57f697f80dd0 Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 24 Jul 2023 07:30:32 +0000 Subject: [PATCH 1087/2047] fix --- src/Common/IntervalTree.h | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Common/IntervalTree.h b/src/Common/IntervalTree.h index ad079a312f2..9a42aadf70e 100644 --- a/src/Common/IntervalTree.h +++ b/src/Common/IntervalTree.h @@ -32,6 +32,12 @@ auto operator<=>(const Interval & lhs, const Interval std::tie(rhs.left, rhs.right); } +template +bool operator==(const Interval & lhs, const Interval & rhs) +{ + return std::tie(lhs.left, lhs.right) == std::tie(rhs.left, rhs.right); +} + struct IntervalTreeVoidValue { }; From 0401dc453e9502697328879728bf0dbf7c1dd9e0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Jul 2023 10:14:23 +0200 Subject: [PATCH 1088/2047] Fix flakiness of test_version_update_after_mutation by enabling force_remove_data_recursively_on_drop Since there can be some leftovers: 2023.07.24 07:08:25.238066 [ 140 ] {} Application: Code: 219. DB::Exception: Cannot drop: filesystem error: in remove: Directory not empty ["/var/lib/clickhouse/data/system/"]. Probably database contain some detached tables or metadata leftovers from Ordinary engine. If you want to remove all data anyway, try to attach database back and drop it again with enabled force_remove_data_recursively_on_drop setting: Exception while trying to convert database system from Ordinary to Atomic. It may be in some intermediate state. You can finish conversion manually by moving the rest tables from system to .tmp_convert.system.9396432095832455195 (using RENAME TABLE) and executing DROP DATABASE system and RENAME DATABASE .tmp_convert.system.9396432095832455195 TO system. (DATABASE_NOT_EMPTY), Stack trace (when copying this message, always include the lines below): 0. DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000e68af57 in /usr/bin/clickhouse 1. ? @ 0x000000000cab443c in /usr/bin/clickhouse 2. DB::DatabaseOnDisk::drop(std::shared_ptr) @ 0x000000001328d617 in /usr/bin/clickhouse 3. DB::DatabaseCatalog::detachDatabase(std::shared_ptr, String const&, bool, bool) @ 0x0000000013524a6c in /usr/bin/clickhouse 4. DB::InterpreterDropQuery::executeToDatabaseImpl(DB::ASTDropQuery const&, std::shared_ptr&, std::vector, DB::UUIDTag>, std::allocator, DB::UUIDTag>>>&) @ 0x0000000013bc05e4 in /usr/bin/clickhouse 5. DB::InterpreterDropQuery::executeToDatabase(DB::ASTDropQuery const&) @ 0x0000000013bbc6b8 in /usr/bin/clickhouse 6. DB::InterpreterDropQuery::execute() @ 0x0000000013bbba22 in /usr/bin/clickhouse 7. ? @ 0x00000000140b13a5 in /usr/bin/clickhouse 8. DB::executeQuery(String const&, std::shared_ptr, bool, DB::QueryProcessingStage::Enum) @ 0x00000000140ad20e in /usr/bin/clickhouse 9. ? @ 0x00000000140d2ef0 in /usr/bin/clickhouse 10. DB::maybeConvertSystemDatabase(std::shared_ptr) @ 0x00000000140d0aaf in /usr/bin/clickhouse 11. DB::Server::main(std::vector> const&) @ 0x000000000e724e55 in /usr/bin/clickhouse 12. Poco::Util::Application::run() @ 0x0000000017ead086 in /usr/bin/clickhouse 13. DB::Server::run() @ 0x000000000e714a5d in /usr/bin/clickhouse 14. Poco::Util::ServerApplication::run(int, char**) @ 0x0000000017ec07b9 in /usr/bin/clickhouse 15. mainEntryClickHouseServer(int, char**) @ 0x000000000e711a26 in /usr/bin/clickhouse 16. main @ 0x0000000008cf13cf in /usr/bin/clickhouse 17. __libc_start_main @ 0x0000000000021b97 in /lib/x86_64-linux-gnu/libc-2.27.so 18. _start @ 0x00000000080705ae in /usr/bin/clickhouse (version 23.7.1.2012) Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 9 +++++++++ .../force_remove_data_recursively_on_drop.xml | 7 +++++++ .../test_version_update_after_mutation/test.py | 13 ++++++++++--- 3 files changed, 26 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index eff44de842a..0448eb2437f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3199,6 +3199,7 @@ class ClickHouseInstance: ): self.name = name self.base_cmd = cluster.base_cmd + self.base_dir = base_path self.docker_id = cluster.get_instance_docker_id(self.name) self.cluster = cluster self.hostname = hostname if hostname is not None else self.name @@ -4193,6 +4194,14 @@ class ClickHouseInstance: ["bash", "-c", f"sed -i 's/{replace}/{replacement}/g' {path_to_config}"] ) + def put_users_config(self, config_path): + """Put new config (useful if you cannot put it at the start)""" + + instance_config_dir = p.abspath(p.join(self.path, "configs")) + users_d_dir = p.abspath(p.join(instance_config_dir, "users.d")) + config_path = p.join(self.base_dir, config_path) + shutil.copy(config_path, users_d_dir) + def create_dir(self): """Create the instance directory and all the needed files there.""" diff --git a/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml b/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml new file mode 100644 index 00000000000..7a00648b28e --- /dev/null +++ b/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index c80205d48c1..416220c93c3 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -51,6 +51,12 @@ def start_cluster(): cluster.shutdown() +def restart_node(node): + # set force_remove_data_recursively_on_drop (cannot be done before, because the version is too old) + node.put_users_config("configs/force_remove_data_recursively_on_drop.xml") + node.restart_with_latest_version(signal=9, fix_metadata=True) + + def test_mutate_and_upgrade(start_cluster): for node in [node1, node2]: node.query("DROP TABLE IF EXISTS mt") @@ -67,8 +73,9 @@ def test_mutate_and_upgrade(start_cluster): node2.query("DETACH TABLE mt") # stop being leader node1.query("DETACH TABLE mt") # stop being leader - node1.restart_with_latest_version(signal=9, fix_metadata=True) - node2.restart_with_latest_version(signal=9, fix_metadata=True) + + restart_node(node1) + restart_node(node2) # After hard restart table can be in readonly mode exec_query_with_retry( @@ -124,7 +131,7 @@ def test_upgrade_while_mutation(start_cluster): # (We could be in process of creating some system table, which will leave empty directory on restart, # so when we start moving system tables from ordinary to atomic db, it will complain about some undeleted files) node3.query("SYSTEM FLUSH LOGS") - node3.restart_with_latest_version(signal=9, fix_metadata=True) + restart_node(node3) # checks for readonly exec_query_with_retry(node3, "OPTIMIZE TABLE mt1", sleep_time=5, retry_count=60) From efa638ef3cc7db3c6149b7c031cc4c7904987abd Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Wed, 19 Jul 2023 12:53:27 +0200 Subject: [PATCH 1089/2047] MaterializedMySQL: Support unquoted utf-8 strings in DDL MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Since ClickHouse does not support unquoted utf-8 strings but MySQL does. Instead of fixing Lexer to recognize utf-8 chars as TokenType::BareWord, suggesting to quote all unrecognized tokens before applying any DDL. Actual parsing and validating the syntax will be done by particular Parser. If there is any TokenType::Error, the query is unable to be parsed anyway. Quoting such tokens can provide the support of utf-8 names. See `tryQuoteUnrecognizedTokens` and `QuoteUnrecognizedTokensTest`. mysql> CREATE TABLE 道.渠(... is converted to CREATE TABLE `道`.`渠`(... Also fixed the bug with missing * while doing SELECT in full sync because db or table name are back quoted when not needed. --- src/Common/quoteString.cpp | 11 + src/Common/quoteString.h | 3 + .../MySQL/MaterializedMySQLSyncThread.cpp | 7 +- .../gtest_try_quote_unrecognized_tokens.cpp | 289 ++++++++++++++++++ .../MySQL/tryQuoteUnrecognizedTokens.cpp | 96 ++++++ .../MySQL/tryQuoteUnrecognizedTokens.h | 10 + src/Storages/StorageMySQL.cpp | 11 +- .../materialized_with_ddl.py | 122 ++++++++ .../test_materialized_mysql_database/test.py | 6 + 9 files changed, 542 insertions(+), 13 deletions(-) create mode 100644 src/Databases/MySQL/tests/gtest_try_quote_unrecognized_tokens.cpp create mode 100644 src/Databases/MySQL/tryQuoteUnrecognizedTokens.cpp create mode 100644 src/Databases/MySQL/tryQuoteUnrecognizedTokens.h diff --git a/src/Common/quoteString.cpp b/src/Common/quoteString.cpp index b464f4837a1..17129441c8f 100644 --- a/src/Common/quoteString.cpp +++ b/src/Common/quoteString.cpp @@ -44,4 +44,15 @@ String backQuoteIfNeed(StringRef x) return res; } + +String backQuoteMySQL(StringRef x) +{ + String res(x.size, '\0'); + { + WriteBufferFromString wb(res); + writeBackQuotedStringMySQL(x, wb); + } + return res; +} + } diff --git a/src/Common/quoteString.h b/src/Common/quoteString.h index b83988258e2..3f17d6e7621 100644 --- a/src/Common/quoteString.h +++ b/src/Common/quoteString.h @@ -24,4 +24,7 @@ String backQuote(StringRef x); /// Quote the identifier with backquotes, if required. String backQuoteIfNeed(StringRef x); +/// Quote the identifier with backquotes, for use in MySQL queries. +String backQuoteMySQL(StringRef x); + } diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 603bf3d0166..673bd155f77 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -342,9 +343,8 @@ static inline String rewriteMysqlQueryColumn(mysqlxx::Pool::Entry & connection, { std::make_shared(), "column_type" } }; - const String & query = "SELECT COLUMN_NAME AS column_name, COLUMN_TYPE AS column_type FROM INFORMATION_SCHEMA.COLUMNS" - " WHERE TABLE_SCHEMA = '" + backQuoteIfNeed(database_name) + - "' AND TABLE_NAME = '" + backQuoteIfNeed(table_name) + "' ORDER BY ORDINAL_POSITION"; + String query = "SELECT COLUMN_NAME AS column_name, COLUMN_TYPE AS column_type FROM INFORMATION_SCHEMA.COLUMNS" + " WHERE TABLE_SCHEMA = '" + database_name + "' AND TABLE_NAME = '" + table_name + "' ORDER BY ORDINAL_POSITION"; StreamSettings mysql_input_stream_settings(global_settings, false, true); auto mysql_source = std::make_unique(connection, query, tables_columns_sample_block, mysql_input_stream_settings); @@ -812,6 +812,7 @@ void MaterializedMySQLSyncThread::executeDDLAtomic(const QueryEvent & query_even CurrentThread::QueryScope query_scope(query_context); String query = query_event.query; + tryQuoteUnrecognizedTokens(query, query); if (!materialized_tables_list.empty()) { auto table_id = tryParseTableIDFromDDL(query, query_event.schema); diff --git a/src/Databases/MySQL/tests/gtest_try_quote_unrecognized_tokens.cpp b/src/Databases/MySQL/tests/gtest_try_quote_unrecognized_tokens.cpp new file mode 100644 index 00000000000..9c76deb2712 --- /dev/null +++ b/src/Databases/MySQL/tests/gtest_try_quote_unrecognized_tokens.cpp @@ -0,0 +1,289 @@ +#include + +#include + +using namespace DB; + +struct TestCase +{ + String query; + String res; + bool ok; + + TestCase( + const String & query_, + const String & res_, + bool ok_) + : query(query_) + , res(res_) + , ok(ok_) + { + } +}; + +std::ostream & operator<<(std::ostream & ostr, const TestCase & test_case) +{ + return ostr << '"' << test_case.query << "\" -> \"" << test_case.res << "\" ok:" << test_case.ok; +} + +class QuoteUnrecognizedTokensTest : public ::testing::TestWithParam +{ +}; + +TEST_P(QuoteUnrecognizedTokensTest, escape) +{ + const auto & [query, expected, ok] = GetParam(); + String actual; + bool res = tryQuoteUnrecognizedTokens(query, actual); + EXPECT_EQ(ok, res); + EXPECT_EQ(expected, actual); +} + +INSTANTIATE_TEST_SUITE_P(MaterializedMySQL, QuoteUnrecognizedTokensTest, ::testing::ValuesIn(std::initializer_list{ + { + "", + "", + false + }, + { + "test '\"`", + "", + false + }, + { + "SELECT * FROM db.`table`", + "", + false + }, + { + "道渠", + "`道渠`", + true + }, + { + "道", + "`道`", + true + }, + { + "道道(skip) 道(", + "`道道`(skip) `道`(", + true + }, + { + "`道渠`", + "", + false + }, + { + "'道'", + "", + false + }, + { + "\"道\"", + "", + false + }, + { + "` 道 test 渠 `", + "", + false + }, + { + "skip 道 skip 123", + "skip `道` skip 123", + true + }, + { + "skip 123 `道` skip", + "", + false + }, + { + "skip `道 skip 123", + "", + false + }, + { + "skip test道 skip", + "skip `test道` skip", + true + }, + { + "test道2test", + "`test道2test`", + true + }, + { + "skip test道2test 123", + "skip `test道2test` 123", + true + }, + { + "skip 您a您a您a a您a您a您a 1您2您3您4 skip", + "skip `您a您a您a` `a您a您a您a` `1您2您3您4` skip", + true + }, + { + "skip 您a 您a您a b您2您c您4 skip", + "skip `您a` `您a您a` `b您2您c您4` skip", + true + }, + { + "123您a skip 56_您a 您a2 b_您2_您c123您_a4 skip", + "`123您a` skip `56_您a` `您a2` `b_您2_您c123您_a4` skip", + true + }, + { + "_您_ 123 skip 56_您_您_您_您_您_您_您_您_您_a 您a2 abc 123_您_您_321 a1b2c3 aaaaa您您_a4 skip", + "`_您_` 123 skip `56_您_您_您_您_您_您_您_您_您_a` `您a2` abc `123_您_您_321` a1b2c3 `aaaaa您您_a4` skip", + true + }, + { + "TABLE 您2 您(", + "TABLE `您2` `您`(", + true + }, + { + "TABLE 您.a您2(日2日2 INT", + "TABLE `您`.`a您2`(`日2日2` INT", + true + }, + { + "TABLE 您$.a_您2a_($日2日_2 INT, 您Hi好 a您b好c)", + "TABLE `您`$.`a_您2a_`($`日2日_2` INT, `您Hi好` `a您b好c`)", + true + }, + { + "TABLE 您a日.您a您a您a(test INT", + "TABLE `您a日`.`您a您a您a`(test INT", + true + }, + { + "TABLE 您a日.您a您a您a(Hi您Hi好Hi INT", + "TABLE `您a日`.`您a您a您a`(`Hi您Hi好Hi` INT", + true + }, + { + "--TABLE 您a日.您a您a您a(test INT", + "", + false + }, + { + "--您a日.您a您a您a(\n您Hi好", + "--您a日.您a您a您a(\n`您Hi好`", + true + }, + { + " /* TABLE 您a日.您a您a您a(test INT", + "", + false + }, + { + "/*您a日.您a您a您a(*/\n您Hi好", + "/*您a日.您a您a您a(*/\n`您Hi好`", + true + }, + { + " 您a日.您您aa您a /* 您a日.您a您a您a */ a您a日a.a您您您a", + " `您a日`.`您您aa您a` /* 您a日.您a您a您a */ `a您a日a`.`a您您您a`", + true + }, + //{ TODO + // "TABLE 您2.您a您a您a(test INT", + // "TABLE `您2`.`您a您a您a`(test INT", + // true + //}, + { + "skip 您a您a您a skip", + "skip `您a您a您a` skip", + true + }, + { + "test 您a2您3a您a 4 again", + "test `您a2您3a您a` 4 again", + true + }, + { + "CREATE TABLE db.`道渠`", + "", + false + }, + { + "CREATE TABLE db.`道渠", + "", + false + }, + { + "CREATE TABLE db.道渠", + "CREATE TABLE db.`道渠`", + true + }, + { + "CREATE TABLE db. 道渠", + "CREATE TABLE db. `道渠`", + true + }, + { + R"sql( + CREATE TABLE gb2312.`道渠` ( `id` int NOT NULL, + 您 INT, + 道渠 DATETIME, + 您test INT, test您 INT, test您test INT, + 道渠test INT, test道渠 INT, test道渠test INT, + 您_ INT, _您 INT, _您_ INT, + 您您__ INT, __您您 INT, __您您__ INT, + 您2 INT, 2您 INT, 2您2 INT, + 您您22 INT, 22您您 INT, 22您您22 INT, + 您_2 INT, _2您 INT, _2您_2 INT, _2您2_ INT, 2_您_2 INT, + 您您__22 INT, __22您您 INT, __22您您__22 INT, __22您您22__ INT, 22__您您__22 INT, + 您2_ INT, 2_您 INT, 2_您2_ INT, + 您您22__ INT, 22__您您 INT, 22__您您22__ INT, + 您_test INT, _test您 INT, _test您_test INT, _test您test_ INT, test_您test_ INT, test_您_test INT, + 您您_test INT, _test您您 INT, _test您您_test INT, _test您您test_ INT, test_您您test_ INT, test_您您_test INT, + 您test3 INT, test3您 INT, test3您test3 INT, test3您3test INT, + 您您test3 INT, test3您您 INT, test3您您test3 INT, test3您您3test INT, + 您3test INT, 3test您 INT, 3test您3test INT, 3test您test3 INT, + 您您3test INT, 3test您您 INT, 3test您您3test INT, 3test您您test3 INT, + 您_test4 INT, _test4您 INT, _test4您_test4 INT, test4_您_test4 INT, _test4您4test_ INT, _test4您test4_ INT, + 您您_test4 INT, _test4您您 INT, _test4您您_test4 INT, test4_您您_test4 INT, _test4您您4test_ INT, _test4您您test4_ INT, + 您_5test INT, _5test您 INT, _5test您_5test INT, 5test_您_test5 INT, _4test您test4_ INT, + test_日期 varchar(256), test_道_2 varchar(256) NOT NULL , + test_道渠您_3 + BIGINT NOT NULL, + 道您3_test INT, + PRIMARY KEY (`id`)) ENGINE=InnoDB DEFAULT CHARSET=gb2312; + )sql", + R"sql( + CREATE TABLE gb2312.`道渠` ( `id` int NOT NULL, + `您` INT, + `道渠` DATETIME, + `您test` INT, `test您` INT, `test您test` INT, + `道渠test` INT, `test道渠` INT, `test道渠test` INT, + `您_` INT, `_您` INT, `_您_` INT, + `您您__` INT, `__您您` INT, `__您您__` INT, + `您2` INT, `2您` INT, `2您2` INT, + `您您22` INT, `22您您` INT, `22您您22` INT, + `您_2` INT, `_2您` INT, `_2您_2` INT, `_2您2_` INT, `2_您_2` INT, + `您您__22` INT, `__22您您` INT, `__22您您__22` INT, `__22您您22__` INT, `22__您您__22` INT, + `您2_` INT, `2_您` INT, `2_您2_` INT, + `您您22__` INT, `22__您您` INT, `22__您您22__` INT, + `您_test` INT, `_test您` INT, `_test您_test` INT, `_test您test_` INT, `test_您test_` INT, `test_您_test` INT, + `您您_test` INT, `_test您您` INT, `_test您您_test` INT, `_test您您test_` INT, `test_您您test_` INT, `test_您您_test` INT, + `您test3` INT, `test3您` INT, `test3您test3` INT, `test3您3test` INT, + `您您test3` INT, `test3您您` INT, `test3您您test3` INT, `test3您您3test` INT, + `您3test` INT, `3test您` INT, `3test您3test` INT, `3test您test3` INT, + `您您3test` INT, `3test您您` INT, `3test您您3test` INT, `3test您您test3` INT, + `您_test4` INT, `_test4您` INT, `_test4您_test4` INT, `test4_您_test4` INT, `_test4您4test_` INT, `_test4您test4_` INT, + `您您_test4` INT, `_test4您您` INT, `_test4您您_test4` INT, `test4_您您_test4` INT, `_test4您您4test_` INT, `_test4您您test4_` INT, + `您_5test` INT, `_5test您` INT, `_5test您_5test` INT, `5test_您_test5` INT, `_4test您test4_` INT, + `test_日期` varchar(256), `test_道_2` varchar(256) NOT NULL , + `test_道渠您_3` + BIGINT NOT NULL, + `道您3_test` INT, + PRIMARY KEY (`id`)) ENGINE=InnoDB DEFAULT CHARSET=gb2312; + )sql", + true + }, +})); diff --git a/src/Databases/MySQL/tryQuoteUnrecognizedTokens.cpp b/src/Databases/MySQL/tryQuoteUnrecognizedTokens.cpp new file mode 100644 index 00000000000..cd4603ddaec --- /dev/null +++ b/src/Databases/MySQL/tryQuoteUnrecognizedTokens.cpp @@ -0,0 +1,96 @@ +#include +#include +#include + +namespace DB +{ + +/// Checks if there are no any tokens (like whitespaces) between current and previous pos +static bool noWhitespaces(const char * to, const char * from) +{ + return static_cast(from - to) == 0; +} + +/// Checks if the token should be quoted too together with unrecognized +static bool isWordOrNumber(TokenType type) +{ + return type == TokenType::BareWord || type == TokenType::Number; +} + +static void quoteLiteral( + IParser::Pos & pos, + IParser::Pos & pos_prev, + const char *& pos_unrecognized, + const char *& copy_from, + String & rewritten_query) +{ + /// Copy also whitespaces if any + const auto * end = + isWordOrNumber(pos->type) && noWhitespaces(pos_prev->end, pos->begin) + ? pos->end + : pos_prev->end; + String literal(pos_unrecognized, static_cast(end - pos_unrecognized)); + rewritten_query.append(copy_from, pos_unrecognized - copy_from).append(backQuoteMySQL(literal)); + copy_from = end; +} + +bool tryQuoteUnrecognizedTokens(const String & query, String & res) +{ + Tokens tokens(query.data(), query.data() + query.size()); + IParser::Pos pos(tokens, 0); + Expected expected; + String rewritten_query; + const char * copy_from = query.data(); + auto pos_prev = pos; + const char * pos_unrecognized = nullptr; + for (;pos->type != TokenType::EndOfStream; ++pos) + { + /// Commit quotes if any whitespaces found or the token is not a word + bool commit = !noWhitespaces(pos_prev->end, pos->begin) || (pos->type != TokenType::Error && !isWordOrNumber(pos->type)); + if (pos_unrecognized && commit) + { + quoteLiteral( + pos, + pos_prev, + pos_unrecognized, + copy_from, + rewritten_query); + pos_unrecognized = nullptr; + } + if (pos->type == TokenType::Error) + { + /// Find first appearance of the error token + if (!pos_unrecognized) + { + pos_unrecognized = + isWordOrNumber(pos_prev->type) && noWhitespaces(pos_prev->end, pos->begin) + ? pos_prev->begin + : pos->begin; + } + } + pos_prev = pos; + } + + /// There was EndOfStream but not committed unrecognized token + if (pos_unrecognized) + { + quoteLiteral( + pos, + pos_prev, + pos_unrecognized, + copy_from, + rewritten_query); + pos_unrecognized = nullptr; + } + + /// If no Errors found + if (copy_from == query.data()) + return false; + + auto size = static_cast(pos->end - copy_from); + rewritten_query.append(copy_from, size); + res = rewritten_query; + return true; +} + +} diff --git a/src/Databases/MySQL/tryQuoteUnrecognizedTokens.h b/src/Databases/MySQL/tryQuoteUnrecognizedTokens.h new file mode 100644 index 00000000000..582a297c485 --- /dev/null +++ b/src/Databases/MySQL/tryQuoteUnrecognizedTokens.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ + +bool tryQuoteUnrecognizedTokens(const String & query, String & res); + +} diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 3e928c3a811..b0a220eb1d2 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -34,16 +35,6 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -static String backQuoteMySQL(const String & x) -{ - String res(x.size(), '\0'); - { - WriteBufferFromString wb(res); - writeBackQuotedStringMySQL(x, wb); - } - return res; -} - StorageMySQL::StorageMySQL( const StorageID & table_id_, mysqlxx::PoolWithFailover && pool_, diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index c97c3e5e2a8..9130ccc359c 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -1581,6 +1581,128 @@ def utf8mb4_test(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE utf8mb4_test") +def utf8mb4_column_test(clickhouse_node, mysql_node, service_name): + db = "utf8mb4_column_test" + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"CREATE DATABASE {db}") + + # Full sync + mysql_node.query(f"CREATE TABLE {db}.unquoted (id INT primary key, 日期 DATETIME)") + mysql_node.query(f"CREATE TABLE {db}.quoted (id INT primary key, `日期` DATETIME)") + mysql_node.query(f"INSERT INTO {db}.unquoted VALUES(1, now())") + mysql_node.query(f"INSERT INTO {db}.quoted VALUES(1, now())") + clickhouse_node.query( + f"CREATE DATABASE {db} ENGINE = MaterializedMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')" + ) + + # Full sync replicated unquoted columns names since they use SHOW CREATE TABLE + # which returns quoted column names + check_query( + clickhouse_node, + f"/* expect: quoted unquoted */ SHOW TABLES FROM {db}", + "quoted\nunquoted\n", + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM {db}.unquoted", + "1\n", + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM {db}.quoted", + "1\n", + ) + + # Inc sync + mysql_node.query( + f"CREATE TABLE {db}.unquoted_new (id INT primary key, 日期 DATETIME)" + ) + mysql_node.query( + f"CREATE TABLE {db}.quoted_new (id INT primary key, `日期` DATETIME)" + ) + mysql_node.query(f"INSERT INTO {db}.unquoted_new VALUES(1, now())") + mysql_node.query(f"INSERT INTO {db}.quoted_new VALUES(1, now())") + mysql_node.query(f"INSERT INTO {db}.unquoted VALUES(2, now())") + mysql_node.query(f"INSERT INTO {db}.quoted VALUES(2, now())") + check_query( + clickhouse_node, + f"/* expect: 2 */ SELECT COUNT() FROM {db}.quoted", + "2\n", + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM {db}.quoted_new", + "1\n", + ) + check_query( + clickhouse_node, + f"/* expect: 2 */ SELECT COUNT() FROM {db}.unquoted", + "2\n", + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM {db}.unquoted_new", + "1\n", + ) + + clickhouse_node.query(f"DROP DATABASE IF EXISTS `{db}`") + mysql_node.query(f"DROP DATABASE IF EXISTS `{db}`") + + +def utf8mb4_name_test(clickhouse_node, mysql_node, service_name): + db = "您Hi您" + table = "日期" + mysql_node.query(f"DROP DATABASE IF EXISTS `{db}`") + clickhouse_node.query(f"DROP DATABASE IF EXISTS `{db}`") + mysql_node.query(f"CREATE DATABASE `{db}`") + mysql_node.query( + f"CREATE TABLE `{db}`.`{table}` (id INT(11) NOT NULL PRIMARY KEY, `{table}` DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" + ) + mysql_node.query(f"INSERT INTO `{db}`.`{table}` VALUES(1, now())") + mysql_node.query( + f"CREATE TABLE {db}.{table}_unquoted (id INT(11) NOT NULL PRIMARY KEY, {table} DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" + ) + mysql_node.query(f"INSERT INTO {db}.{table}_unquoted VALUES(1, now())") + clickhouse_node.query( + f"CREATE DATABASE `{db}` ENGINE = MaterializedMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')" + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}`", + "1\n", + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}_unquoted`", + "1\n", + ) + + # Inc sync + mysql_node.query( + f"CREATE TABLE `{db}`.`{table}2` (id INT(11) NOT NULL PRIMARY KEY, `{table}` DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" + ) + mysql_node.query(f"INSERT INTO `{db}`.`{table}2` VALUES(1, now())") + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}2`", + "1\n", + ) + + mysql_node.query( + f"CREATE TABLE {db}.{table}2_unquoted (id INT(11) NOT NULL PRIMARY KEY, {table} DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" + ) + mysql_node.query(f"INSERT INTO {db}.{table}2_unquoted VALUES(1, now())") + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}2_unquoted`", + "1\n", + ) + + clickhouse_node.query(f"DROP DATABASE IF EXISTS `{db}`") + mysql_node.query(f"DROP DATABASE IF EXISTS `{db}`") + + def system_parts_test(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS system_parts_test") clickhouse_node.query("DROP DATABASE IF EXISTS system_parts_test") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 32c1da8a2bd..e31ef70b4ad 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -381,6 +381,12 @@ def test_utf8mb4( ): materialized_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql57") materialized_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql80") + materialized_with_ddl.utf8mb4_column_test( + clickhouse_node, started_mysql_8_0, "mysql80" + ) + materialized_with_ddl.utf8mb4_name_test( + clickhouse_node, started_mysql_8_0, "mysql80" + ) def test_system_parts_table(started_cluster, started_mysql_8_0, clickhouse_node): From 3710c7238d9eaf0328170bafb03eb4b15ea5d67c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 24 Jul 2023 09:19:06 +0000 Subject: [PATCH 1090/2047] Fix test_throttling --- tests/integration/test_throttling/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index ff8e7154d0d..2b5e9312a4c 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -114,7 +114,7 @@ def node_update_config(mode, setting, value=None): def assert_took(took, should_took): - assert took >= should_took[0] * 0.9 and took < should_took[1] + assert took >= should_took[0] * 0.85 and took < should_took[1] @pytest.mark.parametrize( From 2471b032ab7a504d1997e9d3681bf97f0564273d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jul 2023 09:52:49 +0000 Subject: [PATCH 1091/2047] fix lightweight delete after drop of projection --- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++- .../02792_drop_projection_lwd.reference | 1 + .../0_stateless/02792_drop_projection_lwd.sql | 26 +++++++++++++++++++ 3 files changed, 29 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02792_drop_projection_lwd.reference create mode 100644 tests/queries/0_stateless/02792_drop_projection_lwd.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d773f380377..06a9b62d9de 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5693,7 +5693,8 @@ bool MergeTreeData::supportsLightweightDelete() const auto lock = lockParts(); for (const auto & part : data_parts_by_info) { - if (!part->supportLightweightDeleteMutate()) + if (part->getState() == MergeTreeDataPartState::Active + && !part->supportLightweightDeleteMutate()) return false; } return true; diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.reference b/tests/queries/0_stateless/02792_drop_projection_lwd.reference new file mode 100644 index 00000000000..6529ff889b0 --- /dev/null +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.reference @@ -0,0 +1 @@ +98 diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.sql b/tests/queries/0_stateless/02792_drop_projection_lwd.sql new file mode 100644 index 00000000000..fd446a8efe8 --- /dev/null +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS t_projections_lwd; + +CREATE TABLE t_projections_lwd (a UInt32, b UInt32) ENGINE = MergeTree ORDER BY a; + +INSERT INTO t_projections_lwd SELECT number, number FROM numbers(100); + +-- LWD works +DELETE FROM t_projections_lwd WHERE a = 0; + +-- add projection +ALTER TABLE t_projections_lwd ADD PROJECTION p_t_projections_lwd (SELECT * ORDER BY b); +ALTER TABLE t_projections_lwd MATERIALIZE PROJECTION p_t_projections_lwd; + +-- LWD does not work, as expected +DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError UNFINISHED } +KILL MUTATION WHERE database = currentDatabase() AND table = 't_projections_lwd' SYNC FORMAT Null; + +-- drop projection +SET mutations_sync = 2; +ALTER TABLE t_projections_lwd DROP projection p_t_projections_lwd; + +DELETE FROM t_projections_lwd WHERE a = 2; + +SELECT count() FROM t_projections_lwd; + +DROP TABLE t_projections_lwd; From 5da6c99f6df90ae5a8dde59f9cccce8cee48fc61 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 24 Jul 2023 12:02:27 +0200 Subject: [PATCH 1092/2047] Add comment --- tests/integration/test_throttling/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 2b5e9312a4c..62640394a85 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -114,6 +114,9 @@ def node_update_config(mode, setting, value=None): def assert_took(took, should_took): + # we need to decrease the lower limit because the server limits could + # be enforced by throttling some server background IO instead of query IO + # and we have no control over it assert took >= should_took[0] * 0.85 and took < should_took[1] From 79cc81890316338e35f13576cfd0360494e72645 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jul 2023 11:06:21 +0000 Subject: [PATCH 1093/2047] try to fix test --- .../02726_async_insert_flush_stress.sh | 33 ++++++++++++------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh index 5fafb773d16..876766d0780 100755 --- a/tests/queries/0_stateless/02726_async_insert_flush_stress.sh +++ b/tests/queries/0_stateless/02726_async_insert_flush_stress.sh @@ -11,7 +11,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function insert1() { url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" - while true; do + + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT CSV 1,"a" 2,"b" @@ -22,7 +24,9 @@ function insert1() function insert2() { url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" - while true; do + + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CURL} -sS "$url" -d 'INSERT INTO async_inserts FORMAT JSONEachRow {"id": 5, "s": "e"} {"id": 6, "s": "f"}' done } @@ -30,28 +34,33 @@ function insert2() function insert3() { url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=0" - while true; do + + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO FUNCTION remote('127.0.0.1', $CLICKHOUSE_DATABASE, async_inserts) VALUES (7, 'g') (8, 'h')" done } function select1() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts FORMAT Null" done } function select2() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do ${CLICKHOUSE_CLIENT} -q "SELECT * FROM system.asynchronous_inserts FORMAT Null" done } function flush1() { - while true; do + local TIMELIMIT=$((SECONDS+$1)) + while [ $SECONDS -lt "$TIMELIMIT" ]; do sleep 0.2 ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH ASYNC INSERT QUEUE" done @@ -70,14 +79,14 @@ export -f select2 export -f flush1 for _ in {1..5}; do - timeout $TIMEOUT bash -c insert1 & - timeout $TIMEOUT bash -c insert2 & - timeout $TIMEOUT bash -c insert3 & + insert1 $TIMEOUT & + insert2 $TIMEOUT & + insert3 $TIMEOUT & done -timeout $TIMEOUT bash -c select1 & -timeout $TIMEOUT bash -c select2 & -timeout $TIMEOUT bash -c flush1 & +select1 $TIMEOUT & +select2 $TIMEOUT & +flush1 $TIMEOUT & wait From 21097209d2e709db8022782a02980e52a7bc5df7 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Mon, 24 Jul 2023 15:41:21 +0300 Subject: [PATCH 1094/2047] Revert "Remove `toDecimalString`" --- .../functions/type-conversion-functions.md | 38 +++ .../functions/type-conversion-functions.md | 38 +++ src/Functions/FunctionToDecimalString.cpp | 22 ++ src/Functions/FunctionToDecimalString.h | 312 ++++++++++++++++++ src/IO/WriteHelpers.h | 39 ++- .../02676_to_decimal_string.reference | 21 ++ .../0_stateless/02676_to_decimal_string.sql | 35 ++ 7 files changed, 492 insertions(+), 13 deletions(-) create mode 100644 src/Functions/FunctionToDecimalString.cpp create mode 100644 src/Functions/FunctionToDecimalString.h create mode 100644 tests/queries/0_stateless/02676_to_decimal_string.reference create mode 100644 tests/queries/0_stateless/02676_to_decimal_string.sql diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index c2bd525c483..36f40b37238 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -945,6 +945,44 @@ Result: └────────────┴───────┘ ``` +## toDecimalString + +Converts a numeric value to String with the number of fractional digits in the output specified by the user. + +**Syntax** + +``` sql +toDecimalString(number, scale) +``` + +**Parameters** + +- `number` — Value to be represented as String, [Int, UInt](/docs/en/sql-reference/data-types/int-uint.md), [Float](/docs/en/sql-reference/data-types/float.md), [Decimal](/docs/en/sql-reference/data-types/decimal.md), +- `scale` — Number of fractional digits, [UInt8](/docs/en/sql-reference/data-types/int-uint.md). + * Maximum scale for [Decimal](/docs/en/sql-reference/data-types/decimal.md) and [Int, UInt](/docs/en/sql-reference/data-types/int-uint.md) types is 77 (it is the maximum possible number of significant digits for Decimal), + * Maximum scale for [Float](/docs/en/sql-reference/data-types/float.md) is 60. + +**Returned value** + +- Input value represented as [String](/docs/en/sql-reference/data-types/string.md) with given number of fractional digits (scale). + The number is rounded up or down according to common arithmetic in case requested scale is smaller than original number's scale. + +**Example** + +Query: + +``` sql +SELECT toDecimalString(CAST('64.32', 'Float64'), 5); +``` + +Result: + +```response +┌toDecimalString(CAST('64.32', 'Float64'), 5)─┐ +│ 64.32000 │ +└─────────────────────────────────────────────┘ +``` + ## reinterpretAsUInt(8\|16\|32\|64) ## reinterpretAsInt(8\|16\|32\|64) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 088b1a9a1f1..e53104d8d71 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -762,6 +762,44 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut; └────────────┴───────┘ ``` +## toDecimalString + +Принимает любой численный тип первым аргументом, возвращает строковое десятичное представление числа с точностью, заданной вторым аргументом. + +**Синтаксис** + +``` sql +toDecimalString(number, scale) +``` + +**Параметры** + +- `number` — Значение любого числового типа: [Int, UInt](/docs/ru/sql-reference/data-types/int-uint.md), [Float](/docs/ru/sql-reference/data-types/float.md), [Decimal](/docs/ru/sql-reference/data-types/decimal.md), +- `scale` — Требуемое количество десятичных знаков после запятой, [UInt8](/docs/ru/sql-reference/data-types/int-uint.md). + * Значение `scale` для типов [Decimal](/docs/ru/sql-reference/data-types/decimal.md) и [Int, UInt](/docs/ru/sql-reference/data-types/int-uint.md) должно не превышать 77 (так как это наибольшее количество значимых символов для этих типов), + * Значение `scale` для типа [Float](/docs/ru/sql-reference/data-types/float.md) не должно превышать 60. + +**Возвращаемое значение** + +- Строка ([String](/docs/en/sql-reference/data-types/string.md)), представляющая собой десятичное представление входного числа с заданной длиной дробной части. + При необходимости число округляется по стандартным правилам арифметики. + +**Пример использования** + +Запрос: + +``` sql +SELECT toDecimalString(CAST('64.32', 'Float64'), 5); +``` + +Результат: + +```response +┌─toDecimalString(CAST('64.32', 'Float64'), 5)┐ +│ 64.32000 │ +└─────────────────────────────────────────────┘ +``` + ## reinterpretAsUInt(8\|16\|32\|64) {#reinterpretasuint8163264} ## reinterpretAsInt(8\|16\|32\|64) {#reinterpretasint8163264} diff --git a/src/Functions/FunctionToDecimalString.cpp b/src/Functions/FunctionToDecimalString.cpp new file mode 100644 index 00000000000..fe417b19137 --- /dev/null +++ b/src/Functions/FunctionToDecimalString.cpp @@ -0,0 +1,22 @@ +#include +#include +#include + +namespace DB +{ + +REGISTER_FUNCTION(ToDecimalString) +{ + factory.registerFunction( + FunctionDocumentation{ + .description=R"( +Returns string representation of a number. First argument is the number of any numeric type, +second argument is the desired number of digits in fractional part. Returns String. + + )", + .examples{{"toDecimalString", "SELECT toDecimalString(2.1456,2)", ""}}, + .categories{"String"} + }, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h new file mode 100644 index 00000000000..6ae007e6b66 --- /dev/null +++ b/src/Functions/FunctionToDecimalString.h @@ -0,0 +1,312 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER; +} + +class FunctionToDecimalString : public IFunction +{ +public: + static constexpr auto name = "toDecimalString"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isNumber(*arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal first argument for formatDecimal function: got {}, expected numeric type", + arguments[0]->getName()); + + if (!isUInt8(*arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal second argument for formatDecimal function: got {}, expected UInt8", + arguments[1]->getName()); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + +private: + /// For operations with Integer/Float + template + void vectorConstant(const FromVectorType & vec_from, UInt8 precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const + { + size_t input_rows_count = vec_from.size(); + result_offsets.resize(input_rows_count); + + /// Buffer is used here and in functions below because resulting size cannot be precisely anticipated, + /// and buffer resizes on-the-go. Also, .count() provided by buffer is convenient in this case. + WriteBufferFromVector buf_to(vec_to); + + for (size_t i = 0; i < input_rows_count; ++i) + { + format(vec_from[i], buf_to, precision); + result_offsets[i] = buf_to.count(); + } + + buf_to.finalize(); + } + + template + void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector::Container & vec_precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const + { + size_t input_rows_count = vec_from.size(); + result_offsets.resize(input_rows_count); + + WriteBufferFromVector buf_to(vec_to); + + constexpr size_t max_digits = std::numeric_limits::digits10; + + for (size_t i = 0; i < input_rows_count; ++i) + { + if (vec_precision[i] > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested, shall not be more than {}", max_digits); + format(vec_from[i], buf_to, vec_precision[i]); + result_offsets[i] = buf_to.count(); + } + + buf_to.finalize(); + } + + template + void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const + { + size_t input_rows_count = vec_precision.size(); + result_offsets.resize(input_rows_count); + + WriteBufferFromVector buf_to(vec_to); + + constexpr size_t max_digits = std::numeric_limits::digits10; + + for (size_t i = 0; i < input_rows_count; ++i) + { + if (vec_precision[i] > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested, shall not be more than {}", max_digits); + format(value_from, buf_to, vec_precision[i]); + result_offsets[i] = buf_to.count(); + } + + buf_to.finalize(); + } + + /// For operations with Decimal + template + void vectorConstant(const FirstArgVectorType & vec_from, UInt8 precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const + { + /// There are no more than 77 meaning digits (as it is the max length of UInt256). So we can limit it with 77. + constexpr size_t max_digits = std::numeric_limits::digits10; + if (precision > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); + + WriteBufferFromVector buf_to(vec_to); + size_t input_rows_count = vec_from.size(); + result_offsets.resize(input_rows_count); + + for (size_t i = 0; i < input_rows_count; ++i) + { + writeText(vec_from[i], from_scale, buf_to, true, true, precision); + writeChar(0, buf_to); + result_offsets[i] = buf_to.count(); + } + buf_to.finalize(); + } + + template + void vectorVector(const FirstArgVectorType & vec_from, const ColumnVector::Container & vec_precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const + { + size_t input_rows_count = vec_from.size(); + result_offsets.resize(input_rows_count); + + WriteBufferFromVector buf_to(vec_to); + + constexpr size_t max_digits = std::numeric_limits::digits10; + + for (size_t i = 0; i < input_rows_count; ++i) + { + if (vec_precision[i] > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); + writeText(vec_from[i], from_scale, buf_to, true, true, vec_precision[i]); + writeChar(0, buf_to); + result_offsets[i] = buf_to.count(); + } + buf_to.finalize(); + } + + template + void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, + ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const + { + size_t input_rows_count = vec_precision.size(); + result_offsets.resize(input_rows_count); + + WriteBufferFromVector buf_to(vec_to); + + constexpr size_t max_digits = std::numeric_limits::digits10; + + for (size_t i = 0; i < input_rows_count; ++i) + { + if (vec_precision[i] > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); + writeText(value_from, from_scale, buf_to, true, true, vec_precision[i]); + writeChar(0, buf_to); + result_offsets[i] = buf_to.count(); + } + buf_to.finalize(); + } + + template + static void format(T value, DB::WriteBuffer & out, UInt8 precision) + { + /// Maximum of 60 is hard-coded in 'double-conversion/double-conversion.h' for floating point values, + /// Catch this here to give user a more reasonable error. + if (precision > 60) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too high precision requested for Float, must not be more than 60, got {}", Int8(precision)); + + DB::DoubleConverter::BufferType buffer; + double_conversion::StringBuilder builder{buffer, sizeof(buffer)}; + + const auto result = DB::DoubleConverter::instance().ToFixed(value, precision, &builder); + + if (!result) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, "Error processing number: {}", value); + + out.write(buffer, builder.position()); + writeChar(0, out); + } + + template + static void format(T value, DB::WriteBuffer & out, UInt8 precision) + { + /// Fractional part for Integer is just trailing zeros. Let's limit it with 77 (like with Decimals). + constexpr size_t max_digits = std::numeric_limits::digits10; + if (precision > max_digits) + throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, + "Too many fractional digits requested, shall not be more than {}", max_digits); + writeText(value, out); + if (precision > 0) [[likely]] + { + writeChar('.', out); + for (int i = 0; i < precision; ++i) + writeChar('0', out); + writeChar(0, out); + } + } + +public: + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + switch (arguments[0].type->getTypeId()) + { + case TypeIndex::UInt8: return executeType(arguments); + case TypeIndex::UInt16: return executeType(arguments); + case TypeIndex::UInt32: return executeType(arguments); + case TypeIndex::UInt64: return executeType(arguments); + case TypeIndex::UInt128: return executeType(arguments); + case TypeIndex::UInt256: return executeType(arguments); + case TypeIndex::Int8: return executeType(arguments); + case TypeIndex::Int16: return executeType(arguments); + case TypeIndex::Int32: return executeType(arguments); + case TypeIndex::Int64: return executeType(arguments); + case TypeIndex::Int128: return executeType(arguments); + case TypeIndex::Int256: return executeType(arguments); + case TypeIndex::Float32: return executeType(arguments); + case TypeIndex::Float64: return executeType(arguments); + case TypeIndex::Decimal32: return executeType(arguments); + case TypeIndex::Decimal64: return executeType(arguments); + case TypeIndex::Decimal128: return executeType(arguments); + case TypeIndex::Decimal256: return executeType(arguments); + default: + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", + arguments[0].column->getName(), getName()); + } + } + +private: + template + ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const + { + const auto * from_col_const = typeid_cast(arguments[0].column.get()); + const auto * precision_col = checkAndGetColumn>(arguments[1].column.get()); + const auto * precision_col_const = typeid_cast(arguments[1].column.get()); + + auto result_col = ColumnString::create(); + auto * result_col_string = assert_cast(result_col.get()); + ColumnString::Chars & result_chars = result_col_string->getChars(); + ColumnString::Offsets & result_offsets = result_col_string->getOffsets(); + + if constexpr (is_decimal) + { + const auto * from_col = checkAndGetColumn>(arguments[0].column.get()); + UInt8 from_scale = from_col->getScale(); + + if (from_col) + { + if (precision_col_const) + vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets, from_scale); + else + vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale); + } + else if (from_col_const) + constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets, from_scale); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); + } + else + { + const auto * from_col = checkAndGetColumn>(arguments[0].column.get()); + if (from_col) + { + if (precision_col_const) + vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets); + else + vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets); + } + else if (from_col_const) + constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); + } + + return result_col; + } +}; + +} diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 76778543bd0..aa4c9b17e48 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -905,26 +905,26 @@ inline void writeText(const IPv4 & x, WriteBuffer & buf) { writeIPv4Text(x, buf) inline void writeText(const IPv6 & x, WriteBuffer & buf) { writeIPv6Text(x, buf); } template -void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros) +void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros, + bool fixed_fractional_length, UInt32 fractional_length) { /// If it's big integer, but the number of digits is small, /// use the implementation for smaller integers for more efficient arithmetic. - if constexpr (std::is_same_v) { if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); return; } } @@ -932,24 +932,36 @@ void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool { if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); return; } else if (x <= std::numeric_limits::max()) { - writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros); + writeDecimalFractional(static_cast(x), scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); return; } } constexpr size_t max_digits = std::numeric_limits::digits10; assert(scale <= max_digits); + assert(fractional_length <= max_digits); + char buf[max_digits]; - memset(buf, '0', scale); + memset(buf, '0', std::max(scale, fractional_length)); T value = x; Int32 last_nonzero_pos = 0; - for (Int32 pos = scale - 1; pos >= 0; --pos) + + if (fixed_fractional_length && fractional_length < scale) + { + T new_value = value / DecimalUtils::scaleMultiplier(scale - fractional_length - 1); + auto round_carry = new_value % 10; + value = new_value / 10; + if (round_carry >= 5) + value += 1; + } + + for (Int32 pos = fixed_fractional_length ? std::min(scale - 1, fractional_length - 1) : scale - 1; pos >= 0; --pos) { auto remainder = value % 10; value /= 10; @@ -961,11 +973,12 @@ void writeDecimalFractional(const T & x, UInt32 scale, WriteBuffer & ostr, bool } writeChar('.', ostr); - ostr.write(buf, trailing_zeros ? scale : last_nonzero_pos + 1); + ostr.write(buf, fixed_fractional_length ? fractional_length : (trailing_zeros ? scale : last_nonzero_pos + 1)); } template -void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros) +void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zeros, + bool fixed_fractional_length = false, UInt32 fractional_length = 0) { T part = DecimalUtils::getWholePart(x, scale); @@ -976,7 +989,7 @@ void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer writeIntText(part, ostr); - if (scale) + if (scale || (fixed_fractional_length && fractional_length > 0)) { part = DecimalUtils::getFractionalPart(x, scale); if (part || trailing_zeros) @@ -984,7 +997,7 @@ void writeText(Decimal x, UInt32 scale, WriteBuffer & ostr, bool trailing_zer if (part < 0) part *= T(-1); - writeDecimalFractional(part, scale, ostr, trailing_zeros); + writeDecimalFractional(part, scale, ostr, trailing_zeros, fixed_fractional_length, fractional_length); } } } diff --git a/tests/queries/0_stateless/02676_to_decimal_string.reference b/tests/queries/0_stateless/02676_to_decimal_string.reference new file mode 100644 index 00000000000..4c27ee5b528 --- /dev/null +++ b/tests/queries/0_stateless/02676_to_decimal_string.reference @@ -0,0 +1,21 @@ +2.00000000000000000000000000000000000000000000000000000000000000000000000000000 +2.12 +-2.00000000000000000000000000000000000000000000000000000000000000000000000000000 +-2.12 +2.987600000000000033395508580724708735942840576171875000000000 +2.15 +-2.987600000000000033395508580724708735942840576171875000000000 +-2.15 +64.1230010986 +64.2340000000 +-64.1230010986 +-64.2340000000 +-32.345 +32.34500000000000000000000000000000000000000000000000000000000000000000000000000 +32.46 +-64.5671232345 +128.78932312332132985464 +-128.78932312332132985464 +128.78932312332132985464000000000000000000000000000000000000000000000000000000000 +128.7893231233 +-128.78932312332132985464123123789323123321329854600000000000000000000000000000000 diff --git a/tests/queries/0_stateless/02676_to_decimal_string.sql b/tests/queries/0_stateless/02676_to_decimal_string.sql new file mode 100644 index 00000000000..563d60c62c7 --- /dev/null +++ b/tests/queries/0_stateless/02676_to_decimal_string.sql @@ -0,0 +1,35 @@ +-- Regular types +SELECT toDecimalString(2, 77); -- more digits required than exist +SELECT toDecimalString(2.123456, 2); -- rounding +SELECT toDecimalString(-2, 77); -- more digits required than exist +SELECT toDecimalString(-2.123456, 2); -- rounding + +SELECT toDecimalString(2.9876, 60); -- more digits required than exist (took 60 as it is float by default) +SELECT toDecimalString(2.1456, 2); -- rounding +SELECT toDecimalString(-2.9876, 60); -- more digits required than exist +SELECT toDecimalString(-2.1456, 2); -- rounding + +-- Float32 and Float64 tests. No sense to test big float precision -- the result will be a mess anyway. +SELECT toDecimalString(64.123::Float32, 10); +SELECT toDecimalString(64.234::Float64, 10); +SELECT toDecimalString(-64.123::Float32, 10); +SELECT toDecimalString(-64.234::Float64, 10); + +-- Decimals +SELECT toDecimalString(-32.345::Decimal32(3), 3); +SELECT toDecimalString(32.345::Decimal32(3), 77); -- more digits required than exist +SELECT toDecimalString(32.456::Decimal32(3), 2); -- rounding +SELECT toDecimalString('-64.5671232345'::Decimal64(10), 10); +SELECT toDecimalString('128.78932312332132985464'::Decimal128(20), 20); +SELECT toDecimalString('-128.78932312332132985464123123'::Decimal128(26), 20); -- rounding +SELECT toDecimalString('128.78932312332132985464'::Decimal128(20), 77); -- more digits required than exist +SELECT toDecimalString('128.789323123321329854641231237893231233213298546'::Decimal256(45), 10); -- rounding +SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 77); -- more digits required than exist + +-- Max number of decimal fractional digits is defined as 77 for Int/UInt/Decimal and 60 for Float. +-- These values shall work OK. +SELECT toDecimalString('32.32'::Float32, 61); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} +SELECT toDecimalString('64.64'::Float64, 61); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} +SELECT toDecimalString('88'::UInt8, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} +SELECT toDecimalString('646464'::Int256, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} +SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} From df5ff1383c5c6f7e24cb6933246fc04cf5dfe702 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 24 Jul 2023 14:57:05 +0200 Subject: [PATCH 1095/2047] Fix settings not applied for explain query when format provided (#51859) --- src/Interpreters/InterpreterSetQuery.cpp | 3 +++ ..._explain_settings_not_applied_bug.reference | 11 +++++++++++ .../02798_explain_settings_not_applied_bug.sql | 18 ++++++++++++++++++ 3 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference create mode 100644 tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index 6db57a4f950..e9118b747e5 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -65,6 +65,9 @@ void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMuta } else if (const auto * explain_query = ast->as()) { + if (explain_query->settings_ast) + InterpreterSetQuery(explain_query->settings_ast, context_).executeForCurrentContext(); + applySettingsFromQuery(explain_query->getExplainedQuery(), context_); } else if (const auto * query_with_output = dynamic_cast(ast.get())) diff --git a/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference new file mode 100644 index 00000000000..6fc36a0ba01 --- /dev/null +++ b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference @@ -0,0 +1,11 @@ + explain + + (Expression) + ExpressionTransform + (Aggregating) + FinalizeAggregatedTransform + AggregatingInOrderTransform + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeInOrder 0 → 1 diff --git a/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql new file mode 100644 index 00000000000..76f2129abfa --- /dev/null +++ b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.sql @@ -0,0 +1,18 @@ +SET read_in_order_two_level_merge_threshold=1000000; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(a UInt64) +ENGINE = MergeTree +ORDER BY a; + +INSERT INTO t SELECT * FROM numbers_mt(1e3); +OPTIMIZE TABLE t FINAL; + +EXPLAIN PIPELINE +SELECT a +FROM t +GROUP BY a +FORMAT PrettySpace +SETTINGS optimize_aggregation_in_order = 1; + +DROP TABLE t; From 9ff409f16ed9ba875904651e353b80d1782d9b1d Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Mon, 24 Jul 2023 13:40:06 +0800 Subject: [PATCH 1096/2047] Enforce type check for function arguments in preimage optimization Date/Date32/DateTime/DateTime64 columns are required for arguments of time converters, such as toYear and toYYYYMM. --- .../Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp index f7f69598c4b..7205ac299a9 100644 --- a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp +++ b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp @@ -91,6 +91,9 @@ public: const auto * column_id = func_node->getArguments().getNodes()[0]->as(); if (!column_id) return; + const auto * column_type = column_id->getColumnType().get(); + if (!isDateOrDate32(column_type) && !isDateTime(column_type) && !isDateTime64(column_type)) return; + const auto & converter = FunctionFactory::instance().tryGet(func_node->getFunctionName(), getContext()); if (!converter) return; From c7239c64ea36a6994cd88d34edc3774243472a68 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 24 Jul 2023 15:16:44 +0200 Subject: [PATCH 1097/2047] Remove unused code --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 48adf36e678..3eba9a9de24 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -145,9 +145,6 @@ bool IMergeTreeSelectAlgorithm::getNewTask() ChunkAndProgress IMergeTreeSelectAlgorithm::read() { - size_t num_read_rows = 0; - size_t num_read_bytes = 0; - while (!is_cancelled) { try @@ -178,10 +175,6 @@ ChunkAndProgress IMergeTreeSelectAlgorithm::read() ordered_columns.push_back(res.block.getByName(name).column); } - /// Account a progress from previous empty chunks. - res.num_read_rows += num_read_rows; - res.num_read_bytes += num_read_bytes; - return ChunkAndProgress{ .chunk = Chunk(ordered_columns, res.row_count), .num_read_rows = res.num_read_rows, @@ -194,7 +187,7 @@ ChunkAndProgress IMergeTreeSelectAlgorithm::read() } } - return {Chunk(), num_read_rows, num_read_bytes, true}; + return {Chunk(), 0, 0, true}; } void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask( From 94fa2f18b174a0f88b59479aac40fbc2abbe73a7 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 24 Jul 2023 15:16:56 +0200 Subject: [PATCH 1098/2047] fix analyzer --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 17 +++++++++++++++++ .../02701_non_parametric_function.reference | 1 - .../02701_non_parametric_function.sql | 9 --------- 3 files changed, 17 insertions(+), 10 deletions(-) delete mode 100644 tests/queries/0_stateless/02701_non_parametric_function.reference delete mode 100644 tests/queries/0_stateless/02701_non_parametric_function.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index da8933aabaa..fd16b6e168b 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -116,6 +116,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; + extern const int FUNCTION_CANNOT_HAVE_PARAMETERS; } /** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h first. @@ -4896,6 +4897,12 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi lambda_expression_untyped->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); + if (!parameters.empty()) + { + throw Exception( + ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS, "Function {} is not parametric", function_node.formatASTForErrorMessage()); + } + auto lambda_expression_clone = lambda_expression_untyped->clone(); IdentifierResolveScope lambda_scope(lambda_expression_clone, &scope /*parent_scope*/); @@ -5012,9 +5019,13 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi } FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, scope.context, parameters); + bool is_executable_udf = true; if (!function) + { function = FunctionFactory::instance().tryGet(function_name, scope.context); + is_executable_udf = false; + } if (!function) { @@ -5065,6 +5076,12 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi return result_projection_names; } + /// Executable UDFs may have parameters. They are checked in UserDefinedExecutableFunctionFactory. + if (!parameters.empty() && !is_executable_udf) + { + throw Exception(ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS, "Function {} is not parametric", function_name); + } + /** For lambda arguments we need to initialize lambda argument types DataTypeFunction using `getLambdaArgumentTypes` function. * Then each lambda arguments are initialized with columns, where column source is lambda. * This information is important for later steps of query processing. diff --git a/tests/queries/0_stateless/02701_non_parametric_function.reference b/tests/queries/0_stateless/02701_non_parametric_function.reference deleted file mode 100644 index 00750edc07d..00000000000 --- a/tests/queries/0_stateless/02701_non_parametric_function.reference +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/tests/queries/0_stateless/02701_non_parametric_function.sql b/tests/queries/0_stateless/02701_non_parametric_function.sql deleted file mode 100644 index 11c03372a73..00000000000 --- a/tests/queries/0_stateless/02701_non_parametric_function.sql +++ /dev/null @@ -1,9 +0,0 @@ --- Tags: no-parallel - -SELECT * FROM system.numbers WHERE number > toUInt64(10)(number) LIMIT 10; -- { serverError 309 } - -CREATE FUNCTION IF NOT EXISTS sum_udf as (x, y) -> (x + y); - -SELECT sum_udf(1)(1, 2); - -DROP FUNCTION IF EXISTS sum_udf; From c6e6fd761317662c05532d695c20be72f8e847d2 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 24 Jul 2023 15:58:21 +0200 Subject: [PATCH 1099/2047] Shard `OpenedFileCache` to avoid lock contention (#51341) * shard OpenedFileCache to avoid lock contention * Update OpenedFileCache.h * fix build --------- Co-authored-by: Alexey Milovidov --- src/Common/ProfileEvents.cpp | 1 + src/IO/OpenedFileCache.h | 109 +++++++++++++++++++++-------------- 2 files changed, 68 insertions(+), 42 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 4a656e38edf..f18a67fa565 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -45,6 +45,7 @@ M(MMappedFileCacheMisses, "Number of times a file has not been found in the MMap cache (for the 'mmap' read_method), so we had to mmap it again.") \ M(OpenedFileCacheHits, "Number of times a file has been found in the opened file cache, so we didn't have to open it again.") \ M(OpenedFileCacheMisses, "Number of times a file has been found in the opened file cache, so we had to open it again.") \ + M(OpenedFileCacheMicroseconds, "Amount of time spent executing OpenedFileCache methods.") \ M(AIOWrite, "Number of writes with Linux or FreeBSD AIO interface") \ M(AIOWriteBytes, "Number of bytes written with Linux or FreeBSD AIO interface") \ M(AIORead, "Number of reads with Linux or FreeBSD AIO interface") \ diff --git a/src/IO/OpenedFileCache.h b/src/IO/OpenedFileCache.h index 61e502a494b..2cecc675af7 100644 --- a/src/IO/OpenedFileCache.h +++ b/src/IO/OpenedFileCache.h @@ -4,14 +4,18 @@ #include #include -#include #include +#include +#include + +#include namespace ProfileEvents { extern const Event OpenedFileCacheHits; extern const Event OpenedFileCacheMisses; + extern const Event OpenedFileCacheMicroseconds; } namespace DB @@ -26,57 +30,79 @@ namespace DB */ class OpenedFileCache { -private: - using Key = std::pair; + class OpenedFileMap + { + using Key = std::pair; - using OpenedFileWeakPtr = std::weak_ptr; - using Files = std::map; + using OpenedFileWeakPtr = std::weak_ptr; + using Files = std::map; - Files files; - std::mutex mutex; + Files files; + std::mutex mutex; + + public: + using OpenedFilePtr = std::shared_ptr; + + OpenedFilePtr get(const std::string & path, int flags) + { + Key key(path, flags); + + std::lock_guard lock(mutex); + + auto [it, inserted] = files.emplace(key, OpenedFilePtr{}); + if (!inserted) + { + if (auto res = it->second.lock()) + { + ProfileEvents::increment(ProfileEvents::OpenedFileCacheHits); + return res; + } + } + ProfileEvents::increment(ProfileEvents::OpenedFileCacheMisses); + + OpenedFilePtr res + { + new OpenedFile(path, flags), + [key, this](auto ptr) + { + { + std::lock_guard another_lock(mutex); + files.erase(key); + } + delete ptr; + } + }; + + it->second = res; + return res; + } + + void remove(const std::string & path, int flags) + { + Key key(path, flags); + std::lock_guard lock(mutex); + files.erase(key); + } + }; + + static constexpr size_t buckets = 1024; + std::vector impls{buckets}; public: - using OpenedFilePtr = std::shared_ptr; + using OpenedFilePtr = OpenedFileMap::OpenedFilePtr; OpenedFilePtr get(const std::string & path, int flags) { - Key key(path, flags); - - std::lock_guard lock(mutex); - - auto [it, inserted] = files.emplace(key, OpenedFilePtr{}); - if (!inserted) - { - if (auto res = it->second.lock()) - { - ProfileEvents::increment(ProfileEvents::OpenedFileCacheHits); - return res; - } - } - ProfileEvents::increment(ProfileEvents::OpenedFileCacheMisses); - - OpenedFilePtr res - { - new OpenedFile(path, flags), - [key, this](auto ptr) - { - { - std::lock_guard another_lock(mutex); - files.erase(key); - } - delete ptr; - } - }; - - it->second = res; - return res; + ProfileEventTimeIncrement watch(ProfileEvents::OpenedFileCacheMicroseconds); + const auto bucket = CityHash_v1_0_2::CityHash64(path.data(), path.length()) % buckets; + return impls[bucket].get(path, flags); } void remove(const std::string & path, int flags) { - Key key(path, flags); - std::lock_guard lock(mutex); - files.erase(key); + ProfileEventTimeIncrement watch(ProfileEvents::OpenedFileCacheMicroseconds); + const auto bucket = CityHash_v1_0_2::CityHash64(path.data(), path.length()) % buckets; + impls[bucket].remove(path, flags); } static OpenedFileCache & instance() @@ -87,5 +113,4 @@ public: }; using OpenedFileCachePtr = std::shared_ptr; - } From fbe9f8d0f653cedd30640c42968df1ac8cf78dff Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jul 2023 14:12:01 +0000 Subject: [PATCH 1100/2047] fix recalculation of skip indexes and projcetion in ALTER DELETE --- src/Interpreters/MutationsInterpreter.cpp | 89 +++++++++++------ src/Interpreters/MutationsInterpreter.h | 3 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 ++ src/Storages/MergeTree/IMergeTreeDataPart.h | 3 + src/Storages/MergeTree/MutateTask.cpp | 98 +++++++++++++++++-- src/Storages/StorageInMemoryMetadata.cpp | 6 +- src/Storages/StorageInMemoryMetadata.h | 4 +- ...alter_delete_indexes_projections.reference | 6 ++ ...02832_alter_delete_indexes_projections.sql | 26 +++++ 9 files changed, 201 insertions(+), 40 deletions(-) create mode 100644 tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference create mode 100644 tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 25c52ad8925..1a7cbb45999 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -113,13 +113,14 @@ QueryTreeNodePtr prepareQueryAffectedQueryTree(const std::vector & has_index_or_projection) + const StorageInMemoryMetadata::HasDependencyCallback & has_dependency) { NameSet new_updated_columns = updated_columns; ColumnDependencies dependencies; + while (!new_updated_columns.empty()) { - auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_index_or_projection); + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_dependency); new_updated_columns.clear(); for (const auto & dependency : new_dependencies) { @@ -292,9 +293,14 @@ bool MutationsInterpreter::Source::materializeTTLRecalculateOnly() const return data && data->getSettings()->materialize_ttl_recalculate_only; } -bool MutationsInterpreter::Source::hasIndexOrProjection(const String & file_name) const +bool MutationsInterpreter::Source::hasSecondaryIndex(const String & name) const { - return part && part->checksums.has(file_name); + return part && part->hasSecondaryIndex(name); +} + +bool MutationsInterpreter::Source::hasProjection(const String & name) const +{ + return part && part->hasProjection(name); } static Names getAvailableColumnsWithVirtuals(StorageMetadataPtr metadata_snapshot, const IStorage & storage) @@ -533,13 +539,24 @@ void MutationsInterpreter::prepare(bool dry_run) validateUpdateColumns(source, metadata_snapshot, updated_columns, column_to_affected_materialized); } - std::function has_index_or_projection - = [&](const String & file_name) { return source.hasIndexOrProjection(file_name); }; + StorageInMemoryMetadata::HasDependencyCallback has_dependency = + [&](const String & name, ColumnDependency::Kind kind) + { + if (kind == ColumnDependency::PROJECTION) + return source.hasProjection(name); + + if (kind == ColumnDependency::SKIP_INDEX) + return source.hasSecondaryIndex(name); + + return true; + }; if (settings.recalculate_dependencies_of_updated_columns) - dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns, has_index_or_projection); + dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns, has_dependency); + bool has_alter_delete = false; std::vector read_columns; + /// First, break a sequence of commands into stages. for (auto & command : commands) { @@ -558,6 +575,7 @@ void MutationsInterpreter::prepare(bool dry_run) predicate = makeASTFunction("isZeroOrNull", predicate); stages.back().filters.push_back(predicate); + has_alter_delete = true; } else if (command.type == MutationCommand::UPDATE) { @@ -692,8 +710,7 @@ void MutationsInterpreter::prepare(bool dry_run) if (it == std::cend(indices_desc)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown index: {}", command.index_name); - if (!source.hasIndexOrProjection("skp_idx_" + it->name + ".idx") - && !source.hasIndexOrProjection("skp_idx_" + it->name + ".idx2")) + if (!source.hasSecondaryIndex(it->name)) { auto query = (*it).expression_list_ast->clone(); auto syntax_result = TreeRewriter(context).analyze(query, all_columns); @@ -707,7 +724,7 @@ void MutationsInterpreter::prepare(bool dry_run) { mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); const auto & projection = projections_desc.get(command.projection_name); - if (!source.hasIndexOrProjection(projection.getDirectoryName())) + if (!source.hasProjection(projection.name)) { for (const auto & column : projection.required_columns) dependencies.emplace(column, ColumnDependency::PROJECTION); @@ -732,7 +749,7 @@ void MutationsInterpreter::prepare(bool dry_run) // just recalculate ttl_infos without remove expired data auto all_columns_vec = all_columns.getNames(); auto new_dependencies = metadata_snapshot->getColumnDependencies( - NameSet(all_columns_vec.begin(), all_columns_vec.end()), false, has_index_or_projection); + NameSet(all_columns_vec.begin(), all_columns_vec.end()), false, has_dependency); for (const auto & dependency : new_dependencies) { if (dependency.kind == ColumnDependency::TTL_EXPRESSION) @@ -758,7 +775,7 @@ void MutationsInterpreter::prepare(bool dry_run) auto all_columns_vec = all_columns.getNames(); auto all_dependencies = getAllColumnDependencies( - metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end()), has_index_or_projection); + metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end()), has_dependency); for (const auto & dependency : all_dependencies) { @@ -767,7 +784,7 @@ void MutationsInterpreter::prepare(bool dry_run) } /// Recalc only skip indices and projections of columns which could be updated by TTL. - auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_index_or_projection); + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_dependency); for (const auto & dependency : new_dependencies) { if (dependency.kind == ColumnDependency::SKIP_INDEX || dependency.kind == ColumnDependency::PROJECTION) @@ -861,30 +878,44 @@ void MutationsInterpreter::prepare(bool dry_run) for (const auto & index : metadata_snapshot->getSecondaryIndices()) { - if (source.hasIndexOrProjection("skp_idx_" + index.name + ".idx") || source.hasIndexOrProjection("skp_idx_" + index.name + ".idx2")) + if (!source.hasSecondaryIndex(index.name)) + continue; + + if (has_alter_delete) { - const auto & index_cols = index.expression->getRequiredColumns(); - bool changed = std::any_of( - index_cols.begin(), - index_cols.end(), - [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); - if (changed) - materialized_indices.insert(index.name); + materialized_indices.insert(index.name); + continue; } + + const auto & index_cols = index.expression->getRequiredColumns(); + bool changed = std::any_of( + index_cols.begin(), + index_cols.end(), + [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); + + if (changed) + materialized_indices.insert(index.name); } for (const auto & projection : metadata_snapshot->getProjections()) { - if (source.hasIndexOrProjection(projection.getDirectoryName())) + if (!source.hasProjection(projection.name)) + continue; + + if (has_alter_delete) { - const auto & projection_cols = projection.required_columns; - bool changed = std::any_of( - projection_cols.begin(), - projection_cols.end(), - [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); - if (changed) - materialized_projections.insert(projection.name); + materialized_projections.insert(projection.name); + continue; } + + const auto & projection_cols = projection.required_columns; + bool changed = std::any_of( + projection_cols.begin(), + projection_cols.end(), + [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); + + if (changed) + materialized_projections.insert(projection.name); } /// Stages might be empty when we materialize skip indices or projections which don't add any diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index d783b503531..9b4caaae231 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -120,7 +120,8 @@ public: bool supportsLightweightDelete() const; bool hasLightweightDeleteMask() const; bool materializeTTLRecalculateOnly() const; - bool hasIndexOrProjection(const String & file_name) const; + bool hasSecondaryIndex(const String & name) const; + bool hasProjection(const String & name) const; void read( Stage & first_stage, diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 7050a98a4bc..1ab8dc7fb05 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1983,6 +1983,12 @@ IndexSize IMergeTreeDataPart::getSecondaryIndexSize(const String & secondary_ind return ColumnSize{}; } +bool IMergeTreeDataPart::hasSecondaryIndex(const String & index_name) const +{ + auto file_name = INDEX_FILE_PREFIX + index_name; + return checksums.has(file_name + ".idx") || checksums.has(file_name + ".idx2"); +} + void IMergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) const { for (const auto & [column_name, size] : columns_sizes) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index af6906e004d..bfb472ca50d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -122,6 +122,9 @@ public: /// Otherwise return information about secondary index size on disk. IndexSize getSecondaryIndexSize(const String & secondary_index_name) const; + /// Returns true if there is materialized index with specified name in part. + bool hasSecondaryIndex(const String & index_name) const; + /// Return information about column size on disk for all columns in part ColumnSize getTotalColumnsSize() const { return total_columns_size; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 491c36433ca..fa5ad9858e8 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -69,6 +69,7 @@ static void splitAndModifyMutationCommands( { NameSet mutated_columns; NameSet dropped_columns; + NameSet extra_columns_for_indices_and_projections; for (const auto & command : commands) { @@ -85,6 +86,41 @@ static void splitAndModifyMutationCommands( if (command.type == MutationCommand::Type::MATERIALIZE_COLUMN) mutated_columns.emplace(command.column_name); + + if (command.type == MutationCommand::Type::MATERIALIZE_INDEX) + { + const auto & all_indices = metadata_snapshot->getSecondaryIndices(); + for (const auto & index : all_indices) + { + if (index.name == command.index_name) + { + auto required_columns = index.expression->getRequiredColumns(); + for (const auto & column : required_columns) + { + if (!part_columns.has(column)) + extra_columns_for_indices_and_projections.insert(column); + } + break; + } + } + } + + if (command.type == MutationCommand::Type::MATERIALIZE_PROJECTION) + { + const auto & all_projections = metadata_snapshot->getProjections(); + for (const auto & projection : all_projections) + { + if (projection.name == command.projection_name) + { + for (const auto & column : projection.required_columns) + { + if (!part_columns.has(column)) + extra_columns_for_indices_and_projections.insert(column); + } + break; + } + } + } } else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION) { @@ -187,6 +223,25 @@ static void splitAndModifyMutationCommands( }); } } + + for (const auto & column_name : extra_columns_for_indices_and_projections) + { + if (mutated_columns.contains(column_name)) + continue; + + auto data_type = metadata_snapshot->getColumns().getColumn( + GetColumnsOptions::AllPhysical, + column_name).type; + + for_interpreter.push_back( + MutationCommand + { + .type = MutationCommand::Type::READ_COLUMN, + .column_name = column_name, + .data_type = std::move(data_type), + } + ); + } } else { @@ -453,6 +508,7 @@ static ExecuteTTLType shouldExecuteTTL(const StorageMetadataPtr & metadata_snaps /// Return set of indices which should be recalculated during mutation also /// wraps input stream into additional expression stream static std::set getIndicesToRecalculate( + const MergeTreeDataPartPtr & source_part, QueryPipelineBuilder & builder, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, @@ -463,10 +519,15 @@ static std::set getIndicesToRecalculate( std::set indices_to_recalc; ASTPtr indices_recalc_expr_list = std::make_shared(); const auto & indices = metadata_snapshot->getSecondaryIndices(); + bool is_full_part_storage = isFullPartStorage(source_part->getDataPartStorage()); for (const auto & index : indices) { - if (materialized_indices.contains(index.name)) + bool need_recalculate = + materialized_indices.contains(index.name) + || (!is_full_part_storage && source_part->hasSecondaryIndex(index.name)); + + if (need_recalculate) { if (indices_to_recalc.insert(index_factory.get(index)).second) { @@ -496,15 +557,23 @@ static std::set getIndicesToRecalculate( } static std::set getProjectionsToRecalculate( + const MergeTreeDataPartPtr & source_part, const StorageMetadataPtr & metadata_snapshot, const NameSet & materialized_projections) { std::set projections_to_recalc; + bool is_full_part_storage = isFullPartStorage(source_part->getDataPartStorage()); + for (const auto & projection : metadata_snapshot->getProjections()) { - if (materialized_projections.contains(projection.name)) + bool need_recalculate = + materialized_projections.contains(projection.name) + || (!is_full_part_storage && source_part->hasProjection(projection.name)); + + if (need_recalculate) projections_to_recalc.insert(&projection); } + return projections_to_recalc; } @@ -1279,14 +1348,20 @@ private: removed_indices.insert(command.column_name); } + bool is_full_part_storage = isFullPartStorage(ctx->new_data_part->getDataPartStorage()); const auto & indices = ctx->metadata_snapshot->getSecondaryIndices(); + MergeTreeIndices skip_indices; for (const auto & idx : indices) { if (removed_indices.contains(idx.name)) continue; - if (ctx->materialized_indices.contains(idx.name)) + bool need_recalculate = + ctx->materialized_indices.contains(idx.name) + || (!is_full_part_storage && ctx->source_part->hasSecondaryIndex(idx.name)); + + if (need_recalculate) { skip_indices.push_back(MergeTreeIndexFactory::instance().get(idx)); } @@ -1319,7 +1394,11 @@ private: if (removed_projections.contains(projection.name)) continue; - if (ctx->materialized_projections.contains(projection.name)) + bool need_recalculate = + ctx->materialized_projections.contains(projection.name) + || (!is_full_part_storage && ctx->source_part->hasProjection(projection.name)); + + if (need_recalculate) { ctx->projections_to_build.push_back(&projection); } @@ -1920,9 +1999,16 @@ bool MutateTask::prepare() else /// TODO: check that we modify only non-key columns in this case. { ctx->indices_to_recalc = MutationHelpers::getIndicesToRecalculate( - ctx->mutating_pipeline_builder, ctx->metadata_snapshot, ctx->context, ctx->materialized_indices); + ctx->source_part, + ctx->mutating_pipeline_builder, + ctx->metadata_snapshot, + ctx->context, + ctx->materialized_indices); - ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate(ctx->metadata_snapshot, ctx->materialized_projections); + ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( + ctx->source_part, + ctx->metadata_snapshot, + ctx->materialized_projections); ctx->files_to_skip = MutationHelpers::collectFilesToSkip( ctx->source_part, diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index afe75349864..b1da509c9b0 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -239,7 +239,7 @@ bool StorageInMemoryMetadata::hasAnyGroupByTTL() const ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( const NameSet & updated_columns, bool include_ttl_target, - const std::function & has_indice_or_projection) const + const HasDependencyCallback & has_dependency) const { if (updated_columns.empty()) return {}; @@ -268,13 +268,13 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( for (const auto & index : getSecondaryIndices()) { - if (has_indice_or_projection("skp_idx_" + index.name + ".idx") || has_indice_or_projection("skp_idx_" + index.name + ".idx2")) + if (has_dependency(index.name, ColumnDependency::SKIP_INDEX)) add_dependent_columns(index.expression, indices_columns); } for (const auto & projection : getProjections()) { - if (has_indice_or_projection(projection.getDirectoryName())) + if (has_dependency(projection.getDirectoryName(), ColumnDependency::PROJECTION)) add_dependent_columns(&projection, projections_columns); } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 4ed7eb8bf29..30b2b303492 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -147,12 +147,14 @@ struct StorageInMemoryMetadata TTLDescriptions getGroupByTTLs() const; bool hasAnyGroupByTTL() const; + using HasDependencyCallback = std::function; + /// Returns columns, which will be needed to calculate dependencies (skip indices, projections, /// TTL expressions) if we update @updated_columns set of columns. ColumnDependencies getColumnDependencies( const NameSet & updated_columns, bool include_ttl_target, - const std::function & has_indice_or_projection) const; + const HasDependencyCallback & has_dependency) const; /// Block with ordinary + materialized columns. Block getSampleBlock() const; diff --git a/tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference new file mode 100644 index 00000000000..f14acdf9e6d --- /dev/null +++ b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference @@ -0,0 +1,6 @@ +2 +0 +3355402240 +3355402240 +3321851904 +3321851904 diff --git a/tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql new file mode 100644 index 00000000000..b87230e57d1 --- /dev/null +++ b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql @@ -0,0 +1,26 @@ +set mutations_sync = 2; + +drop table if exists t_delete_skip_index; + +create table t_delete_skip_index (x UInt32, y String, index i y type minmax granularity 3) engine = MergeTree order by tuple(); +insert into t_delete_skip_index select number, toString(number) from numbers(8192 * 10); + +select count() from t_delete_skip_index where y in (4, 5); +alter table t_delete_skip_index delete where x < 8192; +select count() from t_delete_skip_index where y in (4, 5); + +drop table if exists t_delete_skip_index; +drop table if exists t_delete_projection; + +create table t_delete_projection (x UInt32, y UInt64, projection p (select sum(y))) engine = MergeTree order by tuple(); +insert into t_delete_projection select number, toString(number) from numbers(8192 * 10); + +select sum(y) from t_delete_projection settings optimize_use_projections = 0; +select sum(y) from t_delete_projection settings optimize_use_projections = 0, force_optimize_projection = 1; + +alter table t_delete_projection delete where x < 8192; + +select sum(y) from t_delete_projection settings optimize_use_projections = 0; +select sum(y) from t_delete_projection settings optimize_use_projections = 0, force_optimize_projection = 1; + +drop table if exists t_delete_projection; From d2d100b68a4fc1765708a276b217faf403722fb4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Jul 2023 17:05:57 +0200 Subject: [PATCH 1101/2047] Cancel execution in PipelineExecutor in case of exception in graph->updateNode --- src/Processors/Executors/PipelineExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index f523e7b7cf9..1508d834592 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -272,7 +272,7 @@ void PipelineExecutor::executeStepImpl(size_t thread_num, std::atomic_bool * yie /// Prepare processor after execution. if (!graph->updateNode(context.getProcessorID(), queue, async_queue)) - finish(); + cancel(); /// Push other tasks to global queue. tasks.pushTasks(queue, async_queue, context); From f067f8c46d2aec217c3f835441ca1a2a281c72fd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 24 Jul 2023 15:37:16 +0000 Subject: [PATCH 1102/2047] Make 01951_distributed_push_down_limit analyzer agnostic --- tests/analyzer_tech_debt.txt | 1 - ...1951_distributed_push_down_limit.reference | 32 +++++++++---------- .../01951_distributed_push_down_limit.sql | 4 +-- 3 files changed, 18 insertions(+), 19 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index b746d1610a4..1d56b2c3a71 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -72,7 +72,6 @@ 01925_test_storage_merge_aliases 01930_optimize_skip_unused_shards_rewrite_in 01947_mv_subquery -01951_distributed_push_down_limit 01952_optimize_distributed_group_by_sharding_key 02000_join_on_const 02001_shard_num_shard_count diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference index b9a7d17e955..d175d31846b 100644 --- a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference @@ -1,19 +1,19 @@ -- { echo } -explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; -Expression (Projection) - Limit (preliminary LIMIT (without OFFSET)) - Sorting (Merge sorted streams after aggregation stage for ORDER BY) +explain description=0 select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; +Expression + Limit + Sorting Union - Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) - ReadFromStorage (SystemNumbers) - ReadFromRemote (Read from remote replica) -explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; -Expression (Projection) - Limit (preliminary LIMIT (without OFFSET)) - Sorting (Merge sorted streams after aggregation stage for ORDER BY) + Sorting + Expression + ReadFromStorage + ReadFromRemote +explain description=0 select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; +Expression + Limit + Sorting Union - Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) - ReadFromStorage (SystemNumbers) - ReadFromRemote (Read from remote replica) + Sorting + Expression + ReadFromStorage + ReadFromRemote diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.sql b/tests/queries/0_stateless/01951_distributed_push_down_limit.sql index 184e6321988..aee714a494e 100644 --- a/tests/queries/0_stateless/01951_distributed_push_down_limit.sql +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.sql @@ -3,5 +3,5 @@ set prefer_localhost_replica = 1; -- { echo } -explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; -explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; +explain description=0 select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; +explain description=0 select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; From 7b03f3277e5097000bda03c93c27efc5d8c29a34 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 24 Jul 2023 17:59:50 +0200 Subject: [PATCH 1103/2047] Try fix session on close test --- src/Coordination/KeeperDispatcher.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 02308f245d5..99c28674273 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -599,6 +599,10 @@ void KeeperDispatcher::sessionCleanerTask() void KeeperDispatcher::finishSession(int64_t session_id) { + /// shutdown() method will cleanup sessions if needed + if (shutdown_called) + return; + { std::lock_guard lock(session_to_response_callback_mutex); auto session_it = session_to_response_callback.find(session_id); From da0a332d46a7939377dca08147897d0f5b1dd9c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 18:09:38 +0200 Subject: [PATCH 1104/2047] Keep symtab --- cmake/split_debug_symbols.cmake | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/cmake/split_debug_symbols.cmake b/cmake/split_debug_symbols.cmake index a9c2158359a..8ba7669d9e2 100644 --- a/cmake/split_debug_symbols.cmake +++ b/cmake/split_debug_symbols.cmake @@ -22,8 +22,9 @@ macro(clickhouse_split_debug_symbols) # Splits debug symbols into separate file, leaves the binary untouched: COMMAND "${OBJCOPY_PATH}" --only-keep-debug "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" COMMAND chmod 0644 "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" - # Strips binary, sections '.note' & '.comment' are removed in line with Debian's stripping policy: www.debian.org/doc/debian-policy/ch-files.html, section '.clickhouse.hash' is needed for integrity check: - COMMAND "${STRIP_PATH}" --remove-section=.comment --remove-section=.note --keep-section=.clickhouse.hash "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" + # Strips binary, sections '.note' & '.comment' are removed in line with Debian's stripping policy: www.debian.org/doc/debian-policy/ch-files.html, section '.clickhouse.hash' is needed for integrity check. + # Also, after we disabled the export of symbols for dynamic linking, we still to keep a static symbol table for good stack traces. + COMMAND "${STRIP_PATH}" --remove-section=.comment --remove-section=.note --keep-section=.clickhouse.hash --keep-section=.symtab --keep-section=.strtab --keep-section=.shstrtab "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" # Associate stripped binary with debug symbols: COMMAND "${OBJCOPY_PATH}" --add-gnu-debuglink "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" COMMENT "Stripping clickhouse binary" VERBATIM From 0bbf26549f4fb49c599b4a58475c71bccfe9b37b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 18:13:15 +0200 Subject: [PATCH 1105/2047] Fix test --- tests/integration/test_drop_is_lock_free/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_drop_is_lock_free/test.py b/tests/integration/test_drop_is_lock_free/test.py index 8d92d784226..9f595800bea 100644 --- a/tests/integration/test_drop_is_lock_free/test.py +++ b/tests/integration/test_drop_is_lock_free/test.py @@ -104,7 +104,7 @@ def test_query_is_lock_free(lock_free_query, exclusive_table): select_handler = node.get_query_request( f""" - SELECT sleepEachRow(3) FROM {exclusive_table}; + SELECT sleepEachRow(3) FROM {exclusive_table} SETTINGS function_sleep_max_microseconds_per_block = 0; """, query_id=query_id, ) @@ -173,7 +173,7 @@ def test_query_is_permanent(transaction, permanent, exclusive_table): select_handler = node.get_query_request( f""" - SELECT sleepEachRow(3) FROM {exclusive_table}; + SELECT sleepEachRow(3) FROM {exclusive_table} SETTINGS function_sleep_max_microseconds_per_block = 0; """, query_id=query_id, ) From c755fde3a41f95466db1b5b9aeeb54decafa97c8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 20 Jul 2023 18:36:00 +0000 Subject: [PATCH 1106/2047] Include query cache usage in SYSTEM.QUERY_LOG --- docs/en/operations/query-cache.md | 6 +- docs/en/operations/system-tables/query_log.md | 6 ++ src/Interpreters/AsynchronousInsertQueue.cpp | 2 +- src/Interpreters/Cache/QueryCache.h | 9 +++ src/Interpreters/QueryLog.cpp | 13 ++++ src/Interpreters/QueryLog.h | 5 +- src/Interpreters/executeQuery.cpp | 22 ++++--- src/Interpreters/executeQuery.h | 1 + .../02494_query_cache_query_log.reference | 16 +++++ .../02494_query_cache_query_log.sql | 64 +++++++++++++++++++ 10 files changed, 132 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02494_query_cache_query_log.reference create mode 100644 tests/queries/0_stateless/02494_query_cache_query_log.sql diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index bfa51650cd8..547105c65cc 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -62,8 +62,10 @@ may return cached results then. The query cache can be cleared using statement `SYSTEM DROP QUERY CACHE`. The content of the query cache is displayed in system table `system.query_cache`. The number of query cache hits and misses are shown as events "QueryCacheHits" and "QueryCacheMisses" in system table -`system.events`. Both counters are only updated for `SELECT` queries which run with setting "use_query_cache = true". Other queries do not -affect the cache miss counter. +[system.events](system-tables/events.md). Both counters are only updated for `SELECT` queries which run with setting "use_query_cache = +true". Other queries do not affect the cache miss counter. Field `query_log_usage` in system table +[system.query_log](system-tables/query_log.md) shows for each ran query whether the query result was written into or read from the query +cache. The query cache exists once per ClickHouse server process. However, cache results are by default not shared between users. This can be changed (see below) but doing so is not recommended for security reasons. diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index b9fdd19c643..b8dc0c0224c 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -111,6 +111,11 @@ Columns: - `used_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `functions`, which were used during query execution. - `used_storages` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `storages`, which were used during query execution. - `used_table_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `table functions`, which were used during query execution. +- `query_cache_usage` ([Enum8](../../sql-reference/data-types/enum.md)) — Usage of the [query cache](../query-cache.md) during query execution. Values: + - `'None' = 1` = The query result was neither written into nor read from the query cache. + - `'Write' = 1` = The query result was written into the query cache. + - `'Read' = 1` = The query result was read from the query cache. + - `'Unknown' = 1` = Unknown status. **Example** @@ -186,6 +191,7 @@ used_formats: [] used_functions: [] used_storages: [] used_table_functions: [] +query_cache_usage: None ``` **See Also** diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 8d0f18cc305..c6596f50eda 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -605,7 +605,7 @@ try total_rows, total_bytes, key.query_str); bool pulling_pipeline = false; - logQueryFinish(query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, internal); + logQueryFinish(query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); } catch (...) { diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index c24b09c8e46..973015b8003 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -24,6 +24,15 @@ bool astContainsNonDeterministicFunctions(ASTPtr ast, ContextPtr context); class QueryCache { public: + enum class Usage + { + /// starts at 1 for compatibitity with DataTypeEnum8 + None = 1, /// query result neither written nor read into/from query cache + Write, /// query result wrote into query cache + Read, /// query result read from query cache + Unknown, /// we don't know what what happened + }; + /// Represents a query result in the cache. struct Key { diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index ec0315c2f95..c3294512f14 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -41,6 +41,15 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"ExceptionWhileProcessing", static_cast(EXCEPTION_WHILE_PROCESSING)} }); + auto query_cache_usage_datatype = std::make_shared( + DataTypeEnum8::Values + { + {"None", static_cast(QueryCache::Usage::None)}, + {"Write", static_cast(QueryCache::Usage::Write)}, + {"Read", static_cast(QueryCache::Usage::Read)}, + {"Unknown", static_cast(QueryCache::Usage::Unknown)} + }); + auto low_cardinality_string = std::make_shared(std::make_shared()); auto array_low_cardinality_string = std::make_shared(low_cardinality_string); @@ -126,6 +135,8 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"transaction_id", getTransactionIDDataType()}, + {"query_cache_usage", std::move(query_cache_usage_datatype)}, + {"asynchronous_read_counters", std::make_shared(low_cardinality_string, std::make_shared())}, }; } @@ -277,6 +288,8 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(Tuple{tid.start_csn, tid.local_tid, tid.host_id}); + columns[i++]->insert(query_cache_usage); + if (async_read_counters) async_read_counters->dumpToMapColumn(columns[i++].get()); else diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 570d1297239..5bc80280eac 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -4,8 +4,9 @@ #include #include #include -#include +#include #include +#include #include #include #include @@ -96,6 +97,8 @@ struct QueryLogElement TransactionID tid; + QueryCache::Usage query_cache_usage = QueryCache::Usage::Unknown; + static std::string name() { return "QueryLog"; } static NamesAndTypesList getNamesAndTypes(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 688d3b9967d..578ca3b41f9 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -209,7 +209,7 @@ static void logException(ContextPtr context, QueryLogElement & elem, bool log_er } static void -addStatusInfoToQueryElement(QueryLogElement & element, const QueryStatusInfo & info, const ASTPtr query_ast, const ContextPtr context_ptr) +addStatusInfoToQueryLogElement(QueryLogElement & element, const QueryStatusInfo & info, const ASTPtr query_ast, const ContextPtr context_ptr) { const auto time_now = std::chrono::system_clock::now(); UInt64 elapsed_microseconds = info.elapsed_microseconds; @@ -347,6 +347,7 @@ void logQueryFinish( const QueryPipeline & query_pipeline, bool pulling_pipeline, std::shared_ptr query_span, + QueryCache::Usage query_cache_usage, bool internal) { const Settings & settings = context->getSettingsRef(); @@ -364,7 +365,7 @@ void logQueryFinish( QueryStatusInfo info = process_list_elem->getInfo(true, context->getSettingsRef().log_profile_events); elem.type = QueryLogElementType::QUERY_FINISH; - addStatusInfoToQueryElement(elem, info, query_ast, context); + addStatusInfoToQueryLogElement(elem, info, query_ast, context); if (pulling_pipeline) { @@ -399,6 +400,8 @@ void logQueryFinish( ReadableSize(elem.read_bytes / elapsed_seconds)); } + elem.query_cache_usage = query_cache_usage; + if (log_queries && elem.type >= log_queries_min_type && static_cast(elem.query_duration_ms) >= log_queries_min_query_duration_ms) { @@ -499,13 +502,15 @@ void logQueryException( if (process_list_elem) { QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events, false); - addStatusInfoToQueryElement(elem, info, query_ast, context); + addStatusInfoToQueryLogElement(elem, info, query_ast, context); } else { elem.query_duration_ms = start_watch.elapsedMilliseconds(); } + elem.query_cache_usage = QueryCache::Usage::None; + if (settings.calculate_text_stack_trace && log_error) setExceptionStackTrace(elem); logException(context, elem, log_error); @@ -975,7 +980,7 @@ static std::tuple executeQueryImpl( QueryCachePtr query_cache = context->getQueryCache(); const bool can_use_query_cache = query_cache != nullptr && settings.use_query_cache && !internal && (ast->as() || ast->as()); - bool write_into_query_cache = false; + QueryCache::Usage query_cache_usage = QueryCache::Usage::None; if (!async_insert) { @@ -992,6 +997,7 @@ static std::tuple executeQueryImpl( QueryPipeline pipeline; pipeline.readFromQueryCache(reader.getSource(), reader.getSourceTotals(), reader.getSourceExtremes()); res.pipeline = std::move(pipeline); + query_cache_usage = QueryCache::Usage::Read; return true; } } @@ -1095,7 +1101,7 @@ static std::tuple executeQueryImpl( settings.query_cache_max_size_in_bytes, settings.query_cache_max_entries)); res.pipeline.writeResultIntoQueryCache(query_cache_writer); - write_into_query_cache = true; + query_cache_usage = QueryCache::Usage::Write; } } @@ -1147,19 +1153,19 @@ static std::tuple executeQueryImpl( auto finish_callback = [elem, context, ast, - write_into_query_cache, + query_cache_usage, internal, implicit_txn_control, execute_implicit_tcl_query, pulling_pipeline = pipeline.pulling(), query_span](QueryPipeline & query_pipeline) mutable { - if (write_into_query_cache) + if (query_cache_usage == QueryCache::Usage::Write) /// Trigger the actual write of the buffered query result into the query cache. This is done explicitly to prevent /// partial/garbage results in case of exceptions during query execution. query_pipeline.finalizeWriteInQueryCache(); - logQueryFinish(elem, context, ast, query_pipeline, pulling_pipeline, query_span, internal); + logQueryFinish(elem, context, ast, query_pipeline, pulling_pipeline, query_span, query_cache_usage, internal); if (*implicit_txn_control) execute_implicit_tcl_query(context, ASTTransactionControl::COMMIT); diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index 53624f8c812..f2a12bbef18 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -92,6 +92,7 @@ void logQueryFinish( const QueryPipeline & query_pipeline, bool pulling_pipeline, std::shared_ptr query_span, + QueryCache::Usage query_cache_usage, bool internal); void logQueryException( diff --git a/tests/queries/0_stateless/02494_query_cache_query_log.reference b/tests/queries/0_stateless/02494_query_cache_query_log.reference new file mode 100644 index 00000000000..9037909d121 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_query_log.reference @@ -0,0 +1,16 @@ +-- Run a query with query cache not enabled +124437993 +QueryStart SELECT 124437993; Unknown +QueryFinish SELECT 124437993; None +-- Run a query with query cache enabled +124437994 +QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown +QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Write +-- Run the same query with query cache enabled +124437994 +QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown +QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown +QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Read +QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Write +-- Throw exception with query cache enabled +SELECT 124437995, throwIf(1) SETTINGS use_query_cache = 1; None diff --git a/tests/queries/0_stateless/02494_query_cache_query_log.sql b/tests/queries/0_stateless/02494_query_cache_query_log.sql new file mode 100644 index 00000000000..79a8f4cb62b --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_query_log.sql @@ -0,0 +1,64 @@ +-- Tags: no-parallel +-- Tag no-parallel: Messes with internal cache + +SYSTEM DROP QUERY CACHE; + +-- DROP TABLE system.query_log; -- debugging + + + +SELECT '-- Run a query with query cache not enabled'; +SELECT 124437993; + +SYSTEM FLUSH LOGS; + +-- Field 'query_cache_usage' should be 'None' +SELECT type, query, query_cache_usage +FROM system.query_log +WHERE current_database = currentDatabase() + AND query = 'SELECT 124437993;' +ORDER BY type; + + + +SELECT '-- Run a query with query cache enabled'; +SELECT 124437994 SETTINGS use_query_cache = 1; + +SYSTEM FLUSH LOGS; + +-- Field 'query_cache_usage' should be 'Write' +SELECT type, query, query_cache_usage +FROM system.query_log +WHERE current_database = currentDatabase() + AND query = 'SELECT 124437994 SETTINGS use_query_cache = 1;' +ORDER BY type; + + + +SELECT '-- Run the same query with query cache enabled'; +SELECT 124437994 SETTINGS use_query_cache = 1; + +SYSTEM FLUSH LOGS; + +-- Field 'query_cache_usage' should be 'Read' +SELECT type, query, query_cache_usage +FROM system.query_log +WHERE current_database = currentDatabase() + AND query = 'SELECT 124437994 SETTINGS use_query_cache = 1;' +ORDER BY type; + + + +SELECT '-- Throw exception with query cache enabled'; +SELECT 124437995, throwIf(1) SETTINGS use_query_cache = 1; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + +SYSTEM FLUSH LOGS; + +-- Field 'query_cache_usage' should be 'None' +SELECT query, query_cache_usage +FROM system.query_log +WHERE current_database = currentDatabase() + AND query = 'SELECT 124437995, throwIf(1) SETTINGS use_query_cache = 1;' + AND type = 'ExceptionWhileProcessing'; + +SYSTEM DROP QUERY CACHE; From 032956dd1eeca994d6fa5a66f974cfa10203c205 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 24 Jul 2023 18:42:02 +0200 Subject: [PATCH 1107/2047] fix --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../test.py | 27 ++++++++++--------- .../01111_create_drop_replicated_db_stress.sh | 2 +- 3 files changed, 17 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 2393f45ebb6..e11913fc3d2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -576,7 +576,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper /// It's ok if replica became readonly due to connection loss after we got current zookeeper (in this case zookeeper must be expired). /// And it's ok if replica became readonly after shutdown. /// In other cases it's likely that someone called pullLogsToQueue(...) when queue is not initialized yet by RestartingThread. - bool not_completely_initialized = storage.is_readonly && !zookeeper->expired() && !storage.shutdown_called; + bool not_completely_initialized = storage.is_readonly && !zookeeper->expired() && !storage.shutdown_prepared_called; if (not_completely_initialized) throw Exception(ErrorCodes::LOGICAL_ERROR, "Tried to pull logs to queue (reason: {}) on readonly replica {}, it's a bug", reason, storage.getStorageID().getNameForLogs()); diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index a2a4ec92cf7..20b6a6c977f 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -49,25 +49,28 @@ def test_shutdown_and_wait(start_cluster): node.stop_clickhouse(kill=False, stop_wait_sec=60) p = Pool(50) - pm = PartitionManager() - - pm.partition_instances(node1, node2) def insert(value): node1.query(f"INSERT INTO test_table VALUES ({value})") - p.map(insert, range(1, 50)) + with PartitionManager() as pm: + pm.partition_instances(node1, node2) + # iptables rules must be applied immediately, but looks like sometimes they are not... + time.sleep(3) - # Start shutdown async - waiter = p.apply_async(soft_shutdown, (node1,)) - # to be sure that shutdown started - time.sleep(5) + p.map(insert, range(1, 50)) - # node 2 partitioned and don't see any data - assert node2.query("SELECT * FROM test_table") == "0\n" + # Start shutdown async + waiter = p.apply_async(soft_shutdown, (node1,)) + # to be sure that shutdown started + time.sleep(5) + + # node 2 partitioned and don't see any data + assert node2.query("SELECT * FROM test_table") == "0\n" + + # Restore network + pm.heal_all() - # Restore network - pm.heal_all() # wait for shutdown to finish waiter.get() diff --git a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh index cc63af3676b..59899e1c14a 100755 --- a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh +++ b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh @@ -56,7 +56,7 @@ function create_table() if [ -z "$database" ]; then continue; fi $CLICKHOUSE_CLIENT --distributed_ddl_task_timeout=0 -q \ "create table $database.rmt_${RANDOM}_${RANDOM}_${RANDOM} (n int) engine=ReplicatedMergeTree order by tuple() -- suppress $CLICKHOUSE_TEST_ZOOKEEPER_PREFIX" \ - 2>&1| grep -Fa "Exception: " | grep -Fv "Macro 'uuid' and empty arguments" | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE + 2>&1| grep -Fa "Exception: " | grep -Fv "Macro 'uuid' and empty arguments" | grep -Fv "Cannot enqueue query" | grep -Fv "ZooKeeper session expired" | grep -Fv UNKNOWN_DATABASE | grep -Fv TABLE_IS_DROPPED sleep 0.$RANDOM done } From 393481181d5fb7c878af2f1e0bbbd5d9c8d56807 Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 24 Jul 2023 13:35:07 -0400 Subject: [PATCH 1108/2047] Remove Trailing whitespaces --- src/Parsers/Access/ASTCreateRowPolicyQuery.cpp | 2 +- src/Parsers/Access/ASTCreateUserQuery.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp index a423581e300..e95959703ee 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp @@ -172,7 +172,7 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " IN " << (settings.hilite ? IAST::hilite_none : "") << backQuoteIfNeed(storage_name); - + formatOnCluster(settings); assert(names->cluster.empty()); diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index dbb4c1c6d57..b7290c16013 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -212,7 +212,7 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " IN " << (format.hilite ? IAST::hilite_none : "") << backQuoteIfNeed(storage_name); - + formatOnCluster(format); if (new_name) From 22a2fa097f3795cb2a483e899482b97f80aa8189 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 19:40:02 +0200 Subject: [PATCH 1109/2047] Improve error messages --- src/Functions/GregorianDate.cpp | 2 +- src/Functions/parseDateTime.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/GregorianDate.cpp b/src/Functions/GregorianDate.cpp index aaaeeb7339d..f28194781c2 100644 --- a/src/Functions/GregorianDate.cpp +++ b/src/Functions/GregorianDate.cpp @@ -125,7 +125,7 @@ void GregorianDate::init(ReadBuffer & in) assertEOF(in); if (month_ < 1 || month_ > 12 || day_of_month_ < 1 || day_of_month_ > monthLength(is_leap_year(year_), month_)) - throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Invalid date"); + throw Exception(ErrorCodes::CANNOT_PARSE_DATE, "Invalid date, out of range (year: {}, month: {}, day_of_month: {})."); } bool GregorianDate::tryInit(ReadBuffer & in) diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index c3fbc08c4a9..2381def9151 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -398,7 +398,7 @@ namespace static Int32 daysSinceEpochFromDayOfYear(Int32 year_, Int32 day_of_year_) { if (!isDayOfYearValid(year_, day_of_year_)) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid day of year, year:{} day of year:{}", year_, day_of_year_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid day of year, out of range (year: {} day of year: {})", year_, day_of_year_); Int32 res = daysSinceEpochFromDate(year_, 1, 1); res += day_of_year_ - 1; @@ -408,7 +408,7 @@ namespace static Int32 daysSinceEpochFromDate(Int32 year_, Int32 month_, Int32 day_) { if (!isDateValid(year_, month_, day_)) - throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid date, year:{} month:{} day:{}", year_, month_, day_); + throw Exception(ErrorCodes::CANNOT_PARSE_DATETIME, "Invalid date, out of range (year: {} month: {} day_of_month: {})", year_, month_, day_); Int32 res = cumulativeYearDays[year_ - 1970]; res += isLeapYear(year_) ? cumulativeLeapDays[month_ - 1] : cumulativeDays[month_ - 1]; From 654af41431423907fdffed93287e9160f78698b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jul 2023 19:45:55 +0200 Subject: [PATCH 1110/2047] Fix race --- src/Functions/transform.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 79168d82c54..e03701327b1 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -658,13 +658,13 @@ namespace std::unique_ptr table_string_to_idx; std::unique_ptr table_anything_to_idx; - bool is_empty = false; - ColumnPtr from_column; ColumnPtr to_column; ColumnPtr default_column; - std::atomic initialized{false}; + bool is_empty = false; + bool initialized = false; + std::mutex mutex; }; @@ -697,13 +697,12 @@ namespace /// Can be called from different threads. It works only on the first call. void initialize(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const { + std::lock_guard lock(cache.mutex); if (cache.initialized) return; const DataTypePtr & from_type = arguments[0].type; - std::lock_guard lock(cache.mutex); - if (from_type->onlyNull()) { cache.is_empty = true; From c35da36ff2b78dff5b964774673b8c713aa22e95 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Jul 2023 19:50:53 +0200 Subject: [PATCH 1111/2047] Fix default value --- base/poco/Foundation/include/Poco/URI.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/poco/Foundation/include/Poco/URI.h b/base/poco/Foundation/include/Poco/URI.h index f4505147ced..eba8109253d 100644 --- a/base/poco/Foundation/include/Poco/URI.h +++ b/base/poco/Foundation/include/Poco/URI.h @@ -57,7 +57,7 @@ public: URI(); /// Creates an empty URI. - explicit URI(const std::string & uri, bool disable_url_encoding = true); + explicit URI(const std::string & uri, bool disable_url_encoding = false); /// Parses an URI from the given string. Throws a /// SyntaxException if the uri is not valid. @@ -362,7 +362,7 @@ private: std::string _query; std::string _fragment; - bool _disable_url_encoding = true; + bool _disable_url_encoding = false; }; From 2f99363db0356f146db427934b63e9158b7b9858 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 24 Jul 2023 20:51:53 +0300 Subject: [PATCH 1112/2047] Update 02136_scalar_subquery_metrics.sql --- tests/queries/0_stateless/02136_scalar_subquery_metrics.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02136_scalar_subquery_metrics.sql b/tests/queries/0_stateless/02136_scalar_subquery_metrics.sql index 180610288aa..17ff367a58d 100644 --- a/tests/queries/0_stateless/02136_scalar_subquery_metrics.sql +++ b/tests/queries/0_stateless/02136_scalar_subquery_metrics.sql @@ -6,7 +6,7 @@ SELECT '#02136_scalar_subquery_4', (SELECT max(number) FROM numbers(1000)) as n SYSTEM FLUSH LOGS; SELECT read_rows, query FROM system.query_log WHERE - event_date > yesterday() + event_date >= yesterday() AND type = 'QueryFinish' AND current_database == currentDatabase() AND query LIKE 'SELECT ''#02136_scalar_subquery_%' From 8a753de296b5d6914b2cbeecd8b4cdda941db1d3 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 24 Jul 2023 19:57:29 +0200 Subject: [PATCH 1113/2047] Delete 02817_structure_to_schema.out --- .../0_stateless/02817_structure_to_schema.out | 468 ------------------ 1 file changed, 468 deletions(-) delete mode 100644 tests/queries/0_stateless/02817_structure_to_schema.out diff --git a/tests/queries/0_stateless/02817_structure_to_schema.out b/tests/queries/0_stateless/02817_structure_to_schema.out deleted file mode 100644 index ad45343139b..00000000000 --- a/tests/queries/0_stateless/02817_structure_to_schema.out +++ /dev/null @@ -1,468 +0,0 @@ -CapnProto -Numbers - -struct Message -{ - int8 @0 : Int8; - uint8 @1 : UInt8; - int16 @2 : Int16; - uint16 @3 : UInt16; - int32 @4 : Int32; - uint32 @5 : UInt32; - int64 @6 : Int64; - uint64 @7 : UInt64; - int128 @8 : Data; - uint128 @9 : Data; - int256 @10 : Data; - uint256 @11 : Data; - float32 @12 : Float32; - float64 @13 : Float64; - decimal32 @14 : Int32; - decimal64 @15 : Int64; - decimal128 @16 : Data; - decimal256 @17 : Data; -} -Dates - -struct Message -{ - data @0 : UInt16; - date32 @1 : Int32; - datetime @2 : UInt32; - datatime64 @3 : Int64; -} -Strings - -struct Message -{ - string @0 : Data; - fixedstring @1 : Data; -} -Special - -struct Message -{ - ipv4 @0 : UInt32; - ipv6 @1 : Data; - uuid @2 : Data; -} -Nullable - -struct Message -{ - struct Nullable - { - union - { - value @0 : UInt32; - null @1 : Void; - } - } - nullable @0 : Nullable; -} -Enums - -struct Message -{ - enum Enum8 - { - v1 @0; - v2 @1; - v3 @2; - v4 @3; - } - enum8 @0 : Enum8; - enum Enum16 - { - v5 @0; - v6 @1; - v7 @2; - v8 @3; - v9 @4; - } - enum16 @1 : Enum16; -} -Arrays - -struct Message -{ - arr1 @0 : List(UInt32); - arr2 @1 : List(List(List(UInt32))); -} -Tuples - -struct Message -{ - struct Tuple1 - { - e1 @0 : UInt32; - e2 @1 : Data; - e3 @2 : UInt32; - } - tuple1 @0 : Tuple1; - struct Tuple2 - { - struct E1 - { - e1 @0 : UInt32; - struct E2 - { - e1 @0 : Data; - e2 @1 : UInt32; - } - e2 @1 : E2; - e3 @2 : Data; - } - e1 @0 : E1; - struct E2 - { - e1 @0 : Data; - e2 @1 : UInt32; - } - e2 @1 : E2; - } - tuple2 @1 : Tuple2; -} -Maps - -struct Message -{ - struct Map1 - { - struct Entry - { - key @0 : Data; - value @1 : UInt32; - } - entries @0 : List(Entry); - } - map1 @0 : Map1; - struct Map2 - { - struct Entry - { - struct Value - { - struct Entry - { - struct Value - { - struct Entry - { - key @0 : Data; - value @1 : UInt32; - } - entries @0 : List(Entry); - } - key @0 : Data; - value @1 : Value; - } - entries @0 : List(Entry); - } - key @0 : Data; - value @1 : Value; - } - entries @0 : List(Entry); - } - map2 @1 : Map2; -} -Complex - -struct Message -{ - struct C1 - { - struct E1 - { - struct Entry - { - struct Value - { - union - { - value @0 : UInt32; - null @1 : Void; - } - } - key @0 : Data; - value @1 : List(List(Value)); - } - entries @0 : List(Entry); - } - e1 @0 : List(E1); - struct E2 - { - struct Entry - { - struct Value - { - struct E1 - { - union - { - value @0 : Data; - null @1 : Void; - } - } - e1 @0 : List(List(E1)); - struct E2 - { - e1 @0 : UInt32; - struct E2 - { - struct E1 - { - union - { - value @0 : Data; - null @1 : Void; - } - } - e1 @0 : List(List(E1)); - e2 @1 : UInt32; - } - e2 @1 : E2; - } - e2 @1 : List(E2); - } - key @0 : Data; - value @1 : Value; - } - entries @0 : List(Entry); - } - e2 @1 : List(E2); - } - c1 @0 : C1; -} -Read/write with no schema -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 -Output schema -@0xda5297bc0d5b501a; - -struct Message -{ - number @0 : UInt64; -} -Bad output schema path -2 -2 -Protobuf -Numbers - -message Message -{ - int32 int8 = 1; - uint32 uint8 = 2; - int32 int16 = 3; - uint32 uint16 = 4; - int32 int32 = 5; - uint32 uint32 = 6; - int64 int64 = 7; - uint64 uint64 = 8; - bytes int128 = 9; - bytes uint128 = 10; - bytes int256 = 11; - bytes uint256 = 12; - float float32 = 13; - double float64 = 14; - bytes decimal32 = 15; - bytes decimal64 = 16; - bytes decimal128 = 17; - bytes decimal256 = 18; -} -Dates - -message Message -{ - uint32 data = 1; - int32 date32 = 2; - uint32 datetime = 3; - uint64 datatime64 = 4; -} -Strings - -message Message -{ - bytes string = 1; - bytes fixedstring = 2; -} -Special - -message Message -{ - uint32 ipv4 = 1; - bytes ipv6 = 2; - bytes uuid = 3; -} -Nullable - -message Message -{ - uint32 nullable = 1; -} -Enums - -message Message -{ - enum Enum8 - { - v1 = 0; - v2 = 1; - v3 = 2; - v4 = 3; - } - Enum8 enum8 = 1; - enum Enum16 - { - v5 = 0; - v6 = 1; - v7 = 2; - v8 = 3; - v9 = 4; - } - Enum16 enum16 = 2; -} -Arrays - -message Message -{ - repeated uint32 arr1 = 1; - message Arr2 - { - message Arr2 - { - repeated uint32 arr2 = 1; - } - repeated Arr2 arr2 = 1; - } - repeated Arr2 arr2 = 2; -} -Tuples - -message Message -{ - message Tuple1 - { - uint32 e1 = 1; - bytes e2 = 2; - uint32 e3 = 3; - } - Tuple1 tuple1 = 1; - message Tuple2 - { - message E1 - { - uint32 e1 = 1; - message E2 - { - bytes e1 = 1; - uint32 e2 = 2; - } - E2 e2 = 2; - bytes e3 = 3; - } - E1 e1 = 1; - message E2 - { - bytes e1 = 1; - uint32 e2 = 2; - } - E2 e2 = 2; - } - Tuple2 tuple2 = 2; -} -Maps - -message Message -{ - map map1 = 1; - message Map2Value - { - message Map2ValueValue - { - map map2ValueValue = 1; - } - map map2Value = 1; - } - map map2 = 2; -} -Complex - -message Message -{ - message C1 - { - message E1 - { - message E1Value - { - message E1Value - { - repeated uint32 e1Value = 1; - } - repeated E1Value e1Value = 1; - } - map e1 = 1; - } - repeated E1 e1 = 1; - message E2 - { - message E2Value - { - message E1 - { - repeated bytes e1 = 1; - } - repeated E1 e1 = 1; - message E2 - { - uint32 e1 = 1; - message E2 - { - message E1 - { - repeated bytes e1 = 1; - } - repeated E1 e1 = 1; - uint32 e2 = 2; - } - E2 e2 = 2; - } - repeated E2 e2 = 2; - } - map e2 = 1; - } - repeated E2 e2 = 2; - } - C1 c1 = 1; -} -Read/write with no schema -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 -Output schema -syntax = "proto3"; - -message Message -{ - uint64 number = 1; -} -Bad output schema path -2 -2 From ab086f15d09048deb30bef84d5d3e7e62fefd898 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 3 Jun 2023 21:09:25 +0200 Subject: [PATCH 1114/2047] try to push down more --- src/Processors/QueryPlan/JoinStep.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 2ff8f161e99..33fa7955e0d 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -54,7 +54,7 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines bool JoinStep::allowPushDownToRight() const { - return join->pipelineType() == JoinPipelineType::YShaped; + return join->pipelineType() == JoinPipelineType::YShaped || join->pipelineType() == JoinPipelineType::FillRightFirst; } void JoinStep::describePipeline(FormatSettings & settings) const From b2acbe42b722f83c0ffde1c8697e5f19bb14747f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 3 Jun 2023 21:15:19 +0200 Subject: [PATCH 1115/2047] add perf test --- tests/performance/join_filter_pushdown.xml | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 tests/performance/join_filter_pushdown.xml diff --git a/tests/performance/join_filter_pushdown.xml b/tests/performance/join_filter_pushdown.xml new file mode 100644 index 00000000000..3adbbb3029e --- /dev/null +++ b/tests/performance/join_filter_pushdown.xml @@ -0,0 +1,9 @@ + + create table t(a UInt64) engine=MergeTree order by tuple() + insert into t select * from numbers_mt(5e6) + + select * from t as t0 inner join t as t1 using(a) where t1.a = 100 + + drop table t + + From d0894532feff599d1e73acca1a9010a53a26b004 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 26 Jun 2023 21:17:34 +0200 Subject: [PATCH 1116/2047] fix --- src/Processors/QueryPlan/Optimizations/filterPushDown.cpp | 4 ++++ .../queries/0_stateless/02514_analyzer_drop_join_on.reference | 4 ++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 4336de41b7b..af47b6ff4cd 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -341,6 +341,10 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (table_join.kind() != JoinKind::Inner && table_join.kind() != JoinKind::Cross && table_join.kind() != kind) return 0; + /// There is no ASOF Right join, so we're talking about pushing to the right side + if (kind == JoinKind::Right && table_join.strictness() == JoinStrictness::Asof) + return 0; + bool is_left = kind == JoinKind::Left; const auto & input_header = is_left ? child->getInputStreams().front().header : child->getInputStreams().back().header; const auto & res_header = child->getOutputStream().header; diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index 0037ab85c07..1b177b84afa 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -107,7 +107,7 @@ Header: bx String bx_0 String c2_5 String c1_3 UInt64 - Filter (( + (JOIN actions + DROP unused columns after JOIN))) + Expression Header: a2_6 String bx_0 String c2_5 String @@ -139,7 +139,7 @@ Header: bx String ReadFromMemoryStorage Header: b1 UInt64 b2 String - Expression ((JOIN actions + Change column names to column identifiers)) + Filter (( + (JOIN actions + Change column names to column identifiers))) Header: c1_3 UInt64 c2_5 String ReadFromMemoryStorage From 104d3bbbae82309d7d55d3a46a28e6f791791fba Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 24 Jul 2023 21:10:00 +0200 Subject: [PATCH 1117/2047] add test --- .../0_stateless/01763_filter_push_down_bugs.sql | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 9a5ef4727c5..8470b4a3379 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -66,3 +66,17 @@ EXPLAIN indexes=1 SELECT id, delete_time FROM t1 DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; + +-- expected to get row (1, 3, 1, 4) from JOIN and empty result from the query +SELECT * +FROM +( + SELECT * + FROM Values('id UInt64, t UInt64', (1, 3)) +) AS t1 +ASOF INNER JOIN +( + SELECT * + FROM Values('id UInt64, t UInt64', (1, 1), (1, 2), (1, 3), (1, 4), (1, 5)) +) AS t2 ON (t1.id = t2.id) AND (t1.t < t2.t) +WHERE t2.t != 4; From edc479bbf8c72a7076b092dd880fc7d8d2252e4d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 24 Jul 2023 23:40:27 +0200 Subject: [PATCH 1118/2047] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 4 +++- src/Storages/StorageReplicatedMergeTree.h | 3 +++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6cdcffab50a..e6431927805 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4861,6 +4861,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) LOG_TRACE(log, "Waiting for RestartingThread to startup table"); } + std::lock_guard lock{flush_and_shutdown_mutex}; if (shutdown_prepared_called.load() || shutdown_called.load()) throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot startup table because it is dropped"); @@ -4906,6 +4907,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) void StorageReplicatedMergeTree::flushAndPrepareForShutdown() { + std::lock_guard lock{flush_and_shutdown_mutex}; if (shutdown_prepared_called.exchange(true)) return; @@ -4922,7 +4924,7 @@ void StorageReplicatedMergeTree::flushAndPrepareForShutdown() attach_thread->shutdown(); restarting_thread.shutdown(/* part_of_full_shutdown */true); - /// Explicetly set the event, because the restarting thread will not set it again + /// Explicitly set the event, because the restarting thread will not set it again startup_event.set(); shutdown_deadline.emplace(std::chrono::system_clock::now() + std::chrono::milliseconds(settings_ptr->wait_for_unique_parts_send_before_shutdown_ms.totalMilliseconds())); } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 1c721e3724b..daa39536fa7 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -481,6 +481,9 @@ private: std::atomic shutdown_prepared_called {false}; std::optional shutdown_deadline; + /// We call flushAndPrepareForShutdown before acquiring DDLGuard, so we can shutdown a table that is being created right now + mutable std::mutex flush_and_shutdown_mutex; + mutable std::mutex last_sent_parts_mutex; std::condition_variable last_sent_parts_cv; From 21382afa2b2c686cde3ac0702b548d872373d3b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Jul 2023 06:10:04 +0200 Subject: [PATCH 1119/2047] Check for punctuation --- .../AggregateFunctionSimpleLinearRegression.cpp | 2 +- src/Common/ConcurrentBoundedQueue.h | 2 +- src/Common/parseRemoteDescription.cpp | 16 ++-------------- src/Common/parseRemoteDescription.h | 2 +- src/Common/tests/gtest_sensitive_data_masker.cpp | 6 +++--- src/Coordination/ZooKeeperDataReader.cpp | 2 +- src/Core/tests/gtest_settings.cpp | 3 +-- src/DataTypes/NumberTraits.h | 2 +- .../getDictionaryConfigurationFromAST.cpp | 2 +- src/Functions/FunctionsStringHash.cpp | 5 ++--- src/Functions/GatherUtils/sliceHasImplAnyAll.h | 4 ++-- src/IO/S3/PocoHTTPClient.cpp | 2 +- src/Interpreters/Aggregator.h | 2 +- src/Interpreters/DDLWorker.cpp | 2 +- src/Interpreters/InterpreterRenameQuery.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 2 +- src/Parsers/Kusto/ParserKQLOperators.h | 16 ++++++++-------- src/Parsers/tests/gtest_Parser.cpp | 4 ++-- .../Formats/Impl/ArrowFieldIndexUtil.h | 2 +- .../Formats/Impl/JSONEachRowRowInputFormat.cpp | 4 ++-- .../QueryPlan/IntersectOrExceptStep.cpp | 2 +- .../Transforms/buildPushingToViewsChain.cpp | 2 +- src/Server/HTTPHandler.cpp | 2 +- src/Storages/StorageFile.cpp | 4 ++-- src/Storages/StorageProxy.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/TableFunctions/TableFunctionFactory.cpp | 2 +- utils/check-style/check-style | 3 +++ 28 files changed, 45 insertions(+), 56 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.cpp b/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.cpp index 1ed6c83af7d..1489db55857 100644 --- a/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.cpp +++ b/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.cpp @@ -66,7 +66,7 @@ AggregateFunctionPtr createAggregateFunctionSimpleLinearRegression( #undef FOR_LEASTSQR_TYPES #undef DISPATCH - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT , + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types ({}, {}) of arguments of aggregate function {}, must " "be Native Ints, Native UInts or Floats", x_arg->getName(), y_arg->getName(), name); } diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index 01910c4caff..922607da813 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -110,7 +110,7 @@ public: /// Returns false if queue is finished [[nodiscard]] bool pushFront(const T & x) { - return emplaceImpl(/* timeout_milliseconds= */ std::nullopt , x); + return emplaceImpl(/* timeout_milliseconds= */ std::nullopt, x); } /// Returns false if queue is finished diff --git a/src/Common/parseRemoteDescription.cpp b/src/Common/parseRemoteDescription.cpp index 0bcd62d30c7..8ea3f4a0aa5 100644 --- a/src/Common/parseRemoteDescription.cpp +++ b/src/Common/parseRemoteDescription.cpp @@ -52,20 +52,8 @@ static bool parseNumber(const String & description, size_t l, size_t r, size_t & } -/* Parse a string that generates shards and replicas. Separator - one of two characters | or , - * depending on whether shards or replicas are generated. - * For example: - * host1,host2,... - generates set of shards from host1, host2, ... - * host1|host2|... - generates set of replicas from host1, host2, ... - * abc{8..10}def - generates set of shards abc8def, abc9def, abc10def. - * abc{08..10}def - generates set of shards abc08def, abc09def, abc10def. - * abc{x,yy,z}def - generates set of shards abcxdef, abcyydef, abczdef. - * abc{x|yy|z} def - generates set of replicas abcxdef, abcyydef, abczdef. - * abc{1..9}de{f,g,h} - is a direct product, 27 shards. - * abc{1..9}de{0|1} - is a direct product, 9 shards, in each 2 replicas. - */ -std::vector -parseRemoteDescription(const String & description, size_t l, size_t r, char separator, size_t max_addresses, const String & func_name) +std::vector parseRemoteDescription( + const String & description, size_t l, size_t r, char separator, size_t max_addresses, const String & func_name) { std::vector res; std::vector cur; diff --git a/src/Common/parseRemoteDescription.h b/src/Common/parseRemoteDescription.h index e3e4a3f523c..d97558c4728 100644 --- a/src/Common/parseRemoteDescription.h +++ b/src/Common/parseRemoteDescription.h @@ -3,7 +3,7 @@ #include namespace DB { -/* Parse a string that generates shards and replicas. Separator - one of two characters | or , +/* Parse a string that generates shards and replicas. Separator - one of two characters '|' or ',' * depending on whether shards or replicas are generated. * For example: * host1,host2,... - generates set of shards from host1, host2, ... diff --git a/src/Common/tests/gtest_sensitive_data_masker.cpp b/src/Common/tests/gtest_sensitive_data_masker.cpp index 92c4edbac2a..f36c4154684 100644 --- a/src/Common/tests/gtest_sensitive_data_masker.cpp +++ b/src/Common/tests/gtest_sensitive_data_masker.cpp @@ -27,7 +27,7 @@ TEST(Common, SensitiveDataMasker) { Poco::AutoPtr empty_xml_config = new Poco::Util::XMLConfiguration(); - DB::SensitiveDataMasker masker(*empty_xml_config , ""); + DB::SensitiveDataMasker masker(*empty_xml_config, ""); masker.addMaskingRule("all a letters", "a+", "--a--"); masker.addMaskingRule("all b letters", "b+", "--b--"); masker.addMaskingRule("all d letters", "d+", "--d--"); @@ -45,7 +45,7 @@ TEST(Common, SensitiveDataMasker) masker.printStats(); #endif - DB::SensitiveDataMasker masker2(*empty_xml_config , ""); + DB::SensitiveDataMasker masker2(*empty_xml_config, ""); masker2.addMaskingRule("hide root password", "qwerty123", "******"); masker2.addMaskingRule("hide SSN", "[0-9]{3}-[0-9]{2}-[0-9]{4}", "000-00-0000"); masker2.addMaskingRule("hide email", "[A-Za-z0-9._%+-]+@[A-Za-z0-9.-]+\\.[A-Za-z]{2,4}", "hidden@hidden.test"); @@ -58,7 +58,7 @@ TEST(Common, SensitiveDataMasker) "SELECT id FROM mysql('localhost:3308', 'database', 'table', 'root', '******') WHERE " "ssn='000-00-0000' or email='hidden@hidden.test'"); - DB::SensitiveDataMasker maskerbad(*empty_xml_config , ""); + DB::SensitiveDataMasker maskerbad(*empty_xml_config, ""); // gtest has not good way to check exception content, so just do it manually (see https://github.com/google/googletest/issues/952 ) try diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 94fc07bcc4a..79929c4e66e 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -40,7 +40,7 @@ void deserializeSnapshotMagic(ReadBuffer & in) Coordination::read(dbid, in); static constexpr int32_t SNP_HEADER = 1514885966; /// "ZKSN" if (magic_header != SNP_HEADER) - throw Exception(ErrorCodes::CORRUPTED_DATA ,"Incorrect magic header in file, expected {}, got {}", SNP_HEADER, magic_header); + throw Exception(ErrorCodes::CORRUPTED_DATA, "Incorrect magic header in file, expected {}, got {}", SNP_HEADER, magic_header); } int64_t deserializeSessionAndTimeout(KeeperStorage & storage, ReadBuffer & in) diff --git a/src/Core/tests/gtest_settings.cpp b/src/Core/tests/gtest_settings.cpp index cbeb84ef2e7..a6d8763bfb8 100644 --- a/src/Core/tests/gtest_settings.cpp +++ b/src/Core/tests/gtest_settings.cpp @@ -121,7 +121,7 @@ GTEST_TEST(SettingMySQLDataTypesSupport, SetString) ASSERT_EQ(Field("decimal,datetime64"), setting); // comma with spaces - setting = " datetime64 , decimal "; + setting = " datetime64 , decimal "; /// bad punctuation is ok here ASSERT_TRUE(setting.changed); ASSERT_TRUE(setting.value.isSet(MySQLDataTypesSupport::DECIMAL)); ASSERT_TRUE(setting.value.isSet(MySQLDataTypesSupport::DATETIME64)); @@ -166,4 +166,3 @@ GTEST_TEST(SettingMySQLDataTypesSupport, SetInvalidString) ASSERT_TRUE(setting.changed); ASSERT_EQ(0, setting.value.getValue()); } - diff --git a/src/DataTypes/NumberTraits.h b/src/DataTypes/NumberTraits.h index 6b068b0d8b1..cf283d3358c 100644 --- a/src/DataTypes/NumberTraits.h +++ b/src/DataTypes/NumberTraits.h @@ -174,7 +174,7 @@ template struct ResultOfBitNot * Float, [U]Int -> Float * Decimal, Decimal -> Decimal * UUID, UUID -> UUID - * UInt64 , Int -> Error + * UInt64, Int -> Error * Float, [U]Int64 -> Error */ template diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 0b7352e9cbb..b12ffc555d4 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -322,7 +322,7 @@ void buildSingleAttribute( /** Transforms - * PRIMARY KEY Attr1 ,..., AttrN + * PRIMARY KEY Attr1, ..., AttrN * to the next configuration * Attr1 * or diff --git a/src/Functions/FunctionsStringHash.cpp b/src/Functions/FunctionsStringHash.cpp index d6873d9490e..ff8ff2d2651 100644 --- a/src/Functions/FunctionsStringHash.cpp +++ b/src/Functions/FunctionsStringHash.cpp @@ -292,8 +292,8 @@ struct SimHashImpl continue; // we need to store the new word hash value to the oldest location. - // for example, N = 5, array |a0|a1|a2|a3|a4|, now , a0 is the oldest location, - // so we need to store new word hash into location of a0, then ,this array become + // for example, N = 5, array |a0|a1|a2|a3|a4|, now, a0 is the oldest location, + // so we need to store new word hash into location of a0, then this array become // |a5|a1|a2|a3|a4|, next time, a1 become the oldest location, we need to store new // word hash value into location of a1, then array become |a5|a6|a2|a3|a4| words[offset] = BytesRef{word_start, length}; @@ -793,4 +793,3 @@ REGISTER_FUNCTION(StringHash) factory.registerFunction(); } } - diff --git a/src/Functions/GatherUtils/sliceHasImplAnyAll.h b/src/Functions/GatherUtils/sliceHasImplAnyAll.h index 21c80b742fd..99bf1a7cc33 100644 --- a/src/Functions/GatherUtils/sliceHasImplAnyAll.h +++ b/src/Functions/GatherUtils/sliceHasImplAnyAll.h @@ -375,14 +375,14 @@ bool sliceHasImplAnyAllImplInt16( _mm256_or_si256( _mm256_andnot_si256( _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8)), - _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data ,first_data, 1), _mm256_set_epi8(7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8)))), + _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(7,6,5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8)))), _mm256_andnot_si256( _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6)), _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(5,4,3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6))))), _mm256_or_si256( _mm256_andnot_si256( _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4)), - _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data ,first_data ,1), _mm256_set_epi8(3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4)))), + _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(3,2,1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4)))), _mm256_andnot_si256( _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_nm_mask, first_nm_mask, 1), _mm256_set_epi8(1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2)), _mm256_cmpeq_epi16(second_data, _mm256_shuffle_epi8(_mm256_permute2x128_si256(first_data, first_data, 1), _mm256_set_epi8(1,0,31,30,29,28,27,26,25,24,23,22,21,20,19,18,17,16,15,14,13,12,11,10,9,8,7,6,5,4,3,2)))))) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 1a367a8199d..fd825720ac9 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -258,7 +258,7 @@ void PocoHTTPClient::addMetric(const Aws::Http::HttpRequest & request, S3MetricT void PocoHTTPClient::makeRequestInternal( Aws::Http::HttpRequest & request, std::shared_ptr & response, - Aws::Utils::RateLimits::RateLimiterInterface * readLimiter , + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const { /// Most sessions in pool are already connected and it is not possible to set proxy host/port to a connected session. diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 05b34e8460f..29096a38be6 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -292,7 +292,7 @@ struct AggregationMethodStringNoCache { } - using State = ColumnsHashing::HashMethodString; + using State = ColumnsHashing::HashMethodString; static const bool low_cardinality_optimization = false; static const bool one_key_nullable_optimization = nullable; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 193bb5b6ab0..92e6bcb326c 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -551,7 +551,7 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper) chassert(!task.completely_processed); /// Setup tracing context on current thread for current DDL - OpenTelemetry::TracingContextHolder tracing_ctx_holder(__PRETTY_FUNCTION__ , + OpenTelemetry::TracingContextHolder tracing_ctx_holder(__PRETTY_FUNCTION__, task.entry.tracing_context, this->context->getOpenTelemetrySpanLog()); tracing_ctx_holder.root_span.kind = OpenTelemetry::CONSUMER; diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 75d43b541e1..ae79b3f932e 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -193,7 +193,7 @@ AccessRightsElements InterpreterRenameQuery::getRequiredAccess(InterpreterRename required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.to.getDatabase(), elem.to.getTable()); if (rename.exchange) { - required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT , elem.from.getDatabase(), elem.from.getTable()); + required_access.emplace_back(AccessType::CREATE_TABLE | AccessType::INSERT, elem.from.getDatabase(), elem.from.getTable()); required_access.emplace_back(AccessType::SELECT | AccessType::DROP_TABLE, elem.to.getDatabase(), elem.to.getTable()); } } diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 6257e617d4a..2ef4f4d6218 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -482,7 +482,7 @@ CSN TransactionLog::finalizeCommittedTransaction(MergeTreeTransaction * txn, CSN bool removed = running_list.erase(txn->tid.getHash()); if (!removed) { - LOG_ERROR(log , "I's a bug: TID {} {} doesn't exist", txn->tid.getHash(), txn->tid); + LOG_ERROR(log, "It's a bug: TID {} {} doesn't exist", txn->tid.getHash(), txn->tid); abort(); } } diff --git a/src/Parsers/Kusto/ParserKQLOperators.h b/src/Parsers/Kusto/ParserKQLOperators.h index 9796ae10c07..72e25cc3cf9 100644 --- a/src/Parsers/Kusto/ParserKQLOperators.h +++ b/src/Parsers/Kusto/ParserKQLOperators.h @@ -31,10 +31,10 @@ protected: not_endswith, endswith_cs, not_endswith_cs, - equal, //=~ - not_equal,//!~ - equal_cs, //= - not_equal_cs,//!= + equal, /// =~ + not_equal, /// !~ + equal_cs, /// = + not_equal_cs, /// != has, not_has, has_all, @@ -49,10 +49,10 @@ protected: not_hassuffix, hassuffix_cs, not_hassuffix_cs, - in_cs, //in - not_in_cs, //!in - in, //in~ - not_in ,//!in~ + in_cs, /// in + not_in_cs, /// !in + in, /// in~ + not_in, /// !in~ matches_regex, startswith, not_startswith, diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index d77ae8d3a27..18e91c533e0 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -359,11 +359,11 @@ INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserTest, "SELECT *\nFROM Customers\nORDER BY LastName DESC" }, { - "Customers | order by Age desc , FirstName asc ", + "Customers | order by Age desc, FirstName asc ", "SELECT *\nFROM Customers\nORDER BY\n Age DESC,\n FirstName ASC" }, { - "Customers | order by Age asc , FirstName desc", + "Customers | order by Age asc, FirstName desc", "SELECT *\nFROM Customers\nORDER BY\n Age ASC,\n FirstName DESC" }, { diff --git a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h index b7adaa35335..676ce50d04f 100644 --- a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h +++ b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h @@ -35,7 +35,7 @@ public: /// - key: field name with full path. eg. a struct field's name is like a.x.i /// - value: a pair, first value refers to this field's start index, second value refers to how many /// indices this field take. eg. - /// For a parquet schema {x: int , y: {i: int, j: int}}, the return will be + /// For a parquet schema {x: int, y: {i: int, j: int}}, the return will be /// - x: (0, 1) /// - y: (1, 2) /// - y.i: (1, 1) diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index e5f52936021..b1b08cdf256 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -236,10 +236,10 @@ bool JSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi bool JSONEachRowRowInputFormat::checkEndOfData(bool is_first_row) { - /// We consume , or \n before scanning a new row, instead scanning to next row at the end. + /// We consume ',' or '\n' before scanning a new row, instead scanning to next row at the end. /// The reason is that if we want an exact number of rows read with LIMIT x /// from a streaming table engine with text data format, like File or Kafka - /// then seeking to next ;, or \n would trigger reading of an extra row at the end. + /// then seeking to next ';,' or '\n' would trigger reading of an extra row at the end. /// Semicolon is added for convenience as it could be used at end of INSERT query. if (!in->eof()) diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index afdff44020f..b132d27670d 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -30,7 +30,7 @@ static Block checkHeaders(const DataStreams & input_streams_) } IntersectOrExceptStep::IntersectOrExceptStep( - DataStreams input_streams_ , Operator operator_ , size_t max_threads_) + DataStreams input_streams_, Operator operator_, size_t max_threads_) : header(checkHeaders(input_streams_)) , current_operator(operator_) , max_threads(max_threads_) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 7f7f9058f1b..1b20778877d 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -72,7 +72,7 @@ struct ViewsData std::atomic_bool has_exception = false; std::exception_ptr first_exception; - ViewsData(ThreadStatusesHolderPtr thread_status_holder_, ContextPtr context_, StorageID source_storage_id_, StorageMetadataPtr source_metadata_snapshot_ , StoragePtr source_storage_) + ViewsData(ThreadStatusesHolderPtr thread_status_holder_, ContextPtr context_, StorageID source_storage_id_, StorageMetadataPtr source_metadata_snapshot_, StoragePtr source_storage_) : thread_status_holder(std::move(thread_status_holder_)) , context(std::move(context_)) , source_storage_id(std::move(source_storage_id_)) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 069670c84a5..29b75fa6552 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -638,7 +638,7 @@ void HTTPHandler::processQuery( throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected MemoryWriteBuffer"); auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); - copyData(*rdbuf , *next_buffer); + copyData(*rdbuf, *next_buffer); return next_buffer; }; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index cbd32460f7e..3126d584964 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -242,8 +242,8 @@ void listFilesWithRegexpMatchingImpl( { if (recursive) { - listFilesWithRegexpMatchingImpl(fs::path(full_path).append(it->path().string()) / "" , - looking_for_directory ? suffix_with_globs.substr(next_slash_after_glob_pos) : current_glob , + listFilesWithRegexpMatchingImpl(fs::path(full_path).append(it->path().string()) / "", + looking_for_directory ? suffix_with_globs.substr(next_slash_after_glob_pos) : current_glob, total_bytes_to_read, result, recursive); } else if (looking_for_directory && re2::RE2::FullMatch(file_name, matcher)) diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index 14b7fc15af2..582dc6f882d 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -149,7 +149,7 @@ public: return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } - CheckResults checkData(const ASTPtr & query , ContextPtr context) override { return getNested()->checkData(query, context); } + CheckResults checkData(const ASTPtr & query, ContextPtr context) override { return getNested()->checkData(query, context); } void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); } bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); } Strings getDataPaths() const override { return getNested()->getDataPaths(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4e053c4598c..c3dedd69d0d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6569,7 +6569,7 @@ void StorageReplicatedMergeTree::fetchPartition( try { - /// part name , metadata, part_path , true, 0, zookeeper + /// part name, metadata, part_path, true, 0, zookeeper if (!fetchPart(part_name, metadata_snapshot, from_zookeeper_name, part_path, true, 0, zookeeper, /* try_fetch_shared = */ false)) throw Exception(ErrorCodes::UNFINISHED, "Failed to fetch part {} from {}", part_name, from_); } diff --git a/src/TableFunctions/TableFunctionFactory.cpp b/src/TableFunctions/TableFunctionFactory.cpp index 76108f1cdd4..ce3daff0785 100644 --- a/src/TableFunctions/TableFunctionFactory.cpp +++ b/src/TableFunctions/TableFunctionFactory.cpp @@ -41,7 +41,7 @@ TableFunctionPtr TableFunctionFactory::get( { auto hints = getHints(table_function->name); if (!hints.empty()) - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function->name , toString(hints)); + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}. Maybe you meant: {}", table_function->name, toString(hints)); else throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", table_function->name); } diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 0b3b86b4772..c28ca1cfc8a 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -410,3 +410,6 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep # The stateful directory should only contain the tests that depend on the test dataset (hits or visits). find $ROOT_PATH/tests/queries/1_stateful -name '*.sql' -or -name '*.sh' | grep -v '00076_system_columns_bytes' | xargs -I{} bash -c 'grep -q -P "hits|visits" "{}" || echo "The test {} does not depend on the test dataset (hits or visits table) and should be located in the 0_stateless directory. You can also add an exception to the check-style script."' + +# Check for bad punctuation: whitespace before comma. +find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '\w ,' | grep -v 'bad punctuation is ok here' && echo "^ There is bad punctuation: whitespace before comma. You should write it like this: 'Hello, world!'" From ab55c294bfa8c04aa94494a1bdff3e3e3b339821 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Tue, 25 Jul 2023 07:25:27 +0000 Subject: [PATCH 1120/2047] Add assert for STANDARD storage class --- tests/integration/test_backup_s3_storage_class/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backup_s3_storage_class/test.py b/tests/integration/test_backup_s3_storage_class/test.py index a4c93d40f0c..3c0a0cf7706 100644 --- a/tests/integration/test_backup_s3_storage_class/test.py +++ b/tests/integration/test_backup_s3_storage_class/test.py @@ -48,4 +48,6 @@ def test_backup_s3_storage_class(started_cluster): """ ) - assert True + minio = cluster.minio_client + lst = list(minio.list_objects(cluster.minio_bucket, "data/.backup")) + assert lst[0].storage_class == "STANDARD" From a5d7391fbf953d6395ecc5adba48a59e1c248811 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Jul 2023 15:34:38 +0800 Subject: [PATCH 1121/2047] add new function startsWithUTF8 and endsWithUTF8 --- .../functions/string-functions.md | 28 +++++++ src/Functions/FunctionStartsEndsWith.h | 77 ++++++++++++++++--- src/Functions/endsWithUTF8.cpp | 16 ++++ src/Functions/startsWithUTF8.cpp | 16 ++++ .../02833_starts_ends_with_utf8.reference | 29 +++++++ .../02833_starts_ends_with_utf8.sql | 19 +++++ 6 files changed, 175 insertions(+), 10 deletions(-) create mode 100644 src/Functions/endsWithUTF8.cpp create mode 100644 src/Functions/startsWithUTF8.cpp create mode 100644 tests/queries/0_stateless/02833_starts_ends_with_utf8.reference create mode 100644 tests/queries/0_stateless/02833_starts_ends_with_utf8.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 9890d257e84..a398a7c0993 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -693,6 +693,22 @@ Returns whether string `str` ends with `suffix`. endsWith(str, suffix) ``` +## endsWithUTF8 + +Returns whether string `str` ends with `suffix`, assuming that both strings contain svalid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +**Syntax** + +```sql +endsWithUTF8(str, suffix) +``` + +**Example** + +``` sql +SELECT endsWithUTF8('中国', '国'); +``` + ## startsWith Returns whether string `str` starts with `prefix`. @@ -709,6 +725,18 @@ startsWith(str, prefix) SELECT startsWith('Spider-Man', 'Spi'); ``` +## startsWithUTF8 + +Returns whether string `str` starts with `prefix`, assuming that both string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + + +**Example** + +``` sql +SELECT startsWithUTF8('中国', '中'); +``` + + ## trim Removes the specified characters from the start or end of a string. If not specified otherwise, the function removes whitespace (ASCII-character 32). diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index b15bab47ae0..3565af326fe 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -28,10 +28,23 @@ namespace ErrorCodes struct NameStartsWith { static constexpr auto name = "startsWith"; + static constexpr auto is_utf8 = false; }; struct NameEndsWith { static constexpr auto name = "endsWith"; + static constexpr auto is_utf8 = false; +}; + +struct NameStartsWithUTF8 +{ + static constexpr auto name = "startsWithUTF8"; + static constexpr auto is_utf8 = true; +}; +struct NameEndsWithUTF8 +{ + static constexpr auto name = "endsWithUTF8"; + static constexpr auto is_utf8 = true; }; DECLARE_MULTITARGET_CODE( @@ -41,6 +54,7 @@ class FunctionStartsEndsWith : public IFunction { public: static constexpr auto name = Name::name; + static constexpr auto is_utf8 = Name::is_utf8; String getName() const override { @@ -64,7 +78,8 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (isStringOrFixedString(arguments[0]) && isStringOrFixedString(arguments[1])) + if (!is_utf8 && isStringOrFixedString(arguments[0]) && isStringOrFixedString(arguments[1]) + || isString(arguments[0]) && isString(arguments[1])) return std::make_shared(); if (isArray(arguments[0]) && isArray(arguments[1])) @@ -78,8 +93,11 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { auto data_type = arguments[0].type; - if (isStringOrFixedString(*data_type)) + + if (!is_utf8 && isStringOrFixedString(*data_type)) return executeImplString(arguments, {}, input_rows_count); + if (is_utf8 && isString(*data_type)) + return executeImplStringUTF8(arguments, {}, input_rows_count); if (isArray(data_type)) return executeImplArray(arguments, {}, input_rows_count); return {}; @@ -131,7 +149,6 @@ private: typename ColumnVector::Container & vec_res = col_res->getData(); vec_res.resize(input_rows_count); - if (const ColumnString * haystack = checkAndGetColumn(haystack_column)) dispatch(StringSource(*haystack), needle_column, vec_res); else if (const ColumnFixedString * haystack_fixed = checkAndGetColumn(haystack_column)) @@ -146,6 +163,26 @@ private: return col_res; } + ColumnPtr executeImplStringUTF8(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const + { + const IColumn * haystack_column = arguments[0].column.get(); + const IColumn * needle_column = arguments[1].column.get(); + + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + + vec_res.resize(input_rows_count); + if (const ColumnString * haystack = checkAndGetColumn(haystack_column)) + dispatchUTF8(UTF8StringSource(*haystack), needle_column, vec_res); + else if (const ColumnConst * haystack_const = checkAndGetColumnConst(haystack_column)) + dispatchUTF8>(ConstSource(*haystack_const), needle_column, vec_res); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName()); + + return col_res; + } + + template void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const { @@ -161,6 +198,17 @@ private: throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName()); } + template + void dispatchUTF8(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const + { + if (const ColumnString * needle = checkAndGetColumn(needle_column)) + execute(haystack_source, UTF8StringSource(*needle), res_data); + else if (const ColumnConst * needle_const = checkAndGetColumnConst(needle_column)) + execute>(haystack_source, ConstSource(*needle_const), res_data); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName()); + } + template static void execute(HaystackSource haystack_source, NeedleSource needle_source, PaddedPODArray & res_data) { @@ -172,18 +220,27 @@ private: auto needle = needle_source.getWhole(); if (needle.size > haystack.size) - { res_data[row_num] = false; - } else { - if constexpr (std::is_same_v) - { + if constexpr (std::is_same_v) /// startsWith res_data[row_num] = StringRef(haystack.data, needle.size) == StringRef(needle.data, needle.size); - } - else /// endsWith - { + else if constexpr (std::is_same_v) /// endsWith res_data[row_num] = StringRef(haystack.data + haystack.size - needle.size, needle.size) == StringRef(needle.data, needle.size); + else /// startsWithUTF8 or endsWithUTF8 + { + auto length = UTF8::countCodePoints(needle.data, needle.size); + + if constexpr (std::is_same_v) + { + auto slice = haystack_source.getSliceFromLeft(0, length); + res_data[row_num] = StringRef(slice.data, slice.size) == StringRef(needle.data, needle.size); + } + else + { + auto slice = haystack_source.getSliceFromRight(length); + res_data[row_num] = StringRef(slice.data, slice.size) == StringRef(needle.data, needle.size); + } } } diff --git a/src/Functions/endsWithUTF8.cpp b/src/Functions/endsWithUTF8.cpp new file mode 100644 index 00000000000..377683ce47c --- /dev/null +++ b/src/Functions/endsWithUTF8.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionEndsWithUTF8 = FunctionStartsEndsWith; + +REGISTER_FUNCTION(EndsWithUTF8) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/startsWithUTF8.cpp b/src/Functions/startsWithUTF8.cpp new file mode 100644 index 00000000000..e623a55f4dc --- /dev/null +++ b/src/Functions/startsWithUTF8.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionStartsWithUTF8 = FunctionStartsEndsWith; + +REGISTER_FUNCTION(StartsWithUTF8) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02833_starts_ends_with_utf8.reference b/tests/queries/0_stateless/02833_starts_ends_with_utf8.reference new file mode 100644 index 00000000000..ca2a5bc50f8 --- /dev/null +++ b/tests/queries/0_stateless/02833_starts_ends_with_utf8.reference @@ -0,0 +1,29 @@ +-- { echoOn } +select startsWithUTF8('富强民主文明和谐', '富强'); +1 +select startsWithUTF8('富强民主文明和谐', '\xe5'); +0 +select startsWithUTF8('富强民主文明和谐', ''); +1 +SELECT startsWithUTF8('123', '123'); +1 +SELECT startsWithUTF8('123', '12'); +1 +SELECT startsWithUTF8('123', '1234'); +0 +SELECT startsWithUTF8('123', ''); +1 +select endsWithUTF8('富强民主文明和谐', '和谐'); +1 +select endsWithUTF8('富强民主文明和谐', '\x90'); +0 +select endsWithUTF8('富强民主文明和谐', ''); +1 +SELECT endsWithUTF8('123', '3'); +1 +SELECT endsWithUTF8('123', '23'); +1 +SELECT endsWithUTF8('123', '32'); +0 +SELECT endsWithUTF8('123', ''); +1 diff --git a/tests/queries/0_stateless/02833_starts_ends_with_utf8.sql b/tests/queries/0_stateless/02833_starts_ends_with_utf8.sql new file mode 100644 index 00000000000..3a783dc280e --- /dev/null +++ b/tests/queries/0_stateless/02833_starts_ends_with_utf8.sql @@ -0,0 +1,19 @@ +-- { echoOn } +select startsWithUTF8('富强民主文明和谐', '富强'); +select startsWithUTF8('富强民主文明和谐', '\xe5'); +select startsWithUTF8('富强民主文明和谐', ''); + +SELECT startsWithUTF8('123', '123'); +SELECT startsWithUTF8('123', '12'); +SELECT startsWithUTF8('123', '1234'); +SELECT startsWithUTF8('123', ''); + +select endsWithUTF8('富强民主文明和谐', '和谐'); +select endsWithUTF8('富强民主文明和谐', '\x90'); +select endsWithUTF8('富强民主文明和谐', ''); + +SELECT endsWithUTF8('123', '3'); +SELECT endsWithUTF8('123', '23'); +SELECT endsWithUTF8('123', '32'); +SELECT endsWithUTF8('123', ''); +-- { echoOff } From acb33c1d810982291e7f208340dfba50d7f6c327 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Jul 2023 15:39:49 +0800 Subject: [PATCH 1122/2047] fix code style --- src/Functions/FunctionStartsEndsWith.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 3565af326fe..7671129fcfc 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -41,6 +41,7 @@ struct NameStartsWithUTF8 static constexpr auto name = "startsWithUTF8"; static constexpr auto is_utf8 = true; }; + struct NameEndsWithUTF8 { static constexpr auto name = "endsWithUTF8"; From 3e3adc7fecd5f6c409320727bec3a0291aa2430b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Jul 2023 10:29:59 +0200 Subject: [PATCH 1123/2047] tests: increase throttling for 01923_network_receive_time_metric_insert In debug builds launching the client can take a while, so let's increase the throttling to avoid flakiness CI: https://s3.amazonaws.com/clickhouse-test-reports/52490/9e2526a5f04861fcfac49c2ce85560d08c68af66/stateless_tests__debug__[1_5].html Signed-off-by: Azat Khuzhin --- .../0_stateless/01923_network_receive_time_metric_insert.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh index ec5aa141859..4d7e79fae52 100755 --- a/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh +++ b/tests/queries/0_stateless/01923_network_receive_time_metric_insert.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = Memory;" # Rate limit is chosen for operation to spent more than one second. -seq 1 1000 | pv --quiet --rate-limit 1000 | ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT TSV" +seq 1 1000 | pv --quiet --rate-limit 500 | ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT TSV" # We check that the value of NetworkReceiveElapsedMicroseconds correctly includes the time spent waiting data from the client. ${CLICKHOUSE_CLIENT} --multiquery --query "SYSTEM FLUSH LOGS; From b02e290d5507419e6166433b0a045eaeb3d124d9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Jul 2023 10:37:02 +0200 Subject: [PATCH 1124/2047] tests: fix 01035_avg_weighted_long flakiness Use one clickhouse-client invocation instead of 300, in debug builds it is significant - each spawn is ~1 second Signed-off-by: Azat Khuzhin --- .../0_stateless/01035_avg_weighted_long.sh | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/tests/queries/0_stateless/01035_avg_weighted_long.sh b/tests/queries/0_stateless/01035_avg_weighted_long.sh index 138aa03fbb3..8838b07a3d7 100755 --- a/tests/queries/0_stateless/01035_avg_weighted_long.sh +++ b/tests/queries/0_stateless/01035_avg_weighted_long.sh @@ -11,36 +11,36 @@ ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, y) FROM (select toDecimal256 ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, y) FROM (select toDecimal32(1, 0) x, toDecimal256(1, 1) y);" types=("Int8" "Int16" "Int32" "Int64" "UInt8" "UInt16" "UInt32" "UInt64" "Float32" "Float64") - -for left in "${types[@]}" -do - for right in "${types[@]}" - do - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0), (10, 2))" - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (0, 0), (1, 0))" - done -done - exttypes=("Int128" "Int256" "UInt256") - -for left in "${exttypes[@]}" -do - for right in "${exttypes[@]}" - do - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(to${left}(1), to${right}(2))" - done -done - # Decimal types dtypes=("32" "64" "128" "256") -for left in "${dtypes[@]}" -do - for right in "${dtypes[@]}" +( + for left in "${types[@]}" do - ${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(toDecimal${left}(2, 4), toDecimal${right}(1, 4))" + for right in "${types[@]}" + do + echo "SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0), (10, 2));" + echo "SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (0, 0), (1, 0));" + done done -done + + for left in "${exttypes[@]}" + do + for right in "${exttypes[@]}" + do + echo "SELECT avgWeighted(to${left}(1), to${right}(2));" + done + done + + for left in "${dtypes[@]}" + do + for right in "${dtypes[@]}" + do + echo "SELECT avgWeighted(toDecimal${left}(2, 4), toDecimal${right}(1, 4));" + done + done +) | clickhouse-client -nm echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="SELECT avgWeighted(['string'], toFloat64(0))" 2>&1)" \ | grep -c 'Code: 43. DB::Exception: .* DB::Exception:.* Types .* are non-conforming as arguments for aggregate function avgWeighted' From 2efbeab5afe50fbd734a6729e4cffa7ef12fff04 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Jul 2023 10:43:29 +0200 Subject: [PATCH 1125/2047] tests: fix 00719_parallel_ddl_table flakiness in debug builds In debug bulds each client invocation takes ~1 second, and on CI it can take more if the node is under some load, so let's decrease number of iterations. Anyway CI runs each test ~1K times daily, and if there will be something even this number of iterations should be enough. Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/00719_parallel_ddl_table.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00719_parallel_ddl_table.sh b/tests/queries/0_stateless/00719_parallel_ddl_table.sh index fdc994aec33..57a7e228341 100755 --- a/tests/queries/0_stateless/00719_parallel_ddl_table.sh +++ b/tests/queries/0_stateless/00719_parallel_ddl_table.sh @@ -10,7 +10,7 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS parallel_ddl" function query() { - for _ in {1..100}; do + for _ in {1..50}; do ${CLICKHOUSE_CLIENT} --query "CREATE TABLE IF NOT EXISTS parallel_ddl(a Int) ENGINE = Memory" ${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS parallel_ddl" done From d500e75569c59d1f91ae3de9c43f24f2be703e21 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 25 Jul 2023 12:07:47 +0200 Subject: [PATCH 1126/2047] fix --- src/Functions/FunctionToDecimalString.h | 67 ++----------------- .../0_stateless/02676_to_decimal_string.sql | 6 ++ 2 files changed, 13 insertions(+), 60 deletions(-) diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h index 6ae007e6b66..68ad978632e 100644 --- a/src/Functions/FunctionToDecimalString.h +++ b/src/Functions/FunctionToDecimalString.h @@ -22,6 +22,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } class FunctionToDecimalString : public IFunction @@ -36,17 +37,14 @@ public: size_t getNumberOfArguments() const override { return 2; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isNumber(*arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal first argument for formatDecimal function: got {}, expected numeric type", - arguments[0]->getName()); + FunctionArgumentDescriptors mandatory_args = { + {"Value", nullptr, nullptr, nullptr}, + {"precision", &isNativeInteger, &isColumnConst, "const Integer [0-77]"} + }; - if (!isUInt8(*arguments[1])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal second argument for formatDecimal function: got {}, expected UInt8", - arguments[1]->getName()); + validateFunctionArgumentTypes(*this, arguments, mandatory_args, {}); return std::make_shared(); } @@ -98,29 +96,6 @@ private: buf_to.finalize(); } - template - void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets) const - { - size_t input_rows_count = vec_precision.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested, shall not be more than {}", max_digits); - format(value_from, buf_to, vec_precision[i]); - result_offsets[i] = buf_to.count(); - } - - buf_to.finalize(); - } - /// For operations with Decimal template void vectorConstant(const FirstArgVectorType & vec_from, UInt8 precision, @@ -168,29 +143,6 @@ private: buf_to.finalize(); } - template - void constantVector(const FirstArgType & value_from, const ColumnVector::Container & vec_precision, - ColumnString::Chars & vec_to, ColumnString::Offsets & result_offsets, UInt8 from_scale) const - { - size_t input_rows_count = vec_precision.size(); - result_offsets.resize(input_rows_count); - - WriteBufferFromVector buf_to(vec_to); - - constexpr size_t max_digits = std::numeric_limits::digits10; - - for (size_t i = 0; i < input_rows_count; ++i) - { - if (vec_precision[i] > max_digits) - throw DB::Exception(DB::ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, - "Too many fractional digits requested for Decimal, must not be more than {}", max_digits); - writeText(value_from, from_scale, buf_to, true, true, vec_precision[i]); - writeChar(0, buf_to); - result_offsets[i] = buf_to.count(); - } - buf_to.finalize(); - } - template static void format(T value, DB::WriteBuffer & out, UInt8 precision) { @@ -263,7 +215,6 @@ private: template ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const { - const auto * from_col_const = typeid_cast(arguments[0].column.get()); const auto * precision_col = checkAndGetColumn>(arguments[1].column.get()); const auto * precision_col_const = typeid_cast(arguments[1].column.get()); @@ -284,8 +235,6 @@ private: else vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale); } - else if (from_col_const) - constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets, from_scale); else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); } @@ -299,8 +248,6 @@ private: else vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets); } - else if (from_col_const) - constantVector(from_col_const->template getValue(), precision_col->getData(), result_chars, result_offsets); else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); } diff --git a/tests/queries/0_stateless/02676_to_decimal_string.sql b/tests/queries/0_stateless/02676_to_decimal_string.sql index 563d60c62c7..1dae139deb1 100644 --- a/tests/queries/0_stateless/02676_to_decimal_string.sql +++ b/tests/queries/0_stateless/02676_to_decimal_string.sql @@ -33,3 +33,9 @@ SELECT toDecimalString('64.64'::Float64, 61); -- {serverError CANNOT_PRINT_FLOAT SELECT toDecimalString('88'::UInt8, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} SELECT toDecimalString('646464'::Int256, 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} SELECT toDecimalString('-128.789323123321329854641231237893231233213298546'::Decimal256(45), 78); -- {serverError CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER} + +-- wrong types: #52407 and similar +SELECT toDecimalString('256.256'::Decimal256(45), *); -- {serverError ILLEGAL_COLUMN} +SELECT toDecimalString('128.128'::Decimal128(30), 'str'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT toDecimalString('64.64'::Decimal64(10)); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT toDecimalString('64.64'::Decimal64(10), 3, 3); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} From 8184a289e5441208110bcd2f8f63b57e31ccde33 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 1 May 2023 01:53:20 +0000 Subject: [PATCH 1127/2047] Partially reimplement Parquet encoder to make it faster and parallelizable --- src/CMakeLists.txt | 4 + src/Common/CurrentMetrics.cpp | 6 +- src/Common/PODArray.cpp | 10 + src/Common/PODArray.h | 11 + src/Core/Settings.h | 4 + src/Formats/FormatFactory.cpp | 7 +- src/Formats/FormatSettings.h | 6 + .../Formats/Impl/CHColumnToArrowColumn.cpp | 5 +- .../Formats/Impl/Parquet/PrepareForWrite.cpp | 618 +++++++++++++ .../Formats/Impl/Parquet/ThriftUtil.cpp | 35 + .../Formats/Impl/Parquet/ThriftUtil.h | 17 + src/Processors/Formats/Impl/Parquet/Write.cpp | 816 ++++++++++++++++++ src/Processors/Formats/Impl/Parquet/Write.h | 135 +++ .../Formats/Impl/ParquetBlockOutputFormat.cpp | 467 +++++++++- .../Formats/Impl/ParquetBlockOutputFormat.h | 116 ++- .../02735_parquet_encoder.reference | 55 ++ .../0_stateless/02735_parquet_encoder.sql | 168 ++++ 17 files changed, 2425 insertions(+), 55 deletions(-) create mode 100644 src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp create mode 100644 src/Processors/Formats/Impl/Parquet/ThriftUtil.cpp create mode 100644 src/Processors/Formats/Impl/Parquet/ThriftUtil.h create mode 100644 src/Processors/Formats/Impl/Parquet/Write.cpp create mode 100644 src/Processors/Formats/Impl/Parquet/Write.h create mode 100644 tests/queries/0_stateless/02735_parquet_encoder.reference create mode 100644 tests/queries/0_stateless/02735_parquet_encoder.sql diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 975bf9bb618..5c66c7e9495 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -267,6 +267,10 @@ add_object_library(clickhouse_processors_queryplan Processors/QueryPlan) add_object_library(clickhouse_processors_queryplan_optimizations Processors/QueryPlan/Optimizations) add_object_library(clickhouse_user_defined_functions Functions/UserDefined) +if (USE_PARQUET) + add_object_library(clickhouse_processors_formats_impl_parquet Processors/Formats/Impl/Parquet) +endif() + if (TARGET ch_contrib::nuraft) add_object_library(clickhouse_coordination Coordination) endif() diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 583b13cf79d..9a4ffb0577a 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -149,8 +149,10 @@ M(RestartReplicaThreadsActive, "Number of threads in the RESTART REPLICA thread pool running a task.") \ M(QueryPipelineExecutorThreads, "Number of threads in the PipelineExecutor thread pool.") \ M(QueryPipelineExecutorThreadsActive, "Number of threads in the PipelineExecutor thread pool running a task.") \ - M(ParquetDecoderThreads, "Number of threads in the ParquetBlockInputFormat thread pool running a task.") \ - M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool.") \ + M(ParquetDecoderThreads, "Number of threads in the ParquetBlockInputFormat thread pool.") \ + M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool running a task.") \ + M(ParquetEncoderThreads, "Number of threads in ParquetBlockOutputFormat thread pool.") \ + M(ParquetEncoderThreadsActive, "Number of threads in ParquetBlockOutputFormat thread pool running a task.") \ M(OutdatedPartsLoadingThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ M(OutdatedPartsLoadingThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ M(DistributedBytesToInsert, "Number of pending bytes to process for asynchronous insertion into Distributed tables. Number of bytes for every shard is summed.") \ diff --git a/src/Common/PODArray.cpp b/src/Common/PODArray.cpp index 07c3cf1af1a..d21dc40867d 100644 --- a/src/Common/PODArray.cpp +++ b/src/Common/PODArray.cpp @@ -15,4 +15,14 @@ template class PODArray, PADDING_FOR_SIMD - 1, PADD template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; + +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; + +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; +template class PODArray, 0, 0>; } diff --git a/src/Common/PODArray.h b/src/Common/PODArray.h index b126afd2a37..68c1e325f0c 100644 --- a/src/Common/PODArray.h +++ b/src/Common/PODArray.h @@ -783,4 +783,15 @@ extern template class PODArray, PADDING_FOR_SIMD - extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; extern template class PODArray, PADDING_FOR_SIMD - 1, PADDING_FOR_SIMD>; + +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; + +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; +extern template class PODArray, 0, 0>; + } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 97c64ba133c..98f7f212aa5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -953,6 +953,10 @@ class IColumn; M(ParquetVersion, output_format_parquet_version, "2.latest", "Parquet format version for output format. Supported versions: 1.0, 2.4, 2.6 and 2.latest (default)", 0) \ M(ParquetCompression, output_format_parquet_compression_method, "lz4", "Compression method for Parquet output format. Supported codecs: snappy, lz4, brotli, zstd, gzip, none (uncompressed)", 0) \ M(Bool, output_format_parquet_compliant_nested_types, true, "In parquet file schema, use name 'element' instead of 'item' for list elements. This is a historical artifact of Arrow library implementation. Generally increases compatibility, except perhaps with some old versions of Arrow.", 0) \ + M(Bool, output_format_parquet_use_custom_encoder, true, "Use experimental faster Parquet encoder implementation.", 0) \ + M(Bool, output_format_parquet_parallel_encoding, true, "Do Parquet encoding in multiple threads. Requires output_format_parquet_use_custom_encoder.", 0) \ + M(UInt64, output_format_parquet_data_page_size, 1024 * 1024, "Target page size in bytes, before compression.", 0) \ + M(UInt64, output_format_parquet_batch_size, 1024, "Check page size every this many rows. Consider decreasing if you have columns with average values size above a few KBs.", 0) \ M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ M(String, output_format_avro_string_column_pattern, "", "For Avro format: regexp of String columns to select as AVRO string.", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 6e3e086859b..663b7f1ba95 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -130,6 +130,10 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.parquet.max_block_size = settings.input_format_parquet_max_block_size; format_settings.parquet.output_compression_method = settings.output_format_parquet_compression_method; format_settings.parquet.output_compliant_nested_types = settings.output_format_parquet_compliant_nested_types; + format_settings.parquet.use_custom_encoder = settings.output_format_parquet_use_custom_encoder; + format_settings.parquet.parallel_encoding = settings.output_format_parquet_parallel_encoding; + format_settings.parquet.data_page_size = settings.output_format_parquet_data_page_size; + format_settings.parquet.write_batch_size = settings.output_format_parquet_batch_size; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; @@ -434,7 +438,7 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( return format; } - return getOutputFormat(name, buf, sample, context, _format_settings); + return getOutputFormat(name, buf, sample, context, format_settings); } @@ -453,6 +457,7 @@ OutputFormatPtr FormatFactory::getOutputFormat( context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Format, name); auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); + format_settings.max_threads = context->getSettingsRef().max_threads; /** TODO: Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index e321e5264ca..3259c46e5ff 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -100,6 +100,8 @@ struct FormatSettings UInt64 max_parser_depth = DBMS_DEFAULT_MAX_PARSER_DEPTH; + size_t max_threads = 1; + enum class ArrowCompression { NONE, @@ -233,10 +235,14 @@ struct FormatSettings bool output_string_as_string = false; bool output_fixed_string_as_fixed_byte_array = true; bool preserve_order = false; + bool use_custom_encoder = true; + bool parallel_encoding = true; UInt64 max_block_size = 8192; ParquetVersion output_version; ParquetCompression output_compression_method = ParquetCompression::SNAPPY; bool output_compliant_nested_types = true; + size_t data_page_size = 1024 * 1024; + size_t write_batch_size = 1024; } parquet; struct Pretty diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index f688efa3290..e2383d1bfab 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -684,9 +684,6 @@ namespace DB bool output_fixed_string_as_fixed_byte_array, std::unordered_map & dictionary_values) { - const String column_type_name = column_type->getFamilyName(); - WhichDataType which(column_type); - switch (column_type->getTypeId()) { case TypeIndex::Nullable: @@ -796,7 +793,7 @@ namespace DB FOR_INTERNAL_NUMERIC_TYPES(DISPATCH) #undef DISPATCH default: - throw Exception(ErrorCodes::UNKNOWN_TYPE, "Internal type '{}' of a column '{}' is not supported for conversion into {} data format.", column_type_name, column_name, format_name); + throw Exception(ErrorCodes::UNKNOWN_TYPE, "Internal type '{}' of a column '{}' is not supported for conversion into {} data format.", column_type->getFamilyName(), column_name, format_name); } } diff --git a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp new file mode 100644 index 00000000000..a70b6fcfc81 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp @@ -0,0 +1,618 @@ +#include "Processors/Formats/Impl/Parquet/Write.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +/// This file deals with schema conversion and with repetition and definition levels. + +/// Schema conversion is pretty straightforward. + +/// "Repetition and definition levels" are a somewhat tricky way of encoding information about +/// optional fields and lists. +/// +/// If you don't want to learn how these work, feel free to skip the updateRepDefLevels* functions. +/// All you need to know is: +/// * values for nulls are not encoded, so we have to filter nullable columns, +/// * information about all array lengths and nulls is encoded in the arrays `def` and `rep`, +/// which need to be encoded next to the data, +/// * `def` and `rep` arrays can be longer than `primitive_column`, because they include nulls and +/// empty arrays; the values in primitive_column correspond to positions where def[i] == max_def. +/// +/// If you do want to learn it, dremel paper: https://research.google/pubs/pub36632/ +/// Instead of reading the whole paper, try staring at figures 2-3 for a while - it might be enough. +/// (Why does Parquet do all this instead of just storing array lengths and null masks? I'm not +/// really sure.) +/// +/// We calculate the levels recursively, from inner to outer columns. +/// This means scanning the whole array for each Array/Nullable nesting level, which is probably not +/// the most efficient way to do it. But there's usually at most one nesting level, so it's fine. +/// +/// Most of this is moot because ClickHouse doesn't support nullable arrays or tuples right now, so +/// almost none of the tricky cases can happen. We implement it in full generality anyway (mostly +/// because I only learned the previous sentence after writing most of the code). + + +namespace DB::ErrorCodes +{ + extern const int UNKNOWN_TYPE; + extern const int TOO_DEEP_RECURSION; // I'm 14 and this is deep + extern const int UNKNOWN_COMPRESSION_METHOD; + extern const int LOGICAL_ERROR; +} + +namespace DB::Parquet +{ + +/// Thrift structs that Parquet uses for various metadata inside the parquet file. +namespace parq = parquet::format; + +namespace +{ + +void assertNoDefOverflow(ColumnChunkWriteState & s) +{ + if (s.max_def == UINT8_MAX) + throw Exception(ErrorCodes::TOO_DEEP_RECURSION, + "Column has more than 255 levels of nested Array/Nullable. Impressive! Unfortunately, " + "this is not supported by this Parquet encoder (but is supported by Parquet, if you " + "really need this for some reason)."); +} + +void updateRepDefLevelsAndFilterColumnForNullable(ColumnChunkWriteState & s, const NullMap & null_map) +{ + /// Increment definition levels for non-nulls. + /// Filter the column to contain only non-null values. + + assertNoDefOverflow(s); + ++s.max_def; + + /// Normal case: no arrays or nullables inside this nullable. + if (s.max_def == 1) + { + chassert(s.def.empty()); + s.def.resize(null_map.size()); + for (size_t i = 0; i < s.def.size(); ++i) + s.def[i] = !null_map[i]; + + /// We could be more efficient with this: + /// * Instead of doing the filter() here, we could defer it to writeColumnChunkBody(), at + /// least in the simple case of Nullable(Primitive). Then it'll parallelize if the table + /// consists of one big tuple. + /// * Instead of filtering explicitly, we could build filtering into the data encoder. + /// * Instead of filling out the `def` values above, we could point to null_map and build + /// the '!' into the encoder. + /// None of these seem worth the complexity right now. + s.primitive_column = s.primitive_column->filter(s.def, /*result_size_hint*/ -1); + + return; + } + + /// Weird general case: Nullable(Array), Nullable(Nullable), or any arbitrary nesting like that. + /// This is currently not allowed in ClickHouse, but let's support it anyway just in case. + + IColumn::Filter filter; + size_t row_idx = static_cast(-1); + for (size_t i = 0; i < s.def.size(); ++i) + { + row_idx += s.max_rep == 0 || s.rep[i] == 0; + if (s.def[i] == s.max_def - 1) + filter.push_back(!null_map[row_idx]); + s.def[i] += !null_map[row_idx]; + } + s.primitive_column = s.primitive_column->filter(filter, /*result_size_hint*/ -1); +} + +void updateRepDefLevelsForArray(ColumnChunkWriteState & s, const IColumn::Offsets & offsets) +{ + /// Increment all definition levels. + /// For non-first elements of arrays, increment repetition levels. + /// For empty arrays, insert a zero into repetition and definition levels arrays. + + assertNoDefOverflow(s); + ++s.max_def; + ++s.max_rep; + + /// Common case: no arrays or nullables inside this array. + if (s.max_rep == 1 && s.max_def == 1) + { + s.def.resize_fill(s.primitive_column->size(), 1); + s.rep.resize_fill(s.primitive_column->size(), 1); + size_t i = 0; + for (ssize_t row = 0; row < static_cast(offsets.size()); ++row) + { + size_t n = offsets[row] - offsets[row - 1]; + if (n) + { + s.rep[i] = 0; + i += n; + } + else + { + s.def.push_back(1); + s.rep.push_back(1); + s.def[i] = 0; + s.rep[i] = 0; + i += 1; + } + } + return; + } + + /// General case: Array(Array), Array(Nullable), or any arbitrary nesting like that. + + for (auto & x : s.def) + ++x; + + if (s.max_rep == 1) + s.rep.resize_fill(s.def.size(), 1); + else + for (auto & x : s.rep) + ++x; + + PaddedPODArray mask(s.def.size(), 1); // for inserting zeroes to rep and def + size_t i = 0; // in the input (s.def/s.rep) + size_t empty_arrays = 0; + for (ssize_t row = 0; row < static_cast(offsets.size()); ++row) + { + size_t n = offsets[row] - offsets[row - 1]; + if (n) + { + /// Un-increment the first rep of the array. + /// Skip n "items" in the nested column; first element of each item has rep = 1 + /// (we incremented it above). + chassert(s.rep[i] == 1); + --s.rep[i]; + do + { + ++i; + if (i == s.rep.size()) + { + --n; + chassert(n == 0); + break; + } + n -= s.rep[i] == 1; + } while (n); + } + else + { + mask.push_back(1); + mask[i + empty_arrays] = 0; + ++empty_arrays; + } + } + + if (empty_arrays != 0) + { + expandDataByMask(s.def, mask, false); + expandDataByMask(s.rep, mask, false); + } +} + +parq::CompressionCodec::type compressionMethodToParquet(CompressionMethod c) +{ + switch (c) + { + case CompressionMethod::None: return parq::CompressionCodec::UNCOMPRESSED; + case CompressionMethod::Snappy: return parq::CompressionCodec::SNAPPY; + case CompressionMethod::Gzip: return parq::CompressionCodec::GZIP; + case CompressionMethod::Brotli: return parq::CompressionCodec::BROTLI; + case CompressionMethod::Lz4: return parq::CompressionCodec::LZ4_RAW; + case CompressionMethod::Zstd: return parq::CompressionCodec::ZSTD; + + default: + throw Exception(ErrorCodes::UNKNOWN_COMPRESSION_METHOD, "Compression method {} is not supported by Parquet", toContentEncodingName(c)); + } +} + +/// Depth-first traversal of the schema tree for this column. +void prepareColumnRecursive( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas); + +void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::string & name, + const WriteOptions & options, ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + /// Add physical column info. + auto & state = states.emplace_back(); + state.primitive_column = column; + state.compression = options.compression; + + state.column_chunk.__isset.meta_data = true; + state.column_chunk.meta_data.__set_path_in_schema({name}); + state.column_chunk.meta_data.__set_codec(compressionMethodToParquet(state.compression)); + + /// Add logical schema leaf. + auto & schema = schemas.emplace_back(); + schema.__set_repetition_type(parq::FieldRepetitionType::REQUIRED); + schema.__set_name(name); + + /// Convert the type enums. + + using T = parq::Type; + using C = parq::ConvertedType; + + auto types = [&](T::type type_, std::optional converted = std::nullopt, std::optional logical = std::nullopt) + { + state.column_chunk.meta_data.__set_type(type_); + schema.__set_type(type_); + if (converted) + schema.__set_converted_type(*converted); + if (logical) + schema.__set_logicalType(*logical); + }; + + auto int_type = [](Int8 bits, bool signed_) + { + parq::LogicalType t; + t.__isset.INTEGER = true; + t.INTEGER.__set_bitWidth(bits); + t.INTEGER.__set_isSigned(signed_); + return t; + }; + + auto fixed_string = [&](size_t size, std::optional converted = std::nullopt, std::optional logical = std::nullopt) + { + state.column_chunk.meta_data.__set_type(parq::Type::FIXED_LEN_BYTE_ARRAY); + schema.__set_type(parq::Type::FIXED_LEN_BYTE_ARRAY); + schema.__set_type_length(static_cast(size)); + if (converted) + schema.__set_converted_type(*converted); + if (logical) + schema.__set_logicalType(*logical); + }; + + auto decimal = [&](Int32 bytes, UInt32 precision, UInt32 scale) + { + state.column_chunk.meta_data.__set_type(parq::Type::FIXED_LEN_BYTE_ARRAY); + schema.__set_type(parq::Type::FIXED_LEN_BYTE_ARRAY); + schema.__set_type_length(bytes); + schema.__set_scale(static_cast(scale)); + schema.__set_precision(static_cast(precision)); + schema.__set_converted_type(parq::ConvertedType::DECIMAL); + parq::DecimalType d; + d.__set_scale(static_cast(scale)); + d.__set_precision(static_cast(precision)); + parq::LogicalType t; + t.__set_DECIMAL(d); + schema.__set_logicalType(t); + }; + + switch (type->getTypeId()) + { + case TypeIndex::UInt8: types(T::INT32, C::UINT_8 , int_type(8 , false)); break; + case TypeIndex::UInt16: types(T::INT32, C::UINT_16, int_type(16, false)); break; + case TypeIndex::UInt32: types(T::INT32, C::UINT_32, int_type(32, false)); break; + case TypeIndex::UInt64: types(T::INT64, C::UINT_64, int_type(64, false)); break; + case TypeIndex::Int8: types(T::INT32, C::INT_8 , int_type(8 , true)); break; + case TypeIndex::Int16: types(T::INT32, C::INT_16 , int_type(16, true)); break; + case TypeIndex::Int32: types(T::INT32); break; + case TypeIndex::Int64: types(T::INT64); break; + case TypeIndex::Float32: types(T::FLOAT); break; + case TypeIndex::Float64: types(T::DOUBLE); break; + + /// These don't have suitable parquet logical types, so we write them as plain numbers. + /// (Parquet has "enums" but they're just strings, with nowhere to declare all possible enum + /// values in advance as part of the data type.) + case TypeIndex::Enum8: types(T::INT32, C::INT_8 , int_type(8 , true)); break; // Int8 + case TypeIndex::Enum16: types(T::INT32, C::INT_16 , int_type(16, true)); break; // Int16 + case TypeIndex::IPv4: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32 + case TypeIndex::Date: types(T::INT32, C::UINT_16, int_type(16, false)); break; // UInt16 + case TypeIndex::DateTime: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32 + + case TypeIndex::Date32: + { + parq::LogicalType t; + t.__set_DATE({}); + types(T::INT32, C::DATE, t); + break; + } + + case TypeIndex::DateTime64: + { + std::optional converted; + std::optional unit; + switch (assert_cast(*type).getScale()) + { + case 3: + converted = parq::ConvertedType::TIMESTAMP_MILLIS; + unit.emplace().__set_MILLIS({}); + break; + case 6: + converted = parq::ConvertedType::TIMESTAMP_MICROS; + unit.emplace().__set_MICROS({}); + break; + case 9: + unit.emplace().__set_NANOS({}); + break; + } + + std::optional t; + if (unit) + { + parq::TimestampType tt; + tt.__set_isAdjustedToUTC(true); + tt.__set_unit(*unit); + t.emplace().__set_TIMESTAMP(tt); + } + types(T::INT64, converted, t); + break; + } + + case TypeIndex::String: + case TypeIndex::FixedString: + { + if (options.output_fixed_string_as_fixed_byte_array && + type->getTypeId() == TypeIndex::FixedString) + { + fixed_string(assert_cast(*type).getN()); + } + else if (options.output_string_as_string) + { + parq::LogicalType t; + t.__set_STRING({}); + types(T::BYTE_ARRAY, C::UTF8, t); + } + else + { + types(T::BYTE_ARRAY); + } + break; + } + + /// Parquet doesn't have logical types for these. + case TypeIndex::UInt128: fixed_string(16); break; + case TypeIndex::UInt256: fixed_string(32); break; + case TypeIndex::Int128: fixed_string(16); break; + case TypeIndex::Int256: fixed_string(32); break; + case TypeIndex::IPv6: fixed_string(16); break; + + case TypeIndex::Decimal32: decimal(4 , getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal64: decimal(8 , getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal128: decimal(16, getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal256: decimal(32, getDecimalPrecision(*type), getDecimalScale(*type)); break; + + default: + throw Exception(ErrorCodes::UNKNOWN_TYPE, "Internal type '{}' of column '{}' is not supported for conversion into Parquet data format.", type->getFamilyName(), name); + } +} + +void prepareColumnNullable( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + const ColumnNullable * column_nullable = assert_cast(column.get()); + ColumnPtr nested_column = column_nullable->getNestedColumnPtr(); + DataTypePtr nested_type = assert_cast(type.get())->getNestedType(); + const NullMap & null_map = column_nullable->getNullMapData(); + + size_t child_states_begin = states.size(); + size_t child_schema_idx = schemas.size(); + + prepareColumnRecursive(nested_column, nested_type, name, options, states, schemas); + + if (schemas[child_schema_idx].repetition_type == parq::FieldRepetitionType::REQUIRED) + { + /// Normal case: we just slap a FieldRepetitionType::OPTIONAL onto the nested column. + schemas[child_schema_idx].repetition_type = parq::FieldRepetitionType::OPTIONAL; + } + else + { + /// Weird case: Nullable(Nullable(...)). Or Nullable(Tuple(Nullable(...))), etc. + /// This is probably not allowed in ClickHouse, but let's support it just in case. + auto & schema = *schemas.insert(schemas.begin() + child_schema_idx, {}); + schema.__set_repetition_type(parq::FieldRepetitionType::OPTIONAL); + schema.__set_name("nullable"); + schema.__set_num_children(1); + for (size_t i = child_states_begin; i < states.size(); ++i) + { + Strings & path = states[i].column_chunk.meta_data.path_in_schema; + path.insert(path.begin(), schema.name + "."); + } + } + + for (size_t i = child_states_begin; i < states.size(); ++i) + { + auto & s = states[i]; + updateRepDefLevelsAndFilterColumnForNullable(s, null_map); + } +} + +void prepareColumnTuple( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + const auto * column_tuple = assert_cast(column.get()); + const auto * type_tuple = assert_cast(type.get()); + + auto & tuple_schema = schemas.emplace_back(); + tuple_schema.__set_repetition_type(parq::FieldRepetitionType::REQUIRED); + tuple_schema.__set_name(name); + tuple_schema.__set_num_children(static_cast(type_tuple->getElements().size())); + + size_t child_states_begin = states.size(); + + for (size_t i = 0; i < type_tuple->getElements().size(); ++i) + prepareColumnRecursive(column_tuple->getColumnPtr(i), type_tuple->getElement(i), type_tuple->getNameByPosition(i + 1), options, states, schemas); + + for (size_t i = child_states_begin; i < states.size(); ++i) + { + Strings & path = states[i].column_chunk.meta_data.path_in_schema; + /// O(nesting_depth^2), but who cares. + path.insert(path.begin(), name); + } +} + +void prepareColumnArray( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + const auto * column_array = assert_cast(column.get()); + ColumnPtr nested_column = column_array->getDataPtr(); + DataTypePtr nested_type = assert_cast(type.get())->getNestedType(); + const auto & offsets = column_array->getOffsets(); + + /// Schema for lists https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists + /// + /// required group `name` (List): + /// repeated group "list": + /// "element" + + /// Add the groups schema. + + schemas.emplace_back(); + schemas.emplace_back(); + auto & list_schema = schemas[schemas.size() - 2]; + auto & item_schema = schemas[schemas.size() - 1]; + + list_schema.__set_repetition_type(parq::FieldRepetitionType::REQUIRED); + list_schema.__set_name(name); + list_schema.__set_num_children(1); + list_schema.__set_converted_type(parq::ConvertedType::LIST); + list_schema.__isset.logicalType = true; + list_schema.logicalType.__set_LIST({}); + + item_schema.__set_repetition_type(parq::FieldRepetitionType::REPEATED); + item_schema.__set_name("list"); + item_schema.__set_num_children(1); + + std::array path_prefix = {list_schema.name, item_schema.name}; + size_t child_states_begin = states.size(); + + /// Recurse. + prepareColumnRecursive(nested_column, nested_type, "element", options, states, schemas); + + /// Update repetition+definition levels and fully-qualified column names (x -> myarray.list.x). + for (size_t i = child_states_begin; i < states.size(); ++i) + { + Strings & path = states[i].column_chunk.meta_data.path_in_schema; + path.insert(path.begin(), path_prefix.begin(), path_prefix.end()); + + updateRepDefLevelsForArray(states[i], offsets); + } +} + +void prepareColumnMap( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + const auto * column_map = assert_cast(column.get()); + const auto * column_array = &column_map->getNestedColumn(); + const auto & offsets = column_array->getOffsets(); + ColumnPtr column_tuple = column_array->getDataPtr(); + + const auto * map_type = assert_cast(type.get()); + DataTypePtr tuple_type = std::make_shared(map_type->getKeyValueTypes(), Strings{"key", "value"}); + + /// Map is an array of tuples + /// https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#maps + /// + /// required group `name` (Map): + /// repeated group "key_value": + /// reqiured <...> "key" + /// <...> "value" + + auto & map_schema = schemas.emplace_back(); + map_schema.__set_repetition_type(parq::FieldRepetitionType::REQUIRED); + map_schema.__set_name(name); + map_schema.__set_num_children(1); + map_schema.__set_converted_type(parq::ConvertedType::MAP); + map_schema.__set_logicalType({}); + map_schema.logicalType.__set_MAP({}); + + size_t tuple_schema_idx = schemas.size(); + size_t child_states_begin = states.size(); + + prepareColumnTuple(column_tuple, tuple_type, "key_value", options, states, schemas); + + schemas[tuple_schema_idx].__set_repetition_type(parq::FieldRepetitionType::REPEATED); + schemas[tuple_schema_idx].__set_converted_type(parq::ConvertedType::MAP_KEY_VALUE); + + for (size_t i = child_states_begin; i < states.size(); ++i) + { + Strings & path = states[i].column_chunk.meta_data.path_in_schema; + path.insert(path.begin(), name); + + updateRepDefLevelsForArray(states[i], offsets); + } +} + +void prepareColumnRecursive( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates & states, SchemaElements & schemas) +{ + switch (type->getTypeId()) + { + case TypeIndex::Nullable: prepareColumnNullable(column, type, name, options, states, schemas); break; + case TypeIndex::Array: prepareColumnArray(column, type, name, options, states, schemas); break; + case TypeIndex::Tuple: prepareColumnTuple(column, type, name, options, states, schemas); break; + case TypeIndex::Map: prepareColumnMap(column, type, name, options, states, schemas); break; + case TypeIndex::LowCardinality: + { + auto nested_type = assert_cast(*type).getDictionaryType(); + if (nested_type->isNullable()) + prepareColumnNullable( + column->convertToFullColumnIfLowCardinality(), nested_type, name, options, states, schemas); + else + /// Use nested data type, but keep ColumnLowCardinality. The encoder can deal with it. + preparePrimitiveColumn(column, nested_type, name, options, states, schemas); + break; + } + default: + preparePrimitiveColumn(column, type, name, options, states, schemas); + break; + } +} + +} + +SchemaElements convertSchema(const Block & sample, const WriteOptions & options) +{ + SchemaElements schema; + auto & root = schema.emplace_back(); + root.__set_name("schema"); + root.__set_num_children(static_cast(sample.columns())); + + for (auto & c : sample) + prepareColumnForWrite(c.column, c.type, c.name, options, nullptr, &schema); + + return schema; +} + +void prepareColumnForWrite( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates * out_columns_to_write, SchemaElements * out_schema) +{ + if (column->size() == 0 && out_columns_to_write != nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty column passed to Parquet encoder"); + + ColumnChunkWriteStates states; + SchemaElements schemas; + prepareColumnRecursive(column, type, name, options, states, schemas); + + if (out_columns_to_write) + for (auto & s : states) + out_columns_to_write->push_back(std::move(s)); + if (out_schema) + out_schema->insert(out_schema->end(), schemas.begin(), schemas.end()); + + if (column->empty()) + states.clear(); +} + +} diff --git a/src/Processors/Formats/Impl/Parquet/ThriftUtil.cpp b/src/Processors/Formats/Impl/Parquet/ThriftUtil.cpp new file mode 100644 index 00000000000..2a99b028ae0 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ThriftUtil.cpp @@ -0,0 +1,35 @@ +#include +#include + +namespace DB::Parquet +{ + +class WriteBufferTransport : public apache::thrift::transport::TTransport +{ +public: + WriteBuffer & out; + size_t bytes = 0; + + explicit WriteBufferTransport(WriteBuffer & out_) : out(out_) {} + + void write(const uint8_t* buf, uint32_t len) + { + out.write(reinterpret_cast(buf), len); + bytes += len; + } +}; + +template +size_t serializeThriftStruct(const T & obj, WriteBuffer & out) +{ + auto trans = std::make_shared(out); + auto proto = apache::thrift::protocol::TCompactProtocolFactoryT().getProtocol(trans); + obj.write(proto.get()); + return trans->bytes; +} + +template size_t serializeThriftStruct(const parquet::format::PageHeader &, WriteBuffer & out); +template size_t serializeThriftStruct(const parquet::format::ColumnChunk &, WriteBuffer & out); +template size_t serializeThriftStruct(const parquet::format::FileMetaData &, WriteBuffer & out); + +} diff --git a/src/Processors/Formats/Impl/Parquet/ThriftUtil.h b/src/Processors/Formats/Impl/Parquet/ThriftUtil.h new file mode 100644 index 00000000000..1efbe0002d4 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ThriftUtil.h @@ -0,0 +1,17 @@ +#pragma once + +#include // in contrib/arrow/cpp/src/ , generated from parquet.thrift +#include + +namespace DB::Parquet +{ + +/// Returns number of bytes written. +template +size_t serializeThriftStruct(const T & obj, WriteBuffer & out); + +extern template size_t serializeThriftStruct(const parquet::format::PageHeader &, WriteBuffer & out); +extern template size_t serializeThriftStruct(const parquet::format::ColumnChunk &, WriteBuffer & out); +extern template size_t serializeThriftStruct(const parquet::format::FileMetaData &, WriteBuffer & out); + +} diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp new file mode 100644 index 00000000000..a29bb81f8dc --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -0,0 +1,816 @@ +#include "Processors/Formats/Impl/Parquet/Write.h" +#include "Processors/Formats/Impl/Parquet/ThriftUtil.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "config_version.h" + +namespace DB::ErrorCodes +{ + extern const int CANNOT_COMPRESS; + extern const int LIMIT_EXCEEDED; + extern const int LOGICAL_ERROR; +} + +namespace DB::Parquet +{ + +namespace parq = parquet::format; + +namespace +{ + +template +struct StatisticsNumeric +{ + T min = std::numeric_limits::max(); + T max = std::numeric_limits::min(); + + void add(SourceType x) + { + min = std::min(min, static_cast(x)); + max = std::max(max, static_cast(x)); + } + + void merge(const StatisticsNumeric & s) + { + min = std::min(min, s.min); + max = std::max(max, s.max); + } + + void clear() { *this = {}; } + + parq::Statistics get(const WriteOptions &) + { + parq::Statistics s; + s.__isset.min_value = s.__isset.max_value = true; + s.min_value.resize(sizeof(T)); + s.max_value.resize(sizeof(T)); + memcpy(s.min_value.data(), &min, sizeof(T)); + memcpy(s.max_value.data(), &max, sizeof(T)); + + if constexpr (std::is_signed::value) + { + s.__set_min(s.min_value); + s.__set_max(s.max_value); + } + return s; + } +}; + +struct StatisticsFixedString +{ + size_t fixed_string_size = UINT64_MAX; + const uint8_t * min = nullptr; + const uint8_t * max = nullptr; + + void add(parquet::FixedLenByteArray a) + { + chassert(fixed_string_size != UINT64_MAX); + addMin(a.ptr); + addMax(a.ptr); + } + + void merge(const StatisticsFixedString & s) + { + chassert(fixed_string_size == UINT64_MAX || fixed_string_size == s.fixed_string_size); + fixed_string_size = s.fixed_string_size; + if (s.min == nullptr) + return; + addMin(s.min); + addMax(s.max); + } + + void clear() { min = max = nullptr; } + + parq::Statistics get(const WriteOptions & options) + { + parq::Statistics s; + if (min == nullptr || fixed_string_size > options.max_statistics_size) + return s; + s.__set_min_value(std::string(reinterpret_cast(min), fixed_string_size)); + s.__set_max_value(std::string(reinterpret_cast(max), fixed_string_size)); + return s; + } + + void addMin(const uint8_t * p) + { + if (min == nullptr || memcmp(p, min, fixed_string_size) < 0) + min = p; + } + void addMax(const uint8_t * p) + { + if (max == nullptr || memcmp(p, max, fixed_string_size) > 0) + max = p; + } +}; + +struct StatisticsString +{ + parquet::ByteArray min; + parquet::ByteArray max; + + void add(parquet::ByteArray x) + { + addMin(x); + addMax(x); + } + + void merge(const StatisticsString & s) + { + if (s.min.ptr == nullptr) + return; + addMin(s.min); + addMax(s.max); + } + + void clear() { *this = {}; } + + parq::Statistics get(const WriteOptions & options) + { + parq::Statistics s; + if (min.ptr == nullptr) + return s; + if (static_cast(min.len) <= options.max_statistics_size) + s.__set_min_value(std::string(reinterpret_cast(min.ptr), static_cast(min.len))); + if (static_cast(max.len) <= options.max_statistics_size) + s.__set_max_value(std::string(reinterpret_cast(max.ptr), static_cast(max.len))); + return s; + } + + void addMin(parquet::ByteArray x) + { + if (min.ptr == nullptr || compare(x, min) < 0) + min = x; + } + + void addMax(parquet::ByteArray x) + { + if (max.ptr == nullptr || compare(x, max) > 0) + max = x; + } + + static int compare(parquet::ByteArray a, parquet::ByteArray b) + { + int t = memcmp(a.ptr, b.ptr, std::min(a.len, b.len)); + if (t != 0) + return t; + return a.len - b.len; + } +}; + +/// The column usually needs to be converted to one of Parquet physical types, e.g. UInt16 -> Int32 +/// or [element of ColumnString] -> std::string_view. +/// We do this conversion in small batches rather than all at once, just before encoding the batch, +/// in hopes of getting better performance through cache locality. +/// The Coverter* structs below are responsible for that. +/// When conversion is not needed, getBatch() will just return pointer into original data. + +template ::value, + To, + typename std::make_unsigned::type>::type> +struct ConverterNumeric +{ + using Statistics = StatisticsNumeric; + + const Col & column; + PODArray buf; + + explicit ConverterNumeric(const ColumnPtr & c) : column(assert_cast(*c)) {} + + const To * getBatch(size_t offset, size_t count) + { + if constexpr (sizeof(*column.getData().data()) == sizeof(To)) + return reinterpret_cast(column.getData().data() + offset); + else + { + buf.resize(count); + for (size_t i = 0; i < count; ++i) + buf[i] = static_cast(column.getData()[offset + i]); + return buf.data(); + } + } +}; + +struct ConverterString +{ + using Statistics = StatisticsString; + + const ColumnString & column; + PODArray buf; + + explicit ConverterString(const ColumnPtr & c) : column(assert_cast(*c)) {} + + const parquet::ByteArray * getBatch(size_t offset, size_t count) + { + buf.resize(count); + for (size_t i = 0; i < count; ++i) + { + StringRef s = column.getDataAt(offset + i); + buf[i] = parquet::ByteArray(static_cast(s.size), reinterpret_cast(s.data)); + } + return buf.data(); + } +}; + +struct ConverterFixedString +{ + using Statistics = StatisticsFixedString; + + const ColumnFixedString & column; + PODArray buf; + + explicit ConverterFixedString(const ColumnPtr & c) : column(assert_cast(*c)) {} + + const parquet::FixedLenByteArray * getBatch(size_t offset, size_t count) + { + buf.resize(count); + for (size_t i = 0; i < count; ++i) + buf[i].ptr = reinterpret_cast(column.getChars().data() + (offset + i) * column.getN()); + return buf.data(); + } + + size_t fixedStringSize() { return column.getN(); } +}; + +struct ConverterFixedStringAsString +{ + using Statistics = StatisticsString; + + const ColumnFixedString & column; + PODArray buf; + + explicit ConverterFixedStringAsString(const ColumnPtr & c) : column(assert_cast(*c)) {} + + const parquet::ByteArray * getBatch(size_t offset, size_t count) + { + buf.resize(count); + for (size_t i = 0; i < count; ++i) + buf[i] = parquet::ByteArray(static_cast(column.getN()), reinterpret_cast(column.getChars().data() + (offset + i) * column.getN())); + return buf.data(); + } +}; + +template +struct ConverterNumberAsFixedString +{ + /// Calculate min/max statistics for little-endian fixed strings, not numbers, because parquet + /// doesn't know it's numbers. + using Statistics = StatisticsFixedString; + + const ColumnVector & column; + PODArray buf; + + explicit ConverterNumberAsFixedString(const ColumnPtr & c) : column(assert_cast &>(*c)) {} + + const parquet::FixedLenByteArray * getBatch(size_t offset, size_t count) + { + buf.resize(count); + for (size_t i = 0; i < count; ++i) + buf[i].ptr = reinterpret_cast(column.getData().data() + offset + i); + return buf.data(); + } + + size_t fixedStringSize() { return sizeof(T); } +}; + +/// Like ConverterNumberAsFixedString, but converts to big-endian. Because that's the byte order +/// Parquet uses for decimal types and literally nothing else, for some reason. +template +struct ConverterDecimal +{ + using Statistics = StatisticsFixedString; + + const ColumnDecimal & column; + PODArray data_buf; + PODArray ptr_buf; + + explicit ConverterDecimal(const ColumnPtr & c) : column(assert_cast &>(*c)) {} + + const parquet::FixedLenByteArray * getBatch(size_t offset, size_t count) + { + data_buf.resize(count * sizeof(T)); + ptr_buf.resize(count); + memcpy(data_buf.data(), reinterpret_cast(column.getData().data() + offset), count * sizeof(T)); + for (size_t i = 0; i < count; ++i) + { + std::reverse(data_buf.data() + i * sizeof(T), data_buf.data() + (i + 1) * sizeof(T)); + ptr_buf[i].ptr = data_buf.data() + i * sizeof(T); + } + return ptr_buf.data(); + } + + size_t fixedStringSize() { return sizeof(T); } +}; + +/// Returns either `source` or `scratch`. +PODArray & compress(PODArray & source, PODArray & scratch, CompressionMethod method) +{ + /// We could use wrapWriteBufferWithCompressionMethod() for everything, but I worry about the + /// overhead of creating a bunch of WriteBuffers on each page (thousands of values). + switch (method) + { + case CompressionMethod::None: + return source; + + case CompressionMethod::Lz4: + { + #pragma clang diagnostic push + #pragma clang diagnostic ignored "-Wold-style-cast" + + size_t max_dest_size = LZ4_COMPRESSBOUND(source.size()); + + #pragma clang diagnostic pop + + if (max_dest_size > std::numeric_limits::max()) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress column of size {}", formatReadableSizeWithBinarySuffix(source.size())); + + scratch.resize(max_dest_size); + + int compressed_size = LZ4_compress_default( + source.data(), + scratch.data(), + static_cast(source.size()), + static_cast(max_dest_size)); + + scratch.resize(static_cast(compressed_size)); + return scratch; + } + + default: + { + auto dest_buf = std::make_unique>>(scratch); + auto compressed_buf = wrapWriteBufferWithCompressionMethod( + std::move(dest_buf), + method, + /*level*/ 3, + source.size(), + /*existing_memory*/ source.data()); + chassert(compressed_buf->position() == source.data()); + chassert(compressed_buf->available() == source.size()); + compressed_buf->position() += source.size(); + compressed_buf->finalize(); + return scratch; + } + } +} + +void encodeRepDefLevelsRLE(const UInt8 * data, size_t size, UInt8 max_level, PODArray & out) +{ + using arrow::util::RleEncoder; + + chassert(max_level > 0); + size_t offset = out.size(); + size_t prefix_size = sizeof(Int32); + + int bit_width = bitScanReverse(max_level) + 1; + int max_rle_size = RleEncoder::MaxBufferSize(bit_width, static_cast(size)) + + RleEncoder::MinBufferSize(bit_width); + + out.resize(offset + prefix_size + max_rle_size); + + RleEncoder encoder(reinterpret_cast(out.data() + offset + prefix_size), max_rle_size, bit_width); + for (size_t i = 0; i < size; ++i) + encoder.Put(data[i]); + encoder.Flush(); + Int32 len = encoder.len(); + + memcpy(out.data() + offset, &len, prefix_size); + out.resize(offset + prefix_size + len); +} + +void addToEncodingsUsed(ColumnChunkWriteState & s, parq::Encoding::type e) +{ + if (!std::count(s.column_chunk.meta_data.encodings.begin(), s.column_chunk.meta_data.encodings.end(), e)) + s.column_chunk.meta_data.encodings.push_back(e); +} + +void writePage(const parq::PageHeader & header, const PODArray & compressed, ColumnChunkWriteState & s, WriteBuffer & out) +{ + size_t header_size = serializeThriftStruct(header, out); + out.write(compressed.data(), compressed.size()); + + /// Remember first data page and first dictionary page. + if (header.__isset.data_page_header && s.column_chunk.meta_data.data_page_offset == -1) + s.column_chunk.meta_data.__set_data_page_offset(s.column_chunk.meta_data.total_compressed_size); + if (header.__isset.dictionary_page_header && !s.column_chunk.meta_data.__isset.dictionary_page_offset) + s.column_chunk.meta_data.__set_dictionary_page_offset(s.column_chunk.meta_data.total_compressed_size); + + s.column_chunk.meta_data.total_uncompressed_size += header.uncompressed_page_size + header_size; + s.column_chunk.meta_data.total_compressed_size += header.compressed_page_size + header_size; +} + +template +void writeColumnImpl( + ColumnChunkWriteState & s, const WriteOptions & options, WriteBuffer & out, Converter && converter) +{ + size_t num_values = s.max_def > 0 ? s.def.size() : s.primitive_column->size(); + auto encoding = options.encoding; + + typename Converter::Statistics page_statistics; + typename Converter::Statistics total_statistics; + + /// We start with dictionary encoding, then switch to `encoding` (non-dictionary) if the + /// dictionary gets too big. That's how arrow does it too. + bool initially_used_dictionary = options.use_dictionary_encoding; + bool currently_using_dictionary = initially_used_dictionary; + + std::optional fixed_string_descr; + if constexpr (std::is_same::value) + { + /// This just communicates one number to MakeTypedEncoder(): the fixed string length. + fixed_string_descr.emplace(parquet::schema::PrimitiveNode::Make( + "", parquet::Repetition::REQUIRED, parquet::Type::FIXED_LEN_BYTE_ARRAY, + parquet::ConvertedType::NONE, static_cast(converter.fixedStringSize())), 0, 0); + + page_statistics.fixed_string_size = converter.fixedStringSize(); + } + + /// Could use an arena here (by passing a custom MemoryPool), to reuse memory across pages. + /// Alternatively, we could avoid using arrow's dictionary encoding code and leverage + /// ColumnLowCardinality instead. It would work basically the same way as what this function + /// currently does: add values to the ColumnRowCardinality (instead of `encoder`) in batches, + /// checking dictionary size after each batch; if it gets big, flush the dictionary and the + /// indices and switch to non-dictionary encoding. Feels like it could even be slightly less code. + auto encoder = parquet::MakeTypedEncoder( + // ignored if using dictionary + static_cast(encoding), + currently_using_dictionary, fixed_string_descr ? &*fixed_string_descr : nullptr); + + struct PageData + { + parq::PageHeader header; + PODArray data; + }; + std::vector dict_encoded_pages; // can't write them out until we have full dictionary + + /// Reused across pages to reduce number of allocations and improve locality. + PODArray encoded; + PODArray compressed_maybe; + + /// Start of current page. + size_t def_offset = 0; // index in def and rep + size_t data_offset = 0; // index in primitive_column + + auto flush_page = [&](size_t def_count, size_t data_count) + { + encoded.clear(); + + /// Concatenate encoded rep, def, and data. + + if (s.max_rep > 0) + encodeRepDefLevelsRLE(s.rep.data() + def_offset, def_count, s.max_rep, encoded); + if (s.max_def > 0) + encodeRepDefLevelsRLE(s.def.data() + def_offset, def_count, s.max_def, encoded); + + std::shared_ptr values = encoder->FlushValues(); // resets it for next page + + encoded.resize(encoded.size() + values->size()); + memcpy(encoded.data() + encoded.size() - values->size(), values->data(), values->size()); + values.reset(); + + if (encoded.size() > INT32_MAX) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Uncompressed page is too big: {}", encoded.size()); + + size_t uncompressed_size = encoded.size(); + auto & compressed = compress(encoded, compressed_maybe, s.compression); + + if (compressed.size() > INT32_MAX) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Compressed page is too big: {}", compressed.size()); + + parq::PageHeader header; + header.__set_type(parq::PageType::DATA_PAGE); + header.__set_uncompressed_page_size(static_cast(uncompressed_size)); + header.__set_compressed_page_size(static_cast(compressed.size())); + header.__isset.data_page_header = true; + auto & d = header.data_page_header; + d.__set_num_values(static_cast(def_count)); + d.__set_encoding(currently_using_dictionary ? parq::Encoding::RLE_DICTIONARY : encoding); + d.__set_definition_level_encoding(parq::Encoding::RLE); + d.__set_repetition_level_encoding(parq::Encoding::RLE); + /// We could also put checksum in `header.crc`, but apparently no one uses it: + /// https://issues.apache.org/jira/browse/PARQUET-594 + + if (options.write_page_statistics) + { + d.__set_statistics(page_statistics.get(options)); + + if (s.max_def == 1 && s.max_rep == 0) + d.statistics.__set_null_count(static_cast(def_count - data_count)); + } + + total_statistics.merge(page_statistics); + page_statistics.clear(); + + if (currently_using_dictionary) + { + dict_encoded_pages.push_back({.header = std::move(header)}); + std::swap(dict_encoded_pages.back().data, compressed); + } + else + { + writePage(header, compressed, s, out); + } + + def_offset += def_count; + data_offset += data_count; + }; + + auto flush_dict = [&] -> bool + { + auto * dict_encoder = dynamic_cast *>(encoder.get()); + int dict_size = dict_encoder->dict_encoded_size(); + + encoded.resize(static_cast(dict_size)); + dict_encoder->WriteDict(reinterpret_cast(encoded.data())); + + auto & compressed = compress(encoded, compressed_maybe, s.compression); + + if (compressed.size() > INT32_MAX) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Compressed dictionary page is too big: {}", compressed.size()); + + parq::PageHeader header; + header.__set_type(parq::PageType::DICTIONARY_PAGE); + header.__set_uncompressed_page_size(dict_size); + header.__set_compressed_page_size(static_cast(compressed.size())); + header.__isset.dictionary_page_header = true; + header.dictionary_page_header.__set_num_values(dict_encoder->num_entries()); + header.dictionary_page_header.__set_encoding(parq::Encoding::PLAIN); + + writePage(header, compressed, s, out); + + for (auto & p : dict_encoded_pages) + writePage(p.header, p.data, s, out); + + dict_encoded_pages.clear(); + encoder.reset(); + + return true; + }; + + auto is_dict_too_big = [&] { + auto * dict_encoder = dynamic_cast *>(encoder.get()); + int dict_size = dict_encoder->dict_encoded_size(); + return static_cast(dict_size) >= options.dictionary_size_limit; + }; + + while (def_offset < num_values) + { + /// Pick enough data for a page. + size_t next_def_offset = def_offset; + size_t next_data_offset = data_offset; + while (true) + { + /// Bite off a batch of defs and corresponding data values. + size_t def_count = std::min(options.write_batch_size, num_values - next_def_offset); + size_t data_count = 0; + if (s.max_def == 0) + data_count = def_count; + else + for (size_t i = 0; i < def_count; ++i) + data_count += s.def[next_def_offset + i] == s.max_def; + + /// Encode the data (but not the levels yet), so that we can estimate its encoded size. + const typename ParquetDType::c_type * converted = converter.getBatch(next_data_offset, data_count); + + if (options.write_page_statistics || options.write_column_chunk_statistics) + for (size_t i = 0; i < data_count; ++i) + page_statistics.add(converted[i]); + + encoder->Put(converted, static_cast(data_count)); + + next_def_offset += def_count; + next_data_offset += data_count; + + if (currently_using_dictionary && is_dict_too_big()) + { + /// Fallback to non-dictionary encoding. + flush_page(next_def_offset - def_offset, next_data_offset - data_offset); + flush_dict(); + + currently_using_dictionary = false; + encoder = parquet::MakeTypedEncoder( + static_cast(encoding)); + break; + } + + if (next_def_offset == num_values || + static_cast(encoder->EstimatedDataEncodedSize()) >= options.data_page_size) + { + flush_page(next_def_offset - def_offset, next_data_offset - data_offset); + break; + } + } + } + + if (currently_using_dictionary) + flush_dict(); + + chassert(data_offset == s.primitive_column->size()); + + if (options.write_column_chunk_statistics) + { + s.column_chunk.meta_data.__set_statistics(total_statistics.get(options)); + + if (s.max_def == 1 && s.max_rep == 0) + s.column_chunk.meta_data.statistics.__set_null_count(static_cast(def_offset - data_offset)); + } + + /// Report which encodings we've used. + if (s.max_rep > 0 || s.max_def > 0) + addToEncodingsUsed(s, parq::Encoding::RLE); // levels + if (!currently_using_dictionary) + addToEncodingsUsed(s, encoding); // non-dictionary encoding + if (initially_used_dictionary) + { + addToEncodingsUsed(s, parq::Encoding::PLAIN); // dictionary itself + addToEncodingsUsed(s, parq::Encoding::RLE_DICTIONARY); // ids + } +} + +} + +void writeColumnChunkBody(ColumnChunkWriteState & s, const WriteOptions & options, WriteBuffer & out) +{ + s.column_chunk.meta_data.__set_num_values(s.max_def > 0 ? s.def.size() : s.primitive_column->size()); + + /// We'll be updating these as we go. + s.column_chunk.meta_data.__set_encodings({}); + s.column_chunk.meta_data.__set_total_compressed_size(0); + s.column_chunk.meta_data.__set_total_uncompressed_size(0); + s.column_chunk.meta_data.__set_data_page_offset(-1); + + s.primitive_column = s.primitive_column->convertToFullColumnIfLowCardinality(); + + switch (s.primitive_column->getDataType()) + { + /// Numeric conversion to Int32 or Int64. + #define N(source_type, parquet_dtype) \ + writeColumnImpl(s, options, out, \ + ConverterNumeric, parquet::parquet_dtype::c_type>( \ + s.primitive_column)) + + case TypeIndex::UInt8 : N(UInt8 , Int32Type); break; + case TypeIndex::UInt16 : N(UInt16, Int32Type); break; + case TypeIndex::UInt32 : N(UInt32, Int32Type); break; + case TypeIndex::UInt64 : N(UInt64, Int64Type); break; + case TypeIndex::Int8 : N(Int8 , Int32Type); break; + case TypeIndex::Int16 : N(Int16 , Int32Type); break; + case TypeIndex::Int32 : N(Int32 , Int32Type); break; + case TypeIndex::Int64 : N(Int64 , Int64Type); break; + + case TypeIndex::Enum8: N(Int8 , Int32Type); break; + case TypeIndex::Enum16: N(Int16 , Int32Type); break; + case TypeIndex::Date: N(UInt16, Int32Type); break; + case TypeIndex::Date32: N(Int32 , Int32Type); break; + case TypeIndex::DateTime: N(UInt32, Int32Type); break; + + #undef N + + case TypeIndex::Float32: + writeColumnImpl( + s, options, out, ConverterNumeric, Float32, Float32>( + s.primitive_column)); + break; + + case TypeIndex::Float64: + writeColumnImpl( + s, options, out, ConverterNumeric, Float64, Float64>( + s.primitive_column)); + break; + + case TypeIndex::DateTime64: + writeColumnImpl( + s, options, out, ConverterNumeric, Int64, Int64>( + s.primitive_column)); + break; + + case TypeIndex::IPv4: + writeColumnImpl( + s, options, out, ConverterNumeric, Int32, UInt32>( + s.primitive_column)); + break; + + case TypeIndex::String: + writeColumnImpl( + s, options, out, ConverterString(s.primitive_column)); + break; + + case TypeIndex::FixedString: + if (options.output_fixed_string_as_fixed_byte_array) + writeColumnImpl( + s, options, out, ConverterFixedString(s.primitive_column)); + else + writeColumnImpl( + s, options, out, ConverterFixedStringAsString(s.primitive_column)); + break; + + #define F(source_type) \ + writeColumnImpl( \ + s, options, out, ConverterNumberAsFixedString(s.primitive_column)) + case TypeIndex::UInt128: F(UInt128); break; + case TypeIndex::UInt256: F(UInt256); break; + case TypeIndex::Int128: F(Int128); break; + case TypeIndex::Int256: F(Int256); break; + case TypeIndex::IPv6: F(IPv6); break; + #undef F + + #define D(source_type) \ + writeColumnImpl( \ + s, options, out, ConverterDecimal(s.primitive_column)) + case TypeIndex::Decimal32: D(Decimal32); break; + case TypeIndex::Decimal64: D(Decimal64); break; + case TypeIndex::Decimal128: D(Decimal128); break; + case TypeIndex::Decimal256: D(Decimal256); break; + #undef D + + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected column type: {}", s.primitive_column->getFamilyName()); + } + + /// Free some memory. + s.primitive_column = {}; + s.def = {}; + s.rep = {}; +} + +void writeFileHeader(WriteBuffer & out) +{ + /// Write the magic bytes. We're a wizard now. + out.write("PAR1", 4); +} + +parq::ColumnChunk finalizeColumnChunkAndWriteFooter( + size_t offset_in_file, ColumnChunkWriteState s, const WriteOptions &, WriteBuffer & out) +{ + if (s.column_chunk.meta_data.data_page_offset != -1) + s.column_chunk.meta_data.data_page_offset += offset_in_file; + if (s.column_chunk.meta_data.__isset.dictionary_page_offset) + s.column_chunk.meta_data.dictionary_page_offset += offset_in_file; + s.column_chunk.file_offset = offset_in_file + s.column_chunk.meta_data.total_compressed_size; + + serializeThriftStruct(s.column_chunk, out); + + return std::move(s.column_chunk); +} + +parq::RowGroup makeRowGroup(std::vector column_chunks, size_t num_rows) +{ + parq::RowGroup r; + r.__set_num_rows(num_rows); + r.__set_columns(std::move(column_chunks)); + r.__set_total_compressed_size(0); + for (auto & c : r.columns) + { + r.total_byte_size += c.meta_data.total_uncompressed_size; + r.total_compressed_size += c.meta_data.total_compressed_size; + } + if (!r.columns.empty()) + { + auto & m = r.columns[0].meta_data; + r.__set_file_offset(m.__isset.dictionary_page_offset ? m.dictionary_page_offset : m.data_page_offset); + } + return r; +} + +void writeFileFooter(std::vector row_groups, SchemaElements schema, const WriteOptions & options, WriteBuffer & out) +{ + parq::FileMetaData meta; + meta.version = 2; + meta.schema = std::move(schema); + meta.row_groups = std::move(row_groups); + for (auto & r : meta.row_groups) + meta.num_rows += r.num_rows; + meta.__set_created_by(VERSION_NAME " " VERSION_DESCRIBE); + + if (options.write_page_statistics || options.write_column_chunk_statistics) + { + meta.__set_column_orders({}); + for (auto & s : meta.schema) + if (!s.__isset.num_children) + meta.column_orders.emplace_back(); + for (auto & c : meta.column_orders) + c.__set_TYPE_ORDER({}); + } + + size_t footer_size = serializeThriftStruct(meta, out); + + if (footer_size > INT32_MAX) + throw Exception(ErrorCodes::LIMIT_EXCEEDED, "Parquet file metadata too big: {}", footer_size); + + writeIntBinary(static_cast(footer_size), out); + out.write("PAR1", 4); +} + +} diff --git a/src/Processors/Formats/Impl/Parquet/Write.h b/src/Processors/Formats/Impl/Parquet/Write.h new file mode 100644 index 00000000000..333a32e191f --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/Write.h @@ -0,0 +1,135 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB::Parquet +{ + +/// A good resource for learning how Parquet format works is +/// contrib/arrow/cpp/src/parquet/parquet.thrift + +struct WriteOptions +{ + bool output_string_as_string = false; + bool output_fixed_string_as_fixed_byte_array = true; + + CompressionMethod compression = CompressionMethod::Lz4; + + size_t data_page_size = 1024 * 1024; + size_t write_batch_size = 1024; + + bool use_dictionary_encoding = true; + size_t dictionary_size_limit = 1024 * 1024; + /// If using dictionary, this encoding is used as a fallback when dictionary gets too big. + /// Otherwise, this is used for everything. + parquet::format::Encoding::type encoding = parquet::format::Encoding::PLAIN; + + bool write_page_statistics = true; + bool write_column_chunk_statistics = true; + size_t max_statistics_size = 4096; +}; + +/// Information about a primitive column (leaf of the schema tree) to write to Parquet file. +struct ColumnChunkWriteState +{ + /// After writeColumnChunkBody(), offsets in this struct are relative to the start of column chunk. + /// Then finalizeColumnChunkAndWriteFooter() fixes them up before writing to file. + parquet::format::ColumnChunk column_chunk; + + ColumnPtr primitive_column; + CompressionMethod compression; // must match what's inside column_chunk + + /// Repetition and definition levels. Produced by prepareColumnForWrite(). + /// def is empty iff max_def == 0, which means no arrays or nullables. + /// rep is empty iff max_rep == 0, which means no arrays. + PaddedPODArray def; // definition levels + PaddedPODArray rep; // repetition levels + /// Max possible levels, according to schema. Actual max in def/rep may be smaller. + UInt8 max_def = 0; + UInt8 max_rep = 0; + + ColumnChunkWriteState() = default; + /// Prevent accidental copying. + ColumnChunkWriteState(ColumnChunkWriteState &&) = default; + ColumnChunkWriteState & operator=(ColumnChunkWriteState &&) = default; + + /// Estimated memory usage. + size_t allocatedBytes() const + { + size_t r = def.allocated_bytes() + rep.allocated_bytes(); + if (primitive_column) + r += primitive_column->allocatedBytes(); + return r; + } +}; + +using SchemaElements = std::vector; +using ColumnChunkWriteStates = std::vector; + +/// Parquet file consists of row groups, which consist of column chunks. +/// +/// Column chunks can be encoded mostly independently of each other, in parallel. +/// But there are two small complications: +/// 1. One ClickHouse column can translate to multiple leaf columns in parquet. +/// E.g. tuples and maps. +/// If all primitive columns are in one big tuple, we'd like to encode them in parallel too, +/// even though they're one top-level ClickHouse column. +/// 2. At the end of each encoded column chunk there's a footer (struct ColumnMetaData) that +/// contains some absolute offsets in the file. We can't encode it until we know the exact +/// position in the file where the column chunk will go. So these footers have to be serialized +/// sequentially, after we know sizes of all previous column chunks. +/// +/// With that in mind, here's how to write a parquet file: +/// +/// (1) writeFileHeader() +/// (2) For each row group: +/// | (3) For each ClickHouse column: +/// | (4) Call prepareColumnForWrite(). +/// | It'll produce one or more ColumnChunkWriteStates, corresponding to primitive columns that +/// | we need to write. +/// | It'll also produce SchemaElements as a byproduct, describing the logical types and +/// | groupings of the physical columns (e.g. tuples, arrays, maps). +/// | (5) For each ColumnChunkWriteState: +/// | (6) Call writeColumnChunkBody() to write the actual data to the given WriteBuffer. +/// | (7) Call finalizeColumnChunkAndWriteFooter() to write the footer of the column chunk. +/// | (8) Call makeRowGroup() using the ColumnChunk metadata structs from previous step. +/// (9) Call writeFileFooter() using the row groups from previous step and SchemaElements from +/// convertSchema(). +/// +/// Steps (4) and (6) can be parallelized, both within and across row groups. + +/// Parquet schema is a tree of SchemaElements, flattened into a list in depth-first order. +/// Leaf nodes correspond to physical columns of primitive types. Inner nodes describe logical +/// groupings of those columns, e.g. tuples or structs. +SchemaElements convertSchema(const Block & sample, const WriteOptions & options); + +void prepareColumnForWrite( + ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, + ColumnChunkWriteStates * out_columns_to_write, SchemaElements * out_schema = nullptr); + +void writeFileHeader(WriteBuffer & out); + +/// Encodes a column chunk, without the footer. +/// The ColumnChunkWriteState-s should then passed to finalizeColumnChunkAndWriteFooter(). +void writeColumnChunkBody(ColumnChunkWriteState & s, const WriteOptions & options, WriteBuffer & out); + +/// Unlike most of the column chunk data, the footer (`ColumnMetaData`) needs to know its absolute +/// offset in the file. So we encode it separately, after all previous row groups and column chunks +/// have been encoded. +/// (If you're wondering if the 8-byte offset values can be patched inside the encoded blob - no, +/// they're varint-encoded and can't be padded to a fixed length.) +/// `offset_in_file` is the absolute position in the file where the writeColumnChunkBody()'s output +/// starts. +/// Returns a ColumnChunk to add to the RowGroup. +parquet::format::ColumnChunk finalizeColumnChunkAndWriteFooter( + size_t offset_in_file, ColumnChunkWriteState s, const WriteOptions & options, WriteBuffer & out); + +parquet::format::RowGroup makeRowGroup(std::vector column_chunks, size_t num_rows); + +void writeFileFooter(std::vector row_groups, SchemaElements schema, const WriteOptions & options, WriteBuffer & out); + +} diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 91840cd2c50..9a2d9072860 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -3,14 +3,23 @@ #if USE_PARQUET #include +#include #include #include "ArrowBufferedStreams.h" #include "CHColumnToArrowColumn.h" +namespace CurrentMetrics +{ + extern const Metric ParquetEncoderThreads; + extern const Metric ParquetEncoderThreadsActive; +} + namespace DB { +using namespace Parquet; + namespace ErrorCodes { extern const int UNKNOWN_EXCEPTION; @@ -67,11 +76,219 @@ namespace ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) : IOutputFormat(header_, out_), format_settings{format_settings_} { + if (format_settings.parquet.use_custom_encoder) + { + if (format_settings.parquet.parallel_encoding && format_settings.max_threads > 1) + pool = std::make_unique( + CurrentMetrics::ParquetEncoderThreads, CurrentMetrics::ParquetEncoderThreadsActive, + format_settings.max_threads); + + using C = FormatSettings::ParquetCompression; + switch (format_settings.parquet.output_compression_method) + { + case C::NONE: options.compression = CompressionMethod::None; break; + case C::SNAPPY: options.compression = CompressionMethod::Snappy; break; + case C::ZSTD: options.compression = CompressionMethod::Zstd; break; + case C::LZ4: options.compression = CompressionMethod::Lz4; break; + case C::GZIP: options.compression = CompressionMethod::Gzip; break; + case C::BROTLI: options.compression = CompressionMethod::Brotli; break; + } + options.output_string_as_string = format_settings.parquet.output_string_as_string; + options.output_fixed_string_as_fixed_byte_array = format_settings.parquet.output_fixed_string_as_fixed_byte_array; + options.data_page_size = format_settings.parquet.data_page_size; + options.write_batch_size = format_settings.parquet.write_batch_size; + + schema = convertSchema(header_, options); + } } -void ParquetBlockOutputFormat::consumeStaged() +ParquetBlockOutputFormat::~ParquetBlockOutputFormat() { - const size_t columns_num = staging_chunks.at(0).getNumColumns(); + if (pool) + { + is_stopped = true; + pool->wait(); + } +} + +void ParquetBlockOutputFormat::consume(Chunk chunk) +{ + /// Poll background tasks. + if (pool) + { + std::unique_lock lock(mutex); + while (true) + { + /// If some row groups are ready to be written to the file, write them. + reapCompletedRowGroups(lock); + + if (background_exception) + std::rethrow_exception(background_exception); + + if (is_stopped) + return; + + /// If there's too much work in flight, wait for some of it to complete. + if (row_groups.size() < 2) + break; + if (bytes_in_flight <= format_settings.parquet.row_group_bytes * 4 && + task_queue.size() <= format_settings.max_threads * 4) + break; + + condvar.wait(lock); + } + } + + /// Do something like SquashingTransform to produce big enough row groups. + /// Because the real SquashingTransform is only used for INSERT, not for SELECT ... INTO OUTFILE. + /// The latter doesn't even have a pipeline where a transform could be inserted, so it's more + /// convenient to do the squashing here. It's also parallelized here. + + if (chunk.getNumRows() != 0) + { + staging_rows += chunk.getNumRows(); + staging_bytes += chunk.bytes(); + staging_chunks.push_back(std::move(chunk)); + } + + const size_t target_rows = std::max(static_cast(1), format_settings.parquet.row_group_rows); + + if (staging_rows < target_rows && + staging_bytes < format_settings.parquet.row_group_bytes) + return; + + /// In the rare case that more than `row_group_rows` rows arrived in one chunk, split the + /// staging chunk into multiple row groups. + if (staging_rows >= target_rows * 2) + { + /// Increase row group size slightly (by < 2x) to avoid a small row group at the end. + size_t num_row_groups = std::max(static_cast(1), staging_rows / target_rows); + size_t row_group_size = (staging_rows - 1) / num_row_groups + 1; // round up + + Chunk concatenated = std::move(staging_chunks[0]); + for (size_t i = 1; i < staging_chunks.size(); ++i) + concatenated.append(staging_chunks[i]); + staging_chunks.clear(); + + for (size_t offset = 0; offset < staging_rows; offset += row_group_size) + { + size_t count = std::min(row_group_size, staging_rows - offset); + MutableColumns columns = concatenated.cloneEmptyColumns(); + for (size_t i = 0; i < columns.size(); ++i) + columns[i]->insertRangeFrom(*concatenated.getColumns()[i], offset, count); + + Chunks piece; + piece.emplace_back(std::move(columns), count, concatenated.getChunkInfo()); + writeRowGroup(std::move(piece)); + } + } + else + { + writeRowGroup(std::move(staging_chunks)); + } + + staging_chunks.clear(); + staging_rows = 0; + staging_bytes = 0; +} + +void ParquetBlockOutputFormat::finalizeImpl() +{ + if (!staging_chunks.empty()) + writeRowGroup(std::move(staging_chunks)); + + if (format_settings.parquet.use_custom_encoder) + { + if (pool) + { + std::unique_lock lock(mutex); + + /// Wait for background work to complete. + while (true) + { + reapCompletedRowGroups(lock); + + if (background_exception) + std::rethrow_exception(background_exception); + + if (is_stopped) + return; + + if (row_groups.empty()) + break; + + condvar.wait(lock); + } + } + + if (row_groups_complete.empty()) + writeFileHeader(out); + writeFileFooter(std::move(row_groups_complete), schema, options, out); + } + else + { + if (!file_writer) + { + Block header = materializeBlock(getPort(PortKind::Main).getHeader()); + std::vector chunks; + chunks.push_back(Chunk(header.getColumns(), 0)); + writeRowGroup(std::move(chunks)); + } + + if (file_writer) + { + auto status = file_writer->Close(); + if (!status.ok()) + throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while closing a table: {}", status.ToString()); + } + } +} + +void ParquetBlockOutputFormat::resetFormatterImpl() +{ + if (pool) + { + is_stopped = true; + pool->wait(); + is_stopped = false; + } + + background_exception = nullptr; + threads_running = 0; + task_queue.clear(); + row_groups.clear(); + file_writer.reset(); + row_groups_complete.clear(); + staging_chunks.clear(); + staging_rows = 0; + staging_bytes = 0; +} + +void ParquetBlockOutputFormat::onCancel() +{ + is_stopped = true; +} + +void ParquetBlockOutputFormat::writeRowGroup(std::vector chunks) +{ + if (pool) + writeRowGroupInParallel(std::move(chunks)); + else if (!format_settings.parquet.use_custom_encoder) + writeUsingArrow(std::move(chunks)); + else + { + Chunk concatenated = std::move(chunks[0]); + for (size_t i = 1; i < chunks.size(); ++i) + concatenated.append(chunks[i]); + chunks.clear(); + + writeRowGroupInOneThread(std::move(concatenated)); + } +} + +void ParquetBlockOutputFormat::writeUsingArrow(std::vector chunks) +{ + const size_t columns_num = chunks.at(0).getNumColumns(); std::shared_ptr arrow_table; if (!ch_column_to_arrow_column) @@ -85,7 +302,7 @@ void ParquetBlockOutputFormat::consumeStaged() format_settings.parquet.output_fixed_string_as_fixed_byte_array); } - ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, staging_chunks, columns_num); + ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunks, columns_num); if (!file_writer) { @@ -112,64 +329,228 @@ void ParquetBlockOutputFormat::consumeStaged() file_writer = std::move(result.ValueOrDie()); } - // TODO: calculate row_group_size depending on a number of rows and table size - - // allow slightly bigger than row_group_size to avoid a very small tail row group - auto status = file_writer->WriteTable(*arrow_table, std::max(format_settings.parquet.row_group_rows, staging_rows)); + auto status = file_writer->WriteTable(*arrow_table, INT64_MAX); if (!status.ok()) throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while writing a table: {}", status.ToString()); } -void ParquetBlockOutputFormat::consume(Chunk chunk) +void ParquetBlockOutputFormat::writeRowGroupInOneThread(Chunk chunk) { - /// Do something like SquashingTransform to produce big enough row groups. - /// Because the real SquashingTransform is only used for INSERT, not for SELECT ... INTO OUTFILE. - /// The latter doesn't even have a pipeline where a transform could be inserted, so it's more - /// convenient to do the squashing here. - staging_rows += chunk.getNumRows(); - staging_bytes += chunk.bytes(); - staging_chunks.push_back(std::move(chunk)); - chassert(staging_chunks.back().getNumColumns() == staging_chunks.front().getNumColumns()); - if (staging_rows < format_settings.parquet.row_group_rows && - staging_bytes < format_settings.parquet.row_group_bytes) - { + if (chunk.getNumRows() == 0) return; - } - else + + const Block & header = getPort(PortKind::Main).getHeader(); + Parquet::ColumnChunkWriteStates columns_to_write; + chassert(header.columns() == chunk.getNumColumns()); + for (size_t i = 0; i < header.columns(); ++i) + prepareColumnForWrite( + chunk.getColumns()[i], header.getByPosition(i).type, header.getByPosition(i).name, + options, &columns_to_write); + + if (row_groups_complete.empty()) + writeFileHeader(out); + + std::vector column_chunks; + for (auto & s : columns_to_write) { - consumeStaged(); - staging_chunks.clear(); - staging_rows = 0; - staging_bytes = 0; + size_t offset = out.count(); + writeColumnChunkBody(s, options, out); + auto c = finalizeColumnChunkAndWriteFooter(offset, std::move(s), options, out); + column_chunks.push_back(std::move(c)); + } + + auto r = makeRowGroup(std::move(column_chunks), chunk.getNumRows()); + row_groups_complete.push_back(std::move(r)); +} + +void ParquetBlockOutputFormat::writeRowGroupInParallel(std::vector chunks) +{ + std::unique_lock lock(mutex); + + const Block & header = getPort(PortKind::Main).getHeader(); + + RowGroupState & r = row_groups.emplace_back(); + r.column_chunks.resize(header.columns()); + r.tasks_in_flight = r.column_chunks.size(); + + std::vector columnses; + for (auto & chunk : chunks) + { + chassert(header.columns() == chunk.getNumColumns()); + r.num_rows += chunk.getNumRows(); + columnses.push_back(chunk.detachColumns()); + } + + for (size_t i = 0; i < header.columns(); ++i) + { + Task & t = task_queue.emplace_back(&r, i, this); + t.column_type = header.getByPosition(i).type; + t.column_name = header.getByPosition(i).name; + + /// Defer concatenating the columns to the threads. + size_t bytes = 0; + for (size_t j = 0; j < chunks.size(); ++j) + { + auto & col = columnses[j][i]; + bytes += col->allocatedBytes(); + t.column_pieces.push_back(std::move(col)); + } + t.mem.set(bytes); + } + + startMoreThreadsIfNeeded(lock); +} + +void ParquetBlockOutputFormat::reapCompletedRowGroups(std::unique_lock & lock) +{ + while (!row_groups.empty() && row_groups.front().tasks_in_flight == 0 && !is_stopped) + { + RowGroupState & r = row_groups.front(); + + /// Write to the file. + + lock.unlock(); + + if (row_groups_complete.empty()) + writeFileHeader(out); + + std::vector metadata; + for (auto & cols : r.column_chunks) + { + for (ColumnChunk & col : cols) + { + size_t offset = out.count(); + + out.write(col.serialized.data(), col.serialized.size()); + auto m = finalizeColumnChunkAndWriteFooter(offset, std::move(col.state), options, out); + + metadata.push_back(std::move(m)); + } + } + + row_groups_complete.push_back(makeRowGroup(std::move(metadata), r.num_rows)); + + lock.lock(); + + row_groups.pop_front(); } } -void ParquetBlockOutputFormat::finalizeImpl() +void ParquetBlockOutputFormat::startMoreThreadsIfNeeded(const std::unique_lock &) { - if (!file_writer && staging_chunks.empty()) + /// Speculate that all current are already working on tasks. + size_t to_add = std::min(task_queue.size(), format_settings.max_threads - threads_running); + for (size_t i = 0; i < to_add; ++i) { - Block header = materializeBlock(getPort(PortKind::Main).getHeader()); + auto job = [this, thread_group = CurrentThread::getGroup()]() + { + if (thread_group) + CurrentThread::attachToGroupIfDetached(thread_group); + SCOPE_EXIT_SAFE(if (thread_group) CurrentThread::detachFromGroupIfNotDetached();); - consume(Chunk(header.getColumns(), 0)); // this will make staging_chunks non-empty + try + { + setThreadName("ParquetEncoder"); + + threadFunction(); + } + catch (...) + { + std::lock_guard lock(mutex); + background_exception = std::current_exception(); + condvar.notify_all(); + --threads_running; + } + }; + + if (threads_running == 0) + { + /// First thread. We need it to succeed; otherwise we may get stuck. + pool->scheduleOrThrowOnError(job); + ++threads_running; + } + else + { + /// More threads. This may be called from inside the thread pool, so avoid waiting; + /// otherwise it may deadlock. + if (!pool->trySchedule(job)) + break; + } } - - if (!staging_chunks.empty()) - { - consumeStaged(); - staging_chunks.clear(); - staging_rows = 0; - staging_bytes = 0; - } - - auto status = file_writer->Close(); - if (!status.ok()) - throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while closing a table: {}", status.ToString()); } -void ParquetBlockOutputFormat::resetFormatterImpl() +void ParquetBlockOutputFormat::threadFunction() { - file_writer.reset(); + std::unique_lock lock(mutex); + + while (true) + { + if (task_queue.empty() || is_stopped) + { + /// The check and the decrement need to be in the same critical section, to make sure + /// we never get stuck with tasks but no threads. + --threads_running; + return; + } + + auto task = std::move(task_queue.front()); + task_queue.pop_front(); + + if (task.column_type) + { + lock.unlock(); + + IColumn::MutablePtr concatenated = IColumn::mutate(std::move(task.column_pieces[0])); + for (size_t i = 1; i < task.column_pieces.size(); ++i) + { + auto & c = task.column_pieces[i]; + concatenated->insertRangeFrom(*c, 0, c->size()); + c.reset(); + } + task.column_pieces.clear(); + + std::vector subcolumns; + prepareColumnForWrite( + std::move(concatenated), task.column_type, task.column_name, options, &subcolumns); + + lock.lock(); + + for (size_t i = 0; i < subcolumns.size(); ++i) + { + task.row_group->column_chunks[task.column_idx].emplace_back(this); + task.row_group->tasks_in_flight += 1; + + auto & t = task_queue.emplace_back(task.row_group, task.column_idx, this); + t.subcolumn_idx = i; + t.state = std::move(subcolumns[i]); + t.mem.set(t.state.allocatedBytes()); + } + + startMoreThreadsIfNeeded(lock); + } + else + { + lock.unlock(); + + PODArray serialized; + { + WriteBufferFromVector buf(serialized); + writeColumnChunkBody(task.state, options, buf); + } + + lock.lock(); + + auto & c = task.row_group->column_chunks[task.column_idx][task.subcolumn_idx]; + c.state = std::move(task.state); + c.serialized = std::move(serialized); + c.mem.set(c.serialized.size() + c.state.allocatedBytes()); + } + + --task.row_group->tasks_in_flight; + + condvar.notify_all(); + } } void registerOutputFormatParquet(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index 482c778bc52..4c73de007fe 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -2,8 +2,11 @@ #include "config.h" #if USE_PARQUET -# include -# include + +#include +#include +#include +#include namespace arrow { @@ -28,25 +31,128 @@ class ParquetBlockOutputFormat : public IOutputFormat { public: ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); + ~ParquetBlockOutputFormat() override; String getName() const override { return "ParquetBlockOutputFormat"; } String getContentType() const override { return "application/octet-stream"; } private: - void consumeStaged(); + struct MemoryToken + { + ParquetBlockOutputFormat * parent; + size_t bytes = 0; + + explicit MemoryToken(ParquetBlockOutputFormat * p, size_t b = 0) : parent(p) + { + set(b); + } + + MemoryToken(MemoryToken && t) + : parent(std::exchange(t.parent, nullptr)), bytes(std::exchange(t.bytes, 0)) {} + + MemoryToken & operator=(MemoryToken && t) + { + parent = std::exchange(t.parent, nullptr); + bytes = std::exchange(t.bytes, 0); + return *this; + } + + ~MemoryToken() + { + set(0); + } + + void set(size_t new_size) + { + if (new_size == bytes) + return; + parent->bytes_in_flight += new_size - bytes; // overflow is fine + bytes = new_size; + } + }; + + struct ColumnChunk + { + Parquet::ColumnChunkWriteState state; + PODArray serialized; + + MemoryToken mem; + + ColumnChunk(ParquetBlockOutputFormat * p) : mem(p) {} + }; + + struct RowGroupState + { + size_t tasks_in_flight = 0; + std::vector> column_chunks; + size_t num_rows = 0; + }; + + struct Task + { + RowGroupState * row_group; + size_t column_idx; + size_t subcolumn_idx = 0; + + MemoryToken mem; + + /// If not null, we need to call prepareColumnForWrite(). + /// Otherwise we need to call writeColumnChunkBody(). + DataTypePtr column_type; + std::string column_name; + std::vector column_pieces; + + Parquet::ColumnChunkWriteState state; + + Task(RowGroupState * rg, size_t ci, ParquetBlockOutputFormat * p) + : row_group(rg), column_idx(ci), mem(p) {} + }; + void consume(Chunk) override; void finalizeImpl() override; void resetFormatterImpl() override; + void onCancel() override; + void writeRowGroup(std::vector chunks); + void writeUsingArrow(std::vector chunks); + void writeRowGroupInOneThread(Chunk chunk); + void writeRowGroupInParallel(std::vector chunks); + + void threadFunction(); + void startMoreThreadsIfNeeded(const std::unique_lock & lock); + + /// Called in single-threaded fashion. Writes to the file. + void reapCompletedRowGroups(std::unique_lock & lock); + + const FormatSettings format_settings; + + /// Chunks to squash together to form a row group. std::vector staging_chunks; size_t staging_rows = 0; size_t staging_bytes = 0; - const FormatSettings format_settings; - std::unique_ptr file_writer; std::unique_ptr ch_column_to_arrow_column; + + Parquet::WriteOptions options; + Parquet::SchemaElements schema; + std::vector row_groups_complete; + + + std::mutex mutex; + std::condition_variable condvar; // wakes up consume() + std::unique_ptr pool; + + std::atomic_bool is_stopped{false}; + std::exception_ptr background_exception = nullptr; + + /// Invariant: if there's at least one task then there's at least one thread. + size_t threads_running = 0; + std::atomic bytes_in_flight{0}; + + std::deque task_queue; + std::deque row_groups; }; } diff --git a/tests/queries/0_stateless/02735_parquet_encoder.reference b/tests/queries/0_stateless/02735_parquet_encoder.reference new file mode 100644 index 00000000000..c7d79392d85 --- /dev/null +++ b/tests/queries/0_stateless/02735_parquet_encoder.reference @@ -0,0 +1,55 @@ +u8 Nullable(UInt8) +u16 Nullable(UInt16) +u32 Nullable(UInt32) +u64 Nullable(UInt64) +i8 Nullable(Int8) +i16 Nullable(Int16) +i32 Nullable(Int32) +i64 Nullable(Int64) +date Nullable(UInt16) +date32 Nullable(Date32) +datetime Nullable(UInt32) +datetime64 Nullable(DateTime64(3, \'UTC\')) +enum8 Nullable(Int8) +enum16 Nullable(Int16) +float32 Nullable(Float32) +float64 Nullable(Float64) +str Nullable(String) +fstr Nullable(FixedString(12)) +u128 Nullable(FixedString(16)) +u256 Nullable(FixedString(32)) +i128 Nullable(FixedString(16)) +i256 Nullable(FixedString(32)) +decimal32 Nullable(Decimal(9, 3)) +decimal64 Nullable(Decimal(18, 10)) +decimal128 Nullable(Decimal(38, 20)) +decimal256 Nullable(Decimal(76, 40)) +ipv4 Nullable(UInt32) +ipv6 Nullable(FixedString(16)) +0 +0 +0 +0 +1 2 1 +1 2 2 +1 3 3 +1 1000000 1 +3914219105369203805 +4 1000000 1 +(1000000,0,NULL,'100','299') +(1000000,0,NULL,'0','-1294970296') +(1000000,0,NULL,'-2147483296','2147481000') +(100000,900000,NULL,'100009','999999') +[(2,0,NULL,'','[]')] +1 1 +0 1 +16159458007063698496 +16159458007063698496 +BYTE_ARRAY String +FIXED_LEN_BYTE_ARRAY None +BYTE_ARRAY None +BYTE_ARRAY None +BYTE_ARRAY String +never gonna +give you +up diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql new file mode 100644 index 00000000000..d8d52a13218 --- /dev/null +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -0,0 +1,168 @@ +-- Tags: no-fasttest + +set output_format_parquet_use_custom_encoder = 1; +set output_format_parquet_row_group_size = 1000; +set output_format_parquet_data_page_size = 800; +set output_format_parquet_batch_size = 100; +set output_format_parquet_row_group_size_bytes = 1000000000; +set engine_file_truncate_on_insert=1; + +-- Write random data to parquet file, then read from it and check that it matches what we wrote. +-- Do this for all kinds of data types: primitive, Nullable(primitive), Array(primitive), +-- Array(Nullable(primitive)), Array(Array(primitive)), Map(primitive, primitive), etc. + +drop table if exists basic_types_02735; +create temporary table basic_types_02735 as select * from generateRandom(' + u8 UInt8, + u16 UInt16, + u32 UInt32, + u64 UInt64, + i8 Int8, + i16 Int16, + i32 Int32, + i64 Int64, + date Date, + date32 Date32, + datetime DateTime, + datetime64 DateTime64, + enum8 Enum8(''x'' = 1, ''y'' = 2, ''z'' = 3), + enum16 Enum16(''xx'' = 1000, ''yy'' = 2000, ''zz'' = 3000), + float32 Float32, + float64 Float64, + str String, + fstr FixedString(12), + u128 UInt128, + u256 UInt256, + i128 Int128, + i256 Int256, + decimal32 Decimal32(3), + decimal64 Decimal64(10), + decimal128 Decimal128(20), + decimal256 Decimal256(40), + ipv4 IPv4, + ipv6 IPv6') limit 10101; +insert into function file(basic_types_02735.parquet) select * from basic_types_02735; +desc file(basic_types_02735.parquet); +select (select sum(cityHash64(*)) from basic_types_02735) - (select sum(cityHash64(*)) from file(basic_types_02735.parquet)); +drop table basic_types_02735; + + +drop table if exists nullables_02735; +create temporary table nullables_02735 as select * from generateRandom(' + u16 Nullable(UInt16), + i64 Nullable(Int64), + datetime64 Nullable(DateTime64), + enum8 Nullable(Enum8(''x'' = 1, ''y'' = 2, ''z'' = 3)), + float64 Nullable(Float64), + str Nullable(String), + fstr Nullable(FixedString(12)), + i256 Nullable(Int256), + decimal256 Nullable(Decimal256(40)), + ipv6 Nullable(IPv6)') limit 10000; +insert into function file(nullables_02735.parquet) select * from nullables_02735; +select (select sum(cityHash64(*)) from nullables_02735) - (select sum(cityHash64(*)) from file(nullables_02735.parquet)); +drop table nullables_02735; + + +-- TODO: When cityHash64() fully supports Nullable: https://github.com/ClickHouse/ClickHouse/pull/48625 +-- the next two blocks can be simplified: arrays_out_02735 intermediate table is not needed, +-- a.csv and b.csv are not needed. + +drop table if exists arrays_02735; +drop table if exists arrays_out_02735; +create table arrays_02735 engine = Memory as select * from generateRandom(' + u32 Array(UInt32), + i8 Array(Int8), + datetime Array(DateTime), + enum16 Array(Enum16(''xx'' = 1000, ''yy'' = 2000, ''zz'' = 3000)), + float32 Array(Float32), + str Array(String), + fstr Array(FixedString(12)), + u128 Array(UInt128), + decimal64 Array(Decimal64(10)), + ipv4 Array(IPv4), + msi Map(String, Int16), + tup Tuple(FixedString(3), Array(String), Map(Int8, Date))') limit 10000; +insert into function file(arrays_02735.parquet) select * from arrays_02735; +create temporary table arrays_out_02735 as arrays_02735; +insert into arrays_out_02735 select * from file(arrays_02735.parquet); +select (select sum(cityHash64(*)) from arrays_02735) - (select sum(cityHash64(*)) from arrays_out_02735); +--select (select sum(cityHash64(*)) from arrays_02735) - +-- (select sum(cityHash64(u32, i8, datetime, enum16, float32, str, fstr, arrayMap(x->reinterpret(x, 'UInt128'), u128), decimal64, ipv4, msi, tup)) from file(arrays_02735.parquet)); +drop table arrays_02735; +drop table arrays_out_02735; + + +drop table if exists madness_02735; +create temporary table madness_02735 as select * from generateRandom(' + aa Array(Array(UInt32)), + aaa Array(Array(Array(UInt32))), + an Array(Nullable(String)), + aan Array(Array(Nullable(FixedString(10)))), + l LowCardinality(String), + ln LowCardinality(Nullable(FixedString(11))), + al Array(LowCardinality(UInt128)), + aaln Array(Array(LowCardinality(Nullable(String)))), + mln Map(LowCardinality(String), Nullable(Int8)), + t Tuple(Map(FixedString(5), Tuple(Array(UInt16), Nullable(UInt16), Array(Tuple(Int8, Decimal64(10))))), Tuple(kitchen UInt64, sink String)), + n Nested(hello UInt64, world Tuple(first String, second FixedString(1))) + ') limit 10000; +insert into function file(madness_02735.parquet) select * from madness_02735; +insert into function file(a.csv) select * from madness_02735 order by tuple(*); +insert into function file(b.csv) select aa, aaa, an, aan, l, ln, arrayMap(x->reinterpret(x, 'UInt128'), al) as al_, aaln, mln, t, n.hello, n.world from file(madness_02735.parquet) order by tuple(aa, aaa, an, aan, l, ln, al_, aaln, mln, t, n.hello, n.world); +select (select sum(cityHash64(*)) from file(a.csv, LineAsString)) - (select sum(cityHash64(*)) from file(b.csv, LineAsString)); +--select (select sum(cityHash64(*)) from madness_02735) - +-- (select sum(cityHash64(aa, aaa, an, aan, l, ln, map(x->reinterpret(x, 'UInt128'), al), aaln, mln, t, n.hello, n.world)) from file(madness_02735.parquet)); +drop table madness_02735; + + +-- Merging input blocks into bigger row groups. +insert into function file(squash_02735.parquet) select '012345' union all select '543210' settings max_block_size = 1; +select num_columns, num_rows, num_row_groups from file(squash_02735.parquet, ParquetMetadata); + +-- Row group size limit in bytes. +insert into function file(row_group_bytes_02735.parquet) select '012345' union all select '543210' settings max_block_size = 1, output_format_parquet_row_group_size_bytes = 5; +select num_columns, num_rows, num_row_groups from file(row_group_bytes_02735.parquet, ParquetMetadata); + +-- Row group size limit in rows. +insert into function file(tiny_row_groups_02735.parquet) select * from numbers(3) settings output_format_parquet_row_group_size = 1; +select num_columns, num_rows, num_row_groups from file(tiny_row_groups_02735.parquet, ParquetMetadata); + +-- 1M unique 8-byte values should exceed dictionary_size_limit (1 MB). +insert into function file(big_column_chunk_02735.parquet) select number from numbers(1000000) settings output_format_parquet_row_group_size = 1000000; +select num_columns, num_rows, num_row_groups from file(big_column_chunk_02735.parquet, ParquetMetadata); +select sum(cityHash64(number)) from file(big_column_chunk_02735.parquet); + +-- Check statistics: signed vs unsigned, null count. Use enough rows to produce multiple pages. +insert into function file(statistics_02735.parquet) select 100 + number%200 as a, toUInt32(number * 3000) as u, toInt32(number * 3000) as i, if(number % 10 == 9, toString(number), null) as s from numbers(1000000) settings output_format_parquet_row_group_size = 1000000; +select num_columns, num_rows, num_row_groups from file(statistics_02735.parquet, ParquetMetadata); +select tupleElement(c, 'statistics') from file(statistics_02735.parquet, ParquetMetadata) array join tupleElement(row_groups[1], 'columns') as c; + +-- Statistics string length limit (max_statistics_size). +insert into function file(long_string_02735.parquet) select toString(range(number * 2000)) from numbers(2); +select tupleElement(tupleElement(row_groups[1], 'columns'), 'statistics') from file(long_string_02735.parquet, ParquetMetadata); + +-- Compression setting. +insert into function file(compressed_02735.parquet) select concat('aaaaaaaaaaaaaaaa', toString(number)) as s from numbers(1000) settings output_format_parquet_row_group_size = 10000, output_format_parquet_compression_method='zstd'; +select total_compressed_size < 10000, total_uncompressed_size > 15000 from file(compressed_02735.parquet, ParquetMetadata); +insert into function file(compressed_02735.parquet) select concat('aaaaaaaaaaaaaaaa', toString(number)) as s from numbers(1000) settings output_format_parquet_row_group_size = 10000, output_format_parquet_compression_method='none'; +select total_compressed_size < 10000, total_uncompressed_size > 15000 from file(compressed_02735.parquet, ParquetMetadata); + +-- Single-threaded encoding and Arrow encoder. +drop table if exists other_encoders_02735; +create temporary table other_encoders_02735 as select number, number*2 from numbers(10000); +insert into function file(single_thread_02735.parquet) select * from other_encoders_02735 settings max_threads = 1; +select sum(cityHash64(*)) from file(single_thread_02735.parquet); +insert into function file(arrow_02735.parquet) select * from other_encoders_02735 settings output_format_parquet_use_custom_encoder = 0; +select sum(cityHash64(*)) from file(arrow_02735.parquet); + +-- String -> binary vs string; FixedString -> fixed-length-binary vs binary vs string. +insert into function file(strings1_02735.parquet) select 'never', toFixedString('gonna', 5) settings output_format_parquet_string_as_string = 1, output_format_parquet_fixed_string_as_fixed_byte_array = 1; +select columns.5, columns.6 from file(strings1_02735.parquet, ParquetMetadata) array join columns; +insert into function file(strings2_02735.parquet) select 'give', toFixedString('you', 3) settings output_format_parquet_string_as_string = 0, output_format_parquet_fixed_string_as_fixed_byte_array = 0; +select columns.5, columns.6 from file(strings2_02735.parquet, ParquetMetadata) array join columns; +insert into function file(strings3_02735.parquet) select toFixedString('up', 2) settings output_format_parquet_string_as_string = 1, output_format_parquet_fixed_string_as_fixed_byte_array = 0; +select columns.5, columns.6 from file(strings3_02735.parquet, ParquetMetadata) array join columns; +select * from file(strings1_02735.parquet); +select * from file(strings2_02735.parquet); +select * from file(strings3_02735.parquet); From db5cb960508fc20ff7127aa092b89e6002c9f503 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 17 May 2023 01:56:00 +0000 Subject: [PATCH 1128/2047] Start over when falling back to non-dictionary encoding --- src/Processors/Formats/Impl/Parquet/Write.cpp | 41 +++++++++++-------- 1 file changed, 24 insertions(+), 17 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index a29bb81f8dc..ba67f075a0d 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -421,10 +421,7 @@ void writeColumnImpl( typename Converter::Statistics page_statistics; typename Converter::Statistics total_statistics; - /// We start with dictionary encoding, then switch to `encoding` (non-dictionary) if the - /// dictionary gets too big. That's how arrow does it too. - bool initially_used_dictionary = options.use_dictionary_encoding; - bool currently_using_dictionary = initially_used_dictionary; + bool use_dictionary = options.use_dictionary_encoding; std::optional fixed_string_descr; if constexpr (std::is_same::value) @@ -441,12 +438,11 @@ void writeColumnImpl( /// Alternatively, we could avoid using arrow's dictionary encoding code and leverage /// ColumnLowCardinality instead. It would work basically the same way as what this function /// currently does: add values to the ColumnRowCardinality (instead of `encoder`) in batches, - /// checking dictionary size after each batch; if it gets big, flush the dictionary and the - /// indices and switch to non-dictionary encoding. Feels like it could even be slightly less code. + /// checking dictionary size after each batch. That might be faster. auto encoder = parquet::MakeTypedEncoder( // ignored if using dictionary static_cast(encoding), - currently_using_dictionary, fixed_string_descr ? &*fixed_string_descr : nullptr); + use_dictionary, fixed_string_descr ? &*fixed_string_descr : nullptr); struct PageData { @@ -496,7 +492,7 @@ void writeColumnImpl( header.__isset.data_page_header = true; auto & d = header.data_page_header; d.__set_num_values(static_cast(def_count)); - d.__set_encoding(currently_using_dictionary ? parq::Encoding::RLE_DICTIONARY : encoding); + d.__set_encoding(use_dictionary ? parq::Encoding::RLE_DICTIONARY : encoding); d.__set_definition_level_encoding(parq::Encoding::RLE); d.__set_repetition_level_encoding(parq::Encoding::RLE); /// We could also put checksum in `header.crc`, but apparently no one uses it: @@ -513,7 +509,7 @@ void writeColumnImpl( total_statistics.merge(page_statistics); page_statistics.clear(); - if (currently_using_dictionary) + if (use_dictionary) { dict_encoded_pages.push_back({.header = std::move(header)}); std::swap(dict_encoded_pages.back().data, compressed); @@ -593,13 +589,22 @@ void writeColumnImpl( next_def_offset += def_count; next_data_offset += data_count; - if (currently_using_dictionary && is_dict_too_big()) + if (use_dictionary && is_dict_too_big()) { /// Fallback to non-dictionary encoding. - flush_page(next_def_offset - def_offset, next_data_offset - data_offset); - flush_dict(); + /// + /// Discard encoded data and start over. + /// This is different from what arrow does: arrow writes out the dictionary-encoded + /// data, then uses non-dictionary encoding for later pages. + /// Starting over seems better: it produces slightly smaller files (I saw 1-4%) in + /// exchange for slight decrease in speed (I saw < 5%). This seems like a good + /// trade because encoding speed is much less important than decoding (as evidenced + /// by arrow not supporting parallel encoding, even though it's easy to support). - currently_using_dictionary = false; + def_offset = 0; + data_offset = 0; + dict_encoded_pages.clear(); + use_dictionary = false; encoder = parquet::MakeTypedEncoder( static_cast(encoding)); break; @@ -614,7 +619,7 @@ void writeColumnImpl( } } - if (currently_using_dictionary) + if (use_dictionary) flush_dict(); chassert(data_offset == s.primitive_column->size()); @@ -630,13 +635,15 @@ void writeColumnImpl( /// Report which encodings we've used. if (s.max_rep > 0 || s.max_def > 0) addToEncodingsUsed(s, parq::Encoding::RLE); // levels - if (!currently_using_dictionary) - addToEncodingsUsed(s, encoding); // non-dictionary encoding - if (initially_used_dictionary) + if (use_dictionary) { addToEncodingsUsed(s, parq::Encoding::PLAIN); // dictionary itself addToEncodingsUsed(s, parq::Encoding::RLE_DICTIONARY); // ids } + else + { + addToEncodingsUsed(s, encoding); + } } } From dfdf5de972b0b8ee37fd0e89cfeaa8c3f5ea79cf Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 30 May 2023 01:28:16 +0000 Subject: [PATCH 1129/2047] Fixes --- contrib/arrow-cmake/CMakeLists.txt | 5 +- programs/client/Client.cpp | 5 + .../Formats/Impl/Parquet/PrepareForWrite.cpp | 16 ++- src/Processors/Formats/Impl/Parquet/Write.cpp | 120 +++++++++++++++--- src/Processors/Formats/Impl/Parquet/Write.h | 1 + .../Formats/Impl/ParquetBlockInputFormat.cpp | 7 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 18 ++- .../Formats/Impl/ParquetBlockOutputFormat.h | 1 + .../02581_parquet_arrow_orc_compressions.sh | 2 + .../0_stateless/02735_parquet_encoder.sql | 2 +- 10 files changed, 147 insertions(+), 30 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 46b86cb4ddb..e3ea0381595 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -502,9 +502,10 @@ target_include_directories(_parquet SYSTEM BEFORE "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src" "${CMAKE_CURRENT_SOURCE_DIR}/cpp/src") target_link_libraries(_parquet - PUBLIC _arrow - PRIVATE + PUBLIC + _arrow ch_contrib::thrift + PRIVATE boost::headers_only boost::regex OpenSSL::Crypto OpenSSL::SSL) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e1a33231592..e73f77819ad 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -812,6 +812,11 @@ bool Client::processWithFuzzing(const String & full_query) } catch (...) { + if (!ast_to_process) + fmt::print(stderr, + "Error while forming new query: {}\n", + getCurrentExceptionMessage(true)); + // Some functions (e.g. protocol parsers) don't throw, but // set last_exception instead, so we'll also do it here for // uniformity. diff --git a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp index a70b6fcfc81..0700fc8491c 100644 --- a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp +++ b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp @@ -295,7 +295,17 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin switch (type->getTypeId()) { - case TypeIndex::UInt8: types(T::INT32, C::UINT_8 , int_type(8 , false)); break; + case TypeIndex::UInt8: + if (isBool(type)) + { + types(T::BOOLEAN); + state.is_bool = true; + } + else + { + types(T::INT32, C::UINT_8 , int_type(8 , false)); + } + break; case TypeIndex::UInt16: types(T::INT32, C::UINT_16, int_type(16, false)); break; case TypeIndex::UInt32: types(T::INT32, C::UINT_32, int_type(32, false)); break; case TypeIndex::UInt64: types(T::INT64, C::UINT_64, int_type(64, false)); break; @@ -588,7 +598,7 @@ SchemaElements convertSchema(const Block & sample, const WriteOptions & options) root.__set_name("schema"); root.__set_num_children(static_cast(sample.columns())); - for (auto & c : sample) + for (const auto & c : sample) prepareColumnForWrite(c.column, c.type, c.name, options, nullptr, &schema); return schema; @@ -598,7 +608,7 @@ void prepareColumnForWrite( ColumnPtr column, DataTypePtr type, const std::string & name, const WriteOptions & options, ColumnChunkWriteStates * out_columns_to_write, SchemaElements * out_schema) { - if (column->size() == 0 && out_columns_to_write != nullptr) + if (column->empty() && out_columns_to_write != nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty column passed to Parquet encoder"); ColumnChunkWriteStates states; diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index ba67f075a0d..9664d173f29 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -15,6 +15,10 @@ #include #include "config_version.h" +#if USE_SNAPPY +#include +#endif + namespace DB::ErrorCodes { extern const int CANNOT_COMPRESS; @@ -68,7 +72,7 @@ struct StatisticsNumeric } }; -struct StatisticsFixedString +struct StatisticsFixedStringRef { size_t fixed_string_size = UINT64_MAX; const uint8_t * min = nullptr; @@ -81,7 +85,7 @@ struct StatisticsFixedString addMax(a.ptr); } - void merge(const StatisticsFixedString & s) + void merge(const StatisticsFixedStringRef & s) { chassert(fixed_string_size == UINT64_MAX || fixed_string_size == s.fixed_string_size); fixed_string_size = s.fixed_string_size; @@ -93,7 +97,7 @@ struct StatisticsFixedString void clear() { min = max = nullptr; } - parq::Statistics get(const WriteOptions & options) + parq::Statistics get(const WriteOptions & options) const { parq::Statistics s; if (min == nullptr || fixed_string_size > options.max_statistics_size) @@ -115,7 +119,54 @@ struct StatisticsFixedString } }; -struct StatisticsString +template +struct StatisticsFixedStringCopy +{ + bool empty = true; + std::array min {}; + std::array max {}; + + void add(parquet::FixedLenByteArray a) + { + addMin(a.ptr); + addMax(a.ptr); + empty = false; + } + + void merge(const StatisticsFixedStringCopy & s) + { + if (s.empty) + return; + addMin(&s.min[0]); + addMax(&s.max[0]); + empty = false; + } + + void clear() { empty = true; } + + parq::Statistics get(const WriteOptions &) const + { + parq::Statistics s; + if (empty) + return s; + s.__set_min_value(std::string(reinterpret_cast(min.data()), S)); + s.__set_max_value(std::string(reinterpret_cast(max.data()), S)); + return s; + } + + void addMin(const uint8_t * p) + { + if (empty || memcmp(p, min.data(), S) < 0) + memcpy(min.data(), p, S); + } + void addMax(const uint8_t * p) + { + if (empty || memcmp(p, max.data(), S) > 0) + memcpy(max.data(), p, S); + } +}; + +struct StatisticsStringRef { parquet::ByteArray min; parquet::ByteArray max; @@ -126,7 +177,7 @@ struct StatisticsString addMax(x); } - void merge(const StatisticsString & s) + void merge(const StatisticsStringRef & s) { if (s.min.ptr == nullptr) return; @@ -136,7 +187,7 @@ struct StatisticsString void clear() { *this = {}; } - parq::Statistics get(const WriteOptions & options) + parq::Statistics get(const WriteOptions & options) const { parq::Statistics s; if (min.ptr == nullptr) @@ -197,7 +248,7 @@ struct ConverterNumeric { buf.resize(count); for (size_t i = 0; i < count; ++i) - buf[i] = static_cast(column.getData()[offset + i]); + buf[i] = static_cast(column.getData()[offset + i]); // NOLINT return buf.data(); } } @@ -205,7 +256,7 @@ struct ConverterNumeric struct ConverterString { - using Statistics = StatisticsString; + using Statistics = StatisticsStringRef; const ColumnString & column; PODArray buf; @@ -226,7 +277,7 @@ struct ConverterString struct ConverterFixedString { - using Statistics = StatisticsFixedString; + using Statistics = StatisticsFixedStringRef; const ColumnFixedString & column; PODArray buf; @@ -246,7 +297,7 @@ struct ConverterFixedString struct ConverterFixedStringAsString { - using Statistics = StatisticsString; + using Statistics = StatisticsStringRef; const ColumnFixedString & column; PODArray buf; @@ -267,7 +318,7 @@ struct ConverterNumberAsFixedString { /// Calculate min/max statistics for little-endian fixed strings, not numbers, because parquet /// doesn't know it's numbers. - using Statistics = StatisticsFixedString; + using Statistics = StatisticsFixedStringCopy; const ColumnVector & column; PODArray buf; @@ -290,7 +341,7 @@ struct ConverterNumberAsFixedString template struct ConverterDecimal { - using Statistics = StatisticsFixedString; + using Statistics = StatisticsFixedStringCopy; const ColumnDecimal & column; PODArray data_buf; @@ -348,6 +399,24 @@ PODArray & compress(PODArray & source, PODArray & scratch, Com return scratch; } +#if USE_SNAPPY + case CompressionMethod::Snappy: + { + size_t max_dest_size = snappy::MaxCompressedLength(source.size()); + + if (max_dest_size > std::numeric_limits::max()) + throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress column of size {}", formatReadableSizeWithBinarySuffix(source.size())); + + scratch.resize(max_dest_size); + + size_t compressed_size; + snappy::RawCompress(source.data(), source.size(), scratch.data(), &compressed_size); + + scratch.resize(static_cast(compressed_size)); + return scratch; + } +#endif + default: { auto dest_buf = std::make_unique>>(scratch); @@ -421,7 +490,7 @@ void writeColumnImpl( typename Converter::Statistics page_statistics; typename Converter::Statistics total_statistics; - bool use_dictionary = options.use_dictionary_encoding; + bool use_dictionary = options.use_dictionary_encoding && !s.is_bool; std::optional fixed_string_descr; if constexpr (std::is_same::value) @@ -431,7 +500,8 @@ void writeColumnImpl( "", parquet::Repetition::REQUIRED, parquet::Type::FIXED_LEN_BYTE_ARRAY, parquet::ConvertedType::NONE, static_cast(converter.fixedStringSize())), 0, 0); - page_statistics.fixed_string_size = converter.fixedStringSize(); + if constexpr (std::is_same::value) + page_statistics.fixed_string_size = converter.fixedStringSize(); } /// Could use an arena here (by passing a custom MemoryPool), to reuse memory across pages. @@ -605,8 +675,16 @@ void writeColumnImpl( data_offset = 0; dict_encoded_pages.clear(); use_dictionary = false; + +#ifndef NDEBUG + /// Arrow's DictEncoderImpl destructor asserts that FlushValues() was called, so we + /// call it even though we don't need its output. + encoder->FlushValues(); +#endif + encoder = parquet::MakeTypedEncoder( - static_cast(encoding)); + static_cast(encoding), /* use_dictionary */ false, + fixed_string_descr ? &*fixed_string_descr : nullptr); break; } @@ -668,7 +746,13 @@ void writeColumnChunkBody(ColumnChunkWriteState & s, const WriteOptions & option ConverterNumeric, parquet::parquet_dtype::c_type>( \ s.primitive_column)) - case TypeIndex::UInt8 : N(UInt8 , Int32Type); break; + case TypeIndex::UInt8: + if (s.is_bool) + writeColumnImpl(s, options, out, + ConverterNumeric, bool, bool>(s.primitive_column)); + else + N(UInt8 , Int32Type); + break; case TypeIndex::UInt16 : N(UInt16, Int32Type); break; case TypeIndex::UInt32 : N(UInt32, Int32Type); break; case TypeIndex::UInt64 : N(UInt64, Int64Type); break; @@ -769,14 +853,14 @@ parq::ColumnChunk finalizeColumnChunkAndWriteFooter( serializeThriftStruct(s.column_chunk, out); - return std::move(s.column_chunk); + return s.column_chunk; } parq::RowGroup makeRowGroup(std::vector column_chunks, size_t num_rows) { parq::RowGroup r; r.__set_num_rows(num_rows); - r.__set_columns(std::move(column_chunks)); + r.__set_columns(column_chunks); r.__set_total_compressed_size(0); for (auto & c : r.columns) { diff --git a/src/Processors/Formats/Impl/Parquet/Write.h b/src/Processors/Formats/Impl/Parquet/Write.h index 333a32e191f..9197eae5384 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.h +++ b/src/Processors/Formats/Impl/Parquet/Write.h @@ -42,6 +42,7 @@ struct ColumnChunkWriteState ColumnPtr primitive_column; CompressionMethod compression; // must match what's inside column_chunk + bool is_bool = false; /// Repetition and definition levels. Produced by prepareColumnForWrite(). /// def is empty iff max_def == 0, which means no arrays or nullables. diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 3dde8ad6a6c..be9c600f9bd 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -59,7 +59,12 @@ ParquetBlockInputFormat::ParquetBlockInputFormat( pool = std::make_unique(CurrentMetrics::ParquetDecoderThreads, CurrentMetrics::ParquetDecoderThreadsActive, max_decoding_threads); } -ParquetBlockInputFormat::~ParquetBlockInputFormat() = default; +ParquetBlockInputFormat::~ParquetBlockInputFormat() +{ + is_stopped = true; + if (pool) + pool->wait(); +} void ParquetBlockInputFormat::initializeIfNeeded() { diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 9a2d9072860..fbf8b3a7c87 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -68,9 +68,8 @@ namespace if (method == FormatSettings::ParquetCompression::GZIP) return parquet::Compression::type::GZIP; - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported compression method"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported parquet compression method"); } - } ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) @@ -162,7 +161,7 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) if (staging_rows >= target_rows * 2) { /// Increase row group size slightly (by < 2x) to avoid a small row group at the end. - size_t num_row_groups = std::max(static_cast(1), staging_rows / target_rows); + size_t num_row_groups = std::max(static_cast(1), staging_rows / target_rows); size_t row_group_size = (staging_rows - 1) / num_row_groups + 1; // round up Chunk concatenated = std::move(staging_chunks[0]); @@ -222,7 +221,10 @@ void ParquetBlockOutputFormat::finalizeImpl() } if (row_groups_complete.empty()) + { + base_offset = out.count(); writeFileHeader(out); + } writeFileFooter(std::move(row_groups_complete), schema, options, out); } else @@ -349,12 +351,15 @@ void ParquetBlockOutputFormat::writeRowGroupInOneThread(Chunk chunk) options, &columns_to_write); if (row_groups_complete.empty()) + { + base_offset = out.count(); writeFileHeader(out); + } std::vector column_chunks; for (auto & s : columns_to_write) { - size_t offset = out.count(); + size_t offset = out.count() - base_offset; writeColumnChunkBody(s, options, out); auto c = finalizeColumnChunkAndWriteFooter(offset, std::move(s), options, out); column_chunks.push_back(std::move(c)); @@ -413,14 +418,17 @@ void ParquetBlockOutputFormat::reapCompletedRowGroups(std::unique_lock metadata; for (auto & cols : r.column_chunks) { for (ColumnChunk & col : cols) { - size_t offset = out.count(); + size_t offset = out.count() - base_offset; out.write(col.serialized.data(), col.serialized.size()); auto m = finalizeColumnChunkAndWriteFooter(offset, std::move(col.state), options, out); diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index 4c73de007fe..aededc39dc4 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -138,6 +138,7 @@ private: Parquet::WriteOptions options; Parquet::SchemaElements schema; std::vector row_groups_complete; + size_t base_offset = 0; std::mutex mutex; diff --git a/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh b/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh index 89b5147f026..d00026d516a 100755 --- a/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh +++ b/tests/queries/0_stateless/02581_parquet_arrow_orc_compressions.sh @@ -5,6 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +set -o pipefail + $CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='none'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" $CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='lz4'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" $CLICKHOUSE_LOCAL -q "select * from numbers(10) format Parquet settings output_format_parquet_compression_method='snappy'" | $CLICKHOUSE_LOCAL --input-format=Parquet -q "select count() from table" diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index d8d52a13218..3701c685120 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-parallel set output_format_parquet_use_custom_encoder = 1; set output_format_parquet_row_group_size = 1000; From 6b8752f2931fed6483d9221b6f5388e302245f31 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 25 Jul 2023 12:19:35 +0200 Subject: [PATCH 1130/2047] fix error message --- src/Functions/FunctionToDecimalString.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h index 68ad978632e..c16a72115d6 100644 --- a/src/Functions/FunctionToDecimalString.h +++ b/src/Functions/FunctionToDecimalString.h @@ -41,7 +41,7 @@ public: { FunctionArgumentDescriptors mandatory_args = { {"Value", nullptr, nullptr, nullptr}, - {"precision", &isNativeInteger, &isColumnConst, "const Integer [0-77]"} + {"precision", &isNativeInteger, &isColumnConst, "const Integer"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, {}); From 5ee71bd643caf26b9f533dab1e369f9dc306296b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 25 Jul 2023 10:26:26 +0000 Subject: [PATCH 1131/2047] Work around the clang bug --- src/Processors/Formats/Impl/Parquet/Write.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index 9664d173f29..47ef0c53ab5 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -651,6 +651,10 @@ void writeColumnImpl( const typename ParquetDType::c_type * converted = converter.getBatch(next_data_offset, data_count); if (options.write_page_statistics || options.write_column_chunk_statistics) +/// Workaround for clang bug: https://github.com/llvm/llvm-project/issues/63630 +#ifdef MEMORY_SANITIZER +#pragma clang loop vectorize(disable) +#endif for (size_t i = 0; i < data_count; ++i) page_statistics.add(converted[i]); From 155b90c780733a7712956982367088d856ec139b Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Tue, 25 Jul 2023 13:47:59 +0300 Subject: [PATCH 1132/2047] oops --- src/Functions/FunctionToDecimalString.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h index c16a72115d6..a965e2c2c90 100644 --- a/src/Functions/FunctionToDecimalString.h +++ b/src/Functions/FunctionToDecimalString.h @@ -19,10 +19,8 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } class FunctionToDecimalString : public IFunction From f8c90d5964a4c27dc119fd4417c23785a40b9c5e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 25 Jul 2023 13:36:57 +0200 Subject: [PATCH 1133/2047] Make better --- docs/en/sql-reference/transactions.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/transactions.md b/docs/en/sql-reference/transactions.md index 1ca2db44b13..cb89a091d68 100644 --- a/docs/en/sql-reference/transactions.md +++ b/docs/en/sql-reference/transactions.md @@ -5,7 +5,7 @@ slug: /en/guides/developer/transactional ## Case 1: INSERT into one partition, of one table, of the MergeTree* family -This is transactional (ACID) if the number of rows inserted is less than or equal to `max_insert_block_size rows`, and in the case of data in TSV, TKSV, CSV, or JSONEachRow format if the number of bytes is less than `min_chunk_bytes_for_parallel_parsing`: +This is transactional (ACID) if the inserted rows are packed and inserted as a single block (see Notes): - Atomic: an INSERT succeeds or is rejected as a whole: if a confirmation is sent to the client, then all rows were inserted; if an error is sent to the client, then no rows were inserted. - Consistent: if there are no table constraints violated, then all rows in an INSERT are inserted and the INSERT succeeds; if constraints are violated, then no rows are inserted. - Isolated: concurrent clients observe a consistent snapshot of the table–the state of the table either as it was before the INSERT attempt, or after the successful INSERT; no partial state is seen @@ -33,14 +33,16 @@ Same as Case 1 above, with this detail: - atomicity is ensured even if `async_insert` is enabled and `wait_for_async_insert` is set to 1 (the default), but if `wait_for_async_insert` is set to 0, then atomicity is not ensured. ## Notes -- `max_insert_block_size` is 1 000 000 by default and can be adjusted as needed -- `min_chunk_bytes_for_parallel_parsing` is 1 000 000 by default and can be adjusted as needed +- rows inserted from the client in some data format are packed into a single block when: + - the insert format is row-based (like CSV, TSV, Values, JSONEachRow, etc) and the data contains less then `max_insert_block_size` rows (~1 000 000 by default) or less then `min_chunk_bytes_for_parallel_parsing` bytes (10 MB by default) in case of parallel parsing is used (enabled by default) + - the insert format is column-based (like Native, Parquet, ORC, etc) and the data contains only one block of data +- the size of the inserted block in general may depend on many settings (for example: `max_block_size`, `max_insert_block_size`, `min_insert_block_size_rows`, `min_insert_block_size_bytes`, `preferred_block_size_bytes`, etc) - if the client did not receive an answer from the server, the client does not know if the transaction succeeded, and it can repeat the transaction, using exactly-once insertion properties - ClickHouse is using MVCC with snapshot isolation internally - all ACID properties are valid even in the case of server kill/crash - either insert_quorum into different AZ or fsync should be enabled to ensure durable inserts in the typical setup - "consistency" in ACID terms does not cover the semantics of distributed systems, see https://jepsen.io/consistency which is controlled by different settings (select_sequential_consistency) -- this explanation does not cover a new transactions feature that allow to have full-featured transactions over multiple tables, materialized views, for multiple SELECTs, etc. (see the next section on Transactions, Commit, and Rollback). +- this explanation does not cover a new transactions feature that allow to have full-featured transactions over multiple tables, materialized views, for multiple SELECTs, etc. (see the next section on Transactions, Commit, and Rollback) ## Transactions, Commit, and Rollback From 93e5d7f51c561af4d9236ef7e146b94754bc8fd8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 25 Jul 2023 11:42:22 +0000 Subject: [PATCH 1134/2047] Fix flaky 00995_exception_while_insert --- tests/queries/0_stateless/00995_exception_while_insert.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00995_exception_while_insert.sh b/tests/queries/0_stateless/00995_exception_while_insert.sh index 927ac6a54e5..732dba6c6f1 100755 --- a/tests/queries/0_stateless/00995_exception_while_insert.sh +++ b/tests/queries/0_stateless/00995_exception_while_insert.sh @@ -7,8 +7,8 @@ CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS check;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64, y UInt64 DEFAULT throwIf(x > 1500000)) ENGINE = Memory;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64, y UInt64 DEFAULT throwIf(x = 1500000)) ENGINE = Memory;" -seq 1 2000000 | $CLICKHOUSE_CLIENT --query="INSERT INTO check(x) FORMAT TSV" 2>&1 | grep -q "Value passed to 'throwIf' function is non-zero." && echo 'OK' || echo 'FAIL' ||: +seq 1 1500000 | $CLICKHOUSE_CLIENT --query="INSERT INTO check(x) FORMAT TSV" 2>&1 | grep -q "Value passed to 'throwIf' function is non-zero." && echo 'OK' || echo 'FAIL' ||: $CLICKHOUSE_CLIENT --query="DROP TABLE check;" From 22fec136c132de820c07c32d0508e6c67af51050 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 25 Jul 2023 14:04:55 +0200 Subject: [PATCH 1135/2047] Create new StorageView with substituted parameters for every SELECT query of a parameterized view --- src/Interpreters/Context.cpp | 18 +++++++- src/Interpreters/InterpreterSelectQuery.cpp | 24 +--------- .../TranslateQualifiedNamesVisitor.cpp | 7 --- .../TranslateQualifiedNamesVisitor.h | 6 +-- src/Interpreters/TreeRewriter.cpp | 21 ++------- src/Interpreters/TreeRewriter.h | 5 +-- src/Storages/SelectQueryInfo.h | 1 - src/Storages/StorageSnapshot.cpp | 8 +--- src/Storages/StorageSnapshot.h | 2 +- src/Storages/StorageView.cpp | 44 ++----------------- src/Storages/StorageView.h | 11 +---- 11 files changed, 33 insertions(+), 114 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index cc77e0fe723..123c2ab8f85 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -114,7 +114,10 @@ #include #include #include +#include +#include #include +#include #if USE_ROCKSDB #include @@ -1576,8 +1579,21 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const { if (table.get()->isView() && table->as() && table->as()->isParameterizedView()) { + auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); + NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression); + StorageView::replaceQueryParametersIfParametrizedView(query,parameterized_view_values); + + ASTCreateQuery create; + create.select = query->as(); + auto sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(query, getQueryContext()); + auto res = std::make_shared(StorageID(database_name, table_name), + create, + ColumnsDescription(sample_block.getNamesAndTypesList()), + /* comment */ "", + /* is_parameterized_view */ true); + res->startup(); function->prefer_subquery_to_function_formatting = true; - return table; + return res; } } auto hash = table_expression->getTreeHash(); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d07a6521544..23a879a9426 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -610,27 +610,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( { /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. ASTPtr view_table; - NameToNameMap parameter_types; if (view) { query_info.is_parameterized_view = view->isParameterizedView(); - /// We need to fetch the parameters set for SELECT ... FROM parameterized_view() before the query is replaced. - /// replaceWithSubquery replaces the function child and adds the subquery in its place. - /// the parameters are children of function child, if function (which corresponds to parametrised view and has - /// parameters in its arguments: `parametrised_view()`) is replaced the parameters are also gone from tree - /// So we need to get the parameters before they are removed from the tree - /// and after query is replaced, we use these parameters to substitute in the parameterized view query - if (query_info.is_parameterized_view) - { - query_info.parameterized_view_values = analyzeFunctionParamValues(query_ptr); - parameter_types = view->getParameterTypes(); - } view->replaceWithSubquery(getSelectQuery(), view_table, metadata_snapshot, view->isParameterizedView()); - if (query_info.is_parameterized_view) - { - view->replaceQueryParametersIfParametrizedView(query_ptr, query_info.parameterized_view_values); - } - } syntax_analyzer_result = TreeRewriter(context).analyzeSelect( @@ -639,10 +622,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( options, joined_tables.tablesWithColumns(), required_result_column_names, - table_join, - query_info.is_parameterized_view, - query_info.parameterized_view_values, - parameter_types); + table_join); query_info.syntax_analyzer_result = syntax_analyzer_result; @@ -793,7 +773,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.filter_asts.push_back(parallel_replicas_custom_filter_ast); } - source_header = storage_snapshot->getSampleBlockForColumns(required_columns, query_info.parameterized_view_values); + source_header = storage_snapshot->getSampleBlockForColumns(required_columns); } /// Calculate structure of the result. diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index aeb912ddfbb..130ce2194fd 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -249,13 +249,6 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt if (first_table || !data.join_using_columns.contains(column.name)) { std::string column_name = column.name; - - /// replaceQueryParameterWithValue is used for parameterized view (which are created using query parameters - /// and SELECT is used with substitution of these query parameters ) - if (!data.parameter_values.empty()) - column_name - = StorageView::replaceQueryParameterWithValue(column_name, data.parameter_values, data.parameter_types); - addIdentifier(columns, table.table, column_name); } } diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.h b/src/Interpreters/TranslateQualifiedNamesVisitor.h index 6c804ad6c90..73e45fc7ea0 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -28,15 +28,11 @@ public: const TablesWithColumns & tables; std::unordered_set join_using_columns; bool has_columns; - NameToNameMap parameter_values; - NameToNameMap parameter_types; - Data(const NameSet & source_columns_, const TablesWithColumns & tables_, bool has_columns_ = true, const NameToNameMap & parameter_values_ = {}, const NameToNameMap & parameter_types_ = {}) + Data(const NameSet & source_columns_, const TablesWithColumns & tables_, bool has_columns_ = true) : source_columns(source_columns_) , tables(tables_) , has_columns(has_columns_) - , parameter_values(parameter_values_) - , parameter_types(parameter_types_) {} bool hasColumn(const String & name) const { return source_columns.count(name); } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 65b5d950975..d44d6cc6ac8 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -299,11 +299,10 @@ using ReplacePositionalArgumentsVisitor = InDepthNodeVisitor table_join, - bool is_parameterized_view, - const NameToNameMap parameter_values, - const NameToNameMap parameter_types) const + std::shared_ptr table_join) const { auto * select_query = query->as(); if (!select_query) @@ -1201,7 +1197,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.analyzed_join->setColumnsFromJoinedTable(std::move(columns_from_joined_table), source_columns_set, right_table.table.getQualifiedNamePrefix()); } - translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns, parameter_values, parameter_types); + translateQualifiedNames(query, *select_query, source_columns_set, tables_with_columns); /// Optimizes logical expressions. LogicalExpressionsOptimizer(select_query, tables_with_columns, settings.optimize_min_equality_disjunction_chain_length.value).perform(); @@ -1259,15 +1255,6 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.window_function_asts = getWindowFunctions(query, *select_query); result.expressions_with_window_function = getExpressionsWithWindowFunctions(query); - /// replaceQueryParameterWithValue is used for parameterized view (which are created using query parameters - /// and SELECT is used with substitution of these query parameters ) - /// the replaced column names will be used in the next steps - if (is_parameterized_view) - { - for (auto & column : result.source_columns) - column.name = StorageView::replaceQueryParameterWithValue(column.name, parameter_values, parameter_types); - } - result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key); result.required_source_columns_before_expanding_alias_columns = result.required_source_columns.getNames(); diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index ea16c432d0f..a171133cd08 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -128,10 +128,7 @@ public: const SelectQueryOptions & select_options = {}, const std::vector & tables_with_columns = {}, const Names & required_result_columns = {}, - std::shared_ptr table_join = {}, - bool is_parameterized_view = false, - const NameToNameMap parameter_values = {}, - const NameToNameMap parameter_types = {}) const; + std::shared_ptr table_join = {}) const; private: static void normalize(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, bool ignore_alias, const Settings & settings, bool allow_self_aliases, ContextPtr context_, bool is_create_parameterized_view = false); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 13d6909fd52..0f75562e0c1 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -258,7 +258,6 @@ struct SelectQueryInfo bool parallel_replicas_disabled = false; bool is_parameterized_view = false; - NameToNameMap parameterized_view_values; // If limit is not 0, that means it's a trivial limit query. UInt64 limit = 0; diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index 31770c9a32b..6abca59268f 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -113,7 +113,7 @@ NameAndTypePair StorageSnapshot::getColumn(const GetColumnsOptions & options, co return *column; } -Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names, const NameToNameMap & parameter_values) const +Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) const { Block res; @@ -121,12 +121,6 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names, cons for (const auto & column_name : column_names) { std::string substituted_column_name = column_name; - - /// substituted_column_name is used for parameterized view (which are created using query parameters - /// and SELECT is used with substitution of these query parameters ) - if (!parameter_values.empty()) - substituted_column_name = StorageView::replaceValueWithQueryParameter(column_name, parameter_values); - auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name); auto object_column = object_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name); if (column && !object_column) diff --git a/src/Storages/StorageSnapshot.h b/src/Storages/StorageSnapshot.h index 946e8a98cf2..a69f9b95955 100644 --- a/src/Storages/StorageSnapshot.h +++ b/src/Storages/StorageSnapshot.h @@ -71,7 +71,7 @@ struct StorageSnapshot NameAndTypePair getColumn(const GetColumnsOptions & options, const String & column_name) const; /// Block with ordinary + materialized + aliases + virtuals + subcolumns. - Block getSampleBlockForColumns(const Names & column_names, const NameToNameMap & parameter_values = {}) const; + Block getSampleBlockForColumns(const Names & column_names) const; ColumnsDescription getDescriptionForColumns(const Names & column_names) const; diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 470def7e197..f0f9b9540de 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -107,7 +107,8 @@ StorageView::StorageView( const StorageID & table_id_, const ASTCreateQuery & query, const ColumnsDescription & columns_, - const String & comment) + const String & comment, + const bool is_parameterized_view_) : IStorage(table_id_) { StorageInMemoryMetadata storage_metadata; @@ -123,8 +124,7 @@ StorageView::StorageView( NormalizeSelectWithUnionQueryVisitor::Data data{SetOperationMode::Unspecified}; NormalizeSelectWithUnionQueryVisitor{data}.visit(description.inner_query); - is_parameterized_view = query.isParameterizedView(); - view_parameter_types = analyzeReceiveQueryParamsWithType(description.inner_query); + is_parameterized_view = is_parameterized_view_ || query.isParameterizedView(); storage_metadata.setSelectQuery(description); setInMemoryMetadata(storage_metadata); } @@ -173,7 +173,7 @@ void StorageView::read( query_plan.addStep(std::move(materializing)); /// And also convert to expected structure. - const auto & expected_header = storage_snapshot->getSampleBlockForColumns(column_names, query_info.parameterized_view_values); + const auto & expected_header = storage_snapshot->getSampleBlockForColumns(column_names); const auto & header = query_plan.getCurrentDataStream().header; const auto * select_with_union = current_inner_query->as(); @@ -258,42 +258,6 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_ child = view_query; } -String StorageView::replaceQueryParameterWithValue(const String & column_name, const NameToNameMap & parameter_values, const NameToNameMap & parameter_types) -{ - std::string name = column_name; - std::string::size_type pos = 0u; - for (const auto & parameter : parameter_values) - { - if ((pos = name.find(parameter.first)) != std::string::npos) - { - auto parameter_datatype_iterator = parameter_types.find(parameter.first); - size_t parameter_end = pos + parameter.first.size(); - if (parameter_datatype_iterator != parameter_types.end() && name.size() >= parameter_end && (name[parameter_end] == ',' || name[parameter_end] == ')')) - { - String parameter_name("_CAST(" + parameter.second + ", '" + parameter_datatype_iterator->second + "')"); - name.replace(pos, parameter.first.size(), parameter_name); - break; - } - } - } - return name; -} - -String StorageView::replaceValueWithQueryParameter(const String & column_name, const NameToNameMap & parameter_values) -{ - String name = column_name; - std::string::size_type pos = 0u; - for (const auto & parameter : parameter_values) - { - if ((pos = name.find("_CAST(" + parameter.second)) != std::string::npos) - { - name = name.substr(0,pos) + parameter.first + ")"; - break; - } - } - return name; -} - ASTPtr StorageView::restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name) { ASTTableExpression * table_expression = getFirstTableExpression(select_query); diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index bebecb79ec0..b8bf5585c0f 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -15,7 +15,8 @@ public: const StorageID & table_id_, const ASTCreateQuery & query, const ColumnsDescription & columns_, - const String & comment); + const String & comment, + const bool is_parameterized_view_=false); std::string getName() const override { return "View"; } bool isView() const override { return true; } @@ -44,17 +45,9 @@ public: static void replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_query, ASTPtr & view_name, const bool parameterized_view); static ASTPtr restoreViewName(ASTSelectQuery & select_query, const ASTPtr & view_name); - static String replaceQueryParameterWithValue (const String & column_name, const NameToNameMap & parameter_values, const NameToNameMap & parameter_types); - static String replaceValueWithQueryParameter (const String & column_name, const NameToNameMap & parameter_values); - - const NameToNameMap & getParameterTypes() const - { - return view_parameter_types; - } protected: bool is_parameterized_view; - NameToNameMap view_parameter_types; }; } From ed5393ef035f37ab3acd193c69333ff7a0084be7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 25 Jul 2023 12:09:13 +0000 Subject: [PATCH 1136/2047] Stabilize tests --- .../0_stateless/02494_query_cache_query_log.reference | 6 +----- .../queries/0_stateless/02494_query_cache_query_log.sql | 9 ++++++--- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_query_log.reference b/tests/queries/0_stateless/02494_query_cache_query_log.reference index 9037909d121..f9429064456 100644 --- a/tests/queries/0_stateless/02494_query_cache_query_log.reference +++ b/tests/queries/0_stateless/02494_query_cache_query_log.reference @@ -1,16 +1,12 @@ -- Run a query with query cache not enabled 124437993 -QueryStart SELECT 124437993; Unknown QueryFinish SELECT 124437993; None -- Run a query with query cache enabled 124437994 -QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Write -- Run the same query with query cache enabled 124437994 -QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown -QueryStart SELECT 124437994 SETTINGS use_query_cache = 1; Unknown -QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Read QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Write +QueryFinish SELECT 124437994 SETTINGS use_query_cache = 1; Read -- Throw exception with query cache enabled SELECT 124437995, throwIf(1) SETTINGS use_query_cache = 1; None diff --git a/tests/queries/0_stateless/02494_query_cache_query_log.sql b/tests/queries/0_stateless/02494_query_cache_query_log.sql index 79a8f4cb62b..aedc39c4486 100644 --- a/tests/queries/0_stateless/02494_query_cache_query_log.sql +++ b/tests/queries/0_stateless/02494_query_cache_query_log.sql @@ -17,7 +17,8 @@ SELECT type, query, query_cache_usage FROM system.query_log WHERE current_database = currentDatabase() AND query = 'SELECT 124437993;' -ORDER BY type; + AND type = 'QueryFinish' +ORDER BY type, query_cache_usage; @@ -31,7 +32,8 @@ SELECT type, query, query_cache_usage FROM system.query_log WHERE current_database = currentDatabase() AND query = 'SELECT 124437994 SETTINGS use_query_cache = 1;' -ORDER BY type; + AND type = 'QueryFinish' +ORDER BY type, query_cache_usage; @@ -45,7 +47,8 @@ SELECT type, query, query_cache_usage FROM system.query_log WHERE current_database = currentDatabase() AND query = 'SELECT 124437994 SETTINGS use_query_cache = 1;' -ORDER BY type; + AND type = 'QueryFinish' +ORDER BY type, query_cache_usage; From 568afbbec318d5c38c8281f8ef4bd5873fa76c42 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jul 2023 09:56:29 +0000 Subject: [PATCH 1137/2047] added field with refcount to system.remote_data_paths table --- src/Disks/IDisk.h | 5 ++- .../ObjectStorages/DiskObjectStorage.cpp | 2 +- .../System/StorageSystemRemoteDataPaths.cpp | 8 +++- .../02791_remote_paths_refcount.reference | 28 ++++++++++++ .../02791_remote_paths_refcount.sql | 43 +++++++++++++++++++ 5 files changed, 82 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02791_remote_paths_refcount.reference create mode 100644 tests/queries/0_stateless/02791_remote_paths_refcount.sql diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 2b0ca369a96..a2c5e59237f 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -303,10 +303,11 @@ public: std::string local_path; std::string common_prefix_for_objects; StoredObjects objects; + size_t refcount; LocalPathWithObjectStoragePaths( - const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_) - : local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)) {} + const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_, size_t refcount_) + : local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)), refcount(refcount_) {} }; virtual void getRemotePathsRecursive(const String &, std::vector &) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 762151b3808..001cff4cefe 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -82,7 +82,7 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std:: { try { - paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path)); + paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path), metadata_storage->getHardlinkCount(local_path)); } catch (const Exception & e) { diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index eb514d3b3f4..820b1cf3823 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -25,6 +25,7 @@ StorageSystemRemoteDataPaths::StorageSystemRemoteDataPaths(const StorageID & tab {"local_path", std::make_shared()}, {"remote_path", std::make_shared()}, {"size", std::make_shared()}, + {"refcount", std::make_shared()}, {"common_prefix_for_blobs", std::make_shared()}, {"cache_paths", std::make_shared(std::make_shared())}, })); @@ -48,6 +49,7 @@ Pipe StorageSystemRemoteDataPaths::read( MutableColumnPtr col_local_path = ColumnString::create(); MutableColumnPtr col_remote_path = ColumnString::create(); MutableColumnPtr col_size = ColumnUInt64::create(); + MutableColumnPtr col_refcount = ColumnUInt64::create(); MutableColumnPtr col_namespace = ColumnString::create(); MutableColumnPtr col_cache_paths = ColumnArray::create(ColumnString::create()); @@ -65,19 +67,22 @@ Pipe StorageSystemRemoteDataPaths::read( if (disk->supportsCache()) cache = FileCacheFactory::instance().getByName(disk->getCacheName()).cache; - for (const auto & [local_path, common_prefox_for_objects, storage_objects] : remote_paths_by_local_path) + for (const auto & [local_path, common_prefox_for_objects, storage_objects, refcount] : remote_paths_by_local_path) { for (const auto & object : storage_objects) { col_disk_name->insert(disk_name); col_base_path->insert(disk->getPath()); + if (cache) col_cache_base_path->insert(cache->getBasePath()); else col_cache_base_path->insertDefault(); + col_local_path->insert(local_path); col_remote_path->insert(object.remote_path); col_size->insert(object.bytes_size); + col_refcount->insert(refcount); col_namespace->insert(common_prefox_for_objects); if (cache) @@ -101,6 +106,7 @@ Pipe StorageSystemRemoteDataPaths::read( res_columns.emplace_back(std::move(col_local_path)); res_columns.emplace_back(std::move(col_remote_path)); res_columns.emplace_back(std::move(col_size)); + res_columns.emplace_back(std::move(col_refcount)); res_columns.emplace_back(std::move(col_namespace)); res_columns.emplace_back(std::move(col_cache_paths)); diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.reference b/tests/queries/0_stateless/02791_remote_paths_refcount.reference new file mode 100644 index 00000000000..56fb1536205 --- /dev/null +++ b/tests/queries/0_stateless/02791_remote_paths_refcount.reference @@ -0,0 +1,28 @@ +0_0_0_0 0 +0_0_0_0_1 1 +1_0_0_0 0 +1_0_0_0_1 1 +0_0_0_0_1 checksums.txt 0 +0_0_0_0_1 columns.txt 1 +0_0_0_0_1 count.txt 1 +0_0_0_0_1 default_compression_codec.txt 1 +0_0_0_0_1 id.bin 1 +0_0_0_0_1 id.cmrk2 1 +0_0_0_0_1 metadata_version.txt 1 +0_0_0_0_1 minmax_id.idx 1 +0_0_0_0_1 partition.dat 1 +0_0_0_0_1 primary.cidx 1 +0_0_0_0_1 v.bin 1 +0_0_0_0_1 v.cmrk2 1 +1_0_0_0_1 checksums.txt 0 +1_0_0_0_1 columns.txt 0 +1_0_0_0_1 count.txt 1 +1_0_0_0_1 default_compression_codec.txt 0 +1_0_0_0_1 id.bin 1 +1_0_0_0_1 id.cmrk2 1 +1_0_0_0_1 metadata_version.txt 0 +1_0_0_0_1 minmax_id.idx 1 +1_0_0_0_1 partition.dat 1 +1_0_0_0_1 primary.cidx 1 +1_0_0_0_1 v.bin 0 +1_0_0_0_1 v.cmrk2 0 diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.sql b/tests/queries/0_stateless/02791_remote_paths_refcount.sql new file mode 100644 index 00000000000..e64df599d32 --- /dev/null +++ b/tests/queries/0_stateless/02791_remote_paths_refcount.sql @@ -0,0 +1,43 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS t_refcount SYNC; + +CREATE TABLE t_refcount (id UInt64, v UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/test/{database}/t_refcount', '1') +ORDER BY id PARTITION BY id % 2 +SETTINGS + storage_policy = 's3_cache', + allow_remote_fs_zero_copy_replication = 1, + min_bytes_for_wide_part = 0, + compress_marks = 1, + compress_primary_key = 1, + ratio_of_defaults_for_sparse_serialization = 1.0; + +INSERT INTO t_refcount VALUES (1, 10), (2, 20); + +SET mutations_sync = 2; +ALTER TABLE t_refcount UPDATE v = v * 10 WHERE id % 2 = 1; + +SELECT name, active FROM system.parts WHERE database = currentDatabase() AND table = 't_refcount' ORDER BY name; + +WITH splitByChar('/', full_path) AS path_parts +SELECT path_parts[-2] AS part_name, path_parts[-1] AS file_name, refcount +FROM +( + SELECT + path || local_path AS full_path, + substring(full_path, 1, length(full_path) - position(reverse(full_path), '/') + 1) AS part_path, + refcount + FROM system.remote_data_paths + WHERE disk_name = 's3_cache' +) AS paths +INNER JOIN +( + SELECT path + FROM system.parts + WHERE database = currentDatabase() AND table = 't_refcount' AND active +) AS parts +ON paths.part_path = parts.path +ORDER BY part_name, file_name; + +DROP TABLE IF EXISTS t_refcount SYNC; From 23897b823d7a98c122e385ed9452f8e930132825 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Jul 2023 20:41:16 +0800 Subject: [PATCH 1138/2047] fix code style --- docs/en/sql-reference/functions/string-functions.md | 2 +- .../02415_all_new_functions_must_be_documented.reference | 2 ++ utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index a398a7c0993..f36de0dc0d8 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -695,7 +695,7 @@ endsWith(str, suffix) ## endsWithUTF8 -Returns whether string `str` ends with `suffix`, assuming that both strings contain svalid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns whether string `str` ends with `suffix`, assuming that both strings contain valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. **Syntax** diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 247df64f741..d70214e9930 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -263,6 +263,7 @@ encodeURLComponent encodeURLFormComponent encodeXMLComponent endsWith +endsWithUTF8 equals erf erfc @@ -665,6 +666,7 @@ splitByString splitByWhitespace sqrt startsWith +startsWithUTF8 subBitmap substring substringUTF8 diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 65271830555..3cc25108eb1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1394,6 +1394,7 @@ encodings encryptions endian endsWith +endsWithUTF8 enum enum's enums @@ -2200,6 +2201,7 @@ src stacktrace stacktraces startsWith +startsWithUTF8 statbox stateful stddev From 328d0a5269407eef6899907d6b9869307a56dfa4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 25 Jul 2023 14:50:27 +0200 Subject: [PATCH 1139/2047] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++++--- .../test.py | 4 +++- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e6431927805..9e4a63f6ba9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4861,9 +4861,13 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread) LOG_TRACE(log, "Waiting for RestartingThread to startup table"); } - std::lock_guard lock{flush_and_shutdown_mutex}; - if (shutdown_prepared_called.load() || shutdown_called.load()) - throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot startup table because it is dropped"); + auto lock = std::unique_lock(flush_and_shutdown_mutex, std::defer_lock); + do + { + if (shutdown_prepared_called.load() || shutdown_called.load()) + throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Cannot startup table because it is dropped"); + } + while (!lock.try_lock()); /// And this is just a callback session_expired_callback_handler = EventNotifier::instance().subscribe(Coordination::Error::ZSESSIONEXPIRED, [this]() diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index 20b6a6c977f..d971e4ec658 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -3,6 +3,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry from multiprocessing.dummy import Pool import time @@ -54,9 +55,10 @@ def test_shutdown_and_wait(start_cluster): node1.query(f"INSERT INTO test_table VALUES ({value})") with PartitionManager() as pm: + assert node2.query("SELECT * FROM test_table") == "0\n" pm.partition_instances(node1, node2) # iptables rules must be applied immediately, but looks like sometimes they are not... - time.sleep(3) + assert_eq_with_retry(node1, "select count() from remote('node1,node2', 'system.one')", "1\n", settings={"skip_unavailable_shards": 1}) p.map(insert, range(1, 50)) From d7de8bf797a7444927e80c7c88d9b7c5a4040e01 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 25 Jul 2023 13:03:12 +0000 Subject: [PATCH 1140/2047] Automatic style fix --- .../test_replicated_merge_tree_wait_on_shutdown/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index d971e4ec658..d1373d44d0f 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -58,7 +58,12 @@ def test_shutdown_and_wait(start_cluster): assert node2.query("SELECT * FROM test_table") == "0\n" pm.partition_instances(node1, node2) # iptables rules must be applied immediately, but looks like sometimes they are not... - assert_eq_with_retry(node1, "select count() from remote('node1,node2', 'system.one')", "1\n", settings={"skip_unavailable_shards": 1}) + assert_eq_with_retry( + node1, + "select count() from remote('node1,node2', 'system.one')", + "1\n", + settings={"skip_unavailable_shards": 1}, + ) p.map(insert, range(1, 50)) From b91852de3a311cd03ef571e4470deba3deeba25b Mon Sep 17 00:00:00 2001 From: Julian Maicher Date: Tue, 25 Jul 2023 16:01:19 +0200 Subject: [PATCH 1141/2047] fix(docs): Document correct MODIFY COLUMN REMOVE syntax --- docs/en/sql-reference/statements/alter/column.md | 2 +- docs/ru/sql-reference/statements/alter/column.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index dae2c7dd1d3..6ceb9b5849e 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -213,7 +213,7 @@ Removes one of the column properties: `DEFAULT`, `ALIAS`, `MATERIALIZED`, `CODEC Syntax: ```sql -ALTER TABLE table_name MODIFY column_name REMOVE property; +ALTER TABLE table_name MODIFY COLUMN column_name REMOVE property; ``` **Example** diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index a8ace213075..92be30b101a 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -182,7 +182,7 @@ ALTER TABLE visits MODIFY COLUMN browser Array(String) Синтаксис: ```sql -ALTER TABLE table_name MODIFY column_name REMOVE property; +ALTER TABLE table_name MODIFY COLUMN column_name REMOVE property; ``` **Пример** From bd09ad6736bac2b9e986993e75f1f8f61b1508a6 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Tue, 25 Jul 2023 16:19:44 +0200 Subject: [PATCH 1142/2047] MaterializedMySQL: Fix typos in tests --- .../materialized_with_ddl.py | 27 +++++++++++++------ .../test_materialized_mysql_database/test.py | 9 ++++--- 2 files changed, 24 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 8b2943c2b73..389d430622d 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -13,25 +13,36 @@ from multiprocessing.dummy import Pool from helpers.test_tools import assert_eq_with_retry -def check_query(clickhouse_node, query, result_set, retry_count=10, interval_seconds=3): - lastest_result = "" +def check_query( + clickhouse_node, + query, + result_set, + retry_count=30, + interval_seconds=1, + on_failure=None, +): + latest_result = "" + if "/* expect: " not in query: + query = "/* expect: " + result_set.rstrip("\n") + "*/ " + query for i in range(retry_count): try: - lastest_result = clickhouse_node.query(query) - if result_set == lastest_result: + latest_result = clickhouse_node.query(query) + if result_set == latest_result: return - logging.debug(f"latest_result {lastest_result}") + logging.debug(f"latest_result {latest_result}") time.sleep(interval_seconds) except Exception as e: logging.debug(f"check_query retry {i+1} exception {e}") time.sleep(interval_seconds) else: - result_got = clickhouse_node.query(query) + latest_result = clickhouse_node.query(query) + if on_failure is not None and latest_result != result_set: + on_failure(latest_result, result_set) assert ( - result_got == result_set - ), f"Got result {result_got}, while expected result {result_set}" + latest_result == result_set + ), f"Got result '{latest_result}', expected result '{result_set}'" def dml_with_materialized_mysql_database(clickhouse_node, mysql_node, service_name): diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index c21e04af8db..1fd09f733f0 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -52,6 +52,7 @@ def started_cluster(): cluster.start() yield cluster finally: + node_db.stop_clickhouse() # ensures that coverage report is written to disk, even if cluster.shutdown() times out. cluster.shutdown() @@ -86,7 +87,7 @@ class MySQLConnection: else: self.mysql_connection.ping(reconnect=True) logging.debug( - "MySQL Connection establised: {}:{}".format( + "MySQL Connection established: {}:{}".format( self.ip_address, self.port ) ) @@ -94,7 +95,7 @@ class MySQLConnection: except Exception as e: errors += [str(e)] time.sleep(1) - raise Exception("Connection not establised, {}".format(errors)) + raise Exception("Connection not established, {}".format(errors)) def query(self, execution_query): with self.alloc_connection().cursor() as cursor: @@ -118,9 +119,9 @@ class MySQLConnection: if result is not None: print(cursor.fetchall()) - def query_and_get_data(self, executio_query): + def query_and_get_data(self, execution_query): with self.alloc_connection().cursor() as cursor: - cursor.execute(executio_query) + cursor.execute(execution_query) return cursor.fetchall() def close(self): From 2c7c38950d54c009e5268d371dabe8035b817283 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 25 Jul 2023 14:21:12 +0000 Subject: [PATCH 1143/2047] better check for lightweight deletes --- src/Storages/MergeTree/MergeTreeData.cpp | 7 +++++-- .../02792_drop_projection_lwd.reference | 2 +- .../0_stateless/02792_drop_projection_lwd.sql | 16 +++++----------- 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 06a9b62d9de..6179c70ca57 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5693,8 +5693,11 @@ bool MergeTreeData::supportsLightweightDelete() const auto lock = lockParts(); for (const auto & part : data_parts_by_info) { - if (part->getState() == MergeTreeDataPartState::Active - && !part->supportLightweightDeleteMutate()) + if (part->getState() == MergeTreeDataPartState::Outdated + || part->getState() == MergeTreeDataPartState::Deleting) + continue; + + if (!part->supportLightweightDeleteMutate()) return false; } return true; diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.reference b/tests/queries/0_stateless/02792_drop_projection_lwd.reference index 6529ff889b0..3ad5abd03ae 100644 --- a/tests/queries/0_stateless/02792_drop_projection_lwd.reference +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.reference @@ -1 +1 @@ -98 +99 diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.sql b/tests/queries/0_stateless/02792_drop_projection_lwd.sql index fd446a8efe8..a1d8a9c90f3 100644 --- a/tests/queries/0_stateless/02792_drop_projection_lwd.sql +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.sql @@ -1,23 +1,17 @@ +SET mutations_sync = 2; + DROP TABLE IF EXISTS t_projections_lwd; -CREATE TABLE t_projections_lwd (a UInt32, b UInt32) ENGINE = MergeTree ORDER BY a; +CREATE TABLE t_projections_lwd (a UInt32, b UInt32, PROJECTION p (SELECT * ORDER BY b)) ENGINE = MergeTree ORDER BY a; INSERT INTO t_projections_lwd SELECT number, number FROM numbers(100); --- LWD works -DELETE FROM t_projections_lwd WHERE a = 0; - --- add projection -ALTER TABLE t_projections_lwd ADD PROJECTION p_t_projections_lwd (SELECT * ORDER BY b); -ALTER TABLE t_projections_lwd MATERIALIZE PROJECTION p_t_projections_lwd; - -- LWD does not work, as expected -DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError UNFINISHED } +DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError BAD_ARGUMENTS } KILL MUTATION WHERE database = currentDatabase() AND table = 't_projections_lwd' SYNC FORMAT Null; -- drop projection -SET mutations_sync = 2; -ALTER TABLE t_projections_lwd DROP projection p_t_projections_lwd; +ALTER TABLE t_projections_lwd DROP projection p; DELETE FROM t_projections_lwd WHERE a = 2; From 79d0343becaa001dca587ee1932a8520e086d0ce Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Jul 2023 16:34:40 +0200 Subject: [PATCH 1144/2047] tests: fix 01821_join_table_race_long flakiness (#52559) By grouping multiple queries into one clickhouse-client invocation, since each execve of the binary can take ~1 second in debug builds. But this slightly changes the logic, so be aware. Signed-off-by: Azat Khuzhin Co-authored-by: Alexander Tokmakov --- tests/queries/0_stateless/01821_join_table_race_long.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01821_join_table_race_long.sh b/tests/queries/0_stateless/01821_join_table_race_long.sh index e02fe788653..561b856841b 100755 --- a/tests/queries/0_stateless/01821_join_table_race_long.sh +++ b/tests/queries/0_stateless/01821_join_table_race_long.sh @@ -9,13 +9,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS join_table_race" $CLICKHOUSE_CLIENT -q "CREATE TABLE join_table_race(id Int32, name String) ENGINE = Join(ANY, LEFT, id)" -for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "INSERT INTO join_table_race VALUES ($RANDOM, '$RANDOM')" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do echo "INSERT INTO join_table_race VALUES ($RANDOM, '$RANDOM');"; done | $CLICKHOUSE_CLIENT --ignore-error -nm > /dev/null 2> /dev/null & -for _ in {0..200}; do $CLICKHOUSE_CLIENT -q "SELECT count() FROM join_table_race FORMAT Null" > /dev/null 2> /dev/null; done & +for _ in {0..200}; do echo "SELECT count() FROM join_table_race FORMAT Null;"; done | $CLICKHOUSE_CLIENT --ignore-error -nm > /dev/null 2> /dev/null & -for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE join_table_race" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do echo "TRUNCATE TABLE join_table_race;"; done | $CLICKHOUSE_CLIENT --ignore-error -nm > /dev/null 2> /dev/null & -for _ in {0..100}; do $CLICKHOUSE_CLIENT -q "ALTER TABLE join_table_race DELETE WHERE id % 2 = 0" > /dev/null 2> /dev/null; done & +for _ in {0..100}; do echo "ALTER TABLE join_table_race DELETE WHERE id % 2 = 0;"; done | $CLICKHOUSE_CLIENT --ignore-error -nm > /dev/null 2> /dev/null & wait From 85082ad8f8ee0d1023273d8db888e143e59bd828 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 25 Jul 2023 16:35:01 +0200 Subject: [PATCH 1145/2047] Fix data-race DatabaseReplicated::startupTables()/canExecuteReplicatedMetadataAlter() (#52490) CI founds [1]: Exception: Sanitizer assert found for instance ================== WARNING: ThreadSanitizer: data race (pid=348) Write of size 8 at 0x7b58000044a0 by main thread: 2 DB::DatabaseReplicated::startupTables(ThreadPoolImpl>&, DB::LoadingStrictnessLevel) build_docker/./src/Databases/DatabaseReplicated.cpp:526:16 (clickhouse+0x1ec45092) 3 DB::TablesLoader::startupTables() build_docker/./src/Databases/TablesLoader.cpp:87:26 (clickhouse+0x1f9258ab) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 4 DB::loadMetadata(std::__1::shared_ptr, std::__1::basic_string, std::__1::allocator> const&) build_docker/./src/Interpreters/loadMetadata.cpp:234:12 (clickhouse+0x1fff3834) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 5 DB::Server::main() build_docker/./programs/server/Server.cpp:1615:9 (clickhouse+0x163e7f78) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 6 Poco::Util::Application::run() build_docker/./base/poco/Util/src/Application.cpp:315:8 (clickhouse+0x257608fe) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 7 DB::Server::run() build_docker/./programs/server/Server.cpp:391:25 (clickhouse+0x163d7d7c) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 8 Poco::Util::ServerApplication::run(int, char**) build_docker/./base/poco/Util/src/ServerApplication.cpp:131:9 (clickhouse+0x25780114) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 9 mainEntryClickHouseServer(int, char**) build_docker/./programs/server/Server.cpp:196:20 (clickhouse+0x163d4c23) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 10 main build_docker/./programs/main.cpp:487:12 (clickhouse+0xdf8c877) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) Previous read of size 8 at 0x7b58000044a0 by thread T27 (mutexes: write M0, write M1): 1 DB::DatabaseReplicated::canExecuteReplicatedMetadataAlter() const build_docker/./src/Databases/DatabaseReplicated.cpp:1303:12 (clickhouse+0x1ec5c5bd) 2 DB::ReplicatedMergeTreeQueue::shouldExecuteLogEntry() const build_docker/./src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp:1471:24 (clickhouse+0x2115fb56) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 3 DB::ReplicatedMergeTreeQueue::selectEntryToProcess(DB::MergeTreeDataMergerMutator&, DB::MergeTreeData&) build_docker/./src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp:1676:13 (clickhouse+0x21163c58) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 4 DB::StorageReplicatedMergeTree::selectQueueEntry() build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:3240:26 (clickhouse+0x20823db2) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) 5 DB::StorageReplicatedMergeTree::scheduleDataProcessingJob(DB::BackgroundJobsAssignee&) build_docker/./src/Storages/StorageReplicatedMergeTree.cpp:3304:65 (clickhouse+0x208240fc) (BuildId: 7d4ce55d33d4c3e3df9fd39b304e67e53eb61a63) [1]: https://s3.amazonaws.com/clickhouse-test-reports/52395/0b258dda4ee618a4d002e2b5246d68bbd2c77c7e/integration_tests__tsan__[5_6].html Add ddl_worker_initialized flag to avoid this race. Note, that it should be enough to check this flag only in canExecuteReplicatedMetadataAlter() since only it can be run in parallel with ctor before it had been finished. v0: initialize ddl before startupTables() v2: ddl_worker_initialized Signed-off-by: Azat Khuzhin Co-authored-by: Alexander Tokmakov --- src/Databases/DatabaseReplicated.cpp | 4 +++- src/Databases/DatabaseReplicated.h | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index d3b3d4b545f..ed56edd7503 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -524,6 +524,7 @@ void DatabaseReplicated::startupTables(ThreadPool & thread_pool, LoadingStrictne ddl_worker = std::make_unique(this, getContext()); ddl_worker->startup(); + ddl_worker_initialized = true; } bool DatabaseReplicated::checkDigestValid(const ContextPtr & local_context, bool debug_check /* = true */) const @@ -1155,6 +1156,7 @@ void DatabaseReplicated::stopReplication() void DatabaseReplicated::shutdown() { stopReplication(); + ddl_worker_initialized = false; ddl_worker = nullptr; DatabaseAtomic::shutdown(); } @@ -1299,7 +1301,7 @@ bool DatabaseReplicated::canExecuteReplicatedMetadataAlter() const /// It may update the metadata digest (both locally and in ZooKeeper) /// before DatabaseReplicatedDDLWorker::initializeReplication() has finished. /// We should not update metadata until the database is initialized. - return ddl_worker && ddl_worker->isCurrentlyActive(); + return ddl_worker_initialized && ddl_worker->isCurrentlyActive(); } void DatabaseReplicated::detachTablePermanently(ContextPtr local_context, const String & table_name) diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 8e33f482ac1..7ba91e48085 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -134,6 +134,7 @@ private: std::atomic_bool is_readonly = true; std::atomic_bool is_probably_dropped = false; std::atomic_bool is_recovering = false; + std::atomic_bool ddl_worker_initialized = false; std::unique_ptr ddl_worker; UInt32 max_log_ptr_at_creation = 0; From c75b5bc740cd20ee7f5e6bb5a71b9f8e215eb03c Mon Sep 17 00:00:00 2001 From: Sanjam Panda <36253777+saitama951@users.noreply.github.com> Date: Tue, 25 Jul 2023 20:12:22 +0530 Subject: [PATCH 1146/2047] Update TwoLevelStringHashTable.h --- .../HashTable/TwoLevelStringHashTable.h | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Common/HashTable/TwoLevelStringHashTable.h b/src/Common/HashTable/TwoLevelStringHashTable.h index ee6dcd05d9a..0527ec67e6e 100644 --- a/src/Common/HashTable/TwoLevelStringHashTable.h +++ b/src/Common/HashTable/TwoLevelStringHashTable.h @@ -114,18 +114,18 @@ public: { memcpy(&n[0], p, 8); if constexpr (std::endian::native == std::endian::little) - n[0] &= -1ULL >> s; - else - n[0] &= -1ULL << s; + n[0] &= -1ULL >> s; + else + n[0] &= -1ULL << s; } else { const char * lp = x.data + x.size - 8; memcpy(&n[0], lp, 8); if constexpr (std::endian::native == std::endian::little) - n[0] >>= s; - else - n[0] <<= s; + n[0] >>= s; + else + n[0] <<= s; } auto res = hash(k8); auto buck = getBucketFromHash(res); @@ -138,9 +138,9 @@ public: const char * lp = x.data + x.size - 8; memcpy(&n[1], lp, 8); if constexpr (std::endian::native == std::endian::little) - n[1] >>= s; + n[1] >>= s; else - n[1] <<= s; + n[1] <<= s; auto res = hash(k16); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); @@ -152,9 +152,9 @@ public: const char * lp = x.data + x.size - 8; memcpy(&n[2], lp, 8); if constexpr (std::endian::native == std::endian::little) - n[2] >>= s; + n[2] >>= s; else - n[2] <<= s; + n[2] <<= s; auto res = hash(k24); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); From 11016d4c5f36fa39a36c2c2b6c0eec7c1c3dfd5f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 25 Jul 2023 16:46:50 +0200 Subject: [PATCH 1147/2047] Revert "Rewrite uniq to count" --- src/Analyzer/Passes/UniqToCountPass.cpp | 198 ------------------ src/Analyzer/Passes/UniqToCountPass.h | 30 --- src/Analyzer/QueryTreePassManager.cpp | 2 - src/Core/Settings.h | 1 - src/Interpreters/InterpreterSelectQuery.cpp | 7 - .../RewriteUniqToCountVisitor.cpp | 163 -------------- src/Interpreters/RewriteUniqToCountVisitor.h | 30 --- .../test_rewrite_uniq_to_count/__init__.py | 0 .../test_rewrite_uniq_to_count/test.py | 127 ----------- 9 files changed, 558 deletions(-) delete mode 100644 src/Analyzer/Passes/UniqToCountPass.cpp delete mode 100644 src/Analyzer/Passes/UniqToCountPass.h delete mode 100644 src/Interpreters/RewriteUniqToCountVisitor.cpp delete mode 100644 src/Interpreters/RewriteUniqToCountVisitor.h delete mode 100644 tests/integration/test_rewrite_uniq_to_count/__init__.py delete mode 100644 tests/integration/test_rewrite_uniq_to_count/test.py diff --git a/src/Analyzer/Passes/UniqToCountPass.cpp b/src/Analyzer/Passes/UniqToCountPass.cpp deleted file mode 100644 index 7533a99107b..00000000000 --- a/src/Analyzer/Passes/UniqToCountPass.cpp +++ /dev/null @@ -1,198 +0,0 @@ -#include "UniqToCountPass.h" - -#include -#include - -#include -#include -#include -#include - -namespace DB -{ - -namespace -{ - -bool matchFnUniq(String func_name) -{ - auto name = Poco::toLower(func_name); - return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" - || name == "uniqCombined64"; -} - -/// Extract the corresponding projection columns for group by node list. -/// For example: -/// SELECT a as aa, any(b) FROM table group by a; -> aa(ColumnNode) -NamesAndTypes extractProjectionColumnsForGroupBy(const QueryNode * query_node) -{ - if (!query_node->hasGroupBy()) - return {}; - - NamesAndTypes result; - for (const auto & group_by_ele : query_node->getGroupByNode()->getChildren()) - { - const auto & projection_columns = query_node->getProjectionColumns(); - const auto & projection_nodes = query_node->getProjection().getNodes(); - - assert(projection_columns.size() == projection_nodes.size()); - - for (size_t i = 0; i < projection_columns.size(); i++) - { - if (projection_nodes[i]->isEqual(*group_by_ele)) - result.push_back(projection_columns[i]); - } - } - return result; -} - -/// Whether query_columns equals subquery_columns. -/// query_columns: query columns from query -/// subquery_columns: projection columns from subquery -bool nodeListEquals(const QueryTreeNodes & query_columns, const NamesAndTypes & subquery_columns) -{ - if (query_columns.size() != subquery_columns.size()) - return false; - - for (const auto & query_column : query_columns) - { - auto find = std::find_if( - subquery_columns.begin(), - subquery_columns.end(), - [&](const auto & subquery_column) -> bool - { - if (auto * column_node = query_column->as()) - { - return subquery_column == column_node->getColumn(); - } - return false; - }); - - if (find == subquery_columns.end()) - return false; - } - return true; -} - -/// Whether subquery_columns contains all columns in subquery_columns. -/// query_columns: query columns from query -/// subquery_columns: projection columns from subquery -bool nodeListContainsAll(const QueryTreeNodes & query_columns, const NamesAndTypes & subquery_columns) -{ - if (query_columns.size() > subquery_columns.size()) - return false; - - for (const auto & query_column : query_columns) - { - auto find = std::find_if( - subquery_columns.begin(), - subquery_columns.end(), - [&](const auto & subquery_column) -> bool - { - if (auto * column_node = query_column->as()) - { - return subquery_column == column_node->getColumn(); - } - return false; - }); - - if (find == subquery_columns.end()) - return false; - } - return true; -} - -} - -class UniqToCountVisitor : public InDepthQueryTreeVisitor -{ -public: - using Base = InDepthQueryTreeVisitor; - using Base::Base; - - void visitImpl(QueryTreeNodePtr & node) - { - auto * query_node = node->as(); - if (!query_node) - return; - - /// Check that query has only single table expression which is subquery - auto * subquery_node = query_node->getJoinTree()->as(); - if (!subquery_node) - return; - - /// Check that query has only single node in projection - auto & projection_nodes = query_node->getProjection().getNodes(); - if (projection_nodes.size() != 1) - return; - - /// Check that projection_node is a function - auto & projection_node = projection_nodes[0]; - auto * function_node = projection_node->as(); - if (!function_node) - return; - - /// Check that query single projection node is `uniq` or its variants - if (!matchFnUniq(function_node->getFunctionName())) - return; - - auto & uniq_arguments_nodes = function_node->getArguments().getNodes(); - - /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' - auto match_subquery_with_distinct = [&]() -> bool - { - if (!subquery_node->isDistinct()) - return false; - - /// uniq expression list == subquery projection columns - if (!nodeListEquals(uniq_arguments_nodes, subquery_node->getProjectionColumns())) - return false; - - return true; - }; - - /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' - auto match_subquery_with_group_by = [&]() -> bool - { - if (!subquery_node->hasGroupBy()) - return false; - - /// uniq argument node list == subquery group by node list - auto group_by_columns = extractProjectionColumnsForGroupBy(subquery_node); - - if (!nodeListEquals(uniq_arguments_nodes, group_by_columns)) - return false; - - /// subquery projection columns must contain all columns in uniq argument node list - if (!nodeListContainsAll(uniq_arguments_nodes, subquery_node->getProjectionColumns())) - return false; - - return true; - }; - - /// Replace uniq of initial query to count - if (match_subquery_with_distinct() || match_subquery_with_group_by()) - { - AggregateFunctionProperties properties; - auto aggregate_function = AggregateFunctionFactory::instance().get("count", {}, {}, properties); - - function_node->resolveAsAggregateFunction(std::move(aggregate_function)); - function_node->getArguments().getNodes().clear(); - - /// Update projection columns - query_node->resolveProjectionColumns({{"count()", function_node->getResultType()}}); - } - } -}; - - -void UniqToCountPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) -{ - if (!context->getSettings().optimize_uniq_to_count) - return; - - UniqToCountVisitor visitor; - visitor.visit(query_tree_node); -} - -} diff --git a/src/Analyzer/Passes/UniqToCountPass.h b/src/Analyzer/Passes/UniqToCountPass.h deleted file mode 100644 index 4992d524e5e..00000000000 --- a/src/Analyzer/Passes/UniqToCountPass.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/** Optimize `uniq` and its variants(except uniqUpTo) into `count` over subquery. - * Example: 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to - * Result: 'SELECT count() FROM (SELECT DISTINCT x ...)' - * - * Example: 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to - * Result: 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' - * - * Note that we can rewrite all uniq variants except uniqUpTo. - */ -class UniqToCountPass final : public IQueryTreePass -{ -public: - String getName() override { return "UniqToCount"; } - - String getDescription() override - { - return "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause."; - } - - void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override; -}; - -} diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index dd75b0f586d..a6da2a66615 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -247,7 +246,6 @@ void addQueryTreePasses(QueryTreePassManager & manager) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2ead00cafb4..8bebef5fb00 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -776,7 +776,6 @@ class IColumn; M(Bool, keeper_map_strict_mode, false, "Enforce additional checks during operations on KeeperMap. E.g. throw an exception on an insert for already existing key", 0) \ M(UInt64, extract_kvp_max_pairs_per_row, 1000, "Max number pairs that can be produced by extractKeyValuePairs function. Used to safeguard against consuming too much memory.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ - M(Bool, optimize_uniq_to_count, false, "Rewrite uniq and its variants(except uniqUpTo) to count if subquery has distinct or group by clause.", 0) \ M(Bool, allow_create_index_without_type, false, "Allow CREATE INDEX query without TYPE. Query will be ignored. Made for SQL compatibility tests.", 0)\ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8402165b62b..fc3ea3a13ca 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -39,7 +39,6 @@ #include #include #include -#include #include #include @@ -427,12 +426,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( RewriteCountDistinctFunctionVisitor(data_rewrite_countdistinct).visit(query_ptr); } - if (settings.optimize_uniq_to_count) - { - RewriteUniqToCountMatcher::Data data_rewrite_uniq_count; - RewriteUniqToCountVisitor(data_rewrite_uniq_count).visit(query_ptr); - } - JoinedTables joined_tables(getSubqueryContext(context), getSelectQuery(), options.with_all_cols, options_.is_create_parameterized_view); bool got_storage_from_query = false; diff --git a/src/Interpreters/RewriteUniqToCountVisitor.cpp b/src/Interpreters/RewriteUniqToCountVisitor.cpp deleted file mode 100644 index 7445068207a..00000000000 --- a/src/Interpreters/RewriteUniqToCountVisitor.cpp +++ /dev/null @@ -1,163 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -using Aliases = std::unordered_map; - -namespace -{ - -bool matchFnUniq(String func_name) -{ - auto name = Poco::toLower(func_name); - return name == "uniq" || name == "uniqHLL12" || name == "uniqExact" || name == "uniqTheta" || name == "uniqCombined" - || name == "uniqCombined64"; -} - -bool expressionEquals(const ASTPtr & lhs, const ASTPtr & rhs, const Aliases & alias) -{ - if (lhs->getTreeHash() == rhs->getTreeHash()) - { - return true; - } - else - { - auto * lhs_idf = lhs->as(); - auto * rhs_idf = rhs->as(); - if (lhs_idf && rhs_idf) - { - /// compound identifiers, such as: - if (lhs_idf->shortName() == rhs_idf->shortName()) - return true; - - /// translate alias - if (alias.find(lhs_idf->shortName()) != alias.end()) - lhs_idf = alias.find(lhs_idf->shortName())->second->as(); - - if (alias.find(rhs_idf->shortName()) != alias.end()) - rhs_idf = alias.find(rhs_idf->shortName())->second->as(); - - if (lhs_idf->shortName() == rhs_idf->shortName()) - return true; - } - } - return false; -} - -bool expressionListEquals(ASTExpressionList * lhs, ASTExpressionList * rhs, const Aliases & alias) -{ - if (!lhs || !rhs) - return false; - if (lhs->children.size() != rhs->children.size()) - return false; - for (size_t i = 0; i < lhs->children.size(); i++) - { - if (!expressionEquals(lhs->children[i], rhs->children[i], alias)) - return false; - } - return true; -} - -/// Test whether lhs contains all expressions in rhs. -bool expressionListContainsAll(ASTExpressionList * lhs, ASTExpressionList * rhs, const Aliases & alias) -{ - if (!lhs || !rhs) - return false; - if (lhs->children.size() < rhs->children.size()) - return false; - for (const auto & re : rhs->children) - { - auto predicate = [&re, &alias](ASTPtr & le) { return expressionEquals(le, re, alias); }; - if (std::find_if(lhs->children.begin(), lhs->children.end(), predicate) == lhs->children.end()) - return false; - } - return true; -} - -} - -void RewriteUniqToCountMatcher::visit(ASTPtr & ast, Data & /*data*/) -{ - auto * selectq = ast->as(); - if (!selectq || !selectq->tables() || selectq->tables()->children.size() != 1) - return; - auto expr_list = selectq->select(); - if (!expr_list || expr_list->children.size() != 1) - return; - auto * func = expr_list->children[0]->as(); - if (!func || !matchFnUniq(func->name)) - return; - if (selectq->tables()->as()->children[0]->as()->children.size() != 1) - return; - auto * table_expr = selectq->tables() - ->as() - ->children[0] - ->as() - ->children[0] - ->as(); - if (!table_expr || table_expr->children.size() != 1 || !table_expr->subquery) - return; - auto * subquery = table_expr->subquery->as(); - if (!subquery) - return; - auto * sub_selectq = subquery->children[0] - ->as()->children[0] - ->as()->children[0] - ->as(); - if (!sub_selectq) - return; - auto sub_expr_list = sub_selectq->select(); - if (!sub_expr_list) - return; - - /// collect subquery select expressions alias - Aliases alias; - for (const auto & expr : sub_expr_list->children) - { - if (!expr->tryGetAlias().empty()) - alias.insert({expr->tryGetAlias(), expr}); - } - - /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' - auto match_subquery_with_distinct = [&]() -> bool - { - if (!sub_selectq->distinct) - return false; - /// uniq expression list == subquery group by expression list - if (!expressionListEquals(func->children[0]->as(), sub_expr_list->as(), alias)) - return false; - return true; - }; - - /// Whether query matches 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' - auto match_subquery_with_group_by = [&]() -> bool - { - auto group_by = sub_selectq->groupBy(); - if (!group_by) - return false; - /// uniq expression list == subquery group by expression list - if (!expressionListEquals(func->children[0]->as(), group_by->as(), alias)) - return false; - /// subquery select expression list must contain all columns in uniq expression list - if (!expressionListContainsAll(sub_expr_list->as(), func->children[0]->as(), alias)) - return false; - return true; - }; - - if (match_subquery_with_distinct() || match_subquery_with_group_by()) - expr_list->children[0] = makeASTFunction("count"); -} - -} diff --git a/src/Interpreters/RewriteUniqToCountVisitor.h b/src/Interpreters/RewriteUniqToCountVisitor.h deleted file mode 100644 index 94528ccf2ee..00000000000 --- a/src/Interpreters/RewriteUniqToCountVisitor.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include -#include "Interpreters/TreeRewriter.h" - -namespace DB -{ - -class ASTFunction; - -/** Optimize `uniq` into `count` over subquery. - * Example: 'SELECT uniq(x ...) FROM (SELECT DISTINCT x ...)' to - * Result: 'SELECT count() FROM (SELECT DISTINCT x ...)' - * - * Example: 'SELECT uniq(x ...) FROM (SELECT x ... GROUP BY x ...)' to - * Result: 'SELECT count() FROM (SELECT x ... GROUP BY x ...)' - * - * Note that we can rewrite all uniq variants except uniqUpTo. - */ -class RewriteUniqToCountMatcher -{ -public: - struct Data {}; - static void visit(ASTPtr & ast, Data &); - static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } -}; - -using RewriteUniqToCountVisitor = InDepthNodeVisitor; -} diff --git a/tests/integration/test_rewrite_uniq_to_count/__init__.py b/tests/integration/test_rewrite_uniq_to_count/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_rewrite_uniq_to_count/test.py b/tests/integration/test_rewrite_uniq_to_count/test.py deleted file mode 100644 index e38e57f5cee..00000000000 --- a/tests/integration/test_rewrite_uniq_to_count/test.py +++ /dev/null @@ -1,127 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node") - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - prepare() - yield cluster - finally: - shutdown() - cluster.shutdown() - - -def prepare(): - node.query( - """ - CREATE TABLE IF NOT EXISTS test_rewrite_uniq_to_count - ( - `a` UInt8, - `b` UInt8, - `c` UInt8 - ) - ENGINE = MergeTree - ORDER BY `a` - """ - ) - node.query( - "INSERT INTO test_rewrite_uniq_to_count values ('1', '1', '1'), ('1', '1', '1')" - ) - node.query( - "INSERT INTO test_rewrite_uniq_to_count values ('2', '2', '2'), ('2', '2', '2')" - ) - node.query( - "INSERT INTO test_rewrite_uniq_to_count values ('3', '3', '3'), ('3', '3', '3')" - ) - - -def shutdown(): - node.query("DROP TABLE IF EXISTS test_rewrite_uniq_to_count SYNC") - - -def check(query, result): - # old analyzer - query = query + " settings optimize_uniq_to_count = 1" - assert node.query(query) == f"{result}\n" - assert "count()" in node.query("EXPLAIN SYNTAX " + query) - - # new analyzer - query = query + ", allow_experimental_analyzer = 1" - assert node.query(query) == f"{result}\n" - assert "count()" in node.query("EXPLAIN QUERY TREE " + query) - - -def check_by_old_analyzer(query, result): - # only old analyzer - query = query + " settings optimize_uniq_to_count = 1" - assert node.query(query) == f"{result}\n" - assert "count()" in node.query("EXPLAIN SYNTAX " + query) - - -def test_rewrite_distinct(started_cluster): - # simple test - check( - "SELECT uniq(a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count)", - 3, - ) - - # test subquery alias - check( - "SELECT uniq(t.a) FROM (SELECT DISTINCT a FROM test_rewrite_uniq_to_count) t", - 3, - ) - - # test compound column name - check( - "SELECT uniq(a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a FROM test_rewrite_uniq_to_count) t", - 3, - ) - - # test select expression alias - check( - "SELECT uniq(alias_of_a) FROM (SELECT DISTINCT test_rewrite_uniq_to_count.a as alias_of_a FROM test_rewrite_uniq_to_count) t", - 3, - ) - - # test select expression alias - check( - "SELECT uniq(alias_of_a) FROM (SELECT DISTINCT a as alias_of_a FROM test_rewrite_uniq_to_count) t", - 3, - ) - - -def test_rewrite_group_by(started_cluster): - # simple test - check( - "SELECT uniq(a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a)", - 3, - ) - - # test subquery alias - check( - "SELECT uniq(t.a) FROM (SELECT a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3, - ) - - # test select expression alias - check( - "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3, - ) - - # test select expression alias - check( - "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY alias_of_a) t", - 3, - ) - - # test select expression alias - check( - "SELECT uniq(t.alias_of_a) FROM (SELECT a as alias_of_a, sum(b) FROM test_rewrite_uniq_to_count GROUP BY a) t", - 3, - ) From c341df1949e96f938620f22d9fa064514611046b Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 24 Jul 2023 01:14:50 +0300 Subject: [PATCH 1148/2047] draft with retry ConnectionResetException --- src/IO/S3/Client.cpp | 86 ++++++++++++++++--- src/IO/S3/Client.h | 4 + .../test_checking_s3_blobs_paranoid/test.py | 6 +- 3 files changed, 80 insertions(+), 16 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 7e20b1a9e8f..94a7b5166da 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -13,6 +13,8 @@ #include #include +#include + #include #include #include @@ -23,6 +25,15 @@ #include +namespace ProfileEvents +{ + extern const Event S3WriteRequestsErrors; + extern const Event S3ReadRequestsErrors; + + extern const Event DiskS3WriteRequestsErrors; + extern const Event DiskS3ReadRequestsErrors; +} + namespace DB { @@ -346,12 +357,12 @@ Model::HeadObjectOutcome Client::HeadObject(const HeadObjectRequest & request) c Model::ListObjectsV2Outcome Client::ListObjectsV2(const ListObjectsV2Request & request) const { - return doRequest(request, [this](const Model::ListObjectsV2Request & req) { return ListObjectsV2(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::ListObjectsV2Request & req) { return ListObjectsV2(req); }); } Model::ListObjectsOutcome Client::ListObjects(const ListObjectsRequest & request) const { - return doRequest(request, [this](const Model::ListObjectsRequest & req) { return ListObjects(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::ListObjectsRequest & req) { return ListObjects(req); }); } Model::GetObjectOutcome Client::GetObject(const GetObjectRequest & request) const @@ -361,19 +372,19 @@ Model::GetObjectOutcome Client::GetObject(const GetObjectRequest & request) cons Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(const AbortMultipartUploadRequest & request) const { - return doRequest( + return doRequestWithRetryNetworkErrors( request, [this](const Model::AbortMultipartUploadRequest & req) { return AbortMultipartUpload(req); }); } Model::CreateMultipartUploadOutcome Client::CreateMultipartUpload(const CreateMultipartUploadRequest & request) const { - return doRequest( + return doRequestWithRetryNetworkErrors( request, [this](const Model::CreateMultipartUploadRequest & req) { return CreateMultipartUpload(req); }); } Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(const CompleteMultipartUploadRequest & request) const { - auto outcome = doRequest( + auto outcome = doRequestWithRetryNetworkErrors( request, [this](const Model::CompleteMultipartUploadRequest & req) { return CompleteMultipartUpload(req); }); if (!outcome.IsSuccess() || provider_type != ProviderType::GCS) @@ -403,32 +414,32 @@ Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(const Comp Model::CopyObjectOutcome Client::CopyObject(const CopyObjectRequest & request) const { - return doRequest(request, [this](const Model::CopyObjectRequest & req) { return CopyObject(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::CopyObjectRequest & req) { return CopyObject(req); }); } Model::PutObjectOutcome Client::PutObject(const PutObjectRequest & request) const { - return doRequest(request, [this](const Model::PutObjectRequest & req) { return PutObject(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::PutObjectRequest & req) { return PutObject(req); }); } Model::UploadPartOutcome Client::UploadPart(const UploadPartRequest & request) const { - return doRequest(request, [this](const Model::UploadPartRequest & req) { return UploadPart(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::UploadPartRequest & req) { return UploadPart(req); }); } Model::UploadPartCopyOutcome Client::UploadPartCopy(const UploadPartCopyRequest & request) const { - return doRequest(request, [this](const Model::UploadPartCopyRequest & req) { return UploadPartCopy(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::UploadPartCopyRequest & req) { return UploadPartCopy(req); }); } Model::DeleteObjectOutcome Client::DeleteObject(const DeleteObjectRequest & request) const { - return doRequest(request, [this](const Model::DeleteObjectRequest & req) { return DeleteObject(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::DeleteObjectRequest & req) { return DeleteObject(req); }); } Model::DeleteObjectsOutcome Client::DeleteObjects(const DeleteObjectsRequest & request) const { - return doRequest(request, [this](const Model::DeleteObjectsRequest & req) { return DeleteObjects(req); }); + return doRequestWithRetryNetworkErrors(request, [this](const Model::DeleteObjectsRequest & req) { return DeleteObjects(req); }); } Client::ComposeObjectOutcome Client::ComposeObject(const ComposeObjectRequest & request) const @@ -457,7 +468,7 @@ Client::ComposeObjectOutcome Client::ComposeObject(const ComposeObjectRequest & return ComposeObjectOutcome(MakeRequest(req, endpointResolutionOutcome.GetResult(), Aws::Http::HttpMethod::HTTP_PUT)); }; - return doRequest(request, request_fn); + return doRequestWithRetryNetworkErrors(request, request_fn); } template @@ -538,6 +549,57 @@ Client::doRequest(const RequestType & request, RequestFn request_fn) const throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, "Too many redirects"); } +template +std::invoke_result_t +Client::doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn request_fn) const +{ + auto with_retries = [this, request_fn_ = std::move(request_fn)] (const RequestType & request_) + { + const size_t max_tries = 10; + std::exception_ptr last_exception = nullptr; + for (size_t try_no = 0; try_no < max_tries; ++try_no) + { + try + { + return request_fn_(request_); + } + catch (Poco::Net::ConnectionResetException &) + { + // to do distinguish read/write + if (client_configuration.for_disk_s3) + { + ProfileEvents::increment(ProfileEvents::DiskS3WriteRequestsErrors); + ProfileEvents::increment(ProfileEvents::DiskS3ReadRequestsErrors); + } + else + { + ProfileEvents::increment(ProfileEvents::S3WriteRequestsErrors); + ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors); + } + + tryLogCurrentException(log, "Will retry"); + // to do back off + last_exception = std::current_exception(); + continue; + } + } + + chassert(last_exception); + std::rethrow_exception(last_exception); + +// try +// { +// std::rethrow_exception(last_exception); +// } +// catch (const Poco::Exception & e) +// { +// throw Exception(Exception::CreateFromPocoTag{}, e); +// } + }; + + return doRequest(request, with_retries); +} + bool Client::supportsMultiPartCopy() const { return provider_type != ProviderType::GCS; diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 8904c850553..bb893c21774 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -250,6 +250,10 @@ private: std::invoke_result_t doRequest(const RequestType & request, RequestFn request_fn) const; + template + std::invoke_result_t + doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn request_fn) const; + void updateURIForBucket(const std::string & bucket, S3::URI new_uri) const; std::optional getURIFromError(const Aws::S3::S3Error & error) const; std::optional updateURIForBucketForHead(const std::string & bucket) const; diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 5f7a2a6cc7f..6aa149b9c7b 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -307,8 +307,7 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( assert "Code: 1000" in error, error assert ( "DB::Exception: Connection reset by peer." in error - or - "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error ), error @@ -386,6 +385,5 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( assert "Code: 1000" in error, error assert ( "DB::Exception: Connection reset by peer." in error - or - "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error ), error From 2cc1ac45dd8dda3385e2df1db9ea4fab1789a585 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 25 Jul 2023 18:45:56 +0200 Subject: [PATCH 1149/2047] update missed error --- src/Functions/FunctionToDecimalString.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h index a965e2c2c90..ce52d8b99f6 100644 --- a/src/Functions/FunctionToDecimalString.h +++ b/src/Functions/FunctionToDecimalString.h @@ -214,7 +214,7 @@ private: ColumnPtr executeType(const ColumnsWithTypeAndName & arguments) const { const auto * precision_col = checkAndGetColumn>(arguments[1].column.get()); - const auto * precision_col_const = typeid_cast(arguments[1].column.get()); + const auto * precision_col_const = checkAndGetColumnConst>(arguments[1].column.get()); auto result_col = ColumnString::create(); auto * result_col_string = assert_cast(result_col.get()); From 413ec520b3027d9f377aa1929a2855429994ffe3 Mon Sep 17 00:00:00 2001 From: Sanjam Panda Date: Tue, 25 Jul 2023 18:54:27 +0200 Subject: [PATCH 1150/2047] fix code style --- src/Common/HashTable/TwoLevelStringHashTable.h | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Common/HashTable/TwoLevelStringHashTable.h b/src/Common/HashTable/TwoLevelStringHashTable.h index 0527ec67e6e..54c208c5b60 100644 --- a/src/Common/HashTable/TwoLevelStringHashTable.h +++ b/src/Common/HashTable/TwoLevelStringHashTable.h @@ -113,20 +113,20 @@ public: if ((reinterpret_cast(p) & 2048) == 0) { memcpy(&n[0], p, 8); - if constexpr (std::endian::native == std::endian::little) + if constexpr (std::endian::native == std::endian::little) n[0] &= -1ULL >> s; else n[0] &= -1ULL << s; - } + } else { const char * lp = x.data + x.size - 8; memcpy(&n[0], lp, 8); - if constexpr (std::endian::native == std::endian::little) + if constexpr (std::endian::native == std::endian::little) n[0] >>= s; else n[0] <<= s; - } + } auto res = hash(k8); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); @@ -139,9 +139,9 @@ public: memcpy(&n[1], lp, 8); if constexpr (std::endian::native == std::endian::little) n[1] >>= s; - else + else n[1] <<= s; - auto res = hash(k16); + auto res = hash(k16); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); return func(self.impls[buck].m2, k16, res); @@ -153,9 +153,9 @@ public: memcpy(&n[2], lp, 8); if constexpr (std::endian::native == std::endian::little) n[2] >>= s; - else + else n[2] <<= s; - auto res = hash(k24); + auto res = hash(k24); auto buck = getBucketFromHash(res); keyHolderDiscardKey(key_holder); return func(self.impls[buck].m3, k24, res); From 4f7bdf308d215478a718e1fe3c157c043702213e Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 25 Jul 2023 18:57:54 +0200 Subject: [PATCH 1151/2047] add explicit else --- src/Functions/FunctionToDecimalString.h | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionToDecimalString.h b/src/Functions/FunctionToDecimalString.h index ce52d8b99f6..3dd946203cc 100644 --- a/src/Functions/FunctionToDecimalString.h +++ b/src/Functions/FunctionToDecimalString.h @@ -38,7 +38,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { FunctionArgumentDescriptors mandatory_args = { - {"Value", nullptr, nullptr, nullptr}, + {"Value", &isNumber, nullptr, "Number"}, {"precision", &isNativeInteger, &isColumnConst, "const Integer"} }; @@ -230,8 +230,10 @@ private: { if (precision_col_const) vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets, from_scale); - else + else if (precision_col) vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets, from_scale); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName()); } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); @@ -243,8 +245,11 @@ private: { if (precision_col_const) vectorConstant(from_col->getData(), precision_col_const->template getValue(), result_chars, result_offsets); - else + else if (precision_col) vectorVector(from_col->getData(), precision_col->getData(), result_chars, result_offsets); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of second argument of function formatDecimal", arguments[1].column->getName()); + } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function formatDecimal", arguments[0].column->getName()); From 59db21941034a287eea6c1016ed2ca83e6772774 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 25 Jul 2023 19:21:41 +0200 Subject: [PATCH 1152/2047] Fix possible error "Cannot drain connections: cancel first" --- src/QueryPipeline/RemoteQueryExecutor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index cd6f65b7b43..198c3265a84 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -591,8 +591,8 @@ void RemoteQueryExecutor::finish() /// Send the request to abort the execution of the request, if not already sent. tryCancel("Cancelling query because enough data has been read"); - /// If connections weren't created yet or query wasn't sent, nothing to do. - if (!connections || !sent_query) + /// If connections weren't created yet, query wasn't sent or was already finished, nothing to do. + if (!connections || !sent_query || finished) return; /// Get the remaining packets so that there is no out of sync in the connections to the replicas. From d78b3e560f13a6ba8b85b76e2f0d56bea44f2c62 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 25 Jul 2023 17:45:13 +0000 Subject: [PATCH 1153/2047] Fix 02497_trace_events_stress_long again --- .../0_stateless/02497_trace_events_stress_long.sh | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02497_trace_events_stress_long.sh b/tests/queries/0_stateless/02497_trace_events_stress_long.sh index 91f6a9bb541..c111ed40a29 100755 --- a/tests/queries/0_stateless/02497_trace_events_stress_long.sh +++ b/tests/queries/0_stateless/02497_trace_events_stress_long.sh @@ -45,4 +45,11 @@ thread2 $TIMEOUT >/dev/null & wait -$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' \ No newline at end of file +for _ in {1..10} +do + # process list is cleaned after everything is sent to client + # so this check can be run before process list is cleaned + # to avoid spurious failures we retry the check couple of times + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' && break + sleep 1 +done \ No newline at end of file From 20300804b13187447e8677573b46ee70175c98cc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 00:01:46 +0300 Subject: [PATCH 1154/2047] Update test.py --- .../test_replicated_merge_tree_wait_on_shutdown/test.py | 9 --------- 1 file changed, 9 deletions(-) diff --git a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py index d1373d44d0f..67dd03098e9 100644 --- a/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py +++ b/tests/integration/test_replicated_merge_tree_wait_on_shutdown/test.py @@ -55,16 +55,7 @@ def test_shutdown_and_wait(start_cluster): node1.query(f"INSERT INTO test_table VALUES ({value})") with PartitionManager() as pm: - assert node2.query("SELECT * FROM test_table") == "0\n" pm.partition_instances(node1, node2) - # iptables rules must be applied immediately, but looks like sometimes they are not... - assert_eq_with_retry( - node1, - "select count() from remote('node1,node2', 'system.one')", - "1\n", - settings={"skip_unavailable_shards": 1}, - ) - p.map(insert, range(1, 50)) # Start shutdown async From d85f9ddb35f02564fe9d04f20f0a3451530a2b4c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 00:03:08 +0300 Subject: [PATCH 1155/2047] Update parallel_skip.json --- tests/integration/parallel_skip.json | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 407fe7d1b01..1075fbaa0f8 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -69,6 +69,8 @@ "test_server_reload/test.py::test_remove_tcp_port", "test_keeper_map/test.py::test_keeper_map_without_zk", + + "test_replicated_merge_tree_wait_on_shutdown/test.py::test_shutdown_and_wait", "test_http_failover/test.py::test_url_destination_host_with_multiple_addrs", "test_http_failover/test.py::test_url_invalid_hostname", From 27f3c5cfa27f9aef5df563a9fd90089963afe26c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 26 Jul 2023 10:53:07 +0800 Subject: [PATCH 1156/2047] change as requested --- docs/en/sql-reference/functions/string-functions.md | 4 ++-- src/Functions/endsWithUTF8.cpp | 7 ++++++- src/Functions/startsWithUTF8.cpp | 7 ++++++- .../02415_all_new_functions_must_be_documented.reference | 2 -- 4 files changed, 14 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index f36de0dc0d8..b1ffb32e7cd 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -695,7 +695,7 @@ endsWith(str, suffix) ## endsWithUTF8 -Returns whether string `str` ends with `suffix`, assuming that both strings contain valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns whether string `str` ends with `suffix`, the difference between `endsWithUTF8` and `endsWith` is that `endsWithUTF8` match `str` and `suffix` by UTF-8 characters. **Syntax** @@ -727,7 +727,7 @@ SELECT startsWith('Spider-Man', 'Spi'); ## startsWithUTF8 -Returns whether string `str` starts with `prefix`, assuming that both string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns whether string `str` starts with `prefix`, the difference between `startsWithUTF8` and `startsWith` is that `startsWithUTF8` match `str` and `suffix` by UTF-8 characters. **Example** diff --git a/src/Functions/endsWithUTF8.cpp b/src/Functions/endsWithUTF8.cpp index 377683ce47c..1b042452298 100644 --- a/src/Functions/endsWithUTF8.cpp +++ b/src/Functions/endsWithUTF8.cpp @@ -10,7 +10,12 @@ using FunctionEndsWithUTF8 = FunctionStartsEndsWith; REGISTER_FUNCTION(EndsWithUTF8) { - factory.registerFunction(); + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Returns whether string `str` ends with `suffix`, the difference between `endsWithUTF8` and `endsWith` is that `endsWithUTF8` match `str` and `suffix` by UTF-8 characters. + )", + .examples{{"endsWithUTF8", "select endsWithUTF8('富强民主文明和谐', '富强');", ""}}, + .categories{"String"}}); } } diff --git a/src/Functions/startsWithUTF8.cpp b/src/Functions/startsWithUTF8.cpp index e623a55f4dc..3f46916d760 100644 --- a/src/Functions/startsWithUTF8.cpp +++ b/src/Functions/startsWithUTF8.cpp @@ -10,7 +10,12 @@ using FunctionStartsWithUTF8 = FunctionStartsEndsWith; REGISTER_FUNCTION(StartsWithUTF8) { - factory.registerFunction(); + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Returns whether string `str` starts with `prefix`, the difference between `startsWithUTF8` and `startsWith` is that `startsWithUTF8` match `str` and `suffix` by UTF-8 characters. + )", + .examples{{"startsWithUTF8", "select startsWithUTF8('富强民主文明和谐', '富强');", ""}}, + .categories{"String"}}); } } diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index d70214e9930..247df64f741 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -263,7 +263,6 @@ encodeURLComponent encodeURLFormComponent encodeXMLComponent endsWith -endsWithUTF8 equals erf erfc @@ -666,7 +665,6 @@ splitByString splitByWhitespace sqrt startsWith -startsWithUTF8 subBitmap substring substringUTF8 From 21e12ea0842578043fd4686e16c935052480bb80 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Jul 2023 00:53:45 -0400 Subject: [PATCH 1157/2047] Fix tests --- src/Access/IAccessStorage.cpp | 2 +- tests/integration/test_role/test.py | 2 +- tests/queries/0_stateless/01292_create_user.reference | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 11bc97f6cc3..db73db900aa 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -180,7 +180,7 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity) std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return *insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); + return insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); } std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 43a402d9fda..0e1bbea9cea 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -301,7 +301,7 @@ def test_introspection(): ) assert instance.query( - "SELECT * from system.role_grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, granted_role_name" + "SELECT user_name, role_name, granted_role_name, granted_role_is_default, with_admin_option from system.role_grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, granted_role_name" ) == TSV([["A", "\\N", "R1", 1, 0], ["B", "\\N", "R2", 1, 1]]) assert instance.query( diff --git a/tests/queries/0_stateless/01292_create_user.reference b/tests/queries/0_stateless/01292_create_user.reference index eb89a5ed38c..b7c30b304bf 100644 --- a/tests/queries/0_stateless/01292_create_user.reference +++ b/tests/queries/0_stateless/01292_create_user.reference @@ -106,10 +106,10 @@ CREATE USER u2_01292 DEFAULT ROLE r1_01292, r2_01292 SETTINGS readonly = 1 CREATE USER u3_01292 HOST LIKE \'%.%.myhost.com\' DEFAULT ROLE r1_01292, r2_01292 CREATE USER u4_01292 HOST LIKE \'%.%.myhost.com\' DEFAULT ROLE r1_01292, r2_01292 -- system.users -u1_01292 local directory plaintext_password {} [] ['localhost'] [] [] 1 [] [] -u2_01292 local directory no_password {} [] [] [] ['%.%.myhost.com'] 0 [] [] -u3_01292 local directory sha256_password {} ['192.169.1.1','192.168.0.0/16'] ['localhost'] [] [] 0 ['r1_01292'] [] -u4_01292 local directory double_sha1_password {} ['::/0'] [] [] [] 1 [] ['r1_01292'] +u1_01292 local_directory plaintext_password {} [] ['localhost'] [] [] 1 [] [] +u2_01292 local_directory no_password {} [] [] [] ['%.%.myhost.com'] 0 [] [] +u3_01292 local_directory sha256_password {} ['192.169.1.1','192.168.0.0/16'] ['localhost'] [] [] 0 ['r1_01292'] [] +u4_01292 local_directory double_sha1_password {} ['::/0'] [] [] [] 1 [] ['r1_01292'] -- system.settings_profile_elements \N u1_01292 \N 0 readonly 1 \N \N \N \N \N u2_01292 \N 0 \N \N \N \N \N default From 3928f7ef460f4f4603ceaa065733ac0a7ebc4d16 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Jul 2023 09:19:35 +0200 Subject: [PATCH 1158/2047] Remove peak memory usage from the final message in the client --- src/Common/ProgressIndication.cpp | 3 --- tests/queries/0_stateless/01921_test_progress_bar.py | 1 - 2 files changed, 4 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 960d864660c..5a1929d4ec2 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -101,9 +101,6 @@ void ProgressIndication::writeFinalProgress() << formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)"; else std::cout << ". "; - auto peak_memory_usage = getMemoryUsage().peak; - if (peak_memory_usage >= 0) - std::cout << "\nPeak memory usage (for query) " << formatReadableSizeWithBinarySuffix(peak_memory_usage) << "."; } void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) diff --git a/tests/queries/0_stateless/01921_test_progress_bar.py b/tests/queries/0_stateless/01921_test_progress_bar.py index 9ce2168e2ae..89eecbc3987 100755 --- a/tests/queries/0_stateless/01921_test_progress_bar.py +++ b/tests/queries/0_stateless/01921_test_progress_bar.py @@ -17,4 +17,3 @@ with client(name="client1>", log=log) as client1: client1.send("SELECT number FROM numbers(1000) FORMAT Null") client1.expect("Progress: 1\.00 thousand rows, 8\.00 KB .*" + end_of_block) client1.expect("0 rows in set. Elapsed: [\\w]{1}\.[\\w]{3} sec.") - client1.expect("Peak memory usage \(for query\) .*B" + end_of_block) From 2b91a9da78a3cc273a4ed90169da75c92409e06c Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 26 Jul 2023 07:51:26 +0000 Subject: [PATCH 1159/2047] Remove logging from test --- tests/integration/test_backup_s3_storage_class/test.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/tests/integration/test_backup_s3_storage_class/test.py b/tests/integration/test_backup_s3_storage_class/test.py index 3c0a0cf7706..2b11f20afc6 100644 --- a/tests/integration/test_backup_s3_storage_class/test.py +++ b/tests/integration/test_backup_s3_storage_class/test.py @@ -1,12 +1,6 @@ -import os -import logging - import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", From 991584506f11563d324051236e09bd7c1a3b12d4 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 26 Jul 2023 12:42:18 +0400 Subject: [PATCH 1160/2047] fix a bug when files are finalizated after first write --- src/Storages/MergeTree/GinIndexStore.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/GinIndexStore.cpp b/src/Storages/MergeTree/GinIndexStore.cpp index aa0c1fccbc3..91e831270d4 100644 --- a/src/Storages/MergeTree/GinIndexStore.cpp +++ b/src/Storages/MergeTree/GinIndexStore.cpp @@ -243,6 +243,15 @@ void GinIndexStore::finalize() { if (!current_postings.empty()) writeSegment(); + + if (metadata_file_stream) + metadata_file_stream->finalize(); + + if (dict_file_stream) + dict_file_stream->finalize(); + + if (postings_file_stream) + postings_file_stream->finalize(); } void GinIndexStore::initFileStreams() @@ -319,13 +328,8 @@ void GinIndexStore::writeSegment() current_segment.segment_id = getNextSegmentID(); metadata_file_stream->sync(); - metadata_file_stream->finalize(); - dict_file_stream->sync(); - dict_file_stream->finalize(); - postings_file_stream->sync(); - postings_file_stream->finalize(); } GinIndexStoreDeserializer::GinIndexStoreDeserializer(const GinIndexStorePtr & store_) From d83eabcf772d942c18c8d8b2dccb43aaad1bf235 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 26 Jul 2023 17:39:45 +0800 Subject: [PATCH 1161/2047] fix spell --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 3cc25108eb1..240238cab12 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1394,7 +1394,7 @@ encodings encryptions endian endsWith -endsWithUTF8 +endsWithUTF enum enum's enums @@ -2201,7 +2201,7 @@ src stacktrace stacktraces startsWith -startsWithUTF8 +startsWithUTF statbox stateful stddev From 93e10077bad715235dfe7d4da6d103ffbb30f55a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 26 Jul 2023 11:53:19 +0200 Subject: [PATCH 1162/2047] Fix attaching gdb in stress tests (#51445) * Fix attaching gdb in stress tests * Fix * Update run.sh * Try remove run_with_retry * Return run_with_retry * Don't set -e in run_with_retry if it was't set before * Update tests/ci/utils.lib * Fix bash --------- Co-authored-by: Alexander Tokmakov --- docker/test/stress/run.sh | 3 ++- docker/test/upgrade/run.sh | 1 + tests/ci/stress_tests.lib | 2 -- tests/ci/utils.lib | 11 +++++++++-- 4 files changed, 12 insertions(+), 5 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 4926967d2d2..9217fcfddd9 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -14,6 +14,7 @@ ln -s /usr/share/clickhouse-test/clickhouse-test /usr/bin/clickhouse-test # Stress tests and upgrade check uses similar code that was placed # in a separate bash library. See tests/ci/stress_tests.lib +source /usr/share/clickhouse-test/ci/attach_gdb.lib source /usr/share/clickhouse-test/ci/stress_tests.lib install_packages package_folder @@ -52,7 +53,7 @@ azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & start -shellcheck disable=SC2086 # No quotes because I want to split it into words. +# shellcheck disable=SC2086 # No quotes because I want to split it into words. /s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS chmod 777 -R /var/lib/clickhouse clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index b8061309342..73a2965bf44 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -16,6 +16,7 @@ ln -s /usr/share/clickhouse-test/ci/get_previous_release_tag.py /usr/bin/get_pre # Stress tests and upgrade check uses similar code that was placed # in a separate bash library. See tests/ci/stress_tests.lib +source /usr/share/clickhouse-test/ci/attach_gdb.lib source /usr/share/clickhouse-test/ci/stress_tests.lib azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log & diff --git a/tests/ci/stress_tests.lib b/tests/ci/stress_tests.lib index 190f3f39f9e..85b376ac39d 100644 --- a/tests/ci/stress_tests.lib +++ b/tests/ci/stress_tests.lib @@ -9,8 +9,6 @@ FAIL="\tFAIL\t\\N\t" FAILURE_CONTEXT_LINES=100 FAILURE_CONTEXT_MAX_LINE_WIDTH=300 -source attach_gdb.lib - function escaped() { # That's the simplest way I found to escape a string in bash. Yep, bash is the most convenient programming language. diff --git a/tests/ci/utils.lib b/tests/ci/utils.lib index b5ce4ae0d78..c90b7ebe6f6 100644 --- a/tests/ci/utils.lib +++ b/tests/ci/utils.lib @@ -2,6 +2,11 @@ function run_with_retry() { + if [[ $- =~ e ]]; then + set_e=true + else + set_e=false + fi set +e local total_retries="$1" @@ -12,7 +17,9 @@ function run_with_retry() until [ "$retry" -ge "$total_retries" ] do if "$@"; then - set -e + if $set_e; then + set -e + fi return else retry=$((retry + 1)) @@ -26,4 +33,4 @@ function run_with_retry() function fn_exists() { declare -F "$1" > /dev/null; -} \ No newline at end of file +} From 017d34d40fdd8fe5b03e993b030385ccb20b0ebc Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 28 Jun 2023 23:41:51 +0200 Subject: [PATCH 1163/2047] determine task size by prewhere columns --- src/Core/Settings.h | 1 + src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp | 9 ++++++--- src/Storages/MergeTree/MergeTreeReadPool.cpp | 6 ++++-- 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 24be644ee55..d14121a97a3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -673,6 +673,7 @@ class IColumn; M(UInt64, remote_read_min_bytes_for_seek, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes required for remote read (url, s3) to do seek, instead of read with ignore.", 0) \ M(UInt64, merge_tree_min_bytes_per_task_for_remote_reading, 4 * DBMS_DEFAULT_BUFFER_SIZE, "Min bytes to read per task.", 0) \ M(Bool, merge_tree_use_const_size_tasks_for_remote_reading, true, "Whether to use constant size tasks for reading from a remote table.", 0) \ + M(Bool, merge_tree_determine_task_size_by_prewhere_columns, true, "Whether to use only prewhere columns size to determine reading task size.", 0) \ \ M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ M(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background. If wait_for_async_insert is false, INSERT query is processed almost instantly, otherwise client will wait until data will be flushed to table", 0) \ diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index fbad7d2f7be..e9e2138d995 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -328,7 +328,10 @@ MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInf for (const auto & range : part.ranges) part_info->sum_marks += range.end - range.begin; - part_info->approx_size_of_mark = getApproximateSizeOfGranule(*part_info->data_part, column_names); + const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info + ? prewhere_info->prewhere_actions->getRequiredColumnsNames() + : column_names; + part_info->approx_size_of_mark = getApproximateSizeOfGranule(*part_info->data_part, columns); const auto task_columns = getReadTaskColumns( part_reader_info, @@ -369,9 +372,9 @@ MergeTreePrefetchedReadPool::PartsInfos MergeTreePrefetchedReadPool::getPartsInf } if (prewhere_info) { - for (const auto & columns : task_columns.pre_columns) + for (const auto & cols : task_columns.pre_columns) { - for (const auto & col : columns) + for (const auto & col : cols) { const size_t col_size = part.data_part->getColumnSize(col.name).data_compressed; part_info->estimated_memory_usage_for_single_prefetch += std::min(col_size, settings.prefetch_buffer_size); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 2ab90189f9d..896769d9355 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -73,8 +73,10 @@ MergeTreeReadPool::MergeTreeReadPool( size_t total_marks = 0; for (const auto & part : parts_ranges) { - total_compressed_bytes += getApproxSizeOfPart( - *part.data_part, prewhere_info ? prewhere_info->prewhere_actions->getRequiredColumnsNames() : column_names_); + const auto & columns = settings.merge_tree_determine_task_size_by_prewhere_columns && prewhere_info + ? prewhere_info->prewhere_actions->getRequiredColumnsNames() + : column_names_; + total_compressed_bytes += getApproxSizeOfPart(*part.data_part, columns); total_marks += part.getMarksCount(); } From 04180549b094c231a01642cb70fa051bed2f7abb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 26 Jul 2023 13:15:58 +0200 Subject: [PATCH 1164/2047] Fix possible double-free in Aggregator (#52439) --- src/Interpreters/Aggregator.cpp | 6 ++++-- .../test.py | 2 +- .../0_stateless/02355_control_block_size_in_aggregator.sql | 3 ++- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index c7d4b87694b..36cd32910b5 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2020,7 +2020,8 @@ template NO_INLINE Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const { - const size_t max_block_size = params.max_block_size; + /// +1 for nullKeyData, if `data` doesn't have it - not a problem, just some memory for one excessive row will be preallocated + const size_t max_block_size = (return_single_block ? data.size() : std::min(params.max_block_size, data.size())) + 1; const bool final = true; ConvertToBlockRes res; @@ -2097,7 +2098,8 @@ template Aggregator::ConvertToBlockRes NO_INLINE Aggregator::convertToBlockImplNotFinal(Method & method, Table & data, Arenas & aggregates_pools, size_t) const { - const size_t max_block_size = params.max_block_size; + /// +1 for nullKeyData, if `data` doesn't have it - not a problem, just some memory for one excessive row will be preallocated + const size_t max_block_size = (return_single_block ? data.size() : std::min(params.max_block_size, data.size())) + 1; const bool final = false; ConvertToBlockRes res; diff --git a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py index faa38af6533..e66631460f7 100644 --- a/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py +++ b/tests/integration/test_distributed_directory_monitor_split_batch_on_failure/test.py @@ -68,7 +68,7 @@ def test_distributed_directory_monitor_split_batch_on_failure_OFF(started_cluste settings={ # max_memory_usage is the limit for the batch on the remote node # (local query should not be affected since 30MB is enough for 100K rows) - "max_memory_usage": "30Mi", + "max_memory_usage": "20Mi", "max_untracked_memory": "0", }, ) diff --git a/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql index b4754c6d6fe..f9f9661a7c4 100644 --- a/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql +++ b/tests/queries/0_stateless/02355_control_block_size_in_aggregator.sql @@ -1,6 +1,7 @@ SET max_block_size = 4213; -SELECT DISTINCT (blockSize() <= 4213) +--- We allocate space for one more row in case nullKeyData is present. +SELECT DISTINCT (blockSize() <= 4214) FROM ( SELECT number From bf301867650194b089b14240d121b1b1eb3b4f6e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 26 Jul 2023 13:23:09 +0200 Subject: [PATCH 1165/2047] upd test --- .../queries/0_stateless/02701_non_parametric_function.reference | 0 tests/queries/0_stateless/02701_non_parametric_function.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/02701_non_parametric_function.reference create mode 100644 tests/queries/0_stateless/02701_non_parametric_function.sql diff --git a/tests/queries/0_stateless/02701_non_parametric_function.reference b/tests/queries/0_stateless/02701_non_parametric_function.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02701_non_parametric_function.sql b/tests/queries/0_stateless/02701_non_parametric_function.sql new file mode 100644 index 00000000000..b242bdc72ef --- /dev/null +++ b/tests/queries/0_stateless/02701_non_parametric_function.sql @@ -0,0 +1 @@ +SELECT * FROM system.numbers WHERE number > toUInt64(10)(number) LIMIT 10; -- { serverError 309 } From 06b782d37a8a48f66db88a32f456e6cd9649b49b Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 26 Jul 2023 14:40:50 +0300 Subject: [PATCH 1166/2047] Add storage_policy to RestoreSettings in order to allow it in the SETTINGS clause of RESTORE queries --- src/Backups/RestoreSettings.cpp | 1 + src/Backups/RestoreSettings.h | 3 +++ 2 files changed, 4 insertions(+) diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 2009ca4c1ff..95f575b846a 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -164,6 +164,7 @@ namespace M(Bool, allow_s3_native_copy) \ M(Bool, internal) \ M(String, host_id) \ + M(String, storage_policy) \ M(OptionalUUID, restore_uuid) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 1861e219dba..42afe522ea3 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -117,6 +117,9 @@ struct RestoreSettings /// The current host's ID in the format 'escaped_host_name:port'. String host_id; + /// Alterative storage policy that may be specified in the SETTINGS clause of RESTORE queries + String storage_policy; + /// Internal, should not be specified by user. /// Cluster's hosts' IDs in the format 'escaped_host_name:port' for all shards and replicas in a cluster specified in BACKUP ON CLUSTER. std::vector cluster_host_ids; From b80a334eb703166ff3c02f42adf0b4e26efd9f2d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 26 Jul 2023 14:08:08 +0200 Subject: [PATCH 1167/2047] Return zxid in TestKeeper responses --- src/Common/ZooKeeper/IKeeper.h | 2 ++ src/Common/ZooKeeper/TestKeeper.cpp | 21 +++++++++++++++------ src/Common/ZooKeeper/ZooKeeperCommon.h | 1 - 3 files changed, 17 insertions(+), 7 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 5240acc2616..c9325df0d90 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -136,6 +136,8 @@ using ResponseCallback = std::function; struct Response { Error error = Error::ZOK; + int64_t zxid = 0; + Response() = default; Response(const Response &) = default; Response & operator=(const Response &) = default; diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 87c87c4fc92..dabc0ae4eef 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -195,6 +195,7 @@ struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest std::pair TestKeeperCreateRequest::process(TestKeeper::Container & container, int64_t zxid) const { CreateResponse response; + response.zxid = zxid; Undo undo; if (container.contains(path)) @@ -257,9 +258,10 @@ std::pair TestKeeperCreateRequest::process(TestKeeper::Contai return { std::make_shared(response), undo }; } -std::pair TestKeeperRemoveRequest::process(TestKeeper::Container & container, int64_t) const +std::pair TestKeeperRemoveRequest::process(TestKeeper::Container & container, int64_t zxid) const { RemoveResponse response; + response.zxid = zxid; Undo undo; auto it = container.find(path); @@ -296,9 +298,10 @@ std::pair TestKeeperRemoveRequest::process(TestKeeper::Contai return { std::make_shared(response), undo }; } -std::pair TestKeeperExistsRequest::process(TestKeeper::Container & container, int64_t) const +std::pair TestKeeperExistsRequest::process(TestKeeper::Container & container, int64_t zxid) const { ExistsResponse response; + response.zxid = zxid; auto it = container.find(path); if (it != container.end()) @@ -314,9 +317,10 @@ std::pair TestKeeperExistsRequest::process(TestKeeper::Contai return { std::make_shared(response), {} }; } -std::pair TestKeeperGetRequest::process(TestKeeper::Container & container, int64_t) const +std::pair TestKeeperGetRequest::process(TestKeeper::Container & container, int64_t zxid) const { GetResponse response; + response.zxid = zxid; auto it = container.find(path); if (it == container.end()) @@ -336,6 +340,7 @@ std::pair TestKeeperGetRequest::process(TestKeeper::Container std::pair TestKeeperSetRequest::process(TestKeeper::Container & container, int64_t zxid) const { SetResponse response; + response.zxid = zxid; Undo undo; auto it = container.find(path); @@ -370,9 +375,10 @@ std::pair TestKeeperSetRequest::process(TestKeeper::Container return { std::make_shared(response), undo }; } -std::pair TestKeeperListRequest::process(TestKeeper::Container & container, int64_t) const +std::pair TestKeeperListRequest::process(TestKeeper::Container & container, int64_t zxid) const { ListResponse response; + response.zxid = zxid; auto it = container.find(path); if (it == container.end()) @@ -414,9 +420,10 @@ std::pair TestKeeperListRequest::process(TestKeeper::Containe return { std::make_shared(response), {} }; } -std::pair TestKeeperCheckRequest::process(TestKeeper::Container & container, int64_t) const +std::pair TestKeeperCheckRequest::process(TestKeeper::Container & container, int64_t zxid) const { CheckResponse response; + response.zxid = zxid; auto it = container.find(path); if (it == container.end()) { @@ -434,10 +441,11 @@ std::pair TestKeeperCheckRequest::process(TestKeeper::Contain return { std::make_shared(response), {} }; } -std::pair TestKeeperSyncRequest::process(TestKeeper::Container & /*container*/, int64_t) const +std::pair TestKeeperSyncRequest::process(TestKeeper::Container & /*container*/, int64_t zxid) const { SyncResponse response; response.path = path; + response.zxid = zxid; return { std::make_shared(std::move(response)), {} }; } @@ -456,6 +464,7 @@ std::pair TestKeeperReconfigRequest::process(TestKeeper::Cont std::pair TestKeeperMultiRequest::process(TestKeeper::Container & container, int64_t zxid) const { MultiResponse response; + response.zxid = zxid; response.responses.reserve(requests.size()); std::vector undo_actions; diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 131d19f1ca4..e4b2cc97744 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -28,7 +28,6 @@ using LogElements = std::vector; struct ZooKeeperResponse : virtual Response { XID xid = 0; - int64_t zxid = 0; UInt64 response_created_time_ns = 0; From 42c5c420247af88c9fc7aa7eaad40e38e46918f8 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 26 Jul 2023 14:09:19 +0200 Subject: [PATCH 1168/2047] Set zxid in all multi responses --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index c24eecbafd8..e88d66e5444 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -642,6 +642,8 @@ void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) if (op_error == Error::ZOK || op_num == OpNum::Error) dynamic_cast(*response).readImpl(in); + + response->zxid = zxid; } /// Footer. From 0b5b0df58f22ca5ff6ef35ff74078856dbad09e2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 26 Jul 2023 15:24:35 +0300 Subject: [PATCH 1169/2047] Apply custom storage policy to the restored table --- src/Backups/RestorerFromBackup.cpp | 15 +++++++++++++++ src/Backups/RestorerFromBackup.h | 2 ++ 2 files changed, 17 insertions(+) diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 7981cc2f19f..2e598ae0486 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -322,6 +322,7 @@ void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name read_buffer.reset(); ParserCreateQuery create_parser; ASTPtr create_table_query = parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); + setCustomStoragePolicyIfAny(create_table_query); renameDatabaseAndTableNameInCreateQuery(create_table_query, renaming_map, context->getGlobalContext()); QualifiedTableName table_name = renaming_map.getNewTableName(table_name_in_backup); @@ -625,6 +626,20 @@ void RestorerFromBackup::checkDatabase(const String & database_name) } } +void RestorerFromBackup::setCustomStoragePolicyIfAny(ASTPtr query_ptr) +{ + if (!restore_settings.storage_policy.empty()) + { + auto & create_table_query = query_ptr->as(); + if (create_table_query.storage && create_table_query.storage->settings) + { + auto value = create_table_query.storage->settings->changes.tryGet("storage_policy"); + if (value) + *value = restore_settings.storage_policy; + } + } +} + void RestorerFromBackup::removeUnresolvedDependencies() { auto need_exclude_dependency = [this](const StorageID & table_id) diff --git a/src/Backups/RestorerFromBackup.h b/src/Backups/RestorerFromBackup.h index 93b5a6c7694..dcd1922506c 100644 --- a/src/Backups/RestorerFromBackup.h +++ b/src/Backups/RestorerFromBackup.h @@ -95,6 +95,8 @@ private: void createDatabase(const String & database_name) const; void checkDatabase(const String & database_name); + void setCustomStoragePolicyIfAny(ASTPtr query_ptr); + void removeUnresolvedDependencies(); void createTables(); void createTable(const QualifiedTableName & table_name); From eb689708a80ab86adb9130b86dd3dbd09ebbacd7 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 26 Jul 2023 15:26:38 +0300 Subject: [PATCH 1170/2047] Fix typo --- src/Backups/RestoreSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 42afe522ea3..41c66b37442 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -117,7 +117,7 @@ struct RestoreSettings /// The current host's ID in the format 'escaped_host_name:port'. String host_id; - /// Alterative storage policy that may be specified in the SETTINGS clause of RESTORE queries + /// Alternative storage policy that may be specified in the SETTINGS clause of RESTORE queries String storage_policy; /// Internal, should not be specified by user. From aa25ce9e3d50d1b590821a1a9d93f0e1edf53e8e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 26 Jul 2023 11:19:20 +0000 Subject: [PATCH 1171/2047] Follow-up to "Implement support of encrypted elements in configuration file" Cf. PR #50986 - rename XML attribute "encryption_codec" to "encrypted_by" --- docs/en/operations/configuration-files.md | 67 +++++++++++++++---- docs/ru/operations/configuration-files.md | 7 +- src/Common/Config/ConfigProcessor.cpp | 20 +++--- src/Common/examples/encrypt_decrypt.cpp | 4 +- src/Compression/CompressionCodecEncrypted.cpp | 42 ++---------- src/Compression/CompressionCodecEncrypted.h | 4 +- .../test_config_decryption/configs/config.xml | 7 +- .../configs/config.yaml | 6 +- .../configs/config_invalid_chars.xml | 8 ++- .../configs/config_no_encryption_key.xml | 6 +- .../configs/config_subnodes.xml | 6 +- .../configs/config_wrong_method.xml | 7 +- .../test_wrong_settings.py | 2 +- 13 files changed, 110 insertions(+), 76 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index d1d9fa542ab..a19c55673ed 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -67,7 +67,7 @@ Substitutions can also be performed from ZooKeeper. To do this, specify the attr ## Encrypting Configuration {#encryption} -You can use symmetric encryption to encrypt a configuration element, for example, a password field. To do so, first configure the [encryption codec](../sql-reference/statements/create/table.md#encryption-codecs), then add attribute `encryption_codec` with the name of the encryption codec as value to the element to encrypt. +You can use symmetric encryption to encrypt a configuration element, for example, a password field. To do so, first configure the [encryption codec](../sql-reference/statements/create/table.md#encryption-codecs), then add attribute `encrypted_by` with the name of the encryption codec as value to the element to encrypt. Unlike attributes `from_zk`, `from_env` and `incl` (or element `include`), no substitution, i.e. decryption of the encrypted value, is performed in the preprocessed file. Decryption happens only at runtime in the server process. @@ -75,19 +75,22 @@ Example: ```xml + 00112233445566778899aabbccddeeff + admin - 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + ``` -To get the encrypted value `encrypt_decrypt` example application may be used. +To encrypt a value, you can use the (example) program `encrypt_decrypt`: Example: @@ -138,12 +141,17 @@ Here you can see default config written in YAML: [config.yaml.example](https://g There are some differences between YAML and XML formats in terms of ClickHouse configurations. Here are some tips for writing a configuration in YAML format. -You should use a Scalar node to write a key-value pair: +An XML tag with a text value is represented by a YAML key-value pair ``` yaml key: value ``` -To create a node, containing other nodes you should use a Map: +Corresponding XML: +``` xml +value +``` + +A nested XML node is represented by a YAML map: ``` yaml map_key: key1: val1 @@ -151,7 +159,16 @@ map_key: key3: val3 ``` -To create a list of values or nodes assigned to one tag you should use a Sequence: +Corresponding XML: +``` xml + + val1 + val2 + val3 + +``` + +To create the same XML tag multiple times, use a YAML sequence: ``` yaml seq_key: - val1 @@ -162,8 +179,22 @@ seq_key: key3: val5 ``` -If you want to write an attribute for a Sequence or Map node, you should use a @ prefix before the attribute key. Note, that @ is reserved by YAML standard, so you should also to wrap it into double quotes: +Corresponding XML: +```xml +val1 +val2 + + val3 + + + + val4 + val5 + + +``` +To provide an XML attribute, you can use an attribute key with a `@` prefix. Note that `@` is reserved by YAML standard, so must be wrapped in double quotes: ``` yaml map: "@attr1": value1 @@ -171,16 +202,14 @@ map: key: 123 ``` -From that Map we will get these XML nodes: - +Corresponding XML: ``` xml 123 ``` -You can also set attributes for Sequence: - +It is also possible to use attributes in YAML sequence: ``` yaml seq: - "@attr1": value1 @@ -189,13 +218,25 @@ seq: - abc ``` -So, we can get YAML config equal to this XML one: - +Corresponding XML: ``` xml 123 abc ``` +The aforementioned syntax does not allow to express XML text nodes with XML attributes as YAML. This special case can be achieved using an +`#text` attribute key: +```yaml +map_key: + "@attr1": value1 + "#text": value2 +``` + +Corresponding XML: +```xml +value2 +``` + ## Implementation Details {#implementation-details} For each config file, the server also generates `file-preprocessed.xml` files when starting. These files contain all the completed substitutions and overrides, and they are intended for informational use. If ZooKeeper substitutions were used in the config files but ZooKeeper is not available on the server start, the server loads the configuration from the preprocessed file. diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index 01a91bd41c6..085761d80c7 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -87,7 +87,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ## Шифрование {#encryption} -Вы можете использовать симметричное шифрование для зашифровки элемента конфигурации, например, поля password. Чтобы это сделать, сначала настройте [кодек шифрования](../sql-reference/statements/create/table.md#encryption-codecs), затем добавьте аттибут`encryption_codec` с именем кодека шифрования как значение к элементу, который надо зашифровать. +Вы можете использовать симметричное шифрование для зашифровки элемента конфигурации, например, поля password. Чтобы это сделать, сначала настройте [кодек шифрования](../sql-reference/statements/create/table.md#encryption-codecs), затем добавьте аттибут`encrypted_by` с именем кодека шифрования как значение к элементу, который надо зашифровать. В отличии от аттрибутов `from_zk`, `from_env` и `incl` (или элемента `include`), подстановка, т.е. расшифровка зашифрованного значения, не выподняется в файле предобработки. Расшифровка происходит только во время исполнения в серверном процессе. @@ -95,15 +95,18 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ```xml + 00112233445566778899aabbccddeeff + admin - 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + 961F000000040000000000EEDDEF4F453CFE6457C4234BD7C09258BD651D85 + ``` diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index db3c6909b21..a55183782d8 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -192,13 +192,13 @@ static void mergeAttributes(Element & config_element, Element & with_element) std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string & value) { - EncryptionMethod method = getEncryptionMethod(codec_name); - CompressionCodecEncrypted codec(method); + EncryptionMethod encryption_method = toEncryptionMethod(codec_name); + CompressionCodecEncrypted codec(encryption_method); Memory<> memory; memory.resize(codec.getCompressedReserveSize(static_cast(value.size()))); auto bytes_written = codec.compress(value.data(), static_cast(value.size()), memory.data()); - auto encrypted_value = std::string(memory.data(), bytes_written); + std::string encrypted_value(memory.data(), bytes_written); std::string hex_value; boost::algorithm::hex(encrypted_value.begin(), encrypted_value.end(), std::back_inserter(hex_value)); return hex_value; @@ -206,8 +206,8 @@ std::string ConfigProcessor::encryptValue(const std::string & codec_name, const std::string ConfigProcessor::decryptValue(const std::string & codec_name, const std::string & value) { - EncryptionMethod method = getEncryptionMethod(codec_name); - CompressionCodecEncrypted codec(method); + EncryptionMethod encryption_method = toEncryptionMethod(codec_name); + CompressionCodecEncrypted codec(encryption_method); Memory<> memory; std::string encrypted_value; @@ -223,7 +223,7 @@ std::string ConfigProcessor::decryptValue(const std::string & codec_name, const memory.resize(codec.readDecompressedBlockSize(encrypted_value.data())); codec.decompress(encrypted_value.data(), static_cast(encrypted_value.size()), memory.data()); - std::string decrypted_value = std::string(memory.data(), memory.size()); + std::string decrypted_value(memory.data(), memory.size()); return decrypted_value; } @@ -234,7 +234,7 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) if (node->nodeType() == Node::ELEMENT_NODE) { Element & element = dynamic_cast(*node); - if (element.hasAttribute("encryption_codec")) + if (element.hasAttribute("encrypted_by")) { const NodeListPtr children = element.childNodes(); if (children->length() != 1) @@ -244,8 +244,8 @@ void ConfigProcessor::decryptRecursive(Poco::XML::Node * config_root) if (text_node->nodeType() != Node::TEXT_NODE) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Encrypted node {} should have text node", node->nodeName()); - auto encryption_codec = element.getAttribute("encryption_codec"); - text_node->setNodeValue(decryptValue(encryption_codec, text_node->getNodeValue())); + auto encrypted_by = element.getAttribute("encrypted_by"); + text_node->setNodeValue(decryptValue(encrypted_by, text_node->getNodeValue())); } decryptRecursive(node); } @@ -775,7 +775,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes( void ConfigProcessor::decryptEncryptedElements(LoadedConfig & loaded_config) { - CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); + CompressionCodecEncrypted::Configuration::instance().load(*loaded_config.configuration, "encryption_codecs"); Node * config_root = getRootNode(loaded_config.preprocessed_xml.get()); decryptRecursive(config_root); loaded_config.configuration = new Poco::Util::XMLConfiguration(loaded_config.preprocessed_xml); diff --git a/src/Common/examples/encrypt_decrypt.cpp b/src/Common/examples/encrypt_decrypt.cpp index 503802016cb..c7f949195c8 100644 --- a/src/Common/examples/encrypt_decrypt.cpp +++ b/src/Common/examples/encrypt_decrypt.cpp @@ -3,7 +3,7 @@ #include #include -/** This test program encrypts or decrypts text values using a symmetric encryption codec like AES_128_GCM_SIV or AES_256_GCM_SIV. +/** This program encrypts or decrypts text values using a symmetric encryption codec like AES_128_GCM_SIV or AES_256_GCM_SIV. * Keys for codecs are loaded from section of configuration file. * * How to use: @@ -32,7 +32,7 @@ int main(int argc, char ** argv) DB::ConfigProcessor processor(argv[1], false, true); auto loaded_config = processor.loadConfig(); - DB::CompressionCodecEncrypted::Configuration::instance().tryLoad(*loaded_config.configuration, "encryption_codecs"); + DB::CompressionCodecEncrypted::Configuration::instance().load(*loaded_config.configuration, "encryption_codecs"); if (action == "-e") std::cout << processor.encryptValue(codec_name, value) << std::endl; diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 3f4e35a78a4..5438e02792f 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -31,14 +31,14 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -EncryptionMethod getEncryptionMethod(const std::string & name) +EncryptionMethod toEncryptionMethod(const std::string & name) { if (name == "AES_128_GCM_SIV") return AES_128_GCM_SIV; else if (name == "AES_256_GCM_SIV") return AES_256_GCM_SIV; else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown encryption method. Got {}", name); } namespace @@ -48,34 +48,22 @@ namespace String getMethodName(EncryptionMethod Method) { if (Method == AES_128_GCM_SIV) - { return "AES_128_GCM_SIV"; - } else if (Method == AES_256_GCM_SIV) - { return "AES_256_GCM_SIV"; - } else - { return ""; - } } /// Get method code (used for codec, to understand which one we are using) uint8_t getMethodCode(EncryptionMethod Method) { if (Method == AES_128_GCM_SIV) - { return static_cast(CompressionMethodByte::AES_128_GCM_SIV); - } else if (Method == AES_256_GCM_SIV) - { return static_cast(CompressionMethodByte::AES_256_GCM_SIV); - } else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown encryption method. Got {}", getMethodName(Method)); } } // end of namespace @@ -105,17 +93,11 @@ const String empty_nonce = {"\0\0\0\0\0\0\0\0\0\0\0\0", actual_nonce_size}; UInt64 methodKeySize(EncryptionMethod Method) { if (Method == AES_128_GCM_SIV) - { return 16; - } else if (Method == AES_256_GCM_SIV) - { return 32; - } else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown encryption method. Got {}", getMethodName(Method)); } std::string lastErrorString() @@ -130,17 +112,11 @@ std::string lastErrorString() auto getMethod(EncryptionMethod Method) { if (Method == AES_128_GCM_SIV) - { return EVP_aead_aes_128_gcm_siv; - } else if (Method == AES_256_GCM_SIV) - { return EVP_aead_aes_256_gcm_siv; - } else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown encryption method. Got {}", getMethodName(Method)); } /// Encrypt plaintext with particular algorithm and put result into ciphertext_and_tag. @@ -206,17 +182,11 @@ size_t decrypt(std::string_view ciphertext, char * plaintext, EncryptionMethod m auto getMethod(EncryptionMethod Method) { if (Method == AES_128_GCM_SIV) - { return EVP_aes_128_gcm; - } else if (Method == AES_256_GCM_SIV) - { return EVP_aes_256_gcm; - } else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong encryption method. Got {}", getMethodName(Method)); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown encryption method. Got {}", getMethodName(Method)); } /// Encrypt plaintext with particular algorithm and put result into ciphertext_and_tag. diff --git a/src/Compression/CompressionCodecEncrypted.h b/src/Compression/CompressionCodecEncrypted.h index fafcf4af507..7971cbadab7 100644 --- a/src/Compression/CompressionCodecEncrypted.h +++ b/src/Compression/CompressionCodecEncrypted.h @@ -18,8 +18,8 @@ enum EncryptionMethod MAX_ENCRYPTION_METHOD }; -/// Get method for string name. Throw exception for wrong name. -EncryptionMethod getEncryptionMethod(const std::string & name); +/// Get encryption method for string name. Throw exception for wrong name. +EncryptionMethod toEncryptionMethod(const std::string & name); /** This codec encrypts and decrypts blocks with AES-128 in * GCM-SIV mode (RFC-8452), which is the only cipher currently diff --git a/tests/integration/test_config_decryption/configs/config.xml b/tests/integration/test_config_decryption/configs/config.xml index 5c274128e39..4b0d3a77659 100644 --- a/tests/integration/test_config_decryption/configs/config.xml +++ b/tests/integration/test_config_decryption/configs/config.xml @@ -1,4 +1,5 @@ + 00112233445566778899aabbccddeeff @@ -7,6 +8,8 @@ 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff - 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C - 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/configs/config.yaml b/tests/integration/test_config_decryption/configs/config.yaml index ab4391be3c5..1b20b65b652 100644 --- a/tests/integration/test_config_decryption/configs/config.yaml +++ b/tests/integration/test_config_decryption/configs/config.yaml @@ -3,9 +3,11 @@ encryption_codecs: key_hex: 00112233445566778899aabbccddeeff aes_256_gcm_siv: key_hex: 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff + max_table_size_to_drop: '#text': 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C - '@encryption_codec': AES_128_GCM_SIV + '@encrypted_by': AES_128_GCM_SIV + max_partition_size_to_drop: - '@encryption_codec': AES_256_GCM_SIV + '@encrypted_by': AES_256_GCM_SIV '#text': 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 diff --git a/tests/integration/test_config_decryption/configs/config_invalid_chars.xml b/tests/integration/test_config_decryption/configs/config_invalid_chars.xml index 49bf51b5bad..53345b897dc 100644 --- a/tests/integration/test_config_decryption/configs/config_invalid_chars.xml +++ b/tests/integration/test_config_decryption/configs/config_invalid_chars.xml @@ -1,4 +1,5 @@ + 00112233445566778899aabbccddeeff @@ -7,6 +8,9 @@ 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff - --96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C - 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + + + --96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml b/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml index 5f7769f7403..830c75f7378 100644 --- a/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml +++ b/tests/integration/test_config_decryption/configs/config_no_encryption_key.xml @@ -1,3 +1,7 @@ - 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + + + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + diff --git a/tests/integration/test_config_decryption/configs/config_subnodes.xml b/tests/integration/test_config_decryption/configs/config_subnodes.xml index b0e519ff546..8213270f747 100644 --- a/tests/integration/test_config_decryption/configs/config_subnodes.xml +++ b/tests/integration/test_config_decryption/configs/config_subnodes.xml @@ -1,10 +1,14 @@ + 00112233445566778899aabbccddeeff - + + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + diff --git a/tests/integration/test_config_decryption/configs/config_wrong_method.xml b/tests/integration/test_config_decryption/configs/config_wrong_method.xml index b452ce6374c..b96c13d5105 100644 --- a/tests/integration/test_config_decryption/configs/config_wrong_method.xml +++ b/tests/integration/test_config_decryption/configs/config_wrong_method.xml @@ -1,4 +1,5 @@ + 00112233445566778899aabbccddeeff @@ -7,6 +8,8 @@ 00112233445566778899aabbccddeeff00112233445566778899aabbccddeeff - 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C - 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + + 96260000000B0000000000E8FE3C087CED2205A5071078B29FD5C3B97F824911DED3217E980C + 97260000000B0000000000BFFF70C4DA718754C1DA0E2F25FF9246D4783F7FFEC4089EC1CC14 + diff --git a/tests/integration/test_config_decryption/test_wrong_settings.py b/tests/integration/test_config_decryption/test_wrong_settings.py index b148f9a051a..c6987d12324 100644 --- a/tests/integration/test_config_decryption/test_wrong_settings.py +++ b/tests/integration/test_config_decryption/test_wrong_settings.py @@ -15,7 +15,7 @@ def start_clickhouse(config, err_msg): def test_wrong_method(): start_clickhouse( - "configs/config_wrong_method.xml", "Wrong encryption method. Got WRONG" + "configs/config_wrong_method.xml", "Unknown encryption method. Got WRONG" ) From 7d430b803784084a09fd71f98f1a7e8872feb307 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Tue, 25 Jul 2023 23:38:03 +0400 Subject: [PATCH 1172/2047] retry ConnectionResetException --- src/IO/S3/Client.cpp | 79 +++++++++++-------- src/IO/S3/Client.h | 2 +- .../test_checking_s3_blobs_paranoid/test.py | 22 ++++-- 3 files changed, 65 insertions(+), 38 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 94a7b5166da..51c7ee32579 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -357,12 +357,14 @@ Model::HeadObjectOutcome Client::HeadObject(const HeadObjectRequest & request) c Model::ListObjectsV2Outcome Client::ListObjectsV2(const ListObjectsV2Request & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::ListObjectsV2Request & req) { return ListObjectsV2(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::ListObjectsV2Request & req) { return ListObjectsV2(req); }); } Model::ListObjectsOutcome Client::ListObjects(const ListObjectsRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::ListObjectsRequest & req) { return ListObjects(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::ListObjectsRequest & req) { return ListObjects(req); }); } Model::GetObjectOutcome Client::GetObject(const GetObjectRequest & request) const @@ -372,19 +374,19 @@ Model::GetObjectOutcome Client::GetObject(const GetObjectRequest & request) cons Model::AbortMultipartUploadOutcome Client::AbortMultipartUpload(const AbortMultipartUploadRequest & request) const { - return doRequestWithRetryNetworkErrors( + return doRequestWithRetryNetworkErrors( request, [this](const Model::AbortMultipartUploadRequest & req) { return AbortMultipartUpload(req); }); } Model::CreateMultipartUploadOutcome Client::CreateMultipartUpload(const CreateMultipartUploadRequest & request) const { - return doRequestWithRetryNetworkErrors( + return doRequestWithRetryNetworkErrors( request, [this](const Model::CreateMultipartUploadRequest & req) { return CreateMultipartUpload(req); }); } Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(const CompleteMultipartUploadRequest & request) const { - auto outcome = doRequestWithRetryNetworkErrors( + auto outcome = doRequestWithRetryNetworkErrors( request, [this](const Model::CompleteMultipartUploadRequest & req) { return CompleteMultipartUpload(req); }); if (!outcome.IsSuccess() || provider_type != ProviderType::GCS) @@ -414,32 +416,38 @@ Model::CompleteMultipartUploadOutcome Client::CompleteMultipartUpload(const Comp Model::CopyObjectOutcome Client::CopyObject(const CopyObjectRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::CopyObjectRequest & req) { return CopyObject(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::CopyObjectRequest & req) { return CopyObject(req); }); } Model::PutObjectOutcome Client::PutObject(const PutObjectRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::PutObjectRequest & req) { return PutObject(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::PutObjectRequest & req) { return PutObject(req); }); } Model::UploadPartOutcome Client::UploadPart(const UploadPartRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::UploadPartRequest & req) { return UploadPart(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::UploadPartRequest & req) { return UploadPart(req); }); } Model::UploadPartCopyOutcome Client::UploadPartCopy(const UploadPartCopyRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::UploadPartCopyRequest & req) { return UploadPartCopy(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::UploadPartCopyRequest & req) { return UploadPartCopy(req); }); } Model::DeleteObjectOutcome Client::DeleteObject(const DeleteObjectRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::DeleteObjectRequest & req) { return DeleteObject(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::DeleteObjectRequest & req) { return DeleteObject(req); }); } Model::DeleteObjectsOutcome Client::DeleteObjects(const DeleteObjectsRequest & request) const { - return doRequestWithRetryNetworkErrors(request, [this](const Model::DeleteObjectsRequest & req) { return DeleteObjects(req); }); + return doRequestWithRetryNetworkErrors( + request, [this](const Model::DeleteObjectsRequest & req) { return DeleteObjects(req); }); } Client::ComposeObjectOutcome Client::ComposeObject(const ComposeObjectRequest & request) const @@ -468,7 +476,8 @@ Client::ComposeObjectOutcome Client::ComposeObject(const ComposeObjectRequest & return ComposeObjectOutcome(MakeRequest(req, endpointResolutionOutcome.GetResult(), Aws::Http::HttpMethod::HTTP_PUT)); }; - return doRequestWithRetryNetworkErrors(request, request_fn); + return doRequestWithRetryNetworkErrors( + request, request_fn); } template @@ -549,52 +558,60 @@ Client::doRequest(const RequestType & request, RequestFn request_fn) const throw Exception(ErrorCodes::TOO_MANY_REDIRECTS, "Too many redirects"); } -template +template std::invoke_result_t Client::doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn request_fn) const { auto with_retries = [this, request_fn_ = std::move(request_fn)] (const RequestType & request_) { - const size_t max_tries = 10; + chassert(client_configuration.retryStrategy); + const Int64 max_attempts = client_configuration.retryStrategy->GetMaxAttempts(); std::exception_ptr last_exception = nullptr; - for (size_t try_no = 0; try_no < max_tries; ++try_no) + for (Int64 attempt_no = 0; attempt_no < max_attempts; ++attempt_no) { try { + /// S3 does retries network errors actually. + /// But it is matter when errors occur. + /// This code retries a specific case when + /// network error happens when XML document is being read from the response body. + /// Hence, the response body is a stream, network errors are possible at reading. + /// S3 doesn't retry them. + + /// Not all requests can be retried in that way. + /// Requests that read out response body to build the result are possible to retry. + /// Requests that expose the response stream as an answer are not retried with that code. E.g. GetObject. return request_fn_(request_); } catch (Poco::Net::ConnectionResetException &) { - // to do distinguish read/write - if (client_configuration.for_disk_s3) + + if constexpr (IsReadMethod) { - ProfileEvents::increment(ProfileEvents::DiskS3WriteRequestsErrors); - ProfileEvents::increment(ProfileEvents::DiskS3ReadRequestsErrors); + if (client_configuration.for_disk_s3) + ProfileEvents::increment(ProfileEvents::DiskS3ReadRequestsErrors); + else + ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors); } else { - ProfileEvents::increment(ProfileEvents::S3WriteRequestsErrors); - ProfileEvents::increment(ProfileEvents::S3ReadRequestsErrors); + if (client_configuration.for_disk_s3) + ProfileEvents::increment(ProfileEvents::DiskS3WriteRequestsErrors); + else + ProfileEvents::increment(ProfileEvents::S3WriteRequestsErrors); } tryLogCurrentException(log, "Will retry"); - // to do back off last_exception = std::current_exception(); + + auto error = Aws::Client::AWSError(Aws::Client::CoreErrors::NETWORK_CONNECTION, /*retry*/ true); + client_configuration.retryStrategy->CalculateDelayBeforeNextRetry(error, attempt_no); continue; } } chassert(last_exception); std::rethrow_exception(last_exception); - -// try -// { -// std::rethrow_exception(last_exception); -// } -// catch (const Poco::Exception & e) -// { -// throw Exception(Exception::CreateFromPocoTag{}, e); -// } }; return doRequest(request, with_retries); diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index bb893c21774..1b0fdcefe32 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -250,7 +250,7 @@ private: std::invoke_result_t doRequest(const RequestType & request, RequestFn request_fn) const; - template + template std::invoke_result_t doRequestWithRetryNetworkErrors(const RequestType & request, RequestFn request_fn) const; diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 6aa149b9c7b..28b0c9beeaa 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -248,7 +248,9 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD" + insert_query_id = ( + f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_{send_something}" + ) node.query( f""" INSERT INTO @@ -283,7 +285,9 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( action="connection_reset_by_peer", action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_1" + insert_query_id = ( + f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_UPLOAD_{send_something}_1" + ) error = node.query_and_get_error( f""" INSERT INTO @@ -307,7 +311,8 @@ def test_when_s3_connection_reset_by_peer_at_upload_is_retried( assert "Code: 1000" in error, error assert ( "DB::Exception: Connection reset by peer." in error - or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" + in error ), error @@ -325,7 +330,9 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD" + insert_query_id = ( + f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_{send_something}" + ) node.query( f""" INSERT INTO @@ -361,7 +368,9 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( action_args=["1"] if send_something else ["0"], ) - insert_query_id = f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_1" + insert_query_id = ( + f"TEST_WHEN_S3_CONNECTION_RESET_BY_PEER_AT_MULTIPARTUPLOAD_{send_something}_1" + ) error = node.query_and_get_error( f""" INSERT INTO @@ -385,5 +394,6 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( assert "Code: 1000" in error, error assert ( "DB::Exception: Connection reset by peer." in error - or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error + or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" + in error ), error From 9ea479a1a1b77b658f4d8504cb140c09121d44c8 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 26 Jul 2023 13:37:33 +0000 Subject: [PATCH 1173/2047] Analyzer: Support ARRAY JOIN COLUMNS(...) syntax --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 106 ++++++++++-------- .../02833_array_join_columns.reference | 0 .../0_stateless/02833_array_join_columns.sql | 19 ++++ 3 files changed, 78 insertions(+), 47 deletions(-) create mode 100644 tests/queries/0_stateless/02833_array_join_columns.reference create mode 100644 tests/queries/0_stateless/02833_array_join_columns.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index abf02547ccd..5dbe9e350bf 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6477,55 +6477,69 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif resolveExpressionNode(array_join_expression, scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); - auto result_type = array_join_expression->getResultType(); - bool is_array_type = isArray(result_type); - bool is_map_type = isMap(result_type); - - if (!is_array_type && !is_map_type) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "ARRAY JOIN {} requires expression {} with Array or Map type. Actual {}. In scope {}", - array_join_node_typed.formatASTForErrorMessage(), - array_join_expression->formatASTForErrorMessage(), - result_type->getName(), - scope.scope_node->formatASTForErrorMessage()); - - if (is_map_type) - result_type = assert_cast(*result_type).getNestedType(); - - result_type = assert_cast(*result_type).getNestedType(); - - String array_join_column_name; - - if (!array_join_expression_alias.empty()) + auto process_array_join_expression = [&](QueryTreeNodePtr & expression) { - array_join_column_name = array_join_expression_alias; - } - else if (auto * array_join_expression_inner_column = array_join_expression->as()) + auto result_type = expression->getResultType(); + bool is_array_type = isArray(result_type); + bool is_map_type = isMap(result_type); + + if (!is_array_type && !is_map_type) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "ARRAY JOIN {} requires expression {} with Array or Map type. Actual {}. In scope {}", + array_join_node_typed.formatASTForErrorMessage(), + expression->formatASTForErrorMessage(), + result_type->getName(), + scope.scope_node->formatASTForErrorMessage()); + + if (is_map_type) + result_type = assert_cast(*result_type).getNestedType(); + + result_type = assert_cast(*result_type).getNestedType(); + + String array_join_column_name; + + if (!array_join_expression_alias.empty()) + { + array_join_column_name = array_join_expression_alias; + } + else if (auto * array_join_expression_inner_column = array_join_expression->as()) + { + array_join_column_name = array_join_expression_inner_column->getColumnName(); + } + else if (!identifier_full_name.empty()) + { + array_join_column_name = identifier_full_name; + } + else + { + array_join_column_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter); + ++array_join_expressions_counter; + } + + if (array_join_column_names.contains(array_join_column_name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "ARRAY JOIN {} multiple columns with name {}. In scope {}", + array_join_node_typed.formatASTForErrorMessage(), + array_join_column_name, + scope.scope_node->formatASTForErrorMessage()); + array_join_column_names.emplace(array_join_column_name); + + NameAndTypePair array_join_column(array_join_column_name, result_type); + auto array_join_column_node = std::make_shared(std::move(array_join_column), expression, array_join_node); + array_join_column_node->setAlias(array_join_expression_alias); + array_join_column_expressions.push_back(std::move(array_join_column_node)); + }; + + // Support ARRAY JOIN COLUMNS(...). COLUMNS trasformer is resolved to list of columns. + if (auto * columns_list = array_join_expression->as()) { - array_join_column_name = array_join_expression_inner_column->getColumnName(); - } - else if (!identifier_full_name.empty()) - { - array_join_column_name = identifier_full_name; + for (auto & array_join_subexpression : columns_list->getNodes()) + process_array_join_expression(array_join_subexpression); } else { - array_join_column_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter); - ++array_join_expressions_counter; + process_array_join_expression(array_join_expression); } - - if (array_join_column_names.contains(array_join_column_name)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "ARRAY JOIN {} multiple columns with name {}. In scope {}", - array_join_node_typed.formatASTForErrorMessage(), - array_join_column_name, - scope.scope_node->formatASTForErrorMessage()); - array_join_column_names.emplace(array_join_column_name); - - NameAndTypePair array_join_column(array_join_column_name, result_type); - auto array_join_column_node = std::make_shared(std::move(array_join_column), array_join_expression, array_join_node); - array_join_column_node->setAlias(array_join_expression_alias); - array_join_column_expressions.push_back(std::move(array_join_column_node)); } /** Allow to resolve ARRAY JOIN columns from aliases with types after ARRAY JOIN only after ARRAY JOIN expression list is resolved, because @@ -6537,11 +6551,9 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif * And it is expected that `value_element` inside projection expression list will be resolved as `value_element` expression * with type after ARRAY JOIN. */ - for (size_t i = 0; i < array_join_nodes_size; ++i) + array_join_nodes = std::move(array_join_column_expressions); + for (auto & array_join_column_expression : array_join_nodes) { - auto & array_join_column_expression = array_join_nodes[i]; - array_join_column_expression = std::move(array_join_column_expressions[i]); - auto it = scope.alias_name_to_expression_node.find(array_join_column_expression->getAlias()); if (it != scope.alias_name_to_expression_node.end()) { diff --git a/tests/queries/0_stateless/02833_array_join_columns.reference b/tests/queries/0_stateless/02833_array_join_columns.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02833_array_join_columns.sql b/tests/queries/0_stateless/02833_array_join_columns.sql new file mode 100644 index 00000000000..3f9a33a3959 --- /dev/null +++ b/tests/queries/0_stateless/02833_array_join_columns.sql @@ -0,0 +1,19 @@ +drop table if exists test_array_joins; +drop table if exists v4test_array_joins; + +create table test_array_joins +( + id UInt64 default rowNumberInAllBlocks() + 1, + arr_1 Array(String), + arr_2 Array(String), + arr_3 Array(String), + arr_4 Array(String) +) engine = MergeTree order by id; + +insert into test_array_joins (id,arr_1, arr_2, arr_3, arr_4) +SELECT number,array(randomPrintableASCII(3)),array(randomPrintableASCII(3)),array(randomPrintableASCII(3)),array(randomPrintableASCII(3)) +from numbers(1000); + +create view v4test_array_joins as SELECT * from test_array_joins where id != 10; + +select * from v4test_array_joins array join columns('^arr') where match(arr_4,'a') and id < 100 order by id format Null settings optimize_read_in_order = 0; From dccbe875d247818a17e999ceab5e062537169f80 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 26 Jul 2023 15:37:58 +0200 Subject: [PATCH 1174/2047] check if storage shutdown before we operate MergeTreeDeduplicationLog --- .../MergeTree/MergeTreeDeduplicationLog.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 09a04f13fc7..53481ab06a0 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -10,6 +10,8 @@ #include #include +#include + namespace DB { @@ -231,6 +233,12 @@ std::pair MergeTreeDeduplicationLog::addPart(const std: return std::make_pair(info, false); } + if (stopped) + { + LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we add this part."); + return {}; + } + chassert(current_writer != nullptr); /// Create new record @@ -261,6 +269,12 @@ void MergeTreeDeduplicationLog::dropPart(const MergeTreePartInfo & drop_part_inf if (deduplication_window == 0) return; + if (stopped) + { + LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we drop this part."); + return; + } + chassert(current_writer != nullptr); for (auto itr = deduplication_map.begin(); itr != deduplication_map.end(); /* no increment here, we erasing from map */) From 338188ae7f1ccdb399671cbfae584ff79705097a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 26 Jul 2023 14:10:27 +0000 Subject: [PATCH 1175/2047] fix test --- tests/queries/0_stateless/02791_remote_paths_refcount.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.sql b/tests/queries/0_stateless/02791_remote_paths_refcount.sql index e64df599d32..180601738ad 100644 --- a/tests/queries/0_stateless/02791_remote_paths_refcount.sql +++ b/tests/queries/0_stateless/02791_remote_paths_refcount.sql @@ -2,6 +2,10 @@ DROP TABLE IF EXISTS t_refcount SYNC; +-- Names of parts (on which this test depends) +-- can differ in case of fault injection. +SET insert_keeper_fault_injection_probability = 0.0; + CREATE TABLE t_refcount (id UInt64, v UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/{database}/t_refcount', '1') ORDER BY id PARTITION BY id % 2 From 6205218e2b7a87c348cd8fb49f595cd36520286c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 26 Jul 2023 14:57:32 +0000 Subject: [PATCH 1176/2047] added tests and implementation --- src/DataTypes/DataTypeArray.cpp | 8 ++++++++ src/DataTypes/DataTypeArray.h | 13 +++++++++++++ .../02812_pointwise_array_operations.reference | 8 ++++++++ .../02812_pointwise_array_operations.sql | 8 ++++++++ 4 files changed, 37 insertions(+) create mode 100644 tests/queries/0_stateless/02812_pointwise_array_operations.reference create mode 100644 tests/queries/0_stateless/02812_pointwise_array_operations.sql diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index b2c712b2f36..effea7bfcd7 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -11,6 +11,7 @@ #include #include +#include namespace DB @@ -20,6 +21,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +using FieldType = Array; DataTypeArray::DataTypeArray(const DataTypePtr & nested_) @@ -33,6 +35,12 @@ MutableColumnPtr DataTypeArray::createColumn() const return ColumnArray::create(nested->createColumn(), ColumnArray::ColumnOffsets::create()); } +MutableColumnPtr DataTypeArray::createColumnConst(size_t size, const Field & field) const +{ + auto column = createColumn(); + column->insert(field); + return ColumnConst::create(std::move(column), size); +} Field DataTypeArray::getDefault() const { diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index 2714ca1d023..f1cbaeb0001 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -15,6 +16,8 @@ private: DataTypePtr nested; public: + using FieldType = Array; + using ColumnType = ColumnArray; static constexpr bool is_parametric = true; explicit DataTypeArray(const DataTypePtr & nested_); @@ -41,6 +44,8 @@ public: } MutableColumnPtr createColumn() const override; + + MutableColumnPtr createColumnConst(size_t size, const Field & field) const; Field getDefault() const override; @@ -67,4 +72,12 @@ public: size_t getNumberOfDimensions() const; }; +template inline constexpr bool IsDataTypeArray() { + return false; +} + +template <> inline constexpr bool IsDataTypeArray() { + return std::is_same_v; +} + } diff --git a/tests/queries/0_stateless/02812_pointwise_array_operations.reference b/tests/queries/0_stateless/02812_pointwise_array_operations.reference new file mode 100644 index 00000000000..6a484ce17dc --- /dev/null +++ b/tests/queries/0_stateless/02812_pointwise_array_operations.reference @@ -0,0 +1,8 @@ +[2,5] + +[2,6] + +[4.5,5,12,10.1] + +[(11.1,5.4),(6,21)] + diff --git a/tests/queries/0_stateless/02812_pointwise_array_operations.sql b/tests/queries/0_stateless/02812_pointwise_array_operations.sql new file mode 100644 index 00000000000..f12306da519 --- /dev/null +++ b/tests/queries/0_stateless/02812_pointwise_array_operations.sql @@ -0,0 +1,8 @@ +SELECT (materialize([1,1]) + materialize([1,4])) + +SELECT ([1,2] + [1,4]) + +SELECT ([2.5, 1, 3, 10.1] + [2, 4, 9, 0]) + +SELECT ([(1,3), (2,9)] + [(10.1, 2.4), (4,12)]) + From f8200e50cb5fe3d266352d3c004d11f9c59810e1 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 26 Jul 2023 15:00:25 +0000 Subject: [PATCH 1177/2047] added impl --- src/Functions/FunctionBinaryArithmetic.h | 111 +++++++++++++++++++++++ 1 file changed, 111 insertions(+) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index c699da4eaf6..1dc8e51060c 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -42,6 +42,15 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -102,6 +111,9 @@ template constexpr bool IsFloatingPoint = false; template <> inline constexpr bool IsFloatingPoint = true; template <> inline constexpr bool IsFloatingPoint = true; +template constexpr bool IsArray = false; +template <> inline constexpr bool IsArray = true; + template constexpr bool IsDateOrDateTime = false; template <> inline constexpr bool IsDateOrDateTime = true; template <> inline constexpr bool IsDateOrDateTime = true; @@ -1125,6 +1137,92 @@ class FunctionBinaryArithmetic : public IFunction return function->execute(arguments, result_type, input_rows_count); } + template + ColumnPtr executeArrayPlusMinus(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, + size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const + { + auto function = function_builder->build(arguments); + return function->execute(arguments, result_type, input_rows_count); + } + + static ColumnPtr callFunctionNotEquals(ColumnWithTypeAndName first, ColumnWithTypeAndName second, ContextPtr context) + { + ColumnsWithTypeAndName args{first, second}; + auto eq_func = FunctionFactory::instance().get("notEquals", context)->build(args); + return eq_func->execute(args, eq_func->getResultType(), args.front().column->size()); + } + + ColumnPtr executeArrayImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const + { + bool is_const = false; + const auto * return_type_array = checkAndGetDataType(result_type.get()); + + if (!return_type_array) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Return type for function {} must be array.", getName()); + + ColumnPtr result_column = executeArray(arguments, result_type, input_rows_count); + + if (arguments[0].dumpStructure().contains("Const")) + is_const = true; + + if (is_const) + return result_column; + else + return ColumnArray::create(result_column, typeid_cast(arguments[0].column.get())->getOffsetsPtr()); + } + + template + ColumnPtr executeArray(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const + { + if constexpr (is_multiply || is_division) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot use multiplication or division on arrays"); + + auto num_args = arguments.size(); + DataTypes data_types; + + ColumnsWithTypeAndName new_arguments {num_args}; + DataTypePtr t; + + const auto * left_const = typeid_cast(arguments[0].column.get()); + const auto * right_const = typeid_cast(arguments[1].column.get()); + + /// Unpacking arrays if both are constants. + if (left_const && right_const) + { + new_arguments[0] = {left_const->getDataColumnPtr(), arguments[0].type, arguments[0].name}; + new_arguments[1] = {right_const->getDataColumnPtr(), arguments[1].type, arguments[1].name}; + auto col = executeImpl(new_arguments, result_type, 1); + return ColumnConst::create(std::move(col), input_rows_count); + } + + /// Unpacking arrays if at least one column is constant. + if (left_const || right_const) + { + new_arguments[0] = {arguments[0].column->convertToFullColumnIfConst(), arguments[0].type, arguments[0].name}; + new_arguments[1] = {arguments[1].column->convertToFullColumnIfConst(), arguments[1].type, arguments[1].name}; + return executeImpl(new_arguments, result_type, input_rows_count); + } + + /// Unpacking non-const arrays and checking sizes of them. + UInt64 data = 0; + for (size_t i = 0; i < num_args; ++i) + { + auto a = typeid_cast(arguments[i].column.get())->getData().getPtr(); + + if (i == 0) + data = *typeid_cast(arguments[i].column.get())->getOffsets().data(); + else + { + if (*typeid_cast(arguments[i].column.get())->getOffsets().data() != data) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Arguments must be one size"); + } + + t = typeid_cast(arguments[i].type.get())->getNestedType(); + new_arguments[i] = {a, t, arguments[i].name}; + } + return executeImpl(new_arguments, t, input_rows_count); + } + ColumnPtr executeTupleNumberOperator(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const { @@ -1326,6 +1424,16 @@ public: return getReturnTypeImplStatic(new_arguments, context); } + if (isArray(arguments[0]) || isArray(arguments[1])) + { + DataTypes new_arguments { + static_cast(*arguments[0]).getNestedType(), + static_cast(*arguments[1]).getNestedType(), + }; + + return std::make_shared(getReturnTypeImplStatic(new_arguments, context)); + } + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1], context)) { @@ -2031,6 +2139,9 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A return (res = executeNumeric(arguments, left, right, right_nullmap)) != nullptr; }); + if (isArray(result_type)) + return executeArrayImpl(arguments, result_type, input_rows_count); + if (!valid) { // This is a logical error, because the types should have been checked From 89f2e8cdea8d7f32be735cd86326d1cbed24e158 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 26 Jul 2023 23:02:57 +0800 Subject: [PATCH 1178/2047] Fix S3 table function does not work for pre-signed URL --- src/TableFunctions/TableFunctionS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index c8cc0cddd30..3637b3e9eb2 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -160,7 +160,7 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context configuration.keys = {configuration.url.key}; if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.url.uri.getPath(), true); + configuration.format = FormatFactory::instance().getFormatFromFileName(Poco::URI(configuration.url.uri.getPath()).getPath(), true); } void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr context) From 2479f1352a62adebdc460dbfde4510ad25fc7184 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 17:11:28 +0200 Subject: [PATCH 1179/2047] fix deadlocks in StorageTableFunctionProxy --- src/Storages/StorageTableFunction.h | 2 +- .../02828_create_as_table_function_rename.reference | 1 + .../0_stateless/02828_create_as_table_function_rename.sql | 7 +++++++ 3 files changed, 9 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02828_create_as_table_function_rename.reference create mode 100644 tests/queries/0_stateless/02828_create_as_table_function_rename.sql diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 26cbe1f0233..3939483495e 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -156,7 +156,7 @@ public: void checkTableCanBeDropped() const override {} private: - mutable std::mutex nested_mutex; + mutable std::recursive_mutex nested_mutex; mutable GetNestedStorageFunc get_nested; mutable StoragePtr nested; const bool add_conversion; diff --git a/tests/queries/0_stateless/02828_create_as_table_function_rename.reference b/tests/queries/0_stateless/02828_create_as_table_function_rename.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02828_create_as_table_function_rename.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02828_create_as_table_function_rename.sql b/tests/queries/0_stateless/02828_create_as_table_function_rename.sql new file mode 100644 index 00000000000..7e24e485fb9 --- /dev/null +++ b/tests/queries/0_stateless/02828_create_as_table_function_rename.sql @@ -0,0 +1,7 @@ + +drop table if exists t1; +create table t1 as remote('localhost', 'system.one'); +rename table t1 to t2; +select * from t2; +rename table t2 to t1; +drop table t1; From 10a8a8dd74acfa9d8ea161c60a44ac7c5abff562 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 16 Jun 2023 23:24:10 -0400 Subject: [PATCH 1180/2047] Implementing new commands for keeper-client --- programs/keeper-client/Commands.cpp | 79 ++++++++++++++++++++++++- programs/keeper-client/Commands.h | 25 ++++++++ programs/keeper-client/KeeperClient.cpp | 2 + programs/keeper-client/Parser.cpp | 1 + 4 files changed, 106 insertions(+), 1 deletion(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 05928a0d20b..d63f87ac8a6 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -24,8 +24,11 @@ void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con else path = client->cwd; - for (const auto & child : client->zookeeper->getChildren(path)) + auto children = client->zookeeper->getChildren(path); + std::sort(children.begin(), children.end()); + for (const auto & child : children) std::cout << child << " "; + std::cout << "\n"; } @@ -130,6 +133,80 @@ void GetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co std::cout << client->zookeeper->get(client->getAbsolutePath(query->args[0].safeGet())) << "\n"; } +bool GetStatCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String arg; + if (!parseKeeperPath(pos, expected, arg)) + return true; + + node->args.push_back(std::move(arg)); + return true; +} + +void GetStatCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + Coordination::Stat stat; + String path; + if (!query->args.empty()) + path = client->getAbsolutePath(query->args[0].safeGet()); + else + path = client->cwd; + + client->zookeeper->get(path, &stat); + + std::cout << "cZxid = " << stat.czxid << "\n"; + std::cout << "mZxid = " << stat.mzxid << "\n"; + std::cout << "ctime = " << stat.ctime << "\n"; + std::cout << "mtime = " << stat.mtime << "\n"; + std::cout << "version = " << stat.version << "\n"; + std::cout << "cversion = " << stat.cversion << "\n"; + std::cout << "aversion = " << stat.aversion << "\n"; + std::cout << "ephemeralOwner = " << stat.ephemeralOwner << "\n"; + std::cout << "dataLength = " << stat.dataLength << "\n"; + std::cout << "numChildren = " << stat.numChildren << "\n"; + std::cout << "pzxid = " << stat.pzxid << "\n"; +} + +bool FindSupperNodes::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + ASTPtr threshold; + if (!ParserUnsignedInteger{}.parse(pos, threshold, expected)) + return false; + + node->args.push_back(threshold->as().value); + + String path; + if (!parseKeeperPath(pos, expected, path)) + path = "."; + + node->args.push_back(std::move(path)); + return true; +} + +void FindSupperNodes::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + auto threshold = query->args[0].safeGet(); + auto path = client->getAbsolutePath(query->args[1].safeGet()); + + Coordination::Stat stat; + client->zookeeper->get(path, &stat); + + if (stat.numChildren >= static_cast(threshold)) + { + std::cout << path << "\t" << stat.numChildren << "\n"; + return; + } + + auto children = client->zookeeper->getChildren(path); + std::sort(children.begin(), children.end()); + for (auto & child : children) + { + auto next_query = *query; + next_query.args[1] = DB::Field(path / child); + FindSupperNodes{}.execute(&next_query, client); + } +} + bool RMCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { String arg; diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index e4debd53e42..c066aefdff1 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -84,6 +84,31 @@ class GetCommand : public IKeeperClientCommand String getHelpMessage() const override { return "get -- Returns the node's value"; } }; +class GetStatCommand : public IKeeperClientCommand +{ + String getName() const override { return "getstat"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override { return "getstat [path] -- Returns the node's stat (default `.`)"; } +}; + +class FindSupperNodes : public IKeeperClientCommand +{ + String getName() const override { return "findsupernodes"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override + { + return "findsupernodes [path] -- Finds nodes with number of children larger than some threshold for the given path (default `.`)"; + } +}; + class RMCommand : public IKeeperClientCommand { String getName() const override { return "rm"; } diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index f41dca1e27a..09156b5e226 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -177,6 +177,8 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), + std::make_shared(), + std::make_shared(), std::make_shared(), std::make_shared(), std::make_shared(), diff --git a/programs/keeper-client/Parser.cpp b/programs/keeper-client/Parser.cpp index 3420ccb2219..fe46058fcc1 100644 --- a/programs/keeper-client/Parser.cpp +++ b/programs/keeper-client/Parser.cpp @@ -58,6 +58,7 @@ bool KeeperParser::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; String command_name(pos->begin, pos->end); + std::transform(command_name.begin(), command_name.end(), command_name.begin(), [](unsigned char c) { return std::tolower(c); }); Command command; auto iter = KeeperClient::commands.find(command_name); From 68b78f52ec170b4bcbea7b79090cae7a3bfc642d Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 20 Jun 2023 23:41:48 -0400 Subject: [PATCH 1181/2047] Add DeleteStableBackups, FindBigFamily --- programs/keeper-client/Commands.cpp | 97 ++++++++++++++++++++++++- programs/keeper-client/Commands.h | 59 ++++++++++++--- programs/keeper-client/KeeperClient.cpp | 2 + 3 files changed, 144 insertions(+), 14 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index d63f87ac8a6..5c8c228097e 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -1,5 +1,6 @@ #include "Commands.h" +#include #include "KeeperClient.h" @@ -203,10 +204,102 @@ void FindSupperNodes::execute(const ASTKeeperQuery * query, KeeperClient * clien { auto next_query = *query; next_query.args[1] = DB::Field(path / child); - FindSupperNodes{}.execute(&next_query, client); + execute(&next_query, client); } } +bool DeleteStableBackups::parse(IParser::Pos & /* pos */, std::shared_ptr & /* node */, Expected & /* expected */) const +{ + return true; +} + +void DeleteStableBackups::execute(const ASTKeeperQuery * /* query */, KeeperClient * client) const +{ + client->askConfirmation( + "You are going to delete all inactive backups in /clickhouse/backups.", + [client] + { + String backup_root = "/clickhouse/backups"; + auto backups = client->zookeeper->getChildren(backup_root); + + for (auto & child : backups) + { + String backup_path = backup_root + "/" + child; + std::cout << "Found backup " << backup_path << ", checking if it's active\n"; + + String stage_path = backup_path + "/stage"; + auto stages = client->zookeeper->getChildren(stage_path); + + bool is_active = false; + for (auto & stage : stages) + { + if (startsWith(stage, "alive")) + { + is_active = true; + break; + } + } + + if (is_active) + { + std::cout << "Backup " << backup_path << " is active, not going to delete\n"; + continue; + } + + std::cout << "Backup " << backup_path << " is not active, deleting it\n"; + client->zookeeper->removeRecursive(backup_path); + } + }); +} + +bool FindBigFamily::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String path; + if (!parseKeeperPath(pos, expected, path)) + path = "."; + + node->args.push_back(std::move(path)); + + ASTPtr count; + if (ParserUnsignedInteger{}.parse(pos, count, expected)) + node->args.push_back(count->as().value); + else + node->args.push_back(UInt64(10)); + + return true; +} + +void FindBigFamily::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + auto path = client->getAbsolutePath(query->args[0].safeGet()); + auto n = query->args[1].safeGet(); + + std::vector> result; + + std::queue queue; + queue.push(path); + while (!queue.empty()) + { + auto next_path = queue.front(); + queue.pop(); + + auto children = client->zookeeper->getChildren(next_path); + std::transform(children.cbegin(), children.cend(), children.begin(), [&](const String & child) { return next_path / child; }); + + auto response = client->zookeeper->get(children); + + for (size_t i = 0; i < response.size(); ++i) + { + result.emplace_back(response[i].stat.numChildren, children[i]); + queue.push(children[i]); + } + } + + std::sort(result.begin(), result.end(), std::greater()); + for (UInt64 i = 0; i < std::min(result.size(), static_cast(n)); ++i) + std::cout << std::get<1>(result[i]) << "\t" << std::get<0>(result[i]) << "\n"; +} + bool RMCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { String arg; @@ -247,7 +340,7 @@ bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptrgetHelpMessage() << "\n"; + std::cout << pair.second->generateHelpString() << "\n"; } bool FourLetterWordCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index c066aefdff1..f428191f694 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -21,6 +21,12 @@ public: virtual String getName() const = 0; virtual ~IKeeperClientCommand() = default; + + String generateHelpString() const + { + return fmt::vformat(getHelpMessage(), fmt::make_format_args(getName())); + } + }; using Command = std::shared_ptr; @@ -34,7 +40,7 @@ class LSCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "ls [path] -- Lists the nodes for the given path (default: cwd)"; } + String getHelpMessage() const override { return "{} [path] -- Lists the nodes for the given path (default: cwd)"; } }; class CDCommand : public IKeeperClientCommand @@ -45,7 +51,7 @@ class CDCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "cd [path] -- Change the working path (default `.`)"; } + String getHelpMessage() const override { return "{} [path] -- Change the working path (default `.`)"; } }; class SetCommand : public IKeeperClientCommand @@ -58,7 +64,7 @@ class SetCommand : public IKeeperClientCommand String getHelpMessage() const override { - return "set [version] -- Updates the node's value. Only update if version matches (default: -1)"; + return "{} [version] -- Updates the node's value. Only update if version matches (default: -1)"; } }; @@ -70,7 +76,7 @@ class CreateCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "create -- Creates new node"; } + String getHelpMessage() const override { return "{} -- Creates new node"; } }; class GetCommand : public IKeeperClientCommand @@ -81,7 +87,7 @@ class GetCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "get -- Returns the node's value"; } + String getHelpMessage() const override { return "{} -- Returns the node's value"; } }; class GetStatCommand : public IKeeperClientCommand @@ -92,12 +98,12 @@ class GetStatCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "getstat [path] -- Returns the node's stat (default `.`)"; } + String getHelpMessage() const override { return "{} [path] -- Returns the node's stat (default `.`)"; } }; class FindSupperNodes : public IKeeperClientCommand { - String getName() const override { return "findsupernodes"; } + String getName() const override { return "find_super_nodes"; } bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; @@ -105,10 +111,39 @@ class FindSupperNodes : public IKeeperClientCommand String getHelpMessage() const override { - return "findsupernodes [path] -- Finds nodes with number of children larger than some threshold for the given path (default `.`)"; + return "{} [path] -- Finds nodes with number of children larger than some threshold for the given path (default `.`)"; } }; +class DeleteStableBackups : public IKeeperClientCommand +{ + String getName() const override { return "delete_stable_backups"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override + { + return "{} -- Deletes ClickHouse nodes used for backups that are now inactive"; + } +}; + +class FindBigFamily : public IKeeperClientCommand +{ + String getName() const override { return "find_big_family"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override + { + return "{} [path] [n] -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10)"; + } +}; + + class RMCommand : public IKeeperClientCommand { String getName() const override { return "rm"; } @@ -117,7 +152,7 @@ class RMCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "remove -- Remove the node"; } + String getHelpMessage() const override { return "{} -- Remove the node"; } }; class RMRCommand : public IKeeperClientCommand @@ -128,7 +163,7 @@ class RMRCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "rmr -- Recursively deletes path. Confirmation required"; } + String getHelpMessage() const override { return "{} -- Recursively deletes path. Confirmation required"; } }; class HelpCommand : public IKeeperClientCommand @@ -139,7 +174,7 @@ class HelpCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "help -- Prints this message"; } + String getHelpMessage() const override { return "{} -- Prints this message"; } }; class FourLetterWordCommand : public IKeeperClientCommand @@ -150,7 +185,7 @@ class FourLetterWordCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "flwc -- Executes four-letter-word command"; } + String getHelpMessage() const override { return "{} -- Executes four-letter-word command"; } }; } diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 09156b5e226..cac3e9d2996 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -179,6 +179,8 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), + std::make_shared(), + std::make_shared(), std::make_shared(), std::make_shared(), std::make_shared(), From bb2a749b9478cf1fa1daccc6fb765d80ca456b4a Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 21 Jul 2023 18:12:26 -0400 Subject: [PATCH 1182/2047] Update programs/keeper-client/Commands.cpp Co-authored-by: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> --- programs/keeper-client/Commands.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 5c8c228097e..9b2539506e5 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -157,6 +157,7 @@ void GetStatCommand::execute(const ASTKeeperQuery * query, KeeperClient * client std::cout << "cZxid = " << stat.czxid << "\n"; std::cout << "mZxid = " << stat.mzxid << "\n"; + std::cout << "pZxid = " << stat.pzxid << "\n"; std::cout << "ctime = " << stat.ctime << "\n"; std::cout << "mtime = " << stat.mtime << "\n"; std::cout << "version = " << stat.version << "\n"; @@ -165,7 +166,6 @@ void GetStatCommand::execute(const ASTKeeperQuery * query, KeeperClient * client std::cout << "ephemeralOwner = " << stat.ephemeralOwner << "\n"; std::cout << "dataLength = " << stat.dataLength << "\n"; std::cout << "numChildren = " << stat.numChildren << "\n"; - std::cout << "pzxid = " << stat.pzxid << "\n"; } bool FindSupperNodes::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const From 3a838f09d1dda7b9ab8d01afe4fab2c32a376884 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 21 Jul 2023 18:14:30 -0400 Subject: [PATCH 1183/2047] Update programs/keeper-client/Commands.h Co-authored-by: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> --- programs/keeper-client/Commands.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index f428191f694..6b7f62474eb 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -92,7 +92,7 @@ class GetCommand : public IKeeperClientCommand class GetStatCommand : public IKeeperClientCommand { - String getName() const override { return "getstat"; } + String getName() const override { return "get_stat"; } bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; From ea672ba618a17d864932cc4a9a709ab186b0b27c Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Jul 2023 01:57:16 -0400 Subject: [PATCH 1184/2047] Add docs, add tests, fix small issues. --- .../utilities/clickhouse-keeper-client.md | 4 + programs/keeper-client/Commands.cpp | 10 +- programs/keeper-client/Commands.h | 2 +- programs/keeper-client/KeeperClient.cpp | 2 +- tests/integration/test_keeper_client/test.py | 123 +++++++++++++++--- 5 files changed, 113 insertions(+), 28 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index 77f816fe428..37eb0bb71ff 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -51,3 +51,7 @@ keeper foo bar - `rmr ` -- Recursively deletes path. Confirmation required - `flwc ` -- Executes four-letter-word command - `help` -- Prints this message +- `get_stat [path]` -- Returns the node's stat (default `.`) +- `find_super_nodes [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`) +- `delete_stable_backups` -- Deletes ClickHouse nodes used for backups that are now inactive +- `find_big_family [path] [n]` -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 9b2539506e5..985e844afdf 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -168,7 +168,7 @@ void GetStatCommand::execute(const ASTKeeperQuery * query, KeeperClient * client std::cout << "numChildren = " << stat.numChildren << "\n"; } -bool FindSupperNodes::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +bool FindSuperNodes::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { ASTPtr threshold; if (!ParserUnsignedInteger{}.parse(pos, threshold, expected)) @@ -184,7 +184,7 @@ bool FindSupperNodes::parse(IParser::Pos & pos, std::shared_ptr return true; } -void FindSupperNodes::execute(const ASTKeeperQuery * query, KeeperClient * client) const +void FindSuperNodes::execute(const ASTKeeperQuery * query, KeeperClient * client) const { auto threshold = query->args[0].safeGet(); auto path = client->getAbsolutePath(query->args[1].safeGet()); @@ -200,7 +200,7 @@ void FindSupperNodes::execute(const ASTKeeperQuery * query, KeeperClient * clien auto children = client->zookeeper->getChildren(path); std::sort(children.begin(), children.end()); - for (auto & child : children) + for (const auto & child : children) { auto next_query = *query; next_query.args[1] = DB::Field(path / child); @@ -222,7 +222,7 @@ void DeleteStableBackups::execute(const ASTKeeperQuery * /* query */, KeeperClie String backup_root = "/clickhouse/backups"; auto backups = client->zookeeper->getChildren(backup_root); - for (auto & child : backups) + for (const auto & child : backups) { String backup_path = backup_root + "/" + child; std::cout << "Found backup " << backup_path << ", checking if it's active\n"; @@ -231,7 +231,7 @@ void DeleteStableBackups::execute(const ASTKeeperQuery * /* query */, KeeperClie auto stages = client->zookeeper->getChildren(stage_path); bool is_active = false; - for (auto & stage : stages) + for (const auto & stage : stages) { if (startsWith(stage, "alive")) { diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index 6b7f62474eb..093920cb10d 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -101,7 +101,7 @@ class GetStatCommand : public IKeeperClientCommand String getHelpMessage() const override { return "{} [path] -- Returns the node's stat (default `.`)"; } }; -class FindSupperNodes : public IKeeperClientCommand +class FindSuperNodes : public IKeeperClientCommand { String getName() const override { return "find_super_nodes"; } diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index cac3e9d2996..561a1f41f7a 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -178,7 +178,7 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), - std::make_shared(), + std::make_shared(), std::make_shared(), std::make_shared(), std::make_shared(), diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 00c7908eeed..a7de8db4a8d 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -1,6 +1,7 @@ import pytest from helpers.client import CommandRequest from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) @@ -23,10 +24,8 @@ def started_cluster(): cluster.shutdown() -def test_base_commands(started_cluster): - _ = started_cluster - - command = CommandRequest( +def keeper_query(query): + return CommandRequest( [ started_cluster.server_bin_path, "keeper-client", @@ -35,29 +34,111 @@ def test_base_commands(started_cluster): "--port", str(cluster.zookeeper_port), "-q", - "create test_create_zk_node1 testvalue1;create test_create_zk_node_2 testvalue2;get test_create_zk_node1;", + query, ], stdin="", ) + +def test_big_family(): + command = keeper_query( + "create test_big_family foo;" + "create test_big_family/1 foo;" + "create test_big_family/1/1 foo;" + "create test_big_family/1/2 foo;" + "create test_big_family/1/3 foo;" + "create test_big_family/1/4 foo;" + "create test_big_family/1/5 foo;" + "create test_big_family/2/1 foo;" + "create test_big_family/2/2 foo;" + "create test_big_family/2/3 foo;" + "find_big_family test_big_family;" + ) + + assert command.get_answer() == TSV( + [ + ["/test_big_family/1", "5"], + ["/test_big_family/2", "3"], + ["/test_big_family/2/3", "0"], + ["/test_big_family/2/2", "0"], + ["/test_big_family/2/1", "0"], + ["/test_big_family/1/5", "0"], + ["/test_big_family/1/4", "0"], + ["/test_big_family/1/3", "0"], + ["/test_big_family/1/2", "0"], + ["/test_big_family/1/1", "0"], + ] + ) + + command = keeper_query("find_big_family test_big_family 1;") + + assert command.get_answer() == TSV( + [ + ["/test_big_family/1", "5"], + ] + ) + + +def test_find_super_nodes(): + command = keeper_query( + "create test_find_super_nodes/1 foo;" + "create test_find_super_nodes/1/1 foo;" + "create test_find_super_nodes/1/2 foo;" + "create test_find_super_nodes/1/3 foo;" + "create test_find_super_nodes/1/4 foo;" + "create test_find_super_nodes/1/5 foo;" + "create test_find_super_nodes/2/1 foo;" + "create test_find_super_nodes/2/2 foo;" + "create test_find_super_nodes/2/3 foo;" + "create test_find_super_nodes/2/4 foo;" + "cd test_find_super_nodes;" + "find_super_nodes 4;" + ) + + assert command.get_answer() == TSV( + [ + ["/test_find_super_nodes/1", "5"], + ["/test_find_super_nodes/2", "4"], + ] + ) + + +def test_delete_stable_backups(): + command = keeper_query( + "create /clickhouse foo;" + "create /clickhouse/backups foo;" + "create /clickhouse/backups/1 foo;" + "create /clickhouse/backups/1/stage foo;" + "create /clickhouse/backups/1/stage/alive123 foo;" + "create /clickhouse/backups/2 foo;" + "create /clickhouse/backups/2/stage foo;" + "create /clickhouse/backups/2/stage/dead123 foo;" + "delete_stable_backups;" + "y;" + "ls clickhouse/backups;" + ) + + assert command.get_answer() == ( + "You are going to delete all inactive backups in /clickhouse/backups. Continue?\n" + "Found backup /clickhouse/backups/1, checking if it's active\n" + "Backup /clickhouse/backups/1 is active, not going to delete\n" + "Found backup /clickhouse/backups/2, checking if it's active\n" + "Backup /clickhouse/backups/2 is not active, deleting it\n" + "1" + ) + + +def test_base_commands(): + command = keeper_query( + "create test_create_zk_node1 testvalue1;" + "create test_create_zk_node_2 testvalue2;" + "get test_create_zk_node1;" + ) + assert command.get_answer() == "testvalue1\n" -def test_four_letter_word_commands(started_cluster): - _ = started_cluster - - command = CommandRequest( - [ - started_cluster.server_bin_path, - "keeper-client", - "--host", - str(cluster.get_instance_ip("zoo1")), - "--port", - str(cluster.zookeeper_port), - "-q", - "ruok", - ], - stdin="", - ) +def test_four_letter_word_commands(): + command = keeper_query("ruok") assert command.get_answer() == "imok\n" From 6b845ae255c0b14371c879f912d7144b039337fe Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Jul 2023 11:13:53 -0400 Subject: [PATCH 1185/2047] Fix tests --- tests/integration/test_keeper_client/test.py | 63 ++++++++++++-------- 1 file changed, 38 insertions(+), 25 deletions(-) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index a7de8db4a8d..5115c7d3789 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -24,24 +24,30 @@ def started_cluster(): cluster.shutdown() -def keeper_query(query): - return CommandRequest( - [ - started_cluster.server_bin_path, - "keeper-client", - "--host", - str(cluster.get_instance_ip("zoo1")), - "--port", - str(cluster.zookeeper_port), - "-q", - query, - ], - stdin="", - ) +class KeeperClient: + def __init__(self, started_cluster: ClickHouseCluster): + self.cluster = started_cluster + + def query(self, query: str): + return CommandRequest( + [ + self.cluster.server_bin_path, + "keeper-client", + "--host", + str(cluster.get_instance_ip("zoo1")), + "--port", + str(cluster.zookeeper_port), + "-q", + query, + ], + stdin="", + ) -def test_big_family(): - command = keeper_query( +def test_big_family(started_cluster: ClickHouseCluster): + client = KeeperClient(started_cluster) + + command = client.query( "create test_big_family foo;" "create test_big_family/1 foo;" "create test_big_family/1/1 foo;" @@ -70,7 +76,7 @@ def test_big_family(): ] ) - command = keeper_query("find_big_family test_big_family 1;") + command = client.query("find_big_family test_big_family 1;") assert command.get_answer() == TSV( [ @@ -79,8 +85,10 @@ def test_big_family(): ) -def test_find_super_nodes(): - command = keeper_query( +def test_find_super_nodes(started_cluster: ClickHouseCluster): + client = KeeperClient(started_cluster) + + command = client.query( "create test_find_super_nodes/1 foo;" "create test_find_super_nodes/1/1 foo;" "create test_find_super_nodes/1/2 foo;" @@ -103,8 +111,10 @@ def test_find_super_nodes(): ) -def test_delete_stable_backups(): - command = keeper_query( +def test_delete_stable_backups(started_cluster: ClickHouseCluster): + client = KeeperClient(started_cluster) + + command = client.query( "create /clickhouse foo;" "create /clickhouse/backups foo;" "create /clickhouse/backups/1 foo;" @@ -128,8 +138,10 @@ def test_delete_stable_backups(): ) -def test_base_commands(): - command = keeper_query( +def test_base_commands(started_cluster: ClickHouseCluster): + client = KeeperClient(started_cluster) + + command = client.query( "create test_create_zk_node1 testvalue1;" "create test_create_zk_node_2 testvalue2;" "get test_create_zk_node1;" @@ -138,7 +150,8 @@ def test_base_commands(): assert command.get_answer() == "testvalue1\n" -def test_four_letter_word_commands(): - command = keeper_query("ruok") +def test_four_letter_word_commands(started_cluster: ClickHouseCluster): + client = KeeperClient(started_cluster) + command = client.query("ruok") assert command.get_answer() == "imok\n" From 74f3e76b182411fee1d49e74aa5040cd4a378967 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 17:15:28 +0200 Subject: [PATCH 1186/2047] fix build with clang-15 --- src/Common/SystemLogBase.cpp | 2 +- src/Interpreters/Cache/Metadata.cpp | 3 +++ src/Storages/HDFS/StorageHDFS.h | 2 ++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index ed5ffd78a7b..3d68fe63227 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -185,7 +185,7 @@ void SystemLogQueue::confirm(uint64_t to_flush_end) } template -SystemLogQueue::Index SystemLogQueue::pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread) +typename SystemLogQueue::Index SystemLogQueue::pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread) { std::unique_lock lock(mutex); flush_event.wait_for(lock, diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 6a68d0f21f7..783c71448fc 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -360,6 +360,9 @@ private: struct DownloadInfo { + DownloadInfo(const CacheMetadata::Key & key_, const size_t & offset_, const std::weak_ptr & file_segment_) + : key(key_), offset(offset_), file_segment(file_segment_) {} + CacheMetadata::Key key; size_t offset; /// We keep weak pointer to file segment diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 74801b68f73..13e46bc1023 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -29,6 +29,8 @@ public: struct PathWithInfo { + PathWithInfo() = default; + PathWithInfo(const String & path_, const std::optional & info_) : path(path_), info(info_) {} String path; std::optional info; }; From adb46fa17498e02b1f92c71735f5ab175d4ff4bf Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 26 Jul 2023 15:20:54 +0000 Subject: [PATCH 1187/2047] Revert "Implemented requested changes" This reverts commit db4735a10508e02b50565bba8a1e71161df90f82. --- src/Functions/array/arrayIntersect.cpp | 47 +++++++++++++++----------- 1 file changed, 27 insertions(+), 20 deletions(-) diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 83b26b56071..7a8bde7ab5f 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -510,7 +510,7 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable map.clear(); bool all_has_nullable = all_nullable; - bool current_has_nullable; + bool current_has_nullable = false; for (size_t arg_num = 0; arg_num < args; ++arg_num) { @@ -549,8 +549,7 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable ++(*value); } } - // We update offsets for all the arrays except the first one. Offsets for the first array would be updated later. - // It is needed to iterate the first array again so that the elements in the result would have fixed order. + if (arg_num) { prev_off[arg_num] = off; @@ -571,21 +570,15 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable else off = (*arg.offsets)[row]; - bool is_map_serialized = false; for (auto i : collections::range(prev_off[0], off)) { + all_has_nullable = all_nullable; + current_has_nullable = false; typename Map::LookupResult pair = nullptr; if (arg.null_map && (*arg.null_map)[i]) { current_has_nullable = true; - if (all_has_nullable && !null_added) - { - ++result_offset; - result_data.insertDefault(); - null_map.push_back(1); - null_added = true; - } if (null_added) continue; } @@ -598,37 +591,51 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable else { const char * data = nullptr; - if (!is_map_serialized) - { - pair = map.find(columns[0]->serializeValueIntoArena(i, arena, data)); - is_map_serialized = true; - } + pair = map.find(columns[0]->serializeValueIntoArena(i, arena, data)); } prev_off[0] = off; if (arg.is_const) prev_off[0] = 0; + if (!current_has_nullable) + all_has_nullable = false; + if (pair && pair->getMapped() == args) { - // We increase pair->getMapped() here to not skip duplicate values from the first array. ++pair->getMapped(); ++result_offset; if constexpr (is_numeric_column) { - result_data.insertValue(pair->getKey()); + if (pair->getKey() == columns[0]->getElement(i)) + { + result_data.insertValue(pair->getKey()); + } } else if constexpr (std::is_same_v || std::is_same_v) { - result_data.insertData(pair->getKey().data, pair->getKey().size); + if (pair->getKey() == columns[0]->getDataAt(i)) + { + result_data.insertData(pair->getKey().data, pair->getKey().size); + } } else { const char * data = nullptr; - result_data.deserializeAndInsertFromArena(pair->getKey().data); + if (pair->getKey() == columns[0]->serializeValueIntoArena(i, arena, data)) + { + result_data.deserializeAndInsertFromArena(pair->getKey().data); + } } if (all_nullable) null_map.push_back(0); } + if (all_has_nullable && !null_added) + { + ++result_offset; + result_data.insertDefault(); + null_map.push_back(1); + null_added = true; + } } result_offsets.getElement(row) = result_offset; From b8cac9499d01bd51e4b8a669c7d23104c656dc7c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 26 Jul 2023 13:18:09 +0000 Subject: [PATCH 1188/2047] Add tests to reproduce the problem --- .../0_stateless/02833_local_with_dialect.reference | 2 ++ tests/queries/0_stateless/02833_local_with_dialect.sh | 8 ++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/02833_local_with_dialect.reference create mode 100755 tests/queries/0_stateless/02833_local_with_dialect.sh diff --git a/tests/queries/0_stateless/02833_local_with_dialect.reference b/tests/queries/0_stateless/02833_local_with_dialect.reference new file mode 100644 index 00000000000..dbb67375997 --- /dev/null +++ b/tests/queries/0_stateless/02833_local_with_dialect.reference @@ -0,0 +1,2 @@ +0 +[?2004h[?2004lBye. diff --git a/tests/queries/0_stateless/02833_local_with_dialect.sh b/tests/queries/0_stateless/02833_local_with_dialect.sh new file mode 100755 index 00000000000..2a2e1b09459 --- /dev/null +++ b/tests/queries/0_stateless/02833_local_with_dialect.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +echo "exit" | ${CLICKHOUSE_LOCAL} --query "from s\"SELECT * FROM numbers(1)\"" --dialect prql --interactive From 3a6aaa29c9db0db1bc2875b7323c334148da387f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 26 Jul 2023 15:25:57 +0000 Subject: [PATCH 1189/2047] Do not load suggestions in case not ClickHouse dialects --- programs/local/LocalServer.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 3c2a8ae3152..6ac7edaf1d9 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -569,7 +569,9 @@ void LocalServer::processConfig() } print_stack_trace = config().getBool("stacktrace", false); - load_suggestions = (is_interactive || delayed_interactive) && !config().getBool("disable_suggestion", false); + const std::string clickhouse_dialect{"clickhouse"}; + load_suggestions = (is_interactive || delayed_interactive) && !config().getBool("disable_suggestion", false) + && config().getString("dialect", clickhouse_dialect) == clickhouse_dialect; auto logging = (config().has("logger.console") || config().has("logger.level") From dc5cc0a5418a9aab424939055e8b67200e1f2996 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 26 Jul 2023 16:23:17 +0000 Subject: [PATCH 1190/2047] fixed check --- src/Functions/array/arrayIntersect.cpp | 27 +++++++++++++------------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 7a8bde7ab5f..6abfb8ec5c6 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -550,6 +550,8 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable } } + // We update offsets for all the arrays except the first one. Offsets for the first array would be updated later. + // It is needed to iterate the first array again so that the elements in the result would have fixed order. if (arg_num) { prev_off[arg_num] = off; @@ -573,12 +575,18 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable for (auto i : collections::range(prev_off[0], off)) { all_has_nullable = all_nullable; - current_has_nullable = false; typename Map::LookupResult pair = nullptr; if (arg.null_map && (*arg.null_map)[i]) { current_has_nullable = true; + if (all_has_nullable && !null_added) + { + ++result_offset; + result_data.insertDefault(); + null_map.push_back(1); + null_added = true; + } if (null_added) continue; } @@ -602,29 +610,20 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable if (pair && pair->getMapped() == args) { + // We increase pair->getMapped() here to not skip duplicate values from the first array. ++pair->getMapped(); ++result_offset; if constexpr (is_numeric_column) { - if (pair->getKey() == columns[0]->getElement(i)) - { - result_data.insertValue(pair->getKey()); - } + result_data.insertValue(pair->getKey()); } else if constexpr (std::is_same_v || std::is_same_v) { - if (pair->getKey() == columns[0]->getDataAt(i)) - { - result_data.insertData(pair->getKey().data, pair->getKey().size); - } + result_data.insertData(pair->getKey().data, pair->getKey().size); } else { - const char * data = nullptr; - if (pair->getKey() == columns[0]->serializeValueIntoArena(i, arena, data)) - { - result_data.deserializeAndInsertFromArena(pair->getKey().data); - } + result_data.deserializeAndInsertFromArena(pair->getKey().data); } if (all_nullable) null_map.push_back(0); From 0a838dc6d19af963a021aa1910f2144839f21d4a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 26 Jul 2023 18:30:18 +0200 Subject: [PATCH 1191/2047] Revert "Remove `mmap/mremap/munmap` from Allocator.h" (#52589) --- src/Common/Allocator.cpp | 26 ++- src/Common/Allocator.h | 182 ++++++++++++++---- src/Common/Allocator_fwd.h | 2 +- src/Common/CurrentMetrics.cpp | 2 + src/Common/HashTable/HashTableAllocator.h | 2 +- .../01778_mmap_cache_infra.reference | 2 + 6 files changed, 177 insertions(+), 39 deletions(-) diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 769df70d71e..0fb90e5a47e 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -1,4 +1,26 @@ #include "Allocator.h" -template class Allocator; -template class Allocator; +/** Keep definition of this constant in cpp file; otherwise its value + * is inlined into allocator code making it impossible to override it + * in third-party code. + * + * Note: extern may seem redundant, but is actually needed due to bug in GCC. + * See also: https://gcc.gnu.org/legacy-ml/gcc-help/2017-12/msg00021.html + */ +#ifdef NDEBUG + __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 128 * (1ULL << 20); +#else + /** + * In debug build, use small mmap threshold to reproduce more memory + * stomping bugs. Along with ASLR it will hopefully detect more issues than + * ASan. The program may fail due to the limit on number of memory mappings. + * + * Not too small to avoid too quick exhaust of memory mappings. + */ + __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 16384; +#endif + +template class Allocator; +template class Allocator; +template class Allocator; +template class Allocator; diff --git a/src/Common/Allocator.h b/src/Common/Allocator.h index 1e77e988326..5180fbdaa2d 100644 --- a/src/Common/Allocator.h +++ b/src/Common/Allocator.h @@ -36,26 +36,51 @@ #include +/// Required for older Darwin builds, that lack definition of MAP_ANONYMOUS +#ifndef MAP_ANONYMOUS +#define MAP_ANONYMOUS MAP_ANON +#endif + +/** + * Many modern allocators (for example, tcmalloc) do not do a mremap for + * realloc, even in case of large enough chunks of memory. Although this allows + * you to increase performance and reduce memory consumption during realloc. + * To fix this, we do mremap manually if the chunk of memory is large enough. + * The threshold (64 MB) is chosen quite large, since changing the address + * space is very slow, especially in the case of a large number of threads. We + * expect that the set of operations mmap/something to do/mremap can only be + * performed about 1000 times per second. + * + * P.S. This is also required, because tcmalloc can not allocate a chunk of + * memory greater than 16 GB. + * + * P.P.S. Note that MMAP_THRESHOLD symbol is intentionally made weak. It allows + * to override it during linkage when using ClickHouse as a library in + * third-party applications which may already use own allocator doing mmaps + * in the implementation of alloc/realloc. + */ +extern const size_t MMAP_THRESHOLD; + static constexpr size_t MALLOC_MIN_ALIGNMENT = 8; +namespace CurrentMetrics +{ + extern const Metric MMappedAllocs; + extern const Metric MMappedAllocBytes; +} + namespace DB { - namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int CANNOT_ALLOCATE_MEMORY; + extern const int CANNOT_MUNMAP; + extern const int CANNOT_MREMAP; extern const int LOGICAL_ERROR; } - } -/** Previously there was a code which tried to use manual mmap and mremap (clickhouse_mremap.h) for large allocations/reallocations (64MB+). - * Most modern allocators (including jemalloc) don't use mremap, so the idea was to take advantage from mremap system call for large reallocs. - * Actually jemalloc had support for mremap, but it was intentionally removed from codebase https://github.com/jemalloc/jemalloc/commit/e2deab7a751c8080c2b2cdcfd7b11887332be1bb. - * Our performance tests also shows that without manual mmap/mremap/munmap clickhouse is overall faster for about 1-2% and up to 5-7x for some types of queries. - * That is why we don't do manuall mmap/mremap/munmap here and completely rely on jemalloc for allocations of any size. - */ - /** Responsible for allocating / freeing memory. Used, for example, in PODArray, Arena. * Also used in hash tables. * The interface is different from std::allocator @@ -63,8 +88,10 @@ namespace ErrorCodes * - passing the size into the `free` method; * - by the presence of the `alignment` argument; * - the possibility of zeroing memory (used in hash tables); + * - random hint address for mmap + * - mmap_threshold for using mmap less or more */ -template +template class Allocator { public: @@ -82,7 +109,7 @@ public: try { checkSize(size); - freeNoTrack(buf); + freeNoTrack(buf, size); CurrentMemoryTracker::free(size); } catch (...) @@ -105,26 +132,49 @@ public: /// nothing to do. /// BTW, it's not possible to change alignment while doing realloc. } - else if (alignment <= MALLOC_MIN_ALIGNMENT) + else if (old_size < MMAP_THRESHOLD && new_size < MMAP_THRESHOLD + && alignment <= MALLOC_MIN_ALIGNMENT) { /// Resize malloc'd memory region with no special alignment requirement. CurrentMemoryTracker::realloc(old_size, new_size); void * new_buf = ::realloc(buf, new_size); if (nullptr == new_buf) - { - DB::throwFromErrno( - fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); - } + DB::throwFromErrno(fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); buf = new_buf; if constexpr (clear_memory) if (new_size > old_size) memset(reinterpret_cast(buf) + old_size, 0, new_size - old_size); } + else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD) + { + /// Resize mmap'd memory region. + CurrentMemoryTracker::realloc(old_size, new_size); + + // On apple and freebsd self-implemented mremap used (common/mremap.h) + buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE, + PROT_READ | PROT_WRITE, mmap_flags, -1, 0); + if (MAP_FAILED == buf) + DB::throwFromErrno(fmt::format("Allocator: Cannot mremap memory chunk from {} to {}.", + ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_MREMAP); + + /// No need for zero-fill, because mmap guarantees it. + } + else if (new_size < MMAP_THRESHOLD) + { + /// Small allocs that requires a copy. Assume there's enough memory in system. Call CurrentMemoryTracker once. + CurrentMemoryTracker::realloc(old_size, new_size); + + void * new_buf = allocNoTrack(new_size, alignment); + memcpy(new_buf, buf, std::min(old_size, new_size)); + freeNoTrack(buf, old_size); + buf = new_buf; + } else { /// Big allocs that requires a copy. MemoryTracker is called inside 'alloc', 'free' methods. + void * new_buf = alloc(new_size, alignment); memcpy(new_buf, buf, std::min(old_size, new_size)); free(buf, old_size); @@ -142,38 +192,83 @@ protected: static constexpr bool clear_memory = clear_memory_; + // Freshly mmapped pages are copy-on-write references to a global zero page. + // On the first write, a page fault occurs, and an actual writable page is + // allocated. If we are going to use this memory soon, such as when resizing + // hash tables, it makes sense to pre-fault the pages by passing + // MAP_POPULATE to mmap(). This takes some time, but should be faster + // overall than having a hot loop interrupted by page faults. + // It is only supported on Linux. + static constexpr int mmap_flags = MAP_PRIVATE | MAP_ANONYMOUS +#if defined(OS_LINUX) + | (mmap_populate ? MAP_POPULATE : 0) +#endif + ; + private: void * allocNoTrack(size_t size, size_t alignment) { void * buf; - if (alignment <= MALLOC_MIN_ALIGNMENT) - { - if constexpr (clear_memory) - buf = ::calloc(size, 1); - else - buf = ::malloc(size); + size_t mmap_min_alignment = ::getPageSize(); - if (nullptr == buf) - DB::throwFromErrno(fmt::format("Allocator: Cannot malloc {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + if (size >= MMAP_THRESHOLD) + { + if (alignment > mmap_min_alignment) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, + "Too large alignment {}: more than page size when allocating {}.", + ReadableSize(alignment), ReadableSize(size)); + + buf = mmap(getMmapHint(), size, PROT_READ | PROT_WRITE, + mmap_flags, -1, 0); + if (MAP_FAILED == buf) + DB::throwFromErrno(fmt::format("Allocator: Cannot mmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + /// No need for zero-fill, because mmap guarantees it. + + CurrentMetrics::add(CurrentMetrics::MMappedAllocs); + CurrentMetrics::add(CurrentMetrics::MMappedAllocBytes, size); } else { - buf = nullptr; - int res = posix_memalign(&buf, alignment, size); + if (alignment <= MALLOC_MIN_ALIGNMENT) + { + if constexpr (clear_memory) + buf = ::calloc(size, 1); + else + buf = ::malloc(size); - if (0 != res) - DB::throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)), - DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); + if (nullptr == buf) + DB::throwFromErrno(fmt::format("Allocator: Cannot malloc {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + } + else + { + buf = nullptr; + int res = posix_memalign(&buf, alignment, size); - if constexpr (clear_memory) - memset(buf, 0, size); + if (0 != res) + DB::throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)), + DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); + + if constexpr (clear_memory) + memset(buf, 0, size); + } } return buf; } - void freeNoTrack(void * buf) + void freeNoTrack(void * buf, size_t size) { - ::free(buf); + if (size >= MMAP_THRESHOLD) + { + if (0 != munmap(buf, size)) + DB::throwFromErrno(fmt::format("Allocator: Cannot munmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_MUNMAP); + + CurrentMetrics::sub(CurrentMetrics::MMappedAllocs); + CurrentMetrics::sub(CurrentMetrics::MMappedAllocBytes, size); + } + else + { + ::free(buf); + } } void checkSize(size_t size) @@ -182,6 +277,21 @@ private: if (size >= 0x8000000000000000ULL) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Too large size ({}) passed to allocator. It indicates an error.", size); } + +#ifndef NDEBUG + /// In debug builds, request mmap() at random addresses (a kind of ASLR), to + /// reproduce more memory stomping bugs. Note that Linux doesn't do it by + /// default. This may lead to worse TLB performance. + void * getMmapHint() + { + return reinterpret_cast(std::uniform_int_distribution(0x100000000000UL, 0x700000000000UL)(thread_local_rng)); + } +#else + void * getMmapHint() + { + return nullptr; + } +#endif }; @@ -257,5 +367,7 @@ constexpr size_t allocatorInitialBytes; -extern template class Allocator; +extern template class Allocator; +extern template class Allocator; +extern template class Allocator; +extern template class Allocator; diff --git a/src/Common/Allocator_fwd.h b/src/Common/Allocator_fwd.h index a96bc2a503b..a13a4398654 100644 --- a/src/Common/Allocator_fwd.h +++ b/src/Common/Allocator_fwd.h @@ -3,7 +3,7 @@ * This file provides forward declarations for Allocator. */ -template +template class Allocator; template diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 583b13cf79d..e290fc8ccd3 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -173,6 +173,8 @@ M(PartsInMemory, "In-memory parts.") \ M(MMappedFiles, "Total number of mmapped files.") \ M(MMappedFileBytes, "Sum size of mmapped file regions.") \ + M(MMappedAllocs, "Total number of mmapped allocations") \ + M(MMappedAllocBytes, "Sum bytes of mmapped allocations") \ M(AsynchronousReadWait, "Number of threads waiting for asynchronous read.") \ M(PendingAsyncInsert, "Number of asynchronous inserts that are waiting for flush.") \ M(KafkaConsumers, "Number of active Kafka consumers") \ diff --git a/src/Common/HashTable/HashTableAllocator.h b/src/Common/HashTable/HashTableAllocator.h index 8252265111d..47e3fdfc4b6 100644 --- a/src/Common/HashTable/HashTableAllocator.h +++ b/src/Common/HashTable/HashTableAllocator.h @@ -8,7 +8,7 @@ * table, so it makes sense to pre-fault the pages so that page faults don't * interrupt the resize loop. Set the allocator parameter accordingly. */ -using HashTableAllocator = Allocator; +using HashTableAllocator = Allocator; template using HashTableAllocatorWithStackMemory = AllocatorWithStackMemory; diff --git a/tests/queries/0_stateless/01778_mmap_cache_infra.reference b/tests/queries/0_stateless/01778_mmap_cache_infra.reference index 0e82b277bc1..ed365028ecc 100644 --- a/tests/queries/0_stateless/01778_mmap_cache_infra.reference +++ b/tests/queries/0_stateless/01778_mmap_cache_infra.reference @@ -2,5 +2,7 @@ CreatedReadBufferMMap CreatedReadBufferMMapFailed MMappedFileCacheHits MMappedFileCacheMisses +MMappedAllocBytes +MMappedAllocs MMappedFileBytes MMappedFiles From d89e2e6a27746dbb8febd2990d1ed3c23fcf153b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 26 Jul 2023 19:58:41 +0200 Subject: [PATCH 1192/2047] Add SYSTEM STOP LISTEN query (#51016) Co-authored-by: Nikita Mikhaylov Co-authored-by: Nikita Mikhaylov --- docs/en/sql-reference/statements/system.md | 26 + programs/keeper/CMakeLists.txt | 1 + programs/server/Server.cpp | 448 +++++++++++------- programs/server/Server.h | 14 +- src/Access/Common/AccessType.h | 1 + src/Interpreters/Context.cpp | 34 ++ src/Interpreters/Context.h | 8 + src/Interpreters/InterpreterSystemQuery.cpp | 19 +- src/Parsers/ASTSystemQuery.cpp | 11 + src/Parsers/ASTSystemQuery.h | 7 +- src/Parsers/ParserSystemQuery.cpp | 36 ++ src/Parsers/examples/CMakeLists.txt | 4 +- src/Server/ServerType.cpp | 138 ++++++ src/Server/ServerType.h | 44 ++ .../test_system_start_stop_listen/__init__.py | 0 .../configs/cluster.xml | 16 + .../test_system_start_stop_listen/test.py | 40 ++ .../01271_show_privileges.reference | 1 + .../02117_show_create_table_system.reference | 6 +- 19 files changed, 663 insertions(+), 191 deletions(-) create mode 100644 src/Server/ServerType.cpp create mode 100644 src/Server/ServerType.h create mode 100644 tests/integration/test_system_start_stop_listen/__init__.py create mode 100644 tests/integration/test_system_start_stop_listen/configs/cluster.xml create mode 100644 tests/integration/test_system_start_stop_listen/test.py diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 65a35f03fbe..fb601cd5d35 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -414,3 +414,29 @@ Will do sync syscall. ```sql SYSTEM SYNC FILE CACHE [ON CLUSTER cluster_name] ``` + + +### SYSTEM STOP LISTEN + +Closes the socket and gracefully terminates the existing connections to the server on the specified port with the specified protocol. + +However, if the corresponding protocol settings were not specified in the clickhouse-server configuration, this command will have no effect. + +```sql +SYSTEM STOP LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QUERIES CUSTOM | TCP | TCP_WITH_PROXY | TCP_SECURE | HTTP | HTTPS | MYSQL | GRPC | POSTGRESQL | PROMETHEUS | CUSTOM 'protocol'] +``` + +- If `CUSTOM 'protocol'` modifier is specified, the custom protocol with the specified name defined in the protocols section of the server configuration will be stopped. +- If `QUERIES ALL` modifier is specified, all protocols are stopped. +- If `QUERIES DEFAULT` modifier is specified, all default protocols are stopped. +- If `QUERIES CUSTOM` modifier is specified, all custom protocols are stopped. + +### SYSTEM START LISTEN + +Allows new connections to be established on the specified protocols. + +However, if the server on the specified port and protocol was not stopped using the SYSTEM STOP LISTEN command, this command will have no effect. + +```sql +SYSTEM START LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QUERIES CUSTOM | TCP | TCP_WITH_PROXY | TCP_SECURE | HTTP | HTTPS | MYSQL | GRPC | POSTGRESQL | PROMETHEUS | CUSTOM 'protocol'] +``` diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index abf31a7a499..43a8d84b513 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -65,6 +65,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusRequestHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/PrometheusMetricsWriter.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/waitServersToFinish.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/ServerType.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTPRequestHandlerFactoryMain.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/HTTPServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTP/ReadHeaders.cpp diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 960b6574633..dce52ecdb12 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1457,6 +1457,24 @@ try access_control.reload(AccessControl::ReloadMode::USERS_CONFIG_ONLY); }); + global_context->setStopServersCallback([&](const ServerType & server_type) + { + stopServers(servers, server_type); + }); + + global_context->setStartServersCallback([&](const ServerType & server_type) + { + createServers( + config(), + listen_hosts, + listen_try, + server_pool, + async_metrics, + servers, + /* start_servers= */ true, + server_type); + }); + /// Limit on total number of concurrently executed queries. global_context->getProcessList().setMaxSize(server_settings.max_concurrent_queries); @@ -1998,7 +2016,8 @@ void Server::createServers( Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, std::vector & servers, - bool start_servers) + bool start_servers, + const ServerType & server_type) { const Settings & settings = global_context->getSettingsRef(); @@ -2012,6 +2031,9 @@ void Server::createServers( for (const auto & protocol : protocols) { + if (!server_type.shouldStart(ServerType::Type::CUSTOM, protocol)) + continue; + std::vector hosts; if (config.has("protocols." + protocol + ".host")) hosts.push_back(config.getString("protocols." + protocol + ".host")); @@ -2058,162 +2080,190 @@ void Server::createServers( for (const auto & listen_host : listen_hosts) { - /// HTTP - const char * port_name = "http_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); + const char * port_name; - return ProtocolServerAdapter( - listen_host, - port_name, - "http://" + address.toString(), - std::make_unique( - httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); - }); - - /// HTTPS - port_name = "https_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::HTTP)) { + /// HTTP + port_name = "http_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + + return ProtocolServerAdapter( + listen_host, + port_name, + "http://" + address.toString(), + std::make_unique( + httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params)); + }); + } + + if (server_type.shouldStart(ServerType::Type::HTTPS)) + { + /// HTTPS + port_name = "https_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { #if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "https://" + address.toString(), - std::make_unique( - httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "https://" + address.toString(), + std::make_unique( + httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params)); #else - UNUSED(port); - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "HTTPS protocol is disabled because Poco library was built without NetSSL support."); + UNUSED(port); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "HTTPS protocol is disabled because Poco library was built without NetSSL support."); #endif - }); + }); + } - /// TCP - port_name = "tcp_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::TCP)) { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "native protocol (tcp): " + address.toString(), - std::make_unique( - new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - }); + /// TCP + port_name = "tcp_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "native protocol (tcp): " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }); + } - /// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt - port_name = "tcp_with_proxy_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::TCP_WITH_PROXY)) { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "native protocol (tcp) with PROXY: " + address.toString(), - std::make_unique( - new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - }); + /// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt + port_name = "tcp_with_proxy_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "native protocol (tcp) with PROXY: " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }); + } - /// TCP with SSL - port_name = "tcp_port_secure"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::TCP_SECURE)) { -#if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "secure native protocol (tcp_secure): " + address.toString(), - std::make_unique( - new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false), - server_pool, - socket, - new Poco::Net::TCPServerParams)); -#else - UNUSED(port); - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); -#endif - }); + /// TCP with SSL + port_name = "tcp_port_secure"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + #if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "secure native protocol (tcp_secure): " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + #else + UNUSED(port); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); + #endif + }); + } - port_name = "mysql_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::MYSQL)) { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan()); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "MySQL compatibility protocol: " + address.toString(), - std::make_unique(new MySQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); - }); + port_name = "mysql_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan()); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "MySQL compatibility protocol: " + address.toString(), + std::make_unique(new MySQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); + }); + } - port_name = "postgresql_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::POSTGRESQL)) { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan()); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "PostgreSQL compatibility protocol: " + address.toString(), - std::make_unique(new PostgreSQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); - }); + port_name = "postgresql_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan()); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "PostgreSQL compatibility protocol: " + address.toString(), + std::make_unique(new PostgreSQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams)); + }); + } #if USE_GRPC - port_name = "grpc_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::GRPC)) { - Poco::Net::SocketAddress server_address(listen_host, port); - return ProtocolServerAdapter( - listen_host, - port_name, - "gRPC protocol: " + server_address.toString(), - std::make_unique(*this, makeSocketAddress(listen_host, port, &logger()))); - }); + port_name = "grpc_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::SocketAddress server_address(listen_host, port); + return ProtocolServerAdapter( + listen_host, + port_name, + "gRPC protocol: " + server_address.toString(), + std::make_unique(*this, makeSocketAddress(listen_host, port, &logger()))); + }); + } #endif - - /// Prometheus (if defined and not setup yet with http_port) - port_name = "prometheus.port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::PROMETHEUS)) { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "Prometheus: http://" + address.toString(), - std::make_unique( - httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); - }); + /// Prometheus (if defined and not setup yet with http_port) + port_name = "prometheus.port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "Prometheus: http://" + address.toString(), + std::make_unique( + httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params)); + }); + } } } @@ -2224,7 +2274,8 @@ void Server::createInterserverServers( Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, std::vector & servers, - bool start_servers) + bool start_servers, + const ServerType & server_type) { const Settings & settings = global_context->getSettingsRef(); @@ -2236,52 +2287,97 @@ void Server::createInterserverServers( /// Now iterate over interserver_listen_hosts for (const auto & interserver_listen_host : interserver_listen_hosts) { - /// Interserver IO HTTP - const char * port_name = "interserver_http_port"; - createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, interserver_listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - interserver_listen_host, - port_name, - "replica communication (interserver): http://" + address.toString(), - std::make_unique( - httpContext(), - createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"), - server_pool, - socket, - http_params)); - }); + const char * port_name; - port_name = "interserver_https_port"; - createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTP)) { + /// Interserver IO HTTP + port_name = "interserver_http_port"; + createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, interserver_listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + interserver_listen_host, + port_name, + "replica communication (interserver): http://" + address.toString(), + std::make_unique( + httpContext(), + createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"), + server_pool, + socket, + http_params)); + }); + } + + if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTPS)) + { + port_name = "interserver_https_port"; + createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { #if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, interserver_listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - interserver_listen_host, - port_name, - "secure replica communication (interserver): https://" + address.toString(), - std::make_unique( - httpContext(), - createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"), - server_pool, - socket, - http_params)); + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, interserver_listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + interserver_listen_host, + port_name, + "secure replica communication (interserver): https://" + address.toString(), + std::make_unique( + httpContext(), + createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"), + server_pool, + socket, + http_params)); #else - UNUSED(port); - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); + UNUSED(port); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); #endif - }); + }); + } } } +void Server::stopServers( + std::vector & servers, + const ServerType & server_type +) const +{ + Poco::Logger * log = &logger(); + + /// Remove servers once all their connections are closed + auto check_server = [&log](const char prefix[], auto & server) + { + if (!server.isStopping()) + return false; + size_t current_connections = server.currentConnections(); + LOG_DEBUG(log, "Server {}{}: {} ({} connections)", + server.getDescription(), + prefix, + !current_connections ? "finished" : "waiting", + current_connections); + return !current_connections; + }; + + std::erase_if(servers, std::bind_front(check_server, " (from one of previous remove)")); + + for (auto & server : servers) + { + if (!server.isStopping()) + { + const std::string server_port_name = server.getPortName(); + + if (server_type.shouldStop(server_port_name)) + server.stop(); + } + } + + std::erase_if(servers, std::bind_front(check_server, "")); +} + void Server::updateServers( Poco::Util::AbstractConfiguration & config, Poco::ThreadPool & server_pool, diff --git a/programs/server/Server.h b/programs/server/Server.h index d13378dcd65..3f03dd137ef 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -3,8 +3,9 @@ #include #include -#include "Server/HTTP/HTTPContext.h" +#include #include +#include #include /** Server provides three interfaces: @@ -106,7 +107,8 @@ private: Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, std::vector & servers, - bool start_servers = false); + bool start_servers = false, + const ServerType & server_type = ServerType(ServerType::Type::QUERIES_ALL)); void createInterserverServers( Poco::Util::AbstractConfiguration & config, @@ -115,7 +117,8 @@ private: Poco::ThreadPool & server_pool, AsynchronousMetrics & async_metrics, std::vector & servers, - bool start_servers = false); + bool start_servers = false, + const ServerType & server_type = ServerType(ServerType::Type::QUERIES_ALL)); void updateServers( Poco::Util::AbstractConfiguration & config, @@ -123,6 +126,11 @@ private: AsynchronousMetrics & async_metrics, std::vector & servers, std::vector & servers_to_start_before_tables); + + void stopServers( + std::vector & servers, + const ServerType & server_type + ) const; }; } diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 374a1dd04a4..b253a0e13ce 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -187,6 +187,7 @@ enum class AccessType M(SYSTEM_THREAD_FUZZER, "SYSTEM START THREAD FUZZER, SYSTEM STOP THREAD FUZZER, START THREAD FUZZER, STOP THREAD FUZZER", GLOBAL, SYSTEM) \ M(SYSTEM_UNFREEZE, "SYSTEM UNFREEZE", GLOBAL, SYSTEM) \ M(SYSTEM_FAILPOINT, "SYSTEM ENABLE FAILPOINT, SYSTEM DISABLE FAILPOINT", GLOBAL, SYSTEM) \ + M(SYSTEM_LISTEN, "SYSTEM START LISTEN, SYSTEM STOP LISTEN", GLOBAL, SYSTEM) \ M(SYSTEM, "", GROUP, ALL) /* allows to execute SYSTEM {SHUTDOWN|RELOAD CONFIG|...} */ \ \ M(dictGet, "dictHas, dictGetHierarchy, dictIsIn", DICTIONARY, ALL) /* allows to execute functions dictGet(), dictHas(), dictGetHierarchy(), dictIsIn() */\ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index cc77e0fe723..f83e524ffb9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -357,6 +358,9 @@ struct ContextSharedPart : boost::noncopyable Context::ConfigReloadCallback config_reload_callback; + Context::StartStopServersCallback start_servers_callback; + Context::StartStopServersCallback stop_servers_callback; + bool is_server_completely_started = false; #if USE_ROCKSDB @@ -3688,6 +3692,36 @@ void Context::reloadConfig() const shared->config_reload_callback(); } +void Context::setStartServersCallback(StartStopServersCallback && callback) +{ + /// Is initialized at server startup, so lock isn't required. Otherwise use mutex. + shared->start_servers_callback = std::move(callback); +} + +void Context::setStopServersCallback(StartStopServersCallback && callback) +{ + /// Is initialized at server startup, so lock isn't required. Otherwise use mutex. + shared->stop_servers_callback = std::move(callback); +} + +void Context::startServers(const ServerType & server_type) const +{ + /// Use mutex if callback may be changed after startup. + if (!shared->start_servers_callback) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't start servers because start_servers_callback is not set."); + + shared->start_servers_callback(server_type); +} + +void Context::stopServers(const ServerType & server_type) const +{ + /// Use mutex if callback may be changed after startup. + if (!shared->stop_servers_callback) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't stop servers because stop_servers_callback is not set."); + + shared->stop_servers_callback(server_type); +} + void Context::shutdown() { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index fa210f04451..75752774d4c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -134,6 +134,7 @@ using StoragePolicyPtr = std::shared_ptr; using StoragePoliciesMap = std::map; class StoragePolicySelector; using StoragePolicySelectorPtr = std::shared_ptr; +class ServerType; template class MergeTreeBackgroundExecutor; @@ -1057,6 +1058,13 @@ public: void setConfigReloadCallback(ConfigReloadCallback && callback); void reloadConfig() const; + using StartStopServersCallback = std::function; + void setStartServersCallback(StartStopServersCallback && callback); + void setStopServersCallback(StartStopServersCallback && callback); + + void startServers(const ServerType & server_type) const; + void stopServers(const ServerType & server_type) const; + void shutdown(); bool isInternalQuery() const { return is_internal_query; } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 1bd30e06888..3207da9941a 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -556,6 +556,14 @@ BlockIO InterpreterSystemQuery::execute() ); break; } + case Type::STOP_LISTEN: + getContext()->checkAccess(AccessType::SYSTEM_LISTEN); + getContext()->stopServers(query.server_type); + break; + case Type::START_LISTEN: + getContext()->checkAccess(AccessType::SYSTEM_LISTEN); + getContext()->startServers(query.server_type); + break; case Type::FLUSH_ASYNC_INSERT_QUEUE: { getContext()->checkAccess(AccessType::SYSTEM_FLUSH_ASYNC_INSERT_QUEUE); @@ -567,9 +575,6 @@ BlockIO InterpreterSystemQuery::execute() queue->flushAll(); break; } - case Type::STOP_LISTEN_QUERIES: - case Type::START_LISTEN_QUERIES: - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is not supported yet", query.type); case Type::STOP_THREAD_FUZZER: getContext()->checkAccess(AccessType::SYSTEM_THREAD_FUZZER); ThreadFuzzer::stop(); @@ -1181,8 +1186,12 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_SYNC_FILE_CACHE); break; } - case Type::STOP_LISTEN_QUERIES: - case Type::START_LISTEN_QUERIES: + case Type::STOP_LISTEN: + case Type::START_LISTEN: + { + required_access.emplace_back(AccessType::SYSTEM_LISTEN); + break; + } case Type::STOP_THREAD_FUZZER: case Type::START_THREAD_FUZZER: case Type::ENABLE_FAILPOINT: diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index a91449ff035..754eb825dcc 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -220,6 +220,17 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, { settings.ostr << (settings.hilite ? hilite_none : ""); } + else if (type == Type::START_LISTEN || type == Type::STOP_LISTEN) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " " << ServerType::serverTypeToString(server_type.type) + << (settings.hilite ? hilite_none : ""); + + if (server_type.type == ServerType::CUSTOM) + { + settings.ostr << (settings.hilite ? hilite_identifier : "") << " " << backQuoteIfNeed(server_type.custom_name); + } + + } } diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index b18f8fc7b07..ebaf357c0ab 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -3,6 +3,7 @@ #include #include #include +#include #include "config.h" @@ -35,8 +36,8 @@ public: #if USE_AWS_S3 DROP_S3_CLIENT_CACHE, #endif - STOP_LISTEN_QUERIES, - START_LISTEN_QUERIES, + STOP_LISTEN, + START_LISTEN, RESTART_REPLICAS, RESTART_REPLICA, RESTORE_REPLICA, @@ -116,6 +117,8 @@ public: SyncReplicaMode sync_replica_mode = SyncReplicaMode::DEFAULT; + ServerType server_type; + String getID(char) const override { return "SYSTEM query"; } ASTPtr clone() const override diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 48dbe60e241..9aff0e8879e 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -442,6 +442,42 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & break; } + case Type::START_LISTEN: + case Type::STOP_LISTEN: + { + if (!parseQueryWithOnCluster(res, pos, expected)) + return false; + + ServerType::Type current_type = ServerType::Type::END; + std::string current_custom_name; + + for (const auto & type : magic_enum::enum_values()) + { + if (ParserKeyword{ServerType::serverTypeToString(type)}.ignore(pos, expected)) + { + current_type = type; + break; + } + } + + if (current_type == ServerType::Type::END) + return false; + + if (current_type == ServerType::CUSTOM) + { + ASTPtr ast; + + if (!ParserStringLiteral{}.parse(pos, ast, expected)) + return false; + + current_custom_name = ast->as().value.get(); + } + + res->server_type = ServerType(current_type, current_custom_name); + + break; + } + default: { if (!parseQueryWithOnCluster(res, pos, expected)) diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index 82ca7bc0688..e411574bd65 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -3,8 +3,8 @@ set(SRCS) clickhouse_add_executable(lexer lexer.cpp ${SRCS}) target_link_libraries(lexer PRIVATE clickhouse_parsers) -clickhouse_add_executable(select_parser select_parser.cpp ${SRCS}) +clickhouse_add_executable(select_parser select_parser.cpp ${SRCS} "../../Server/ServerType.cpp") target_link_libraries(select_parser PRIVATE clickhouse_parsers) -clickhouse_add_executable(create_parser create_parser.cpp ${SRCS}) +clickhouse_add_executable(create_parser create_parser.cpp ${SRCS} "../../Server/ServerType.cpp") target_link_libraries(create_parser PRIVATE clickhouse_parsers) diff --git a/src/Server/ServerType.cpp b/src/Server/ServerType.cpp new file mode 100644 index 00000000000..c6916ee39d9 --- /dev/null +++ b/src/Server/ServerType.cpp @@ -0,0 +1,138 @@ +#include + +#include +#include +#include + +#include + +namespace DB +{ + +namespace +{ + std::vector getTypeIndexToTypeName() + { + constexpr std::size_t types_size = magic_enum::enum_count(); + + std::vector type_index_to_type_name; + type_index_to_type_name.resize(types_size); + + auto entries = magic_enum::enum_entries(); + for (const auto & [entry, str] : entries) + { + auto str_copy = String(str); + std::replace(str_copy.begin(), str_copy.end(), '_', ' '); + type_index_to_type_name[static_cast(entry)] = std::move(str_copy); + } + + return type_index_to_type_name; + } +} + +const char * ServerType::serverTypeToString(ServerType::Type type) +{ + /** During parsing if SystemQuery is not parsed properly it is added to Expected variants as description check IParser.h. + * Description string must be statically allocated. + */ + static std::vector type_index_to_type_name = getTypeIndexToTypeName(); + const auto & type_name = type_index_to_type_name[static_cast(type)]; + return type_name.data(); +} + +bool ServerType::shouldStart(Type server_type, const std::string & custom_name_) const +{ + if (type == Type::QUERIES_ALL) + return true; + + if (type == Type::QUERIES_DEFAULT) + { + switch (server_type) + { + case Type::TCP: + case Type::TCP_WITH_PROXY: + case Type::TCP_SECURE: + case Type::HTTP: + case Type::HTTPS: + case Type::MYSQL: + case Type::GRPC: + case Type::POSTGRESQL: + case Type::PROMETHEUS: + case Type::INTERSERVER_HTTP: + case Type::INTERSERVER_HTTPS: + return true; + default: + return false; + } + } + + if (type == Type::QUERIES_CUSTOM) + { + switch (server_type) + { + case Type::CUSTOM: + return true; + default: + return false; + } + } + + return type == server_type && custom_name == custom_name_; +} + +bool ServerType::shouldStop(const std::string & port_name) const +{ + Type port_type; + std::string port_custom_name; + + if (port_name == "http_port") + port_type = Type::HTTP; + + else if (port_name == "https_port") + port_type = Type::HTTPS; + + else if (port_name == "tcp_port") + port_type = Type::TCP; + + else if (port_name == "tcp_with_proxy_port") + port_type = Type::TCP_WITH_PROXY; + + else if (port_name == "tcp_port_secure") + port_type = Type::TCP_SECURE; + + else if (port_name == "mysql_port") + port_type = Type::MYSQL; + + else if (port_name == "postgresql_port") + port_type = Type::POSTGRESQL; + + else if (port_name == "grpc_port") + port_type = Type::GRPC; + + else if (port_name == "prometheus.port") + port_type = Type::PROMETHEUS; + + else if (port_name == "interserver_http_port") + port_type = Type::INTERSERVER_HTTP; + + else if (port_name == "interserver_https_port") + port_type = Type::INTERSERVER_HTTPS; + + else if (port_name.starts_with("protocols.") && port_name.ends_with(".port")) + { + constexpr size_t protocols_size = std::string_view("protocols.").size(); + constexpr size_t port_size = std::string_view("protocols.").size(); + + port_type = Type::CUSTOM; + port_custom_name = port_name.substr(protocols_size, port_name.size() - port_size); + } + else + port_type = Type::UNKNOWN; + + if (port_type == Type::UNKNOWN) + return false; + + return shouldStart(type, port_custom_name); +} + +} diff --git a/src/Server/ServerType.h b/src/Server/ServerType.h new file mode 100644 index 00000000000..345d1a10119 --- /dev/null +++ b/src/Server/ServerType.h @@ -0,0 +1,44 @@ +#pragma once + +#include +namespace DB +{ + +class ServerType +{ +public: + + enum Type + { + UNKNOWN, + TCP, + TCP_WITH_PROXY, + TCP_SECURE, + HTTP, + HTTPS, + MYSQL, + GRPC, + POSTGRESQL, + PROMETHEUS, + CUSTOM, + INTERSERVER_HTTP, + INTERSERVER_HTTPS, + QUERIES_ALL, + QUERIES_DEFAULT, + QUERIES_CUSTOM, + END + }; + + ServerType() = default; + explicit ServerType(Type type_, const std::string & custom_name_ = "") : type(type_), custom_name(custom_name_) {} + + static const char * serverTypeToString(Type type); + + bool shouldStart(Type server_type, const std::string & custom_name_ = "") const; + bool shouldStop(const std::string & port_name) const; + + Type type; + std::string custom_name; +}; + +} diff --git a/tests/integration/test_system_start_stop_listen/__init__.py b/tests/integration/test_system_start_stop_listen/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_system_start_stop_listen/configs/cluster.xml b/tests/integration/test_system_start_stop_listen/configs/cluster.xml new file mode 100644 index 00000000000..93d8f890f40 --- /dev/null +++ b/tests/integration/test_system_start_stop_listen/configs/cluster.xml @@ -0,0 +1,16 @@ + + + + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_system_start_stop_listen/test.py b/tests/integration/test_system_start_stop_listen/test.py new file mode 100644 index 00000000000..ec1a000c599 --- /dev/null +++ b/tests/integration/test_system_start_stop_listen/test.py @@ -0,0 +1,40 @@ +#!/usr/bin/env python3 + + +import pytest +import time +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry +import random +import string +import json + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", main_configs=["configs/cluster.xml"], with_zookeeper=True +) +node2 = cluster.add_instance( + "node2", main_configs=["configs/cluster.xml"], with_zookeeper=True +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_system_start_stop_listen_queries(started_cluster): + node1.query("SYSTEM STOP LISTEN QUERIES ALL") + + assert "Connection refused" in node1.query_and_get_error("SELECT 1", timeout=3) + + node2.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") + + node1.query("SELECT 1") diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index e6f7fa1ed2b..db0f2d8235b 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -138,6 +138,7 @@ SYSTEM FLUSH [] \N SYSTEM SYSTEM THREAD FUZZER ['SYSTEM START THREAD FUZZER','SYSTEM STOP THREAD FUZZER','START THREAD FUZZER','STOP THREAD FUZZER'] GLOBAL SYSTEM SYSTEM UNFREEZE ['SYSTEM UNFREEZE'] GLOBAL SYSTEM SYSTEM FAILPOINT ['SYSTEM ENABLE FAILPOINT','SYSTEM DISABLE FAILPOINT'] GLOBAL SYSTEM +SYSTEM LISTEN ['SYSTEM START LISTEN','SYSTEM STOP LISTEN'] GLOBAL SYSTEM SYSTEM [] \N ALL dictGet ['dictHas','dictGetHierarchy','dictIsIn'] DICTIONARY ALL displaySecretsInShowAndSelect [] GLOBAL ALL diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index d58c76260c5..46d1f0e3a0b 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -297,7 +297,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM' = 140, 'dictGet' = 141, 'displaySecretsInShowAndSelect' = 142, 'addressToLine' = 143, 'addressToLineWithInlines' = 144, 'addressToSymbol' = 145, 'demangle' = 146, 'INTROSPECTION' = 147, 'FILE' = 148, 'URL' = 149, 'REMOTE' = 150, 'MONGO' = 151, 'REDIS' = 152, 'MEILISEARCH' = 153, 'MYSQL' = 154, 'POSTGRES' = 155, 'SQLITE' = 156, 'ODBC' = 157, 'JDBC' = 158, 'HDFS' = 159, 'S3' = 160, 'HIVE' = 161, 'AZURE' = 162, 'SOURCES' = 163, 'CLUSTER' = 164, 'ALL' = 165, 'NONE' = 166), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -584,10 +584,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM' = 140, 'dictGet' = 141, 'displaySecretsInShowAndSelect' = 142, 'addressToLine' = 143, 'addressToLineWithInlines' = 144, 'addressToSymbol' = 145, 'demangle' = 146, 'INTROSPECTION' = 147, 'FILE' = 148, 'URL' = 149, 'REMOTE' = 150, 'MONGO' = 151, 'REDIS' = 152, 'MEILISEARCH' = 153, 'MYSQL' = 154, 'POSTGRES' = 155, 'SQLITE' = 156, 'ODBC' = 157, 'JDBC' = 158, 'HDFS' = 159, 'S3' = 160, 'HIVE' = 161, 'AZURE' = 162, 'SOURCES' = 163, 'CLUSTER' = 164, 'ALL' = 165, 'NONE' = 166), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM' = 140, 'dictGet' = 141, 'displaySecretsInShowAndSelect' = 142, 'addressToLine' = 143, 'addressToLineWithInlines' = 144, 'addressToSymbol' = 145, 'demangle' = 146, 'INTROSPECTION' = 147, 'FILE' = 148, 'URL' = 149, 'REMOTE' = 150, 'MONGO' = 151, 'REDIS' = 152, 'MEILISEARCH' = 153, 'MYSQL' = 154, 'POSTGRES' = 155, 'SQLITE' = 156, 'ODBC' = 157, 'JDBC' = 158, 'HDFS' = 159, 'S3' = 160, 'HIVE' = 161, 'AZURE' = 162, 'SOURCES' = 163, 'CLUSTER' = 164, 'ALL' = 165, 'NONE' = 166)) + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' From 5aeeaebd00c90385b1d9d8e3c6b13d298240c752 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 26 Jul 2023 17:09:23 -0300 Subject: [PATCH 1193/2047] init and destroy ares channel on demand.. --- src/Common/CaresPTRResolver.cpp | 71 ++++++++++--------- src/Common/CaresPTRResolver.h | 29 ++++---- .../tests/gtest_dns_reverse_resolve.cpp | 35 ++++----- 3 files changed, 71 insertions(+), 64 deletions(-) diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index fee4d01eb37..bf61e940745 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -41,9 +41,25 @@ namespace DB } } - std::mutex CaresPTRResolver::mutex; + struct AresChannelRAII + { + AresChannelRAII() + { + if (ares_init(&channel) != ARES_SUCCESS) + { + throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to initialize c-ares channel"); + } + } - CaresPTRResolver::CaresPTRResolver(CaresPTRResolver::provider_token) : channel(nullptr) + ~AresChannelRAII() + { + ares_destroy(channel); + } + + ares_channel channel; + }; + + CaresPTRResolver::CaresPTRResolver(CaresPTRResolver::provider_token) { /* * ares_library_init is not thread safe. Currently, the only other usage of c-ares seems to be in grpc. @@ -57,34 +73,22 @@ namespace DB * */ static const auto library_init_result = ares_library_init(ARES_LIB_INIT_ALL); - if (library_init_result != ARES_SUCCESS || ares_init(&channel) != ARES_SUCCESS) + if (library_init_result != ARES_SUCCESS) { throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to initialize c-ares"); } } - CaresPTRResolver::~CaresPTRResolver() - { - ares_destroy(channel); - /* - * Library initialization is currently done only once in the constructor. Multiple instances of CaresPTRResolver - * will be used in the lifetime of ClickHouse, thus it's problematic to have de-init here. - * In a practical view, it makes little to no sense to de-init a DNS library since DNS requests will happen - * until the end of the program. Hence, ares_library_cleanup() will not be called. - * */ - } - std::unordered_set CaresPTRResolver::resolve(const std::string & ip) { - std::lock_guard guard(mutex); + AresChannelRAII channel_raii; std::unordered_set ptr_records; - resolve(ip, ptr_records); + resolve(ip, ptr_records, channel_raii.channel); - if (!wait_and_process()) + if (!wait_and_process(channel_raii.channel)) { - cancel_requests(); throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to complete reverse DNS query for IP {}", ip); } @@ -93,22 +97,21 @@ namespace DB std::unordered_set CaresPTRResolver::resolve_v6(const std::string & ip) { - std::lock_guard guard(mutex); + AresChannelRAII channel_raii; std::unordered_set ptr_records; - resolve_v6(ip, ptr_records); + resolve_v6(ip, ptr_records, channel_raii.channel); - if (!wait_and_process()) + if (!wait_and_process(channel_raii.channel)) { - cancel_requests(); throw DB::Exception(DB::ErrorCodes::DNS_ERROR, "Failed to complete reverse DNS query for IP {}", ip); } return ptr_records; } - void CaresPTRResolver::resolve(const std::string & ip, std::unordered_set & response) + void CaresPTRResolver::resolve(const std::string & ip, std::unordered_set & response, ares_channel channel) { in_addr addr; @@ -117,7 +120,7 @@ namespace DB ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET, callback, &response); } - void CaresPTRResolver::resolve_v6(const std::string & ip, std::unordered_set & response) + void CaresPTRResolver::resolve_v6(const std::string & ip, std::unordered_set & response, ares_channel channel) { in6_addr addr; inet_pton(AF_INET6, ip.c_str(), &addr); @@ -125,15 +128,15 @@ namespace DB ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET6, callback, &response); } - bool CaresPTRResolver::wait_and_process() + bool CaresPTRResolver::wait_and_process(ares_channel channel) { int sockets[ARES_GETSOCK_MAXNUM]; pollfd pollfd[ARES_GETSOCK_MAXNUM]; while (true) { - auto readable_sockets = get_readable_sockets(sockets, pollfd); - auto timeout = calculate_timeout(); + auto readable_sockets = get_readable_sockets(sockets, pollfd, channel); + auto timeout = calculate_timeout(channel); int number_of_fds_ready = 0; if (!readable_sockets.empty()) @@ -158,11 +161,11 @@ namespace DB if (number_of_fds_ready > 0) { - process_readable_sockets(readable_sockets); + process_readable_sockets(readable_sockets, channel); } else { - process_possible_timeout(); + process_possible_timeout(channel); break; } } @@ -170,12 +173,12 @@ namespace DB return true; } - void CaresPTRResolver::cancel_requests() + void CaresPTRResolver::cancel_requests(ares_channel channel) { ares_cancel(channel); } - std::span CaresPTRResolver::get_readable_sockets(int * sockets, pollfd * pollfd) + std::span CaresPTRResolver::get_readable_sockets(int * sockets, pollfd * pollfd, ares_channel channel) { int sockets_bitmask = ares_getsock(channel, sockets, ARES_GETSOCK_MAXNUM); @@ -205,7 +208,7 @@ namespace DB return std::span(pollfd, number_of_sockets_to_poll); } - int64_t CaresPTRResolver::calculate_timeout() + int64_t CaresPTRResolver::calculate_timeout(ares_channel channel) { timeval tv; if (auto * tvp = ares_timeout(channel, nullptr, &tv)) @@ -218,14 +221,14 @@ namespace DB return 0; } - void CaresPTRResolver::process_possible_timeout() + void CaresPTRResolver::process_possible_timeout(ares_channel channel) { /* Call ares_process() unconditonally here, even if we simply timed out above, as otherwise the ares name resolve won't timeout! */ ares_process_fd(channel, ARES_SOCKET_BAD, ARES_SOCKET_BAD); } - void CaresPTRResolver::process_readable_sockets(std::span readable_sockets) + void CaresPTRResolver::process_readable_sockets(std::span readable_sockets, ares_channel channel) { for (auto readable_socket : readable_sockets) { diff --git a/src/Common/CaresPTRResolver.h b/src/Common/CaresPTRResolver.h index 454509ae43c..24a5e422ca8 100644 --- a/src/Common/CaresPTRResolver.h +++ b/src/Common/CaresPTRResolver.h @@ -28,32 +28,35 @@ namespace DB public: explicit CaresPTRResolver(provider_token); - ~CaresPTRResolver() override; + + /* + * Library initialization is currently done only once in the constructor. Multiple instances of CaresPTRResolver + * will be used in the lifetime of ClickHouse, thus it's problematic to have de-init here. + * In a practical view, it makes little to no sense to de-init a DNS library since DNS requests will happen + * until the end of the program. Hence, ares_library_cleanup() will not be called. + * */ + ~CaresPTRResolver() override = default; std::unordered_set resolve(const std::string & ip) override; std::unordered_set resolve_v6(const std::string & ip) override; private: - bool wait_and_process(); + bool wait_and_process(ares_channel channel); - void cancel_requests(); + void cancel_requests(ares_channel channel); - void resolve(const std::string & ip, std::unordered_set & response); + void resolve(const std::string & ip, std::unordered_set & response, ares_channel channel); - void resolve_v6(const std::string & ip, std::unordered_set & response); + void resolve_v6(const std::string & ip, std::unordered_set & response, ares_channel channel); - std::span get_readable_sockets(int * sockets, pollfd * pollfd); + std::span get_readable_sockets(int * sockets, pollfd * pollfd, ares_channel channel); - int64_t calculate_timeout(); + int64_t calculate_timeout(ares_channel channel); - void process_possible_timeout(); + void process_possible_timeout(ares_channel channel); - void process_readable_sockets(std::span readable_sockets); - - ares_channel channel; - - static std::mutex mutex; + void process_readable_sockets(std::span readable_sockets, ares_channel channel); }; } diff --git a/src/Common/tests/gtest_dns_reverse_resolve.cpp b/src/Common/tests/gtest_dns_reverse_resolve.cpp index 08351564eaf..de33deddac3 100644 --- a/src/Common/tests/gtest_dns_reverse_resolve.cpp +++ b/src/Common/tests/gtest_dns_reverse_resolve.cpp @@ -9,34 +9,35 @@ namespace DB { TEST(Common, ReverseDNS) { - auto addresses = std::vector({ - "8.8.8.8", "2001:4860:4860::8888", // dns.google - "142.250.219.35", // google.com - "157.240.12.35", // facebook - "208.84.244.116", "2600:1419:c400::214:c410", //www.terra.com.br, - "127.0.0.1", "::1" - }); - auto func = [&]() { // Good random seed, good engine auto rnd1 = std::mt19937(std::random_device{}()); - for (int i = 0; i < 50; ++i) + for (int i = 0; i < 10; ++i) { auto & dns_resolver_instance = DNSResolver::instance(); -// unfortunately, DNS cache can't be disabled because we might end up causing a DDoS attack -// dns_resolver_instance.setDisableCacheFlag(); + dns_resolver_instance.setDisableCacheFlag(); - auto addr_index = rnd1() % addresses.size(); + auto val1 = rnd1() % static_cast((pow(2, 31) - 1)); + auto val2 = rnd1() % static_cast((pow(2, 31) - 1)); + auto val3 = rnd1() % static_cast((pow(2, 31) - 1)); + auto val4 = rnd1() % static_cast((pow(2, 31) - 1)); - [[maybe_unused]] auto result = dns_resolver_instance.reverseResolve(Poco::Net::IPAddress{ addresses[addr_index] }); + uint32_t ipv4_buffer[1] = { + static_cast(val1) + }; -// will not assert either because some of the IP addresses might change in the future and -// this test will become flaky -// ASSERT_TRUE(!result.empty()); + uint32_t ipv6_buffer[4] = { + static_cast(val1), + static_cast(val2), + static_cast(val3), + static_cast(val4) + }; + + dns_resolver_instance.reverseResolve(Poco::Net::IPAddress{ ipv4_buffer, sizeof(ipv4_buffer)}); + dns_resolver_instance.reverseResolve(Poco::Net::IPAddress{ ipv6_buffer, sizeof(ipv6_buffer)}); } - }; auto number_of_threads = 200u; From bc86c26e4e75172fa8cb0665c26ff7b1228372a3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Jul 2023 23:37:20 +0300 Subject: [PATCH 1194/2047] Update src/Formats/StructureToFormatSchemaUtils.cpp Co-authored-by: Antonio Andelic --- src/Formats/StructureToFormatSchemaUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/StructureToFormatSchemaUtils.cpp b/src/Formats/StructureToFormatSchemaUtils.cpp index fd187c033ec..edd29d5f6ed 100644 --- a/src/Formats/StructureToFormatSchemaUtils.cpp +++ b/src/Formats/StructureToFormatSchemaUtils.cpp @@ -61,7 +61,7 @@ namespace const auto * it = find_first_symbols<'_', '.'>(begin, end); String first = String(begin, it); String second = it == end ? "" : String(it + 1, end); - return {first, second}; + return {std::move(first), std::move(second)}; } } From 954a1d3edec8117a135c23b7ec60065249fa0f02 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 26 Jul 2023 23:38:14 +0300 Subject: [PATCH 1195/2047] Improve logging macros (#52519) * improve logging macros * fix * Update logger_useful.h * fix * fix --- base/poco/Foundation/include/Poco/Message.h | 2 + base/poco/Foundation/src/Message.cpp | 13 +++ src/Common/LoggingFormatStringHelpers.h | 63 ++++++++---- src/Common/logger_useful.h | 97 ++++++++++++++----- src/Common/tests/gtest_log.cpp | 53 ++++++++++ .../01164_detach_attach_partition_race.sh | 2 +- 6 files changed, 189 insertions(+), 41 deletions(-) diff --git a/base/poco/Foundation/include/Poco/Message.h b/base/poco/Foundation/include/Poco/Message.h index e8f04888ab4..282c7fb5fd1 100644 --- a/base/poco/Foundation/include/Poco/Message.h +++ b/base/poco/Foundation/include/Poco/Message.h @@ -67,6 +67,8 @@ public: Message( const std::string & source, const std::string & text, Priority prio, const char * file, int line, std::string_view fmt_str = {}); + Message( + std::string && source, std::string && text, Priority prio, const char * file, int line, std::string_view fmt_str); /// Creates a Message with the given source, text, priority, /// source file path and line. /// diff --git a/base/poco/Foundation/src/Message.cpp b/base/poco/Foundation/src/Message.cpp index 663c96e47a2..54118cc0fc5 100644 --- a/base/poco/Foundation/src/Message.cpp +++ b/base/poco/Foundation/src/Message.cpp @@ -60,6 +60,19 @@ Message::Message(const std::string& source, const std::string& text, Priority pr } +Message::Message(std::string && source, std::string && text, Priority prio, const char * file, int line, std::string_view fmt_str): + _source(std::move(source)), + _text(std::move(text)), + _prio(prio), + _tid(0), + _file(file), + _line(line), + _pMap(0), + _fmt_str(fmt_str) +{ + init(); +} + Message::Message(const Message& msg): _source(msg._source), _text(msg._text), diff --git a/src/Common/LoggingFormatStringHelpers.h b/src/Common/LoggingFormatStringHelpers.h index 3afa3fb089d..6dffd202807 100644 --- a/src/Common/LoggingFormatStringHelpers.h +++ b/src/Common/LoggingFormatStringHelpers.h @@ -43,6 +43,17 @@ struct PreformattedMessage operator const std::string & () const { return text; } operator std::string () && { return std::move(text); } operator fmt::format_string<> () const { UNREACHABLE(); } + + void apply(std::string & out_text, std::string_view & out_format_string) const & + { + out_text = text; + out_format_string = format_string; + } + void apply(std::string & out_text, std::string_view & out_format_string) && + { + out_text = std::move(text); + out_format_string = format_string; + } }; template @@ -99,10 +110,33 @@ template constexpr std::string_view tryGetStaticFormatString(T && x } } +/// Constexpr ifs are not like ifdefs, and compiler still checks that unneeded code can be compiled +/// This template is useful to avoid compilation failures when condition of some "constexpr if" is false +template struct ConstexprIfsAreNotIfdefs +{ + template constexpr static std::string_view getStaticFormatString(T &&) { return {}; } + template static PreformattedMessage getPreformatted(T &&) { return {}; } +}; + +template<> struct ConstexprIfsAreNotIfdefs +{ + template consteval static std::string_view getStaticFormatString(T && x) + { + /// See tryGetStaticFormatString(...) + static_assert(!std::is_same_v>); + static_assert(std::is_nothrow_convertible::value); + static_assert(!std::is_pointer::value); + return std::string_view(x); + } + + template static T && getPreformatted(T && x) { return std::forward(x); } +}; + template constexpr size_t numArgs(Ts &&...) { return sizeof...(Ts); } template constexpr auto firstArg(T && x, Ts &&...) { return std::forward(x); } /// For implicit conversion of fmt::basic_runtime<> to char* for std::string ctor template constexpr auto firstArg(fmt::basic_runtime && data, Ts &&...) { return data.str.data(); } +template constexpr auto firstArg(const fmt::basic_runtime & data, Ts &&...) { return data.str.data(); } consteval ssize_t formatStringCountArgsNum(const char * const str, size_t len) { @@ -142,26 +176,19 @@ consteval void formatStringCheckArgsNumImpl(std::string_view str, size_t nargs) functionThatFailsCompilationOfConstevalFunctions("unexpected number of arguments in a format string"); } -template -struct CheckArgsNumHelperImpl +template +consteval void formatStringCheckArgsNum(T && str, size_t nargs) { - template - consteval CheckArgsNumHelperImpl(T && str) - { - formatStringCheckArgsNumImpl(tryGetStaticFormatString(str), sizeof...(Args)); - } - - /// No checks for fmt::runtime and PreformattedMessage - template CheckArgsNumHelperImpl(fmt::basic_runtime &&) {} - template<> CheckArgsNumHelperImpl(PreformattedMessage &) {} - template<> CheckArgsNumHelperImpl(const PreformattedMessage &) {} - template<> CheckArgsNumHelperImpl(PreformattedMessage &&) {} - -}; - -template using CheckArgsNumHelper = CheckArgsNumHelperImpl...>; -template void formatStringCheckArgsNum(CheckArgsNumHelper, Args &&...) {} + formatStringCheckArgsNumImpl(tryGetStaticFormatString(str), nargs); +} +template inline void formatStringCheckArgsNum(fmt::basic_runtime &&, size_t) {} +template<> inline void formatStringCheckArgsNum(PreformattedMessage &, size_t) {} +template<> inline void formatStringCheckArgsNum(const PreformattedMessage &, size_t) {} +template<> inline void formatStringCheckArgsNum(PreformattedMessage &&, size_t) {} +template struct FormatStringTypeInfo{ static constexpr bool is_static = true; static constexpr bool has_format = true; }; +template struct FormatStringTypeInfo> { static constexpr bool is_static = false; static constexpr bool has_format = false; }; +template<> struct FormatStringTypeInfo { static constexpr bool is_static = false; static constexpr bool has_format = true; }; /// This wrapper helps to avoid too frequent and noisy log messages. /// For each pair (logger_name, format_string) it remembers when such a message was logged the last time. diff --git a/src/Common/logger_useful.h b/src/Common/logger_useful.h index 3ebb1d25075..d9fe5ac9190 100644 --- a/src/Common/logger_useful.h +++ b/src/Common/logger_useful.h @@ -1,7 +1,7 @@ #pragma once /// Macros for convenient usage of Poco logger. - +#include #include #include #include @@ -28,33 +28,86 @@ namespace #define LOG_IMPL_FIRST_ARG(X, ...) X +/// Copy-paste from contrib/libpq/include/c.h +/// There's no easy way to count the number of arguments without evaluating these arguments... +#define CH_VA_ARGS_NARGS(...) \ + CH_VA_ARGS_NARGS_(__VA_ARGS__, \ + 63,62,61,60, \ + 59,58,57,56,55,54,53,52,51,50, \ + 49,48,47,46,45,44,43,42,41,40, \ + 39,38,37,36,35,34,33,32,31,30, \ + 29,28,27,26,25,24,23,22,21,20, \ + 19,18,17,16,15,14,13,12,11,10, \ + 9, 8, 7, 6, 5, 4, 3, 2, 1, 0) +#define CH_VA_ARGS_NARGS_( \ + _01,_02,_03,_04,_05,_06,_07,_08,_09,_10, \ + _11,_12,_13,_14,_15,_16,_17,_18,_19,_20, \ + _21,_22,_23,_24,_25,_26,_27,_28,_29,_30, \ + _31,_32,_33,_34,_35,_36,_37,_38,_39,_40, \ + _41,_42,_43,_44,_45,_46,_47,_48,_49,_50, \ + _51,_52,_53,_54,_55,_56,_57,_58,_59,_60, \ + _61,_62,_63, N, ...) \ + (N) + +#define LINE_NUM_AS_STRING_IMPL2(x) #x +#define LINE_NUM_AS_STRING_IMPL(x) LINE_NUM_AS_STRING_IMPL2(x) +#define LINE_NUM_AS_STRING LINE_NUM_AS_STRING_IMPL(__LINE__) +#define MESSAGE_FOR_EXCEPTION_ON_LOGGING "Failed to write a log message: " __FILE__ ":" LINE_NUM_AS_STRING "\n" + /// Logs a message to a specified logger with that level. /// If more than one argument is provided, /// the first argument is interpreted as a template with {}-substitutions /// and the latter arguments are treated as values to substitute. /// If only one argument is provided, it is treated as a message without substitutions. -#define LOG_IMPL(logger, priority, PRIORITY, ...) do \ -{ \ - auto _logger = ::getLogger(logger); \ - const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ - (DB::CurrentThread::get().getClientLogsLevel() >= (priority)); \ - if (_is_clients_log || _logger->is((PRIORITY))) \ - { \ - std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \ - formatStringCheckArgsNum(__VA_ARGS__); \ - if (auto _channel = _logger->getChannel()) \ - { \ - std::string file_function; \ - file_function += __FILE__; \ - file_function += "; "; \ - file_function += __PRETTY_FUNCTION__; \ - Poco::Message poco_message(_logger->name(), formatted_message, \ - (PRIORITY), file_function.c_str(), __LINE__, tryGetStaticFormatString(LOG_IMPL_FIRST_ARG(__VA_ARGS__))); \ - _channel->log(poco_message); \ - } \ - ProfileEvents::incrementForLogMessage(PRIORITY); \ - } \ +#define LOG_IMPL(logger, priority, PRIORITY, ...) do \ +{ \ + auto _logger = ::getLogger(logger); \ + const bool _is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ + (DB::CurrentThread::get().getClientLogsLevel() >= (priority)); \ + if (!_is_clients_log && !_logger->is((PRIORITY))) \ + break; \ + \ + try \ + { \ + ProfileEvents::incrementForLogMessage(PRIORITY); \ + auto _channel = _logger->getChannel(); \ + if (!_channel) \ + break; \ + \ + constexpr size_t _nargs = CH_VA_ARGS_NARGS(__VA_ARGS__); \ + using LogTypeInfo = FormatStringTypeInfo>; \ + \ + std::string_view _format_string; \ + std::string _formatted_message; \ + \ + if constexpr (LogTypeInfo::is_static) \ + { \ + formatStringCheckArgsNum(LOG_IMPL_FIRST_ARG(__VA_ARGS__), _nargs - 1); \ + _format_string = ConstexprIfsAreNotIfdefs::getStaticFormatString(LOG_IMPL_FIRST_ARG(__VA_ARGS__)); \ + } \ + \ + constexpr bool is_preformatted_message = !LogTypeInfo::is_static && LogTypeInfo::has_format; \ + if constexpr (is_preformatted_message) \ + { \ + static_assert(_nargs == 1 || !is_preformatted_message); \ + ConstexprIfsAreNotIfdefs::getPreformatted(LOG_IMPL_FIRST_ARG(__VA_ARGS__)).apply(_formatted_message, _format_string); \ + } \ + else \ + { \ + _formatted_message = _nargs == 1 ? firstArg(__VA_ARGS__) : fmt::format(__VA_ARGS__); \ + } \ + \ + std::string _file_function = __FILE__ "; "; \ + _file_function += __PRETTY_FUNCTION__; \ + Poco::Message _poco_message(_logger->name(), std::move(_formatted_message), \ + (PRIORITY), _file_function.c_str(), __LINE__, _format_string); \ + _channel->log(_poco_message); \ + } \ + catch (...) \ + { \ + ::write(STDERR_FILENO, static_cast(MESSAGE_FOR_EXCEPTION_ON_LOGGING), sizeof(MESSAGE_FOR_EXCEPTION_ON_LOGGING)); \ + } \ } while (false) diff --git a/src/Common/tests/gtest_log.cpp b/src/Common/tests/gtest_log.cpp index f92866626f9..e755c22ba75 100644 --- a/src/Common/tests/gtest_log.cpp +++ b/src/Common/tests/gtest_log.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -50,3 +51,55 @@ TEST(Logger, TestLog) } } + +static size_t global_counter = 0; + +static std::string getLogMessage() +{ + ++global_counter; + return "test1 " + std::to_string(thread_local_rng()); +} + +static size_t getLogMessageParam() +{ + ++global_counter; + return thread_local_rng(); +} + +static PreformattedMessage getPreformatted() +{ + ++global_counter; + return PreformattedMessage::create("test3 {}", thread_local_rng()); +} + +static size_t getLogMessageParamOrThrow() +{ + size_t x = thread_local_rng(); + if (x % 1000 == 0) + return x; + throw Poco::Exception("error", 42); +} + +TEST(Logger, SideEffects) +{ + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + auto my_channel = Poco::AutoPtr(new Poco::StreamChannel(oss)); + auto * log = &Poco::Logger::create("Logger", my_channel.get()); + log->setLevel("trace"); + + /// Ensure that parameters are evaluated only once + global_counter = 0; + LOG_TRACE(log, fmt::runtime(getLogMessage())); + EXPECT_EQ(global_counter, 1); + LOG_TRACE(log, "test2 {}", getLogMessageParam()); + EXPECT_EQ(global_counter, 2); + LOG_TRACE(log, getPreformatted()); + EXPECT_EQ(global_counter, 3); + + auto var = PreformattedMessage::create("test4 {}", thread_local_rng()); + LOG_TRACE(log, var); + EXPECT_EQ(var.text.starts_with("test4 "), true); + EXPECT_EQ(var.format_string, "test4 {}"); + + LOG_TRACE(log, "test no throw {}", getLogMessageParamOrThrow()); +} diff --git a/tests/queries/0_stateless/01164_detach_attach_partition_race.sh b/tests/queries/0_stateless/01164_detach_attach_partition_race.sh index e645cb5aae7..07b39723c37 100755 --- a/tests/queries/0_stateless/01164_detach_attach_partition_race.sh +++ b/tests/queries/0_stateless/01164_detach_attach_partition_race.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "drop table if exists mt" -$CLICKHOUSE_CLIENT -q "create table mt (n int) engine=MergeTree order by n settings parts_to_throw_insert=1000" +$CLICKHOUSE_CLIENT -q "create table mt (n int) engine=MergeTree order by n settings parts_to_throw_insert=5000" $CLICKHOUSE_CLIENT -q "insert into mt values (1)" $CLICKHOUSE_CLIENT -q "insert into mt values (2)" $CLICKHOUSE_CLIENT -q "insert into mt values (3)" From 64b2d5ca15f734d098e5e0e1cb75ac6bfcca476d Mon Sep 17 00:00:00 2001 From: hendrik-m <140658004+hendrik-m@users.noreply.github.com> Date: Wed, 26 Jul 2023 23:06:23 +0200 Subject: [PATCH 1196/2047] Allow string literals for table names --- src/Parsers/ExpressionElementParsers.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 0149526da79..354c43d3348 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -188,6 +188,20 @@ bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ++pos; return true; } + else if (pos->type == TokenType::StringLiteral) + { + ReadBufferFromMemory buf(pos->begin, pos->size()); + String s; + + readQuotedStringWithSQLStyle(s, buf); + + if (s.empty()) /// Identifiers "empty string" are not allowed. + return false; + + node = std::make_shared(s); + ++pos; + return true; + } else if (pos->type == TokenType::BareWord) { node = std::make_shared(String(pos->begin, pos->end)); From 571603390efdb7a5b540184015e214435e33c075 Mon Sep 17 00:00:00 2001 From: hendrik-m <140658004+hendrik-m@users.noreply.github.com> Date: Wed, 26 Jul 2023 23:09:12 +0200 Subject: [PATCH 1197/2047] add test cases --- ...use_local_table_name_expressions.reference | 27 +++++++++ ...clickhouse_local_table_name_expressions.sh | 56 +++++++++++++++++++ 2 files changed, 83 insertions(+) create mode 100644 tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.reference create mode 100644 tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.sh diff --git a/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.reference b/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.reference new file mode 100644 index 00000000000..1e3b82ac136 --- /dev/null +++ b/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.reference @@ -0,0 +1,27 @@ +Test 1: check double quotes +1 abc 123 abacaba +2 def 456 bacabaa +3 story 78912 acabaab +4 history 21321321 cabaaba +Test 1a: check double quotes no parsing overflow +1 +Test 1b: check double quotes empty +1 +Test 2: check back quotes +1 abc 123 abacaba +2 def 456 bacabaa +3 story 78912 acabaab +4 history 21321321 cabaaba +Test 2a: check back quotes no parsing overflow +1 +Test 2b: check back quotes empty +1 +Test 3: check literal +1 abc 123 abacaba +2 def 456 bacabaa +3 story 78912 acabaab +4 history 21321321 cabaaba +Test 3a: check literal no parsing overflow +1 +Test 3b: check literal empty +1 diff --git a/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.sh b/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.sh new file mode 100644 index 00000000000..73e64d1e175 --- /dev/null +++ b/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +dir=${CLICKHOUSE_TEST_UNIQUE_NAME} +[[ -d $dir ]] && rm -rd $dir +mkdir $dir + +# Create temporary csv file for tests +echo '"id","str","int","text"' > $dir/tmp.csv +echo '1,"abc",123,"abacaba"' >> $dir/tmp.csv +echo '2,"def",456,"bacabaa"' >> $dir/tmp.csv +echo '3,"story",78912,"acabaab"' >> $dir/tmp.csv +echo '4,"history",21321321,"cabaaba"' >> $dir/tmp.csv + +################# +echo "Test 1: check double quotes" + +$CLICKHOUSE_LOCAL -q "SELECT * FROM \"${dir}/tmp.csv\"" +################# +echo "Test 1a: check double quotes no parsing overflow" + +$CLICKHOUSE_LOCAL -q "SELECT * FROM \"${dir}/tmp.csv\"\"bad\"" 2>&1 | grep -c "UNKNOWN_TABLE" +################# +echo "Test 1b: check double quotes empty" + +$CLICKHOUSE_LOCAL -q "SELECT * FROM \"\"" 2>&1 | grep -c "SYNTAX_ERROR" +################# +echo "Test 2: check back quotes" + +$CLICKHOUSE_LOCAL -q "SELECT * FROM \`${dir}/tmp.csv\`" +################# +echo "Test 2a: check back quotes no parsing overflow" + +$CLICKHOUSE_LOCAL -q "SELECT * FROM \`${dir}/tmp.csv\`\`bad\`" 2>&1 | grep -c "UNKNOWN_TABLE" +################# +echo "Test 2b: check back quotes empty" + +$CLICKHOUSE_LOCAL -q "SELECT * FROM \`\`" 2>&1 | grep -c "SYNTAX_ERROR" +################# +echo "Test 3: check literal" + +$CLICKHOUSE_LOCAL -q "SELECT * FROM '${dir}/tmp.csv'" +################# +echo "Test 3a: check literal no parsing overflow" + +$CLICKHOUSE_LOCAL -q "SELECT * FROM '${dir}/tmp.csv''bad'" 2>&1 | grep -c "UNKNOWN_TABLE" +################# +echo "Test 3b: check literal empty" + +$CLICKHOUSE_LOCAL -q "SELECT * FROM ''" 2>&1 | grep -c "SYNTAX_ERROR" + +# Remove temporary dir with files +rm -rd $dir From 83fd3d4c89c9e247816ed619eecf6cb66bdb0154 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 26 Jul 2023 21:59:13 +0000 Subject: [PATCH 1198/2047] Additional checks related to DistinctSortedChunkTransform In response to suspicious crash reports from sqlancer for DISTINCT queries --- base/base/StringRef.h | 3 +++ .../Transforms/DistinctSortedChunkTransform.cpp | 15 +++++++++++++-- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/base/base/StringRef.h b/base/base/StringRef.h index f428b7c747f..448bc102b41 100644 --- a/base/base/StringRef.h +++ b/base/base/StringRef.h @@ -8,6 +8,7 @@ #include #include +#include #include #include @@ -274,6 +275,8 @@ struct CRC32Hash if (size == 0) return 0; + chassert(pos); + if (size < 8) { return static_cast(hashLessThan8(x.data, x.size)); diff --git a/src/Processors/Transforms/DistinctSortedChunkTransform.cpp b/src/Processors/Transforms/DistinctSortedChunkTransform.cpp index 28a3260d742..720543991c5 100644 --- a/src/Processors/Transforms/DistinctSortedChunkTransform.cpp +++ b/src/Processors/Transforms/DistinctSortedChunkTransform.cpp @@ -126,9 +126,20 @@ bool DistinctSortedChunkTransform::isKey(const size_t key_pos, const size_t row_ bool DistinctSortedChunkTransform::isLatestKeyFromPrevChunk(const size_t row_pos) const { - for (size_t i = 0; i < sorted_columns.size(); ++i) + for (size_t i = 0, s = sorted_columns.size(); i < s; ++i) { - const int res = prev_chunk_latest_key[i]->compareAt(0, row_pos, *sorted_columns[i], sorted_columns_descr[i].nulls_direction); + const auto & sorted_column = *sorted_columns[i]; + /// temporary hardening due to suspious crashes in sqlancer tests + if (unlikely(sorted_column.size() <= row_pos)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Unexpected size of a sorted column: size {}, row_pos {}, column position {}, type {}", + sorted_column.size(), + row_pos, + i, + sorted_column.getFamilyName()); + + const int res = prev_chunk_latest_key[i]->compareAt(0, row_pos, sorted_column, sorted_columns_descr[i].nulls_direction); if (res != 0) return false; } From 0cd2d7449b2057828fd653cc00155e7e4d4f6c09 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 27 Jul 2023 00:01:25 +0200 Subject: [PATCH 1199/2047] Use for-range loop, add comment --- src/Formats/StructureToFormatSchemaUtils.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Formats/StructureToFormatSchemaUtils.cpp b/src/Formats/StructureToFormatSchemaUtils.cpp index edd29d5f6ed..a9374647ebc 100644 --- a/src/Formats/StructureToFormatSchemaUtils.cpp +++ b/src/Formats/StructureToFormatSchemaUtils.cpp @@ -35,11 +35,14 @@ void endNested(WriteBuffer & buf, size_t indent) String getSchemaFieldName(const String & column_name) { String result = column_name; - size_t i = 0; - while (i < result.size() && isupper(result[i])) + /// Replace all first uppercase letters to lower-case, + /// because fields in CapnProto schema must begin with a lower-case letter. + /// Don't replace all letters to lower-case to remain camelCase field names. + for (auto & symbol : result) { - result[i] = tolower(result[i]); - ++i; + if (islower(symbol)) + break; + symbol = tolower(symbol); } return result; } From 6aab4cc83508093d19ed84130be483e73c08b324 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 05:25:40 +0200 Subject: [PATCH 1200/2047] Check for unexpected cyrillic --- src/Compression/CompressionCodecEncrypted.cpp | 2 +- src/Disks/DiskEncrypted.cpp | 4 ++-- src/IO/examples/read_buffer.cpp | 4 ++-- src/IO/examples/write_buffer.cpp | 2 +- src/IO/examples/write_buffer_perf.cpp | 2 +- src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp | 2 +- utils/check-style/check-style | 3 +++ 7 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 3f4e35a78a4..34b621291fd 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -493,7 +493,7 @@ void CompressionCodecEncrypted::Configuration::loadImpl( /// If there is only one key with non zero ID, curren_key_id should be defined. if (new_params->keys_storage[method].size() == 1 && !new_params->keys_storage[method].contains(0)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config has one key with non zero id. сurrent_key_id is required"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Config has one key with non zero id. current_key_id is required"); } /// Try to find which key will be used for encryption. If there is no current_key and only one key without id diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 441e639b967..aa9c4a92adc 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -433,10 +433,10 @@ void DiskEncrypted::applyNewSettings( { auto new_settings = parseDiskEncryptedSettings(name, config, config_prefix, disk_map); if (new_settings->wrapped_disk != delegate) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Сhanging wrapped disk on the fly is not supported. Disk {}", name); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Changing wrapped disk on the fly is not supported. Disk {}", name); if (new_settings->disk_path != disk_path) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Сhanging disk path on the fly is not supported. Disk {}", name); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Changing disk path on the fly is not supported. Disk {}", name); current_settings.set(std::move(new_settings)); IDisk::applyNewSettings(config, context, config_prefix, disk_map); diff --git a/src/IO/examples/read_buffer.cpp b/src/IO/examples/read_buffer.cpp index 85675c0d613..221da24715b 100644 --- a/src/IO/examples/read_buffer.cpp +++ b/src/IO/examples/read_buffer.cpp @@ -40,7 +40,7 @@ int readAndPrint(DB::ReadBuffer & in) int main(int, char **) { { - std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'"; + std::string s = "-123456 123.456 вася pe\\ttya\t'\\'xyz\\\\'"; DB::ReadBufferFromString in(s); if (readAndPrint(in)) std::cout << "readAndPrint from ReadBufferFromString failed" << std::endl; @@ -49,7 +49,7 @@ int main(int, char **) std::shared_ptr in; { - std::string s = "-123456 123.456 вася пе\\tтя\t'\\'xyz\\\\'"; + std::string s = "-123456 123.456 вася pe\\ttya\t'\\'xyz\\\\'"; in = std::make_shared(s); } if (readAndPrint(*in)) diff --git a/src/IO/examples/write_buffer.cpp b/src/IO/examples/write_buffer.cpp index bca0be24b1a..999f9b1bb34 100644 --- a/src/IO/examples/write_buffer.cpp +++ b/src/IO/examples/write_buffer.cpp @@ -14,7 +14,7 @@ int main(int, char **) { DB::Int64 a = -123456; DB::Float64 b = 123.456; - DB::String c = "вася пе\tтя"; + DB::String c = "вася pe\ttya"; DB::String d = "'xyz\\"; std::stringstream s; // STYLE_CHECK_ALLOW_STD_STRING_STREAM diff --git a/src/IO/examples/write_buffer_perf.cpp b/src/IO/examples/write_buffer_perf.cpp index 0b3d0a61241..3f57ddb9a4f 100644 --- a/src/IO/examples/write_buffer_perf.cpp +++ b/src/IO/examples/write_buffer_perf.cpp @@ -14,7 +14,7 @@ int main(int, char **) { DB::Int64 a = -123456; DB::Float64 b = 123.456; - DB::String c = "вася пе\tтя"; + DB::String c = "вася pe\ttya"; DB::String d = "'xyz\\"; std::ofstream s("test"); diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp index 229a0630328..cff83b0ad3b 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp @@ -180,7 +180,7 @@ Chunk ParquetMetadataInputFormat::generate() else if (name == names[3]) { auto column = types[3]->createColumn(); - /// Version сan be only PARQUET_1_0 or PARQUET_2_LATEST (which is 2.6). + /// Version can be only PARQUET_1_0 or PARQUET_2_LATEST (which is 2.6). String version = metadata->version() == parquet::ParquetVersion::PARQUET_1_0 ? "1.0" : "2.6"; assert_cast(*column).insertData(version.data(), version.size()); res.addColumn(std::move(column)); diff --git a/utils/check-style/check-style b/utils/check-style/check-style index c28ca1cfc8a..67c185a0b54 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -413,3 +413,6 @@ find $ROOT_PATH/tests/queries/1_stateful -name '*.sql' -or -name '*.sh' | grep - # Check for bad punctuation: whitespace before comma. find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '\w ,' | grep -v 'bad punctuation is ok here' && echo "^ There is bad punctuation: whitespace before comma. You should write it like this: 'Hello, world!'" + +# Cyrillic characters hiding inside Latin. +find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '[a-zA-Z][а-яА-ЯёЁ]|[а-яА-ЯёЁ][a-zA-Z]' && echo "^ Cyrillic characters found in unexpected place." From eae86f603c24600df39f38f255a700c4acc7432f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 06:34:09 +0200 Subject: [PATCH 1201/2047] Fix the test --- src/Databases/DatabaseFilesystem.cpp | 6 ++---- src/Interpreters/DatabaseCatalog.cpp | 21 ++++++++++++++++--- .../0_stateless/02722_database_filesystem.sh | 2 +- 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index a6365b0348c..59f9ee67d7b 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -77,10 +77,8 @@ bool DatabaseFilesystem::checkTableFilePath(const std::string & table_path, Cont /// Check access for file before checking its existence. if (check_path && !fileOrSymlinkPathStartsWith(table_path, user_files_path)) { - if (throw_on_error) - throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File is not inside {}", user_files_path); - else - return false; + /// Access denied is thrown regardless of 'throw_on_error' + throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File is not inside {}", user_files_path); } /// Check if the corresponding file exists. diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 13cac5afb1b..f8481e3f1d8 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -336,7 +336,6 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( return db_and_table; } - if (table_id.database_name == TEMPORARY_DATABASE) { /// For temporary tables UUIDs are set in Context::resolveStorageID(...). @@ -369,8 +368,24 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( database = it->second; } - auto table = database->tryGetTable(table_id.table_name, context_); - if (!table && exception) + StoragePtr table; + if (exception) + { + try + { + table = database->getTable(table_id.table_name, context_); + } + catch (const Exception & e) + { + exception->emplace(e); + } + } + else + { + table = database->tryGetTable(table_id.table_name, context_); + } + + if (!table && exception && !exception->has_value()) exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs())); if (!table) diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index 3b7a41bb39e..7b2e1bf1a66 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -61,7 +61,7 @@ CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); """ 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||: # FILE_DOESNT_EXIST: unknown file -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 60" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 107" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: Cannot determine the file format by it's extension ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.myext\`;" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||: From 13d9952227162b63ae5e4ae98b0a40677c169b97 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Jul 2023 00:38:51 -0400 Subject: [PATCH 1202/2047] add id to insertImpl --- src/Access/AccessControl.cpp | 10 +++++--- src/Access/AccessControl.h | 2 +- src/Access/DiskAccessStorage.cpp | 16 +++--------- src/Access/DiskAccessStorage.h | 3 +-- src/Access/IAccessStorage.cpp | 25 ++++++++++++------- src/Access/IAccessStorage.h | 11 ++++++-- src/Access/MemoryAccessStorage.cpp | 14 ++--------- src/Access/MemoryAccessStorage.h | 7 +----- src/Access/MultipleAccessStorage.cpp | 11 ++++---- src/Access/MultipleAccessStorage.h | 2 +- src/Access/ReplicatedAccessStorage.cpp | 16 +++--------- src/Access/ReplicatedAccessStorage.h | 3 +-- src/Parsers/Access/ParserCreateQuotaQuery.cpp | 4 +-- src/Parsers/Access/ParserCreateRoleQuery.cpp | 3 ++- .../Access/ParserCreateRowPolicyQuery.cpp | 4 +-- .../ParserCreateSettingsProfileQuery.cpp | 4 +-- src/Parsers/Access/ParserCreateUserQuery.cpp | 3 ++- .../Access/ParserDropAccessEntityQuery.cpp | 3 ++- .../Access/ParserMoveAccessEntityQuery.cpp | 3 ++- src/Parsers/Access/parseUserName.h | 5 ---- 20 files changed, 64 insertions(+), 85 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 22b00d5dde7..3759b6c8919 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -525,12 +525,14 @@ scope_guard AccessControl::subscribeForChanges(const std::vector & ids, co return changes_notifier->subscribeForChanges(ids, handler); } -std::optional AccessControl::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) +bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - auto id = MultipleAccessStorage::insertImpl(entity, replace_if_exists, throw_if_exists, set_id); - if (id) + if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists)) + { changes_notifier->sendNotifications(); - return id; + return true; + } + return false; } bool AccessControl::removeImpl(const UUID & id, bool throw_if_not_exists) diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 85feaa70462..60b40775a56 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -229,7 +229,7 @@ private: class CustomSettingsPrefixes; class PasswordComplexityRules; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index acbd7b12569..190c7567b85 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -498,20 +498,10 @@ std::optional> DiskAccessStorage::readNameWi } -std::optional DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) -{ - UUID id = set_id ? *set_id : generateRandomID(); - if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true)) - return id; - - return std::nullopt; -} - - -bool DiskAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) +bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) { std::lock_guard lock{mutex}; - return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, write_on_disk); + return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk = */ true); } @@ -745,7 +735,7 @@ void DiskAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] { for (const auto & [id, entity] : my_entities) - insertWithID(id, entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true); + insert(id, entity, replace_if_exists, throw_if_exists); }); } diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 3541c3228e3..5d94008b34f 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -39,7 +39,7 @@ private: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; @@ -53,7 +53,6 @@ private: void listsWritingThreadFunc() TSA_NO_THREAD_SAFETY_ANALYSIS; void stopListsWritingThread(); - bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk); bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) TSA_REQUIRES(mutex); bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); bool removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index db73db900aa..468c93c1116 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -180,14 +180,21 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity) std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); + auto id = generateRandomID(); + + if (insert(id, entity, replace_if_exists, throw_if_exists)) + return id; + + return std::nullopt; } -std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) + +bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return insertImpl(entity, replace_if_exists, throw_if_exists, set_id); + return insertImpl(id, entity, replace_if_exists, throw_if_exists); } + std::vector IAccessStorage::insert(const std::vector & multiple_entities, bool replace_if_exists, bool throw_if_exists) { return insert(multiple_entities, /* ids = */ {}, replace_if_exists, throw_if_exists); @@ -216,16 +223,16 @@ std::vector IAccessStorage::insert(const std::vector & mu { const auto & entity = multiple_entities[i]; - std::optional id; + UUID id; if (!ids.empty()) id = ids[i]; + else + id = generateRandomID(); - auto new_id = insertImpl(entity, replace_if_exists, throw_if_exists, id); - - if (new_id) + if (insertImpl(id, entity, replace_if_exists, throw_if_exists)) { successfully_inserted.push_back(entity); - new_ids.push_back(*new_id); + new_ids.push_back(id); } } return new_ids; @@ -274,7 +281,7 @@ std::vector IAccessStorage::insertOrReplace(const std::vector IAccessStorage::insertImpl(const AccessEntityPtr & entity, bool, bool, std::optional) +bool IAccessStorage::insertImpl(const UUID &, const AccessEntityPtr & entity, bool, bool) { if (isReadOnly()) throwReadonlyCannotInsert(entity->getType(), entity->getName()); diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index 894220e95ad..b82537df639 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -132,7 +134,7 @@ public: /// Throws an exception if the specified name already exists. UUID insert(const AccessEntityPtr & entity); std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); - std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); + bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); std::vector insert(const std::vector & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true); std::vector insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists = false, bool throw_if_exists = true); @@ -185,7 +187,7 @@ protected: virtual std::vector findAllImpl(AccessEntityType type) const = 0; virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0; virtual std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const; - virtual std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); + virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); virtual bool removeImpl(const UUID & id, bool throw_if_not_exists); virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); virtual std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const; @@ -284,4 +286,9 @@ std::vector>> IAccessStorage return entities; } +inline bool parseAccessStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) +{ + return parseIdentifierOrStringLiteral(pos, expected, storage_name); +} + } diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp index 16f3b3793a0..999e2c41d0b 100644 --- a/src/Access/MemoryAccessStorage.cpp +++ b/src/Access/MemoryAccessStorage.cpp @@ -63,17 +63,7 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not } -std::optional MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) -{ - UUID id = set_id ? *set_id : generateRandomID(); - if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) - return id; - - return std::nullopt; -} - - -bool MemoryAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) { std::lock_guard lock{mutex}; return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists); @@ -300,7 +290,7 @@ void MemoryAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] { for (const auto & [id, entity] : my_entities) - insertWithID(id, entity, replace_if_exists, throw_if_exists); + insert(id, entity, replace_if_exists, throw_if_exists); }); } diff --git a/src/Access/MemoryAccessStorage.h b/src/Access/MemoryAccessStorage.h index 7c19ab9c4ab..f877860041a 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -22,11 +22,6 @@ public: const char * getStorageType() const override { return STORAGE_TYPE; } - /// Inserts an entity with a specified ID. - /// If `replace_if_exists == true` it can replace an existing entry with such ID and also remove an existing entry - /// with such name & type. - bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists); - /// Removes all entities except the specified list `ids_to_keep`. /// The function skips IDs not contained in the storage. void removeAllExcept(const std::vector & ids_to_keep); @@ -44,7 +39,7 @@ private: std::optional findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index 169dc889509..e6665349ae6 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -316,7 +316,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode) } -std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) +bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { std::shared_ptr storage_for_insertion; @@ -339,13 +339,14 @@ std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & en getStorageName()); } - auto id = storage_for_insertion->insert(entity, replace_if_exists, throw_if_exists, set_id); - if (id) + if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists)) { std::lock_guard lock{mutex}; - ids_cache.set(*id, storage_for_insertion); + ids_cache.set(id, storage_for_insertion); + return true; } - return id; + + return false; } diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 1d95296b796..069d414f601 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -63,7 +63,7 @@ protected: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index efa9dbbdfaa..275672da115 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -108,17 +108,7 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function) } } -std::optional ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) -{ - const UUID id = set_id ? *set_id : generateRandomID(); - if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) - return id; - - return std::nullopt; -} - - -bool ReplicatedAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) { const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(new_entity->getType()); const String & name = new_entity->getName(); @@ -619,7 +609,7 @@ AccessEntityPtr ReplicatedAccessStorage::tryReadEntityFromZooKeeper(const zkutil void ReplicatedAccessStorage::setEntityNoLock(const UUID & id, const AccessEntityPtr & entity) { LOG_DEBUG(getLogger(), "Setting id {} to entity named {}", toString(id), entity->getName()); - memory_storage.insertWithID(id, entity, /* replace_if_exists= */ true, /* throw_if_exists= */ false); + memory_storage.insert(id, entity, /* replace_if_exists= */ true, /* throw_if_exists= */ false); } @@ -711,7 +701,7 @@ void ReplicatedAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] { for (const auto & [id, entity] : my_entities) - insertWithID(id, entity, replace_if_exists, throw_if_exists); + insert(id, entity, replace_if_exists, throw_if_exists); }); } diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index 0a0c53d3bc4..cddb20860f7 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -46,11 +46,10 @@ private: std::unique_ptr watching_thread; std::shared_ptr> watched_queue; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; - bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists); bool insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); bool removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, bool throw_if_not_exists); bool updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); diff --git a/src/Parsers/Access/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp index 05c2820b012..718498ed1d1 100644 --- a/src/Parsers/Access/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -1,11 +1,11 @@ #include +#include #include #include #include #include #include #include -#include #include #include #include @@ -312,7 +312,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateRoleQuery.cpp b/src/Parsers/Access/ParserCreateRoleQuery.cpp index b548c07ec70..99a97f6901d 100644 --- a/src/Parsers/Access/ParserCreateRoleQuery.cpp +++ b/src/Parsers/Access/ParserCreateRoleQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -111,7 +112,7 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index b2177ca4ade..f9e9466e35d 100644 --- a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -1,10 +1,10 @@ +#include #include #include #include #include #include #include -#include #include #include #include @@ -273,7 +273,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp index 634ff68427a..fcb6ebd8806 100644 --- a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp +++ b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp @@ -1,10 +1,10 @@ +#include #include #include #include #include #include #include -#include #include #include #include @@ -132,7 +132,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index d95d41296dc..577b6772ab2 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -481,7 +482,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } } - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp index d5ecdf27449..7fd34a6f8e4 100644 --- a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -78,7 +79,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & } if (ParserKeyword{"FROM"}.ignore(pos, expected)) - parseStorageName(pos, expected, storage_name); + parseAccessStorageName(pos, expected, storage_name); if (cluster.empty()) parseOnCluster(pos, expected, cluster); diff --git a/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp index 6ac41b77b84..798e200bfb3 100644 --- a/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -73,7 +74,7 @@ bool ParserMoveAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & return false; } - if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseStorageName(pos, expected, storage_name)) + if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseAccessStorageName(pos, expected, storage_name)) return false; if (cluster.empty()) diff --git a/src/Parsers/Access/parseUserName.h b/src/Parsers/Access/parseUserName.h index 64fed2ce21c..10c548d8826 100644 --- a/src/Parsers/Access/parseUserName.h +++ b/src/Parsers/Access/parseUserName.h @@ -35,9 +35,4 @@ inline bool parseRoleNames(IParser::Pos & pos, Expected & expected, Strings & ro return parseUserNames(pos, expected, role_names); } -inline bool parseStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) -{ - return parseIdentifierOrStringLiteral(pos, expected, storage_name); -} - } From d35c87c1bdf4cba8848e94acd0ed8ec1f6b34502 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 05:06:16 +0000 Subject: [PATCH 1203/2047] allow positional options for clickhouse-local and populate global udf settings --- programs/local/LocalServer.cpp | 15 +++++++++++++++ src/Client/ClientBase.cpp | 5 ----- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 3c2a8ae3152..3ee268aed0e 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -266,6 +266,10 @@ void LocalServer::tryInitPath() global_context->setUserFilesPath(""); // user's files are everywhere + std::string user_scripts_path = config().getString("user_scripts_path", fs::path(path) / "user_scripts/"); + global_context->setUserScriptsPath(user_scripts_path); + fs::create_directories(user_scripts_path); + /// top_level_domains_lists const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/"); if (!top_level_domains_path.empty()) @@ -490,6 +494,17 @@ try applyCmdSettings(global_context); + /// try to load user defined executable functions, throw on error and die + try + { + global_context->loadOrReloadUserDefinedExecutableFunctions(config()); + } + catch (...) + { + tryLogCurrentException(&logger(), "Caught exception while loading user defined executable functions."); + throw; + } + if (is_interactive) { clearTerminal(); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 3e964d5c6a3..496fc8fce0a 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2624,11 +2624,6 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); } - /// Check positional options (options after ' -- ', ex: clickhouse-client -- ). - unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional); - if (unrecognized_options.size() > 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional options are not supported."); - po::store(parsed, options); } From 65ffe91bf26a3429fe691c755736867e7819d2f5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 07:13:26 +0200 Subject: [PATCH 1204/2047] Fix double whitespace --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index b2c75db0e54..c661e6b782d 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -1011,7 +1011,7 @@ private: using ConfluentSchemaRegistry = AvroConfluentRowInputFormat::SchemaRegistry; #define SCHEMA_REGISTRY_CACHE_MAX_SIZE 1000 /// Cache of Schema Registry URL -> SchemaRegistry -static CacheBase schema_registry_cache(SCHEMA_REGISTRY_CACHE_MAX_SIZE); +static CacheBase schema_registry_cache(SCHEMA_REGISTRY_CACHE_MAX_SIZE); static std::shared_ptr getConfluentSchemaRegistry(const FormatSettings & format_settings) { From 7c1832fcfc7b0b97d6e9d672938c0f26f04e4738 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Thu, 27 Jul 2023 09:41:47 +0300 Subject: [PATCH 1205/2047] Fix named collections related statements #51609 --- .../NamedCollections/NamedCollectionUtils.cpp | 56 +++++++++++-------- .../NamedCollections/NamedCollectionUtils.h | 4 +- .../InterpreterCreateNamedCollectionQuery.cpp | 1 - .../InterpreterDropNamedCollectionQuery.cpp | 6 +- src/Parsers/ASTCreateNamedCollectionQuery.h | 1 + .../ParserAlterNamedCollectionQuery.cpp | 8 ++- src/Parsers/ParserCreateQuery.cpp | 17 ++++-- src/Parsers/ParserCreateQuery.h | 1 + .../ParserDropNamedCollectionQuery.cpp | 3 +- 9 files changed, 57 insertions(+), 40 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index 6ec09fb8a77..12bf100feae 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -225,24 +226,15 @@ public: void remove(const std::string & collection_name) { - if (!removeIfExists(collection_name)) + auto collection_path = getMetadataPath(collection_name); + if (!fs::exists(collection_path)) { throw Exception( ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, "Cannot remove collection `{}`, because it doesn't exist", collection_name); } - } - - bool removeIfExists(const std::string & collection_name) - { - auto collection_path = getMetadataPath(collection_name); - if (fs::exists(collection_path)) - { - fs::remove(collection_path); - return true; - } - return false; + fs::remove(collection_path); } private: @@ -393,27 +385,43 @@ void loadIfNot() return loadIfNotUnlocked(lock); } -void removeFromSQL(const std::string & collection_name, ContextPtr context) +void removeFromSQL(const ASTDropNamedCollectionQuery & query, ContextPtr context) { auto lock = lockNamedCollectionsTransaction(); loadIfNotUnlocked(lock); - LoadFromSQL(context).remove(collection_name); - NamedCollectionFactory::instance().remove(collection_name); -} - -void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr context) -{ - auto lock = lockNamedCollectionsTransaction(); - loadIfNotUnlocked(lock); - LoadFromSQL(context).removeIfExists(collection_name); - NamedCollectionFactory::instance().removeIfExists(collection_name); + auto & instance = NamedCollectionFactory::instance(); + if (!instance.exists(query.collection_name)) + { + if (!query.if_exists) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "Cannot remove collection `{}`, because it doesn't exist", + query.collection_name); + } + return; + } + LoadFromSQL(context).remove(query.collection_name); + instance.remove(query.collection_name); } void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context) { auto lock = lockNamedCollectionsTransaction(); loadIfNotUnlocked(lock); - NamedCollectionFactory::instance().add(query.collection_name, LoadFromSQL(context).create(query)); + auto & instance = NamedCollectionFactory::instance(); + if (instance.exists(query.collection_name)) + { + if (!query.if_not_exists) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "A named collection `{}` already exists", + query.collection_name); + } + return; + } + instance.add(query.collection_name, LoadFromSQL(context).create(query)); } void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context) diff --git a/src/Common/NamedCollections/NamedCollectionUtils.h b/src/Common/NamedCollections/NamedCollectionUtils.h index c929abb5d74..6cbe9101550 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.h +++ b/src/Common/NamedCollections/NamedCollectionUtils.h @@ -8,6 +8,7 @@ namespace DB class ASTCreateNamedCollectionQuery; class ASTAlterNamedCollectionQuery; +class ASTDropNamedCollectionQuery; namespace NamedCollectionUtils { @@ -26,8 +27,7 @@ void reloadFromConfig(const Poco::Util::AbstractConfiguration & config); void loadFromSQL(ContextPtr context); /// Remove collection as well as its metadata from `context->getPath() / named_collections /`. -void removeFromSQL(const std::string & collection_name, ContextPtr context); -void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr context); +void removeFromSQL(const ASTDropNamedCollectionQuery & query, ContextPtr context); /// Create a new collection from AST and put it to `context->getPath() / named_collections /`. void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context); diff --git a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp index bac59998062..3b0fba5fd9f 100644 --- a/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterCreateNamedCollectionQuery.cpp @@ -1,5 +1,4 @@ #include - #include #include #include diff --git a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp index cc3444bb4df..fe49b1cfd7c 100644 --- a/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp +++ b/src/Interpreters/InterpreterDropNamedCollectionQuery.cpp @@ -22,11 +22,7 @@ BlockIO InterpreterDropNamedCollectionQuery::execute() return executeDDLQueryOnCluster(query_ptr, current_context, params); } - if (query.if_exists) - NamedCollectionUtils::removeIfExistsFromSQL(query.collection_name, current_context); - else - NamedCollectionUtils::removeFromSQL(query.collection_name, current_context); - + NamedCollectionUtils::removeFromSQL(query, current_context); return {}; } diff --git a/src/Parsers/ASTCreateNamedCollectionQuery.h b/src/Parsers/ASTCreateNamedCollectionQuery.h index 303d1901931..785aeb781b4 100644 --- a/src/Parsers/ASTCreateNamedCollectionQuery.h +++ b/src/Parsers/ASTCreateNamedCollectionQuery.h @@ -13,6 +13,7 @@ class ASTCreateNamedCollectionQuery : public IAST, public ASTQueryWithOnCluster public: std::string collection_name; SettingsChanges changes; + bool if_not_exists = false; String getID(char) const override { return "CreateNamedCollectionQuery"; } diff --git a/src/Parsers/ParserAlterNamedCollectionQuery.cpp b/src/Parsers/ParserAlterNamedCollectionQuery.cpp index 9108747ad82..8fb84f86246 100644 --- a/src/Parsers/ParserAlterNamedCollectionQuery.cpp +++ b/src/Parsers/ParserAlterNamedCollectionQuery.cpp @@ -13,8 +13,9 @@ bool ParserAlterNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & nod { ParserKeyword s_alter("ALTER"); ParserKeyword s_collection("NAMED COLLECTION"); + ParserKeyword s_if_exists("IF EXISTS"); + ParserKeyword s_on("ON"); ParserKeyword s_delete("DELETE"); - ParserIdentifier name_p; ParserSetQuery set_p; ParserToken s_comma(TokenType::Comma); @@ -32,10 +33,13 @@ bool ParserAlterNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & nod if (!s_collection.ignore(pos, expected)) return false; + if (s_if_exists.ignore(pos, expected)) + if_exists = true; + if (!name_p.parse(pos, collection_name, expected)) return false; - if (ParserKeyword{"ON"}.ignore(pos, expected)) + if (s_on.ignore(pos, expected)) { if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) return false; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 415d3321eb5..82674ab1a35 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -1421,15 +1421,17 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_create("CREATE"); - ParserKeyword s_attach("ATTACH"); ParserKeyword s_named_collection("NAMED COLLECTION"); + ParserKeyword s_if_not_exists("IF NOT EXISTS"); + ParserKeyword s_on("ON"); ParserKeyword s_as("AS"); - - ParserToken s_comma(TokenType::Comma); ParserIdentifier name_p; + ParserToken s_comma(TokenType::Comma); + + String cluster_str; + bool if_not_exists = false; ASTPtr collection_name; - String cluster_str; if (!s_create.ignore(pos, expected)) return false; @@ -1437,10 +1439,13 @@ bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expec if (!s_named_collection.ignore(pos, expected)) return false; + if (s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + if (!name_p.parse(pos, collection_name, expected)) return false; - if (ParserKeyword{"ON"}.ignore(pos, expected)) + if (s_on.ignore(pos, expected)) { if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) return false; @@ -1465,7 +1470,9 @@ bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expec auto query = std::make_shared(); tryGetIdentifierNameInto(collection_name, query->collection_name); + query->if_not_exists = if_not_exists; query->changes = changes; + query->cluster = std::move(cluster_str); node = query; return true; diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 09935e2b608..0a98923436c 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -548,6 +548,7 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +/// CREATE NAMED COLLECTION name [ON CLUSTER cluster] class ParserCreateNamedCollectionQuery : public IParserBase { protected: diff --git a/src/Parsers/ParserDropNamedCollectionQuery.cpp b/src/Parsers/ParserDropNamedCollectionQuery.cpp index 1ea8aa6d75d..b0b010b5ef6 100644 --- a/src/Parsers/ParserDropNamedCollectionQuery.cpp +++ b/src/Parsers/ParserDropNamedCollectionQuery.cpp @@ -12,6 +12,7 @@ bool ParserDropNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node ParserKeyword s_drop("DROP"); ParserKeyword s_collection("NAMED COLLECTION"); ParserKeyword s_if_exists("IF EXISTS"); + ParserKeyword s_on("ON"); ParserIdentifier name_p; String cluster_str; @@ -31,7 +32,7 @@ bool ParserDropNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node if (!name_p.parse(pos, collection_name, expected)) return false; - if (ParserKeyword{"ON"}.ignore(pos, expected)) + if (s_on.ignore(pos, expected)) { if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) return false; From e511072df1b26d466d96c9cf81ca6031a44956ad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Thu, 27 Jul 2023 09:42:38 +0300 Subject: [PATCH 1206/2047] Update named collections related docs --- docs/README.md | 4 +-- .../table-engines/integrations/deltalake.md | 2 +- .../table-engines/integrations/hudi.md | 2 +- .../sql-reference/statements/alter/index.md | 2 ++ .../statements/alter/named-collection.md | 30 ++++++++++++++++ .../sql-reference/statements/create/index.md | 21 ++++++------ .../statements/create/named-collection.md | 34 +++++++++++++++++++ docs/en/sql-reference/statements/drop.md | 17 ++++++++++ .../sql-reference/table-functions/iceberg.md | 2 +- 9 files changed, 99 insertions(+), 15 deletions(-) create mode 100644 docs/en/sql-reference/statements/alter/named-collection.md create mode 100644 docs/en/sql-reference/statements/create/named-collection.md diff --git a/docs/README.md b/docs/README.md index 0cd35a4e3ec..d1260312166 100644 --- a/docs/README.md +++ b/docs/README.md @@ -200,8 +200,8 @@ Templates: - [Server Setting](_description_templates/template-server-setting.md) - [Database or Table engine](_description_templates/template-engine.md) - [System table](_description_templates/template-system-table.md) -- [Data type](_description_templates/data-type.md) -- [Statement](_description_templates/statement.md) +- [Data type](_description_templates/template-data-type.md) +- [Statement](_description_templates/template-statement.md) diff --git a/docs/en/engines/table-engines/integrations/deltalake.md b/docs/en/engines/table-engines/integrations/deltalake.md index b562e9d7fe6..964c952f31a 100644 --- a/docs/en/engines/table-engines/integrations/deltalake.md +++ b/docs/en/engines/table-engines/integrations/deltalake.md @@ -22,7 +22,7 @@ CREATE TABLE deltalake - `url` — Bucket url with path to the existing Delta Lake table. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. -Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) +Engine parameters can be specified using [Named Collections](/docs/en/operations/named-collections.md). **Example** diff --git a/docs/en/engines/table-engines/integrations/hudi.md b/docs/en/engines/table-engines/integrations/hudi.md index c60618af289..b2f599e5c92 100644 --- a/docs/en/engines/table-engines/integrations/hudi.md +++ b/docs/en/engines/table-engines/integrations/hudi.md @@ -22,7 +22,7 @@ CREATE TABLE hudi_table - `url` — Bucket url with the path to an existing Hudi table. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. -Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) +Engine parameters can be specified using [Named Collections](/docs/en/operations/named-collections.md). **Example** diff --git a/docs/en/sql-reference/statements/alter/index.md b/docs/en/sql-reference/statements/alter/index.md index 7dadc2be5b2..dca34d16f25 100644 --- a/docs/en/sql-reference/statements/alter/index.md +++ b/docs/en/sql-reference/statements/alter/index.md @@ -36,6 +36,8 @@ These `ALTER` statements modify entities related to role-based access control: [ALTER TABLE ... MODIFY COMMENT](/docs/en/sql-reference/statements/alter/comment.md) statement adds, modifies, or removes comments to the table, regardless if it was set before or not. +[ALTER NAMED COLLECTION](/docs/en/sql-reference/statements/alter/named-collection.md) statement modifies [Named Collections](/docs/en/operations/named-collections.md). + ## Mutations `ALTER` queries that are intended to manipulate table data are implemented with a mechanism called “mutations”, most notably [ALTER TABLE … DELETE](/docs/en/sql-reference/statements/alter/delete.md) and [ALTER TABLE … UPDATE](/docs/en/sql-reference/statements/alter/update.md). They are asynchronous background processes similar to merges in [MergeTree](/docs/en/engines/table-engines/mergetree-family/index.md) tables that to produce new “mutated” versions of parts. diff --git a/docs/en/sql-reference/statements/alter/named-collection.md b/docs/en/sql-reference/statements/alter/named-collection.md new file mode 100644 index 00000000000..ac6752127c1 --- /dev/null +++ b/docs/en/sql-reference/statements/alter/named-collection.md @@ -0,0 +1,30 @@ +--- +slug: /en/sql-reference/statements/alter/named-collection +sidebar_label: NAMED COLLECTION +--- + +# ALTER NAMED COLLECTION + +This query intends to modify already existing named collections. + +**Syntax** + +```sql +ALTER NAMED COLLECTION [IF EXISTS] name [ON CLUSTER cluster] +[ SET +key_name1 = 'some value', +key_name2 = 'some value', +key_name3 = 'some value', +... ] | +[ DELETE key_name4, key_name5, ... ] +``` + +**Example** + +```sql +CREATE NAMED COLLECTION foobar AS a = '1', b = '2'; + +ALTER NAMED COLLECTION foobar SET a = '2', c = '3'; + +ALTER NAMED COLLECTION foobar DELETE b; +``` diff --git a/docs/en/sql-reference/statements/create/index.md b/docs/en/sql-reference/statements/create/index.md index 14e29d051d7..fa39526a53e 100644 --- a/docs/en/sql-reference/statements/create/index.md +++ b/docs/en/sql-reference/statements/create/index.md @@ -8,13 +8,14 @@ sidebar_label: CREATE Create queries make a new entity of one of the following kinds: -- [DATABASE](../../../sql-reference/statements/create/database.md) -- [TABLE](../../../sql-reference/statements/create/table.md) -- [VIEW](../../../sql-reference/statements/create/view.md) -- [DICTIONARY](../../../sql-reference/statements/create/dictionary.md) -- [FUNCTION](../../../sql-reference/statements/create/function.md) -- [USER](../../../sql-reference/statements/create/user.md) -- [ROLE](../../../sql-reference/statements/create/role.md) -- [ROW POLICY](../../../sql-reference/statements/create/row-policy.md) -- [QUOTA](../../../sql-reference/statements/create/quota.md) -- [SETTINGS PROFILE](../../../sql-reference/statements/create/settings-profile.md) +- [DATABASE](/docs/en/sql-reference/statements/create/database.md) +- [TABLE](/docs/en/sql-reference/statements/create/table.md) +- [VIEW](/docs/en/sql-reference/statements/create/view.md) +- [DICTIONARY](/docs/en/sql-reference/statements/create/dictionary.md) +- [FUNCTION](/docs/en/sql-reference/statements/create/function.md) +- [USER](/docs/en/sql-reference/statements/create/user.md) +- [ROLE](/docs/en/sql-reference/statements/create/role.md) +- [ROW POLICY](/docs/en/sql-reference/statements/create/row-policy.md) +- [QUOTA](/docs/en/sql-reference/statements/create/quota.md) +- [SETTINGS PROFILE](/docs/en/sql-reference/statements/create/settings-profile.md) +- [NAMED COLLECTION](/docs/en/sql-reference/statements/create/named-collection.md) diff --git a/docs/en/sql-reference/statements/create/named-collection.md b/docs/en/sql-reference/statements/create/named-collection.md new file mode 100644 index 00000000000..e30a8cd6df2 --- /dev/null +++ b/docs/en/sql-reference/statements/create/named-collection.md @@ -0,0 +1,34 @@ +--- +slug: /en/sql-reference/statements/create/named-collection +sidebar_label: NAMED COLLECTION +--- + +# CREATE NAMED COLLECTION + +Creates a new named collection. + +**Syntax** + +```sql +CREATE NAMED COLLECTION [IF NOT EXISTS] name [ON CLUSTER cluster] AS +key_name1 = 'some value', +key_name2 = 'some value', +key_name3 = 'some value', +... +``` + +**Example** + +```sql +CREATE NAMED COLLECTION foobar AS a = '1', b = '2'; +``` + +**Related satements** + +- [CREATE NAMED COLLECTION](https://clickhouse.com/docs/en/sql-reference/statements/alter/named-collection) +- [DROP NAMED COLLECTION](https://clickhouse.com/docs/en/sql-reference/statements/drop#drop-function) + + +**See Also** + +- [Named collections guide](/docs/en/operations/named-collections.md) diff --git a/docs/en/sql-reference/statements/drop.md b/docs/en/sql-reference/statements/drop.md index b6208c2fd52..c91457993c4 100644 --- a/docs/en/sql-reference/statements/drop.md +++ b/docs/en/sql-reference/statements/drop.md @@ -119,3 +119,20 @@ DROP FUNCTION [IF EXISTS] function_name [on CLUSTER cluster] CREATE FUNCTION linear_equation AS (x, k, b) -> k*x + b; DROP FUNCTION linear_equation; ``` + +## DROP NAMED COLLECTION + +Deletes a named collection. + +**Syntax** + +``` sql +DROP NAMED COLLECTION [IF EXISTS] name [on CLUSTER cluster] +``` + +**Example** + +``` sql +CREATE NAMED COLLECTION foobar AS a = '1', b = '2'; +DROP NAMED COLLECTION foobar; +``` diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index 30db0ef00aa..fa86b436a5e 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -21,7 +21,7 @@ iceberg(url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure]) - `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. By default `Parquet` is used. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. -Engine parameters can be specified using [Named Collections](../../operations/named-collections.md) +Engine parameters can be specified using [Named Collections](/docs/en/operations/named-collections.md). **Returned value** From d2d7139da3af470a49267047f0ea45f652d59e45 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 08:58:23 +0200 Subject: [PATCH 1207/2047] Changelog for 23.7 --- CHANGELOG.md | 175 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 175 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index bf6b309ef2c..878edfa4add 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v23.7, 2023-07-27](#237)**
**[ClickHouse release v23.6, 2023-06-30](#236)**
**[ClickHouse release v23.5, 2023-06-08](#235)**
**[ClickHouse release v23.4, 2023-04-26](#234)**
@@ -9,6 +10,180 @@ # 2023 Changelog +###
ClickHouse release 23.7, 2023-07-27 + +#### Backward Incompatible Change +* Add `NAMED COLLECTION` access type (aliases `USE NAMED COLLECTION`, `NAMED COLLECTION USAGE`). This PR is backward incompatible because this access type is disabled by default (because a parent access type `NAMED COLLECTION ADMIN` is disabled by default as well). Proposed in [#50277](https://github.com/ClickHouse/ClickHouse/issues/50277). To grant use `GRANT NAMED COLLECTION ON collection_name TO user` or `GRANT NAMED COLLECTION ON * TO user`, to be able to give these grants `named_collection_admin` is required in config (previously it was named `named_collection_control`, so will remain as an alias). [#50625](https://github.com/ClickHouse/ClickHouse/pull/50625) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixing a typo in the `system.parts` column name `last_removal_attemp_time`. Now it is named `last_removal_attempt_time`. [#52104](https://github.com/ClickHouse/ClickHouse/pull/52104) ([filimonov](https://github.com/filimonov)). +* Bump version of the distributed_ddl_entry_format_version to 5 by default (enables opentelemetry and initial_query_idd pass through). This will not allow to process existing entries for distributed DDL after *downgrade* (but note, that usually there should be no such unprocessed entries). [#52128](https://github.com/ClickHouse/ClickHouse/pull/52128) ([Azat Khuzhin](https://github.com/azat)). +* Check projection metadata the same way we check ordinary metadata. This change may prevent the server from starting in case there was a table with an invalid projection. An example is a projection that created positional columns in PK (e.g. `projection p (select * order by 1, 4)` which is not allowed in table PK and can cause a crash during insert/merge). Drop such projections before the update. Fixes [#52353](https://github.com/ClickHouse/ClickHouse/issues/52353). [#52361](https://github.com/ClickHouse/ClickHouse/pull/52361) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* The experimental feature `hashid` is removed due to a bug. The quality of implementation was questionable at the start, and it didn't get through the experimental status. This closes [#52406](https://github.com/ClickHouse/ClickHouse/issues/52406). [#52449](https://github.com/ClickHouse/ClickHouse/pull/52449) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Added support for PRQL as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Add support for external disks in Keeper for storing snapshots and logs. [#50098](https://github.com/ClickHouse/ClickHouse/pull/50098) ([Antonio Andelic](https://github.com/antonio2368)). +* Add support for multi-directory selection (`{}`) globs. [#50559](https://github.com/ClickHouse/ClickHouse/pull/50559) ([Andrey Zvonov](https://github.com/zvonand)). +* Support ZooKeeper `reconfig` command for ClickHouse Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). +* Kafka connector can fetch Avro schema from schema registry with basic authentication using url-encoded credentials. [#49664](https://github.com/ClickHouse/ClickHouse/pull/49664) ([Ilya Golshtein](https://github.com/ilejn)). +* Add function `arrayJaccardIndex` which computes the Jaccard similarity between two arrays. [#50076](https://github.com/ClickHouse/ClickHouse/pull/50076) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Add a column `is_obsolete` to `system.settings` and similar tables. Closes [#50819](https://github.com/ClickHouse/ClickHouse/issues/50819). [#50826](https://github.com/ClickHouse/ClickHouse/pull/50826) ([flynn](https://github.com/ucasfl)). +* Implement support of encrypted elements in configuration file. Added possibility to use encrypted text in leaf elements of configuration file. The text is encrypted using encryption codecs from `` section. [#50986](https://github.com/ClickHouse/ClickHouse/pull/50986) ([Roman Vasin](https://github.com/rvasin)). +* Grace Hash Join algorithm is now applicable to FULL and RIGHT JOINs. [#49483](https://github.com/ClickHouse/ClickHouse/issues/49483). [#51013](https://github.com/ClickHouse/ClickHouse/pull/51013) ([lgbo](https://github.com/lgbo-ustc)). +* Add `SYSTEM STOP LISTEN` query for more graceful termination. Closes [#47972](https://github.com/ClickHouse/ClickHouse/issues/47972). [#51016](https://github.com/ClickHouse/ClickHouse/pull/51016) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add `input_format_csv_allow_variable_number_of_columns` options. [#51273](https://github.com/ClickHouse/ClickHouse/pull/51273) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Another boring feature: add function `substring_index`, as in Spark or MySQL. [#51472](https://github.com/ClickHouse/ClickHouse/pull/51472) ([李扬](https://github.com/taiyang-li)). +* A system table `jemalloc_bins` to show stats for jemalloc bins. Example `SELECT *, size * (nmalloc - ndalloc) AS allocated_bytes FROM system.jemalloc_bins WHERE allocated_bytes > 0 ORDER BY allocated_bytes DESC LIMIT 10`. Enjoy. [#51674](https://github.com/ClickHouse/ClickHouse/pull/51674) ([Alexander Gololobov](https://github.com/davenger)). +* Add `RowBinaryWithDefaults` format with extra byte before each column as a flag for using the column's default value. Closes [#50854](https://github.com/ClickHouse/ClickHouse/issues/50854). [#51695](https://github.com/ClickHouse/ClickHouse/pull/51695) ([Kruglov Pavel](https://github.com/Avogar)). +* Added `default_temporary_table_engine` setting. Same as `default_table_engine` but for temporary tables. [#51292](https://github.com/ClickHouse/ClickHouse/issues/51292). [#51708](https://github.com/ClickHouse/ClickHouse/pull/51708) ([velavokr](https://github.com/velavokr)). +* Added new `initcap` / `initcapUTF8` functions which convert the first letter of each word to upper case and the rest to lower case. [#51735](https://github.com/ClickHouse/ClickHouse/pull/51735) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Create table now supports `PRIMARY KEY` syntax in column definition. Columns are added to primary index in the same order columns are defined. [#51881](https://github.com/ClickHouse/ClickHouse/pull/51881) ([Ilya Yatsishin](https://github.com/qoega)). +* Added the possibility to use date and time format specifiers in log and error log file names, either in config files (`log` and `errorlog` tags) or command line arguments (`--log-file` and `--errorlog-file`). [#51945](https://github.com/ClickHouse/ClickHouse/pull/51945) ([Victor Krasnov](https://github.com/sirvickr)). +* Added Peak Memory Usage statistic to HTTP headers. [#51946](https://github.com/ClickHouse/ClickHouse/pull/51946) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added new `hasSubsequence` (+`CaseInsensitive` and `UTF8` versions) functions to match subsequences in strings. [#52050](https://github.com/ClickHouse/ClickHouse/pull/52050) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Add `array_agg` as alias of `groupArray` for PostgreSQL compatibility. Closes [#52100](https://github.com/ClickHouse/ClickHouse/issues/52100). ### Documentation entry for user-facing changes. [#52135](https://github.com/ClickHouse/ClickHouse/pull/52135) ([flynn](https://github.com/ucasfl)). +* Add `any_value` as a compatibility alias for `any` aggregate function. Closes [#52140](https://github.com/ClickHouse/ClickHouse/issues/52140). [#52147](https://github.com/ClickHouse/ClickHouse/pull/52147) ([flynn](https://github.com/ucasfl)). +* Add aggregate function `array_concat_agg` for compatibility with BigQuery, it's alias of `groupArrayArray`. Closes [#52139](https://github.com/ClickHouse/ClickHouse/issues/52139). [#52149](https://github.com/ClickHouse/ClickHouse/pull/52149) ([flynn](https://github.com/ucasfl)). +* Add `OCTET_LENGTH` as an alias to `length`. Closes [#52153](https://github.com/ClickHouse/ClickHouse/issues/52153). [#52176](https://github.com/ClickHouse/ClickHouse/pull/52176) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Added `firstLine` function to extract the first line from the multi-line string. This closes [#51172](https://github.com/ClickHouse/ClickHouse/issues/51172). [#52209](https://github.com/ClickHouse/ClickHouse/pull/52209) ([Mikhail Koviazin](https://github.com/mkmkme)). +* Implement KQL-style formatting for the `Interval` data type. This is only needed for compatibility with the `Kusto` query language. [#45671](https://github.com/ClickHouse/ClickHouse/pull/45671) ([ltrk2](https://github.com/ltrk2)). +* Added query `SYSTEM FLUSH ASYNC INSERT QUEUE` which flushes all pending asynchronous inserts to the destination tables. Added a server-side setting `async_insert_queue_flush_on_shutdown` (`true` by default) which determines whether to flush queue of asynchronous inserts on graceful shutdown. Setting `async_insert_threads` is now a server-side setting. [#49160](https://github.com/ClickHouse/ClickHouse/pull/49160) ([Anton Popov](https://github.com/CurtizJ)). +* Aliases `current_database` and a new function `current_schemas` for compatibility with PostgreSQL. [#51076](https://github.com/ClickHouse/ClickHouse/pull/51076) ([Pedro Riera](https://github.com/priera)). +* Add alias for functions `today` (now available under the `curdate`/`current_date` names) and `now` (`current_timestamp`). [#52106](https://github.com/ClickHouse/ClickHouse/pull/52106) ([Lloyd-Pottiger](https://github.com/Lloyd-Pottiger)). +* Support `async_deduplication_token` for async insert. [#52136](https://github.com/ClickHouse/ClickHouse/pull/52136) ([Han Fei](https://github.com/hanfei1991)). +* Add new setting `disable_url_encoding` that allows to disable decoding/encoding path in uri in URL engine. [#52337](https://github.com/ClickHouse/ClickHouse/pull/52337) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Performance Improvement +* Writing parquet files is 10x faster, it's multi-threaded now. Almost the same speed as reading. [#49367](https://github.com/ClickHouse/ClickHouse/pull/49367) ([Michael Kolupaev](https://github.com/al13n321)). +* Enable automatic selection of the sparse serialization format by default. It improves performance. The format is supported since version 22.1. After this change, downgrading to versions older than 22.1 might not be possible. You can turn off the usage of the sparse serialization format by providing the `ratio_of_defaults_for_sparse_serialization = 1` setting for your MergeTree tables. [#49631](https://github.com/ClickHouse/ClickHouse/pull/49631) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Enable `move_all_conditions_to_prewhere` and `enable_multiple_prewhere_read_steps` settings by default. [#46365](https://github.com/ClickHouse/ClickHouse/pull/46365) ([Alexander Gololobov](https://github.com/davenger)). +* Improves performance of some queries by tuning allocator. [#46416](https://github.com/ClickHouse/ClickHouse/pull/46416) ([Azat Khuzhin](https://github.com/azat)). +* Now we use fixed-size tasks in `MergeTreePrefetchedReadPool` as in `MergeTreeReadPool`. Also from now we use connection pool for S3 requests. [#49732](https://github.com/ClickHouse/ClickHouse/pull/49732) ([Nikita Taranov](https://github.com/nickitat)). +* More pushdown to the right side of join. [#50532](https://github.com/ClickHouse/ClickHouse/pull/50532) ([Nikita Taranov](https://github.com/nickitat)). +* Improve grace_hash join by reserving hash table's size (resubmit). [#50875](https://github.com/ClickHouse/ClickHouse/pull/50875) ([lgbo](https://github.com/lgbo-ustc)). +* Waiting on lock in `OpenedFileCache` could be noticeable sometimes. We sharded it into multiple sub-maps (each with its own lock) to avoid contention. [#51341](https://github.com/ClickHouse/ClickHouse/pull/51341) ([Nikita Taranov](https://github.com/nickitat)). +* Move conditions with primary key columns to the end of PREWHERE chain. The idea is that conditions with PK columns are likely to be used in PK analysis and will not contribute much more to PREWHERE filtering. [#51958](https://github.com/ClickHouse/ClickHouse/pull/51958) ([Alexander Gololobov](https://github.com/davenger)). +* Speed up `COUNT(DISTINCT)` for String types by inlining SipHash. The performance experiments of *OnTime* on the ICX device (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads) show that this change could bring an improvement of *11.6%* to the QPS of the query *Q8* while having no impact on others. [#52036](https://github.com/ClickHouse/ClickHouse/pull/52036) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Enable `allow_vertical_merges_from_compact_to_wide_parts` by default. It will save memory usage during merges. [#52295](https://github.com/ClickHouse/ClickHouse/pull/52295) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis which invalidates primary keys. This issue only exists when `query_plan_optimize_primary_key = 1, query_plan_optimize_projection = 1`. This fixes [#48823](https://github.com/ClickHouse/ClickHouse/issues/48823). This fixes [#51173](https://github.com/ClickHouse/ClickHouse/issues/51173). [#52308](https://github.com/ClickHouse/ClickHouse/pull/52308) ([Amos Bird](https://github.com/amosbird)). +* Reduce the number of syscalls in `FileCache::loadMetadata` - this speeds up server startup if the filesystem cache is configured. [#52435](https://github.com/ClickHouse/ClickHouse/pull/52435) ([Raúl Marín](https://github.com/Algunenano)). +* Allow to have strict lower boundary for file segment size by downloading remaining data in the background. Minimum size of file segment (if actual file size is bigger) is configured as cache configuration setting `boundary_alignment`, by default `4Mi`. Number of background threads are configured as cache configuration setting `background_download_threads`, by default `2`. Also `max_file_segment_size` was increased from `8Mi` to `32Mi` in this PR. [#51000](https://github.com/ClickHouse/ClickHouse/pull/51000) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Decreased default timeouts for S3 from 30 seconds to 3 seconds, and for other HTTP from 180 seconds to 30 seconds. [#51171](https://github.com/ClickHouse/ClickHouse/pull/51171) ([Michael Kolupaev](https://github.com/al13n321)). +* New setting `merge_tree_determine_task_size_by_prewhere_columns` added. If set to `true` only sizes of the columns from `PREWHERE` section will be considered to determine reading task size. Otherwise all the columns from query are considered. [#52606](https://github.com/ClickHouse/ClickHouse/pull/52606) ([Nikita Taranov](https://github.com/nickitat)). + +#### Improvement +* Use read_bytes/total_bytes_to_read for progress bar in s3/file/url/... table functions for better progress indication. [#51286](https://github.com/ClickHouse/ClickHouse/pull/51286) ([Kruglov Pavel](https://github.com/Avogar)). +* Introduce a table setting `wait_for_unique_parts_send_before_shutdown_ms` which specify the amount of time replica will wait before closing interserver handler for replicated sends. Also fix inconsistency with shutdown of tables and interserver handlers: now server shutdown tables first and only after it shut down interserver handlers. [#51851](https://github.com/ClickHouse/ClickHouse/pull/51851) ([alesapin](https://github.com/alesapin)). +* Allow SQL standard `FETCH` without `OFFSET`. See https://antonz.org/sql-fetch/. [#51293](https://github.com/ClickHouse/ClickHouse/pull/51293) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow filtering HTTP headers for the URL/S3 table functions with the new `http_forbid_headers` section in config. Both exact matching and regexp filters are available. [#51038](https://github.com/ClickHouse/ClickHouse/pull/51038) ([Nikolay Degterinsky](https://github.com/evillique)). +* Don't show messages about `16 EiB` free space in logs, as they don't make sense. This closes [#49320](https://github.com/ClickHouse/ClickHouse/issues/49320). [#49342](https://github.com/ClickHouse/ClickHouse/pull/49342) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Properly check the limit for the `sleepEachRow` function. Add a setting `function_sleep_max_microseconds_per_block`. This is needed for generic query fuzzer. [#49343](https://github.com/ClickHouse/ClickHouse/pull/49343) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix two issues in `geoHash` functions. [#50066](https://github.com/ClickHouse/ClickHouse/pull/50066) ([李扬](https://github.com/taiyang-li)). +* Log async insert flush queries into `system.query_log`. [#51160](https://github.com/ClickHouse/ClickHouse/pull/51160) ([Raúl Marín](https://github.com/Algunenano)). +* Functions `date_diff` and `age` now support millisecond/microsecond unit and work with microsecond precision. [#51291](https://github.com/ClickHouse/ClickHouse/pull/51291) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Improve parsing of path in clickhouse-keeper-client. [#51359](https://github.com/ClickHouse/ClickHouse/pull/51359) ([Azat Khuzhin](https://github.com/azat)). +* A third-party product depending on ClickHouse (Gluten: a Plugin to Double SparkSQL's Performance) had a bug. This fix avoids heap overflow in that third-party product while reading from HDFS. [#51386](https://github.com/ClickHouse/ClickHouse/pull/51386) ([李扬](https://github.com/taiyang-li)). +* Add ability to disable native copy for S3 (setting for BACKUP/RESTORE `allow_s3_native_copy`, and `s3_allow_native_copy` for `s3`/`s3_plain` disks). [#51448](https://github.com/ClickHouse/ClickHouse/pull/51448) ([Azat Khuzhin](https://github.com/azat)). +* Add column `primary_key_size` to `system.parts` table to show compressed primary key size on disk. Closes [#51400](https://github.com/ClickHouse/ClickHouse/issues/51400). [#51496](https://github.com/ClickHouse/ClickHouse/pull/51496) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Allow running `clickhouse-local` without procfs, without home directory existing, and without name resolution plugins from glibc. [#51518](https://github.com/ClickHouse/ClickHouse/pull/51518) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add placeholder `%a` for rull filename in rename_files_after_processing setting. [#51603](https://github.com/ClickHouse/ClickHouse/pull/51603) ([Kruglov Pavel](https://github.com/Avogar)). +* Add column `modification_time` into `system.parts_columns`. [#51685](https://github.com/ClickHouse/ClickHouse/pull/51685) ([Azat Khuzhin](https://github.com/azat)). +* Add new setting `input_format_csv_use_default_on_bad_values` to CSV format that allows to insert default value when parsing of a single field failed. [#51716](https://github.com/ClickHouse/ClickHouse/pull/51716) ([KevinyhZou](https://github.com/KevinyhZou)). +* Added a crash log flush to the disk after the unexpected crash. [#51720](https://github.com/ClickHouse/ClickHouse/pull/51720) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Fix behavior in dashboard page where errors unrelated to authentication are not shown. Also fix 'overlapping' chart behavior. [#51744](https://github.com/ClickHouse/ClickHouse/pull/51744) ([Zach Naimon](https://github.com/ArctypeZach)). +* Allow UUID to UInt128 conversion. [#51765](https://github.com/ClickHouse/ClickHouse/pull/51765) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added support for function `range` of Nullable arguments. [#51767](https://github.com/ClickHouse/ClickHouse/pull/51767) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Convert condition like `toyear(x) = c` to `c1 <= x < c2`. [#51795](https://github.com/ClickHouse/ClickHouse/pull/51795) ([Han Fei](https://github.com/hanfei1991)). +* Improve MySQL compatibility of the statement `SHOW INDEX`. [#51796](https://github.com/ClickHouse/ClickHouse/pull/51796) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix `use_structure_from_insertion_table_in_table_functions` does not work with `MATERIALIZED` and `ALIAS` columns. Closes [#51817](https://github.com/ClickHouse/ClickHouse/issues/51817). Closes [#51019](https://github.com/ClickHouse/ClickHouse/issues/51019). [#51825](https://github.com/ClickHouse/ClickHouse/pull/51825) ([flynn](https://github.com/ucasfl)). +* Cache dictionary now requests only unique keys from source. Closes [#51762](https://github.com/ClickHouse/ClickHouse/issues/51762). [#51853](https://github.com/ClickHouse/ClickHouse/pull/51853) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed the case when settings were not applied for EXPLAIN query when FORMAT was provided. [#51859](https://github.com/ClickHouse/ClickHouse/pull/51859) ([Nikita Taranov](https://github.com/nickitat)). +* Allow SETTINGS before FORMAT in DESCRIBE TABLE query for compatibility with SELECT query. Closes [#51544](https://github.com/ClickHouse/ClickHouse/issues/51544). [#51899](https://github.com/ClickHouse/ClickHouse/pull/51899) ([Nikolay Degterinsky](https://github.com/evillique)). +* Var-Int encoded integers (e.g. used by the native protocol) can now use the full 64-bit range. 3rd party clients are advised to update their var-int code accordingly. [#51905](https://github.com/ClickHouse/ClickHouse/pull/51905) ([Robert Schulze](https://github.com/rschu1ze)). +* Update certificates when they change without the need to manually SYSTEM RELOAD CONFIG. [#52030](https://github.com/ClickHouse/ClickHouse/pull/52030) ([Mike Kot](https://github.com/myrrc)). +* Added `allow_create_index_without_type` setting that allow to ignore `ADD INDEX` queries without specified `TYPE`. Standard SQL queries will just succeed without changing table schema. [#52056](https://github.com/ClickHouse/ClickHouse/pull/52056) ([Ilya Yatsishin](https://github.com/qoega)). +* Log messages are written to the `system.text_log` from the server startup. [#52113](https://github.com/ClickHouse/ClickHouse/pull/52113) ([Dmitry Kardymon](https://github.com/kardymonds)). +* In cases where the HTTP endpoint has multiple IP addresses and the first of them is unreachable, a timeout exception was thrown. Made session creation with handling all resolved endpoints. [#52116](https://github.com/ClickHouse/ClickHouse/pull/52116) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Avro input format now supports Union even if it contains only a single type. Closes [#52131](https://github.com/ClickHouse/ClickHouse/issues/52131). [#52137](https://github.com/ClickHouse/ClickHouse/pull/52137) ([flynn](https://github.com/ucasfl)). +* Add setting `optimize_use_implicit_projections` to disable implicit projections (currently only `min_max_count` projection). [#52152](https://github.com/ClickHouse/ClickHouse/pull/52152) ([Amos Bird](https://github.com/amosbird)). +* It was possible to use the function `hasToken` for infinite loop. Now this possibility is removed. This closes [#52156](https://github.com/ClickHouse/ClickHouse/issues/52156). [#52160](https://github.com/ClickHouse/ClickHouse/pull/52160) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Create ZK ancestors optimistically. [#52195](https://github.com/ClickHouse/ClickHouse/pull/52195) ([Raúl Marín](https://github.com/Algunenano)). +* Fix [#50582](https://github.com/ClickHouse/ClickHouse/issues/50582). Avoid the `Not found column ... in block` error in some cases of reading in-order and constants. [#52259](https://github.com/ClickHouse/ClickHouse/pull/52259) ([Chen768959](https://github.com/Chen768959)). +* Check whether S2 geo primitives are invalid as early as possible on ClickHouse side. This closes: [#27090](https://github.com/ClickHouse/ClickHouse/issues/27090). [#52260](https://github.com/ClickHouse/ClickHouse/pull/52260) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add back missing projection QueryAccessInfo when `query_plan_optimize_projection = 1`. This fixes [#50183](https://github.com/ClickHouse/ClickHouse/issues/50183) . This fixes [#50093](https://github.com/ClickHouse/ClickHouse/issues/50093). [#52327](https://github.com/ClickHouse/ClickHouse/pull/52327) ([Amos Bird](https://github.com/amosbird)). +* When `ZooKeeperRetriesControl` rethrows an error, it's more useful to see its original stack trace, not the one from `ZooKeeperRetriesControl` itself. [#52347](https://github.com/ClickHouse/ClickHouse/pull/52347) ([Vitaly Baranov](https://github.com/vitlibar)). +* Wait for zero copy replication lock even if some disks don't support it. [#52376](https://github.com/ClickHouse/ClickHouse/pull/52376) ([Raúl Marín](https://github.com/Algunenano)). +* Now interserver port will be closed only after tables are shut down. [#52498](https://github.com/ClickHouse/ClickHouse/pull/52498) ([alesapin](https://github.com/alesapin)). + +#### Experimental Feature +* Allow to add disk name for custom disks. Previously custom disks would use an internal generated disk name. Now it will be possible with `disk = disk_(...)` (e.g. disk will have name `name`) . [#51552](https://github.com/ClickHouse/ClickHouse/pull/51552) ([Kseniia Sumarokova](https://github.com/kssenii)). This syntax can be changed in this release. +* (experimental MaterializedMySQL) Fixed crash when `mysqlxx::Pool::Entry` is used after it was disconnected. [#52063](https://github.com/ClickHouse/ClickHouse/pull/52063) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) `CREATE TABLE ... AS SELECT` .. is now supported in MaterializedMySQL. [#52067](https://github.com/ClickHouse/ClickHouse/pull/52067) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) Introduced automatic conversion of text types to utf8 for MaterializedMySQL. [#52084](https://github.com/ClickHouse/ClickHouse/pull/52084) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) Now unquoted UTF-8 strings are supported in DDL for MaterializedMySQL. [#52318](https://github.com/ClickHouse/ClickHouse/pull/52318) ([Val Doroshchuk](https://github.com/valbok)). +* (experimental MaterializedMySQL) Now double quoted comments are supported in MaterializedMySQL. [#52355](https://github.com/ClickHouse/ClickHouse/pull/52355) ([Val Doroshchuk](https://github.com/valbok)). +* Upgrade Intel QPL from v1.1.0 to v1.2.0 2. Upgrade Intel accel-config from v3.5 to v4.0 3. Fixed issue that Device IOTLB miss has big perf. impact for IAA accelerators. [#52180](https://github.com/ClickHouse/ClickHouse/pull/52180) ([jasperzhu](https://github.com/jinjunzh)). +* The `session_timezone` setting (new in version 23.6) is demoted to experimental. [#52445](https://github.com/ClickHouse/ClickHouse/pull/52445) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement +* Add experimental ClickHouse builds for Linux RISC-V 64 to CI. [#31398](https://github.com/ClickHouse/ClickHouse/pull/31398) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add integration test check with the enabled Analyzer. [#50926](https://github.com/ClickHouse/ClickHouse/pull/50926) [#52210](https://github.com/ClickHouse/ClickHouse/pull/52210) ([Dmitry Novik](https://github.com/novikd)). +* Reproducible builds for Rust. [#52395](https://github.com/ClickHouse/ClickHouse/pull/52395) ([Azat Khuzhin](https://github.com/azat)). +* Update Cargo dependencies. [#51721](https://github.com/ClickHouse/ClickHouse/pull/51721) ([Raúl Marín](https://github.com/Algunenano)). +* Make the function `CHColumnToArrowColumn::fillArrowArrayWithArrayColumnData` to work with nullable arrays, which are not possible in ClickHouse, but needed for Gluten. [#52112](https://github.com/ClickHouse/ClickHouse/pull/52112) ([李扬](https://github.com/taiyang-li)). +* We've updated the CCTZ library to master, but there are no user-visible changes. [#52124](https://github.com/ClickHouse/ClickHouse/pull/52124) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The `system.licenses` table now includes the hard-forked library Poco. This closes [#52066](https://github.com/ClickHouse/ClickHouse/issues/52066). [#52127](https://github.com/ClickHouse/ClickHouse/pull/52127) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check that there are no cases of bad punctuation: whitespace before a comma like `Hello ,world` instead of `Hello, world`. [#52549](https://github.com/ClickHouse/ClickHouse/pull/52549) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fix MaterializedPostgreSQL syncTables [#49698](https://github.com/ClickHouse/ClickHouse/pull/49698) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix projection with optimize_aggregators_of_group_by_keys [#49709](https://github.com/ClickHouse/ClickHouse/pull/49709) ([Amos Bird](https://github.com/amosbird)). +* Fix optimize_skip_unused_shards with JOINs [#51037](https://github.com/ClickHouse/ClickHouse/pull/51037) ([Azat Khuzhin](https://github.com/azat)). +* Fix formatDateTime() with fractional negative datetime64 [#51290](https://github.com/ClickHouse/ClickHouse/pull/51290) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Functions `hasToken*` were totally wrong. Add a test for [#43358](https://github.com/ClickHouse/ClickHouse/issues/43358) [#51378](https://github.com/ClickHouse/ClickHouse/pull/51378) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix optimization to move functions before sorting. [#51481](https://github.com/ClickHouse/ClickHouse/pull/51481) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix Block structure mismatch in Pipe::unitePipes for FINAL [#51492](https://github.com/ClickHouse/ClickHouse/pull/51492) ([Nikita Taranov](https://github.com/nickitat)). +* Fix SIGSEGV for clusters with zero weight across all shards (fixes INSERT INTO FUNCTION clusterAllReplicas()) [#51545](https://github.com/ClickHouse/ClickHouse/pull/51545) ([Azat Khuzhin](https://github.com/azat)). +* Fix timeout for hedged requests [#51582](https://github.com/ClickHouse/ClickHouse/pull/51582) ([Azat Khuzhin](https://github.com/azat)). +* Fix logical error in ANTI join with NULL [#51601](https://github.com/ClickHouse/ClickHouse/pull/51601) ([vdimir](https://github.com/vdimir)). +* Fix for moving 'IN' conditions to PREWHERE [#51610](https://github.com/ClickHouse/ClickHouse/pull/51610) ([Alexander Gololobov](https://github.com/davenger)). +* Do not apply PredicateExpressionsOptimizer for ASOF/ANTI join [#51633](https://github.com/ClickHouse/ClickHouse/pull/51633) ([vdimir](https://github.com/vdimir)). +* Fix async insert with deduplication for ReplicatedMergeTree using merging algorithms [#51676](https://github.com/ClickHouse/ClickHouse/pull/51676) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix reading from empty column in `parseSipHashKey` [#51804](https://github.com/ClickHouse/ClickHouse/pull/51804) ([Nikita Taranov](https://github.com/nickitat)). +* Fix segfault when create invalid EmbeddedRocksdb table [#51847](https://github.com/ClickHouse/ClickHouse/pull/51847) ([Duc Canh Le](https://github.com/canhld94)). +* Fix inserts into MongoDB tables [#51876](https://github.com/ClickHouse/ClickHouse/pull/51876) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix deadlock on DatabaseCatalog shutdown [#51908](https://github.com/ClickHouse/ClickHouse/pull/51908) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix error in subquery operators [#51922](https://github.com/ClickHouse/ClickHouse/pull/51922) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix async connect to hosts with multiple ips [#51934](https://github.com/ClickHouse/ClickHouse/pull/51934) ([Kruglov Pavel](https://github.com/Avogar)). +* Do not remove inputs after ActionsDAG::merge [#51947](https://github.com/ClickHouse/ClickHouse/pull/51947) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Check refcount in `RemoveManyObjectStorageOperation::finalize` instead of `execute` [#51954](https://github.com/ClickHouse/ClickHouse/pull/51954) ([vdimir](https://github.com/vdimir)). +* Allow parametric UDFs [#51964](https://github.com/ClickHouse/ClickHouse/pull/51964) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Small fix for toDateTime64() for dates after 2283-12-31 [#52130](https://github.com/ClickHouse/ClickHouse/pull/52130) ([Andrey Zvonov](https://github.com/zvonand)). +* Fix ORDER BY tuple of WINDOW functions [#52145](https://github.com/ClickHouse/ClickHouse/pull/52145) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis when aggregation expression contains monotonic functions [#52151](https://github.com/ClickHouse/ClickHouse/pull/52151) ([Amos Bird](https://github.com/amosbird)). +* Fix error in `groupArrayMoving` functions [#52161](https://github.com/ClickHouse/ClickHouse/pull/52161) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable direct join for range dictionary [#52187](https://github.com/ClickHouse/ClickHouse/pull/52187) ([Duc Canh Le](https://github.com/canhld94)). +* Fix sticky mutations test (and extremely rare race condition) [#52197](https://github.com/ClickHouse/ClickHouse/pull/52197) ([alesapin](https://github.com/alesapin)). +* Fix race in Web disk [#52211](https://github.com/ClickHouse/ClickHouse/pull/52211) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix data race in Connection::setAsyncCallback on unknown packet from server [#52219](https://github.com/ClickHouse/ClickHouse/pull/52219) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix temp data deletion on startup, add test [#52275](https://github.com/ClickHouse/ClickHouse/pull/52275) ([vdimir](https://github.com/vdimir)). +* Don't use minmax_count projections when counting nullable columns [#52297](https://github.com/ClickHouse/ClickHouse/pull/52297) ([Amos Bird](https://github.com/amosbird)). +* MergeTree/ReplicatedMergeTree should use server timezone for log entries [#52325](https://github.com/ClickHouse/ClickHouse/pull/52325) ([Azat Khuzhin](https://github.com/azat)). +* Fix parameterized view with cte and multiple usage [#52328](https://github.com/ClickHouse/ClickHouse/pull/52328) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Disable expression templates for time intervals [#52335](https://github.com/ClickHouse/ClickHouse/pull/52335) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `apply_snapshot` in Keeper [#52358](https://github.com/ClickHouse/ClickHouse/pull/52358) ([Antonio Andelic](https://github.com/antonio2368)). +* Update build-osx.md [#52377](https://github.com/ClickHouse/ClickHouse/pull/52377) ([AlexBykovski](https://github.com/AlexBykovski)). +* Fix `countSubstrings()` hang with empty needle and a column haystack [#52409](https://github.com/ClickHouse/ClickHouse/pull/52409) ([Sergei Trifonov](https://github.com/serxa)). +* Fix normal projection with merge table [#52432](https://github.com/ClickHouse/ClickHouse/pull/52432) ([Amos Bird](https://github.com/amosbird)). +* Fix possible double-free in Aggregator [#52439](https://github.com/ClickHouse/ClickHouse/pull/52439) ([Nikita Taranov](https://github.com/nickitat)). +* Fixed inserting into Buffer engine [#52440](https://github.com/ClickHouse/ClickHouse/pull/52440) ([Vasily Nemkov](https://github.com/Enmk)). +* The implementation of AnyHash was non-conformant. [#52448](https://github.com/ClickHouse/ClickHouse/pull/52448) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check recursion depth in OptimizedRegularExpression [#52451](https://github.com/ClickHouse/ClickHouse/pull/52451) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix data-race DatabaseReplicated::startupTables()/canExecuteReplicatedMetadataAlter() [#52490](https://github.com/ClickHouse/ClickHouse/pull/52490) ([Azat Khuzhin](https://github.com/azat)). +* Fix abort in function `transform` [#52513](https://github.com/ClickHouse/ClickHouse/pull/52513) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix lightweight delete after drop of projection [#52517](https://github.com/ClickHouse/ClickHouse/pull/52517) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible error "Cannot drain connections: cancel first" [#52585](https://github.com/ClickHouse/ClickHouse/pull/52585) ([Kruglov Pavel](https://github.com/Avogar)). + + ### ClickHouse release 23.6, 2023-06-29 #### Backward Incompatible Change From d6afa3de0150dfeaf2c346bf6f5080c095d73091 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Thu, 27 Jul 2023 10:29:36 +0300 Subject: [PATCH 1208/2047] Fix missing 'typename' prior to dependent type name --- src/Common/SystemLogBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index ed5ffd78a7b..3d68fe63227 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -185,7 +185,7 @@ void SystemLogQueue::confirm(uint64_t to_flush_end) } template -SystemLogQueue::Index SystemLogQueue::pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread) +typename SystemLogQueue::Index SystemLogQueue::pop(std::vector& output, bool& should_prepare_tables_anyway, bool& exit_this_thread) { std::unique_lock lock(mutex); flush_event.wait_for(lock, From 5395d34fd719e5ff4b94a522ee42278dcc2acece Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Thu, 27 Jul 2023 10:32:10 +0300 Subject: [PATCH 1209/2047] Fix missing constructor call --- src/Storages/HDFS/StorageHDFS.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index f176ac6f037..fa6cfd824e7 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -114,9 +114,9 @@ namespace { if (next_slash_after_glob_pos == std::string::npos) { - result.emplace_back( + result.emplace_back(StorageHDFS::PathWithInfo{ String(ls.file_info[i].mName), - StorageHDFS::PathInfo{ls.file_info[i].mLastMod, static_cast(ls.file_info[i].mSize)}); + StorageHDFS::PathInfo{ls.file_info[i].mLastMod, static_cast(ls.file_info[i].mSize)}}); } else { From bcc41fbc2f2f32a7187110583d8739b72a311856 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Thu, 27 Jul 2023 10:46:35 +0300 Subject: [PATCH 1210/2047] Fix reference to a local binding declared in an enclosing function --- src/Coordination/KeeperSnapshotManagerS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 0b135442265..cbb5c110eda 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -218,7 +218,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const SnapshotFileInfo & snapsh } catch (...) { - LOG_INFO(log, "Failed to delete lock file for {} from S3", snapshot_path); + LOG_INFO(log, "Failed to delete lock file for {} from S3", snapshot_file_info.path); tryLogCurrentException(__PRETTY_FUNCTION__); } }); From ac51ade45c5581d031c382277b550b2fb2f873fe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Thu, 27 Jul 2023 08:45:53 +0000 Subject: [PATCH 1211/2047] Do not run the test without Rust libraries --- tests/queries/0_stateless/02833_local_with_dialect.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02833_local_with_dialect.sh b/tests/queries/0_stateless/02833_local_with_dialect.sh index 2a2e1b09459..012a6d91269 100755 --- a/tests/queries/0_stateless/02833_local_with_dialect.sh +++ b/tests/queries/0_stateless/02833_local_with_dialect.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest, no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From f61f36800ccd028bb9cf4ef402275006faf2facb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 27 Jul 2023 08:48:23 +0000 Subject: [PATCH 1212/2047] Fix style --- .../Formats/Impl/Parquet/PrepareForWrite.cpp | 14 +++++++------- src/Processors/Formats/Impl/Parquet/Write.cpp | 16 ++++++++-------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp index 0700fc8491c..bc4c9ca3b72 100644 --- a/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp +++ b/src/Processors/Formats/Impl/Parquet/PrepareForWrite.cpp @@ -303,14 +303,14 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin } else { - types(T::INT32, C::UINT_8 , int_type(8 , false)); + types(T::INT32, C::UINT_8, int_type(8, false)); } break; case TypeIndex::UInt16: types(T::INT32, C::UINT_16, int_type(16, false)); break; case TypeIndex::UInt32: types(T::INT32, C::UINT_32, int_type(32, false)); break; case TypeIndex::UInt64: types(T::INT64, C::UINT_64, int_type(64, false)); break; - case TypeIndex::Int8: types(T::INT32, C::INT_8 , int_type(8 , true)); break; - case TypeIndex::Int16: types(T::INT32, C::INT_16 , int_type(16, true)); break; + case TypeIndex::Int8: types(T::INT32, C::INT_8, int_type(8, true)); break; + case TypeIndex::Int16: types(T::INT32, C::INT_16, int_type(16, true)); break; case TypeIndex::Int32: types(T::INT32); break; case TypeIndex::Int64: types(T::INT64); break; case TypeIndex::Float32: types(T::FLOAT); break; @@ -319,8 +319,8 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin /// These don't have suitable parquet logical types, so we write them as plain numbers. /// (Parquet has "enums" but they're just strings, with nowhere to declare all possible enum /// values in advance as part of the data type.) - case TypeIndex::Enum8: types(T::INT32, C::INT_8 , int_type(8 , true)); break; // Int8 - case TypeIndex::Enum16: types(T::INT32, C::INT_16 , int_type(16, true)); break; // Int16 + case TypeIndex::Enum8: types(T::INT32, C::INT_8, int_type(8, true)); break; // Int8 + case TypeIndex::Enum16: types(T::INT32, C::INT_16, int_type(16, true)); break; // Int16 case TypeIndex::IPv4: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32 case TypeIndex::Date: types(T::INT32, C::UINT_16, int_type(16, false)); break; // UInt16 case TypeIndex::DateTime: types(T::INT32, C::UINT_32, int_type(32, false)); break; // UInt32 @@ -392,8 +392,8 @@ void preparePrimitiveColumn(ColumnPtr column, DataTypePtr type, const std::strin case TypeIndex::Int256: fixed_string(32); break; case TypeIndex::IPv6: fixed_string(16); break; - case TypeIndex::Decimal32: decimal(4 , getDecimalPrecision(*type), getDecimalScale(*type)); break; - case TypeIndex::Decimal64: decimal(8 , getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal32: decimal(4, getDecimalPrecision(*type), getDecimalScale(*type)); break; + case TypeIndex::Decimal64: decimal(8, getDecimalPrecision(*type), getDecimalScale(*type)); break; case TypeIndex::Decimal128: decimal(16, getDecimalPrecision(*type), getDecimalScale(*type)); break; case TypeIndex::Decimal256: decimal(32, getDecimalPrecision(*type), getDecimalScale(*type)); break; diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index 47ef0c53ab5..5ebf2be76d2 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -755,20 +755,20 @@ void writeColumnChunkBody(ColumnChunkWriteState & s, const WriteOptions & option writeColumnImpl(s, options, out, ConverterNumeric, bool, bool>(s.primitive_column)); else - N(UInt8 , Int32Type); + N(UInt8, Int32Type); break; case TypeIndex::UInt16 : N(UInt16, Int32Type); break; case TypeIndex::UInt32 : N(UInt32, Int32Type); break; case TypeIndex::UInt64 : N(UInt64, Int64Type); break; - case TypeIndex::Int8 : N(Int8 , Int32Type); break; - case TypeIndex::Int16 : N(Int16 , Int32Type); break; - case TypeIndex::Int32 : N(Int32 , Int32Type); break; - case TypeIndex::Int64 : N(Int64 , Int64Type); break; + case TypeIndex::Int8 : N(Int8, Int32Type); break; + case TypeIndex::Int16 : N(Int16, Int32Type); break; + case TypeIndex::Int32 : N(Int32, Int32Type); break; + case TypeIndex::Int64 : N(Int64, Int64Type); break; - case TypeIndex::Enum8: N(Int8 , Int32Type); break; - case TypeIndex::Enum16: N(Int16 , Int32Type); break; + case TypeIndex::Enum8: N(Int8, Int32Type); break; + case TypeIndex::Enum16: N(Int16, Int32Type); break; case TypeIndex::Date: N(UInt16, Int32Type); break; - case TypeIndex::Date32: N(Int32 , Int32Type); break; + case TypeIndex::Date32: N(Int32, Int32Type); break; case TypeIndex::DateTime: N(UInt32, Int32Type); break; #undef N From 2b18872e86898fe0c0ee40ddecf05c29088a7aca Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 08:55:19 +0000 Subject: [PATCH 1213/2047] Incorporate review feedback --- docs/en/operations/system-tables/query_log.md | 8 ++++---- src/Interpreters/Cache/QueryCache.h | 8 ++++---- src/Interpreters/QueryLog.cpp | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index b8dc0c0224c..835c79129de 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -112,10 +112,10 @@ Columns: - `used_storages` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `storages`, which were used during query execution. - `used_table_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `table functions`, which were used during query execution. - `query_cache_usage` ([Enum8](../../sql-reference/data-types/enum.md)) — Usage of the [query cache](../query-cache.md) during query execution. Values: - - `'None' = 1` = The query result was neither written into nor read from the query cache. - - `'Write' = 1` = The query result was written into the query cache. - - `'Read' = 1` = The query result was read from the query cache. - - `'Unknown' = 1` = Unknown status. + - `'Unknown' = 1` = Status unknown. + - `'None' = 2` = The query result was neither written into nor read from the query cache. + - `'Write' = 3` = The query result was written into the query cache. + - `'Read' = 4` = The query result was read from the query cache. **Example** diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 973015b8003..5fe756268f2 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -27,10 +27,10 @@ public: enum class Usage { /// starts at 1 for compatibitity with DataTypeEnum8 - None = 1, /// query result neither written nor read into/from query cache - Write, /// query result wrote into query cache - Read, /// query result read from query cache - Unknown, /// we don't know what what happened + Unknown = 1, /// we don't know what what happened + None, /// query result neither written nor read into/from query cache + Write, /// query result written into query cache + Read, /// query result read from query cache }; /// Represents a query result in the cache. diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index c3294512f14..df21e82305a 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -44,10 +44,10 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() auto query_cache_usage_datatype = std::make_shared( DataTypeEnum8::Values { + {"Unknown", static_cast(QueryCache::Usage::Unknown)}, {"None", static_cast(QueryCache::Usage::None)}, {"Write", static_cast(QueryCache::Usage::Write)}, - {"Read", static_cast(QueryCache::Usage::Read)}, - {"Unknown", static_cast(QueryCache::Usage::Unknown)} + {"Read", static_cast(QueryCache::Usage::Read)} }); auto low_cardinality_string = std::make_shared(std::make_shared()); From 043ad45ec3f4f2d1ff8b619da4c6ec5cb2b8f0d0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 09:18:24 +0000 Subject: [PATCH 1214/2047] Incorporate review feedback, pt. II --- docs/en/operations/system-tables/query_log.md | 8 ++++---- src/Interpreters/Cache/QueryCache.h | 9 ++++----- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 835c79129de..c6f565b8748 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -112,10 +112,10 @@ Columns: - `used_storages` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `storages`, which were used during query execution. - `used_table_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `table functions`, which were used during query execution. - `query_cache_usage` ([Enum8](../../sql-reference/data-types/enum.md)) — Usage of the [query cache](../query-cache.md) during query execution. Values: - - `'Unknown' = 1` = Status unknown. - - `'None' = 2` = The query result was neither written into nor read from the query cache. - - `'Write' = 3` = The query result was written into the query cache. - - `'Read' = 4` = The query result was read from the query cache. + - `'Unknown'` = Status unknown. + - `'None'` = The query result was neither written into nor read from the query cache. + - `'Write'` = The query result was written into the query cache. + - `'Read'` = The query result was read from the query cache. **Example** diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 5fe756268f2..c2de8ca22dd 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -26,11 +26,10 @@ class QueryCache public: enum class Usage { - /// starts at 1 for compatibitity with DataTypeEnum8 - Unknown = 1, /// we don't know what what happened - None, /// query result neither written nor read into/from query cache - Write, /// query result written into query cache - Read, /// query result read from query cache + Unknown, /// we don't know what what happened + None, /// query result neither written nor read into/from query cache + Write, /// query result written into query cache + Read, /// query result read from query cache }; /// Represents a query result in the cache. From 578ea9c5e8a0e9eed4da2afa15c84951aba9d688 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 27 Jul 2023 09:50:42 +0000 Subject: [PATCH 1215/2047] Update documentation --- docs/en/operations/backup.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 62f931a76b4..63bf3cfeb5c 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -84,6 +84,7 @@ The BACKUP and RESTORE statements take a list of DATABASE and TABLE names, a des - `password` for the file on disk - `base_backup`: the destination of the previous backup of this source. For example, `Disk('backups', '1.zip')` - `structure_only`: if enabled, allows to only backup or restore the CREATE statements without the data of tables + - `s3_storage_class`: the storage class used for S3 backup. For example, `STANDARD` ### Usage examples From 04361908e67380cdea7c185b28d8f056330424af Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 27 Jul 2023 10:36:02 +0000 Subject: [PATCH 1216/2047] Fix test_keeper_reconfig_replace_leader --- tests/integration/test_keeper_reconfig_replace_leader/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_keeper_reconfig_replace_leader/test.py b/tests/integration/test_keeper_reconfig_replace_leader/test.py index ca1ec3a0c92..ef1d5394b67 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader/test.py @@ -83,6 +83,8 @@ def test_reconfig_replace_leader(started_cluster): assert "node3" in config assert "node4" not in config + ku.wait_configs_equal(config, zk2) + with pytest.raises(Exception): zk1.stop() zk1.close() From 228de12d94a206f6eaae74059216886c32c2b53e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 27 Jul 2023 13:59:23 +0300 Subject: [PATCH 1217/2047] Update ReplicatedMergeTreeQueue.cpp (#52648) --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index e11913fc3d2..21d5597e614 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1803,7 +1803,7 @@ std::map ReplicatedMergeTreeQueue::getAlterMutationCo Int64 part_data_version = part->info.getDataVersion(); Int64 part_metadata_version = part->getMetadataVersion(); - LOG_DEBUG(log, "Looking for mutations for part {} (part data version {}, part metadata version {})", part->name, part_data_version, part_metadata_version); + LOG_TEST(log, "Looking for mutations for part {} (part data version {}, part metadata version {})", part->name, part_data_version, part_metadata_version); std::map result; /// Here we return mutation commands for part which has bigger alter version than part metadata version. From e749938e92667ed3195c773726037d44c865993b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 27 Jul 2023 12:07:34 +0000 Subject: [PATCH 1218/2047] fixed test --- src/Functions/array/arrayIntersect.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 6abfb8ec5c6..ee84e3138e8 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -628,13 +628,6 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable if (all_nullable) null_map.push_back(0); } - if (all_has_nullable && !null_added) - { - ++result_offset; - result_data.insertDefault(); - null_map.push_back(1); - null_added = true; - } } result_offsets.getElement(row) = result_offset; From 9d73be6fca63edbca0bd97d07386f41268f1b11d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 12:14:46 +0000 Subject: [PATCH 1219/2047] test is added --- .../0_stateless/02833_local_udf_options.reference | 1 + tests/queries/0_stateless/02833_local_udf_options.sh | 11 +++++++++++ tests/queries/0_stateless/scripts_udf/function.xml | 9 +++++++++ tests/queries/0_stateless/scripts_udf/udf.sh | 3 +++ 4 files changed, 24 insertions(+) create mode 100755 tests/queries/0_stateless/02833_local_udf_options.reference create mode 100755 tests/queries/0_stateless/02833_local_udf_options.sh create mode 100644 tests/queries/0_stateless/scripts_udf/function.xml create mode 100755 tests/queries/0_stateless/scripts_udf/udf.sh diff --git a/tests/queries/0_stateless/02833_local_udf_options.reference b/tests/queries/0_stateless/02833_local_udf_options.reference new file mode 100755 index 00000000000..19f0805d8de --- /dev/null +++ b/tests/queries/0_stateless/02833_local_udf_options.reference @@ -0,0 +1 @@ +qwerty diff --git a/tests/queries/0_stateless/02833_local_udf_options.sh b/tests/queries/0_stateless/02833_local_udf_options.sh new file mode 100755 index 00000000000..149b62d7e2c --- /dev/null +++ b/tests/queries/0_stateless/02833_local_udf_options.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +SCRIPTS_DIR=$CUR_DIR/scripts_udf + +$CLICKHOUSE_LOCAL -q 'select test_function()' -- --user_scripts_path=$SCRIPTS_DIR --user_defined_executable_functions_config=$SCRIPTS_DIR/function.xml diff --git a/tests/queries/0_stateless/scripts_udf/function.xml b/tests/queries/0_stateless/scripts_udf/function.xml new file mode 100644 index 00000000000..69a0abb5cec --- /dev/null +++ b/tests/queries/0_stateless/scripts_udf/function.xml @@ -0,0 +1,9 @@ + + + executable + test_function + String + TabSeparated + udf.sh + + diff --git a/tests/queries/0_stateless/scripts_udf/udf.sh b/tests/queries/0_stateless/scripts_udf/udf.sh new file mode 100755 index 00000000000..add85833c3e --- /dev/null +++ b/tests/queries/0_stateless/scripts_udf/udf.sh @@ -0,0 +1,3 @@ +#!/bin/sh + +echo qwerty From b5fc34b770cdbc555d1f9d45516af530da9ab5e5 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 27 Jul 2023 12:20:33 +0000 Subject: [PATCH 1220/2047] Rename setting disable_url_encoding to enable_url_encoding and add a test --- base/poco/Foundation/include/Poco/URI.h | 4 ++-- base/poco/Foundation/src/URI.cpp | 24 +++++++++---------- docs/en/engines/table-engines/special/url.md | 2 +- docs/en/operations/settings/settings.md | 6 ++--- docs/en/sql-reference/table-functions/url.md | 2 +- src/Core/Settings.h | 2 +- src/Storages/StorageURL.cpp | 2 +- .../02833_url_without_path_encoding.reference | 2 ++ .../02833_url_without_path_encoding.sh | 12 ++++++++++ 9 files changed, 35 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/02833_url_without_path_encoding.reference create mode 100755 tests/queries/0_stateless/02833_url_without_path_encoding.sh diff --git a/base/poco/Foundation/include/Poco/URI.h b/base/poco/Foundation/include/Poco/URI.h index eba8109253d..30654504e0b 100644 --- a/base/poco/Foundation/include/Poco/URI.h +++ b/base/poco/Foundation/include/Poco/URI.h @@ -57,7 +57,7 @@ public: URI(); /// Creates an empty URI. - explicit URI(const std::string & uri, bool disable_url_encoding = false); + explicit URI(const std::string & uri, bool enable_url_encoding = true); /// Parses an URI from the given string. Throws a /// SyntaxException if the uri is not valid. @@ -362,7 +362,7 @@ private: std::string _query; std::string _fragment; - bool _disable_url_encoding = false; + bool _enable_url_encoding = true; }; diff --git a/base/poco/Foundation/src/URI.cpp b/base/poco/Foundation/src/URI.cpp index 3354c69d188..41e331bb080 100644 --- a/base/poco/Foundation/src/URI.cpp +++ b/base/poco/Foundation/src/URI.cpp @@ -36,8 +36,8 @@ URI::URI(): } -URI::URI(const std::string& uri, bool decode_and_encode_path): - _port(0), _disable_url_encoding(decode_and_encode_path) +URI::URI(const std::string& uri, bool enable_url_encoding): + _port(0), _enable_url_encoding(enable_url_encoding) { parse(uri); } @@ -108,7 +108,7 @@ URI::URI(const URI& uri): _path(uri._path), _query(uri._query), _fragment(uri._fragment), - _disable_url_encoding(uri._disable_url_encoding) + _enable_url_encoding(uri._enable_url_encoding) { } @@ -121,7 +121,7 @@ URI::URI(const URI& baseURI, const std::string& relativeURI): _path(baseURI._path), _query(baseURI._query), _fragment(baseURI._fragment), - _disable_url_encoding(baseURI._disable_url_encoding) + _enable_url_encoding(baseURI._enable_url_encoding) { resolve(relativeURI); } @@ -153,7 +153,7 @@ URI& URI::operator = (const URI& uri) _path = uri._path; _query = uri._query; _fragment = uri._fragment; - _disable_url_encoding = uri._disable_url_encoding; + _enable_url_encoding = uri._enable_url_encoding; } return *this; } @@ -184,7 +184,7 @@ void URI::swap(URI& uri) std::swap(_path, uri._path); std::swap(_query, uri._query); std::swap(_fragment, uri._fragment); - std::swap(_disable_url_encoding, uri._disable_url_encoding); + std::swap(_enable_url_encoding, uri._enable_url_encoding); } @@ -687,18 +687,18 @@ void URI::decode(const std::string& str, std::string& decodedStr, bool plusAsSpa void URI::encodePath(std::string & encodedStr) const { - if (_disable_url_encoding) - encodedStr = _path; - else + if (_enable_url_encoding) encode(_path, RESERVED_PATH, encodedStr); + else + encodedStr = _path; } void URI::decodePath(const std::string & encodedStr) { - if (_disable_url_encoding) - _path = encodedStr; - else + if (_enable_url_encoding) decode(encodedStr, _path); + else + _path = encodedStr; } bool URI::isWellKnownPort() const diff --git a/docs/en/engines/table-engines/special/url.md b/docs/en/engines/table-engines/special/url.md index f556df0a088..5a5e1564180 100644 --- a/docs/en/engines/table-engines/special/url.md +++ b/docs/en/engines/table-engines/special/url.md @@ -106,4 +106,4 @@ For partitioning by month, use the `toYYYYMM(date_column)` expression, where `da ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. -- [disable_url_encoding](/docs/en/operations/settings/settings.md#disable_url_encoding) -allows to disable decoding/encoding path in uri. Disabled by default. +- [enable_url_encoding](/docs/en/operations/settings/settings.md#enable_url_encoding) - allows to enable/disable decoding/encoding path in uri. Enabled by default. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8dfb6c0d225..db25ef10c91 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3468,11 +3468,11 @@ Possible values: Default value: `0`. -## disable_url_encoding {#disable_url_encoding} +## enable_url_encoding {#enable_url_encoding} -Allows to disable decoding/encoding path in uri in [URL](../../engines/table-engines/special/url.md) engine tables. +Allows to enable/disable decoding/encoding path in uri in [URL](../../engines/table-engines/special/url.md) engine tables. -Disabled by default. +Enabled by default. ## database_atomic_wait_for_drop_and_detach_synchronously {#database_atomic_wait_for_drop_and_detach_synchronously} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 677ed011960..859de86f019 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -56,7 +56,7 @@ Character `|` inside patterns is used to specify failover addresses. They are it ## Storage Settings {#storage-settings} - [engine_url_skip_empty_files](/docs/en/operations/settings/settings.md#engine_url_skip_empty_files) - allows to skip empty files while reading. Disabled by default. -- [disable_url_encoding](/docs/en/operations/settings/settings.md#disable_url_encoding) - allows to disable decoding/encoding path in uri. Disabled by default. +- [enable_url_encoding](/docs/en/operations/settings/settings.md#enable_url_encoding) - allows to enable/disable decoding/encoding path in uri. Enabled by default. **See Also** diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c69d132ea25..9823f8e9c95 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -622,7 +622,7 @@ class IColumn; M(Bool, engine_file_allow_create_multiple_files, false, "Enables or disables creating a new file on each insert in file engine tables if format has suffix.", 0) \ M(Bool, engine_file_skip_empty_files, false, "Allows to skip empty files in file table engine", 0) \ M(Bool, engine_url_skip_empty_files, false, "Allows to skip empty files in url table engine", 0) \ - M(Bool, disable_url_encoding, false, " Allows to disable decoding/encoding path in uri in URL table engine", 0) \ + M(Bool, enable_url_encoding, false, " Allows to enable/disable decoding/encoding path in uri in URL table engine", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(Bool, database_replicated_enforce_synchronous_settings, false, "Enforces synchronous waiting for some queries (see also database_atomic_wait_for_drop_and_detach_synchronously, mutation_sync, alter_sync). Not recommended to enable these settings.", 0) \ diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index db8cb6b42de..41eb18ab541 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -371,7 +371,7 @@ std::pair> StorageURLSource: for (; option != end; ++option) { bool skip_url_not_found_error = glob_url && read_settings.http_skip_not_found_url_for_globs && option == std::prev(end); - auto request_uri = Poco::URI(*option, context->getSettingsRef().disable_url_encoding); + auto request_uri = Poco::URI(*option, context->getSettingsRef().enable_url_encoding); for (const auto & [param, value] : params) request_uri.addQueryParameter(param, value); diff --git a/tests/queries/0_stateless/02833_url_without_path_encoding.reference b/tests/queries/0_stateless/02833_url_without_path_encoding.reference new file mode 100644 index 00000000000..d5626230d71 --- /dev/null +++ b/tests/queries/0_stateless/02833_url_without_path_encoding.reference @@ -0,0 +1,2 @@ +4 +test%2Fa.tsv diff --git a/tests/queries/0_stateless/02833_url_without_path_encoding.sh b/tests/queries/0_stateless/02833_url_without_path_encoding.sh new file mode 100755 index 00000000000..699b1c8e99b --- /dev/null +++ b/tests/queries/0_stateless/02833_url_without_path_encoding.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=0" + +# Grep 'test%2Fa.tsv' to ensure that path wasn't encoded/decoded +$CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=1" 2>&1 | grep -o "test%2Fa.tsv" -m1 + From f3dc6dd061515054afbbe5c58452a9554998a8b7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 27 Jul 2023 15:23:57 +0300 Subject: [PATCH 1221/2047] Revert "Added field `refcount` to `system.remote_data_paths` table" --- src/Disks/IDisk.h | 5 +- .../ObjectStorages/DiskObjectStorage.cpp | 2 +- .../System/StorageSystemRemoteDataPaths.cpp | 8 +--- .../02791_remote_paths_refcount.reference | 28 ----------- .../02791_remote_paths_refcount.sql | 47 ------------------- 5 files changed, 4 insertions(+), 86 deletions(-) delete mode 100644 tests/queries/0_stateless/02791_remote_paths_refcount.reference delete mode 100644 tests/queries/0_stateless/02791_remote_paths_refcount.sql diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index a2c5e59237f..2b0ca369a96 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -303,11 +303,10 @@ public: std::string local_path; std::string common_prefix_for_objects; StoredObjects objects; - size_t refcount; LocalPathWithObjectStoragePaths( - const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_, size_t refcount_) - : local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)), refcount(refcount_) {} + const std::string & local_path_, const std::string & common_prefix_for_objects_, StoredObjects && objects_) + : local_path(local_path_), common_prefix_for_objects(common_prefix_for_objects_), objects(std::move(objects_)) {} }; virtual void getRemotePathsRecursive(const String &, std::vector &) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 001cff4cefe..762151b3808 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -82,7 +82,7 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std:: { try { - paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path), metadata_storage->getHardlinkCount(local_path)); + paths_map.emplace_back(local_path, metadata_storage->getObjectStorageRootPath(), getStorageObjects(local_path)); } catch (const Exception & e) { diff --git a/src/Storages/System/StorageSystemRemoteDataPaths.cpp b/src/Storages/System/StorageSystemRemoteDataPaths.cpp index 820b1cf3823..eb514d3b3f4 100644 --- a/src/Storages/System/StorageSystemRemoteDataPaths.cpp +++ b/src/Storages/System/StorageSystemRemoteDataPaths.cpp @@ -25,7 +25,6 @@ StorageSystemRemoteDataPaths::StorageSystemRemoteDataPaths(const StorageID & tab {"local_path", std::make_shared()}, {"remote_path", std::make_shared()}, {"size", std::make_shared()}, - {"refcount", std::make_shared()}, {"common_prefix_for_blobs", std::make_shared()}, {"cache_paths", std::make_shared(std::make_shared())}, })); @@ -49,7 +48,6 @@ Pipe StorageSystemRemoteDataPaths::read( MutableColumnPtr col_local_path = ColumnString::create(); MutableColumnPtr col_remote_path = ColumnString::create(); MutableColumnPtr col_size = ColumnUInt64::create(); - MutableColumnPtr col_refcount = ColumnUInt64::create(); MutableColumnPtr col_namespace = ColumnString::create(); MutableColumnPtr col_cache_paths = ColumnArray::create(ColumnString::create()); @@ -67,22 +65,19 @@ Pipe StorageSystemRemoteDataPaths::read( if (disk->supportsCache()) cache = FileCacheFactory::instance().getByName(disk->getCacheName()).cache; - for (const auto & [local_path, common_prefox_for_objects, storage_objects, refcount] : remote_paths_by_local_path) + for (const auto & [local_path, common_prefox_for_objects, storage_objects] : remote_paths_by_local_path) { for (const auto & object : storage_objects) { col_disk_name->insert(disk_name); col_base_path->insert(disk->getPath()); - if (cache) col_cache_base_path->insert(cache->getBasePath()); else col_cache_base_path->insertDefault(); - col_local_path->insert(local_path); col_remote_path->insert(object.remote_path); col_size->insert(object.bytes_size); - col_refcount->insert(refcount); col_namespace->insert(common_prefox_for_objects); if (cache) @@ -106,7 +101,6 @@ Pipe StorageSystemRemoteDataPaths::read( res_columns.emplace_back(std::move(col_local_path)); res_columns.emplace_back(std::move(col_remote_path)); res_columns.emplace_back(std::move(col_size)); - res_columns.emplace_back(std::move(col_refcount)); res_columns.emplace_back(std::move(col_namespace)); res_columns.emplace_back(std::move(col_cache_paths)); diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.reference b/tests/queries/0_stateless/02791_remote_paths_refcount.reference deleted file mode 100644 index 56fb1536205..00000000000 --- a/tests/queries/0_stateless/02791_remote_paths_refcount.reference +++ /dev/null @@ -1,28 +0,0 @@ -0_0_0_0 0 -0_0_0_0_1 1 -1_0_0_0 0 -1_0_0_0_1 1 -0_0_0_0_1 checksums.txt 0 -0_0_0_0_1 columns.txt 1 -0_0_0_0_1 count.txt 1 -0_0_0_0_1 default_compression_codec.txt 1 -0_0_0_0_1 id.bin 1 -0_0_0_0_1 id.cmrk2 1 -0_0_0_0_1 metadata_version.txt 1 -0_0_0_0_1 minmax_id.idx 1 -0_0_0_0_1 partition.dat 1 -0_0_0_0_1 primary.cidx 1 -0_0_0_0_1 v.bin 1 -0_0_0_0_1 v.cmrk2 1 -1_0_0_0_1 checksums.txt 0 -1_0_0_0_1 columns.txt 0 -1_0_0_0_1 count.txt 1 -1_0_0_0_1 default_compression_codec.txt 0 -1_0_0_0_1 id.bin 1 -1_0_0_0_1 id.cmrk2 1 -1_0_0_0_1 metadata_version.txt 0 -1_0_0_0_1 minmax_id.idx 1 -1_0_0_0_1 partition.dat 1 -1_0_0_0_1 primary.cidx 1 -1_0_0_0_1 v.bin 0 -1_0_0_0_1 v.cmrk2 0 diff --git a/tests/queries/0_stateless/02791_remote_paths_refcount.sql b/tests/queries/0_stateless/02791_remote_paths_refcount.sql deleted file mode 100644 index 180601738ad..00000000000 --- a/tests/queries/0_stateless/02791_remote_paths_refcount.sql +++ /dev/null @@ -1,47 +0,0 @@ --- Tags: no-fasttest - -DROP TABLE IF EXISTS t_refcount SYNC; - --- Names of parts (on which this test depends) --- can differ in case of fault injection. -SET insert_keeper_fault_injection_probability = 0.0; - -CREATE TABLE t_refcount (id UInt64, v UInt64) -ENGINE = ReplicatedMergeTree('/clickhouse/test/{database}/t_refcount', '1') -ORDER BY id PARTITION BY id % 2 -SETTINGS - storage_policy = 's3_cache', - allow_remote_fs_zero_copy_replication = 1, - min_bytes_for_wide_part = 0, - compress_marks = 1, - compress_primary_key = 1, - ratio_of_defaults_for_sparse_serialization = 1.0; - -INSERT INTO t_refcount VALUES (1, 10), (2, 20); - -SET mutations_sync = 2; -ALTER TABLE t_refcount UPDATE v = v * 10 WHERE id % 2 = 1; - -SELECT name, active FROM system.parts WHERE database = currentDatabase() AND table = 't_refcount' ORDER BY name; - -WITH splitByChar('/', full_path) AS path_parts -SELECT path_parts[-2] AS part_name, path_parts[-1] AS file_name, refcount -FROM -( - SELECT - path || local_path AS full_path, - substring(full_path, 1, length(full_path) - position(reverse(full_path), '/') + 1) AS part_path, - refcount - FROM system.remote_data_paths - WHERE disk_name = 's3_cache' -) AS paths -INNER JOIN -( - SELECT path - FROM system.parts - WHERE database = currentDatabase() AND table = 't_refcount' AND active -) AS parts -ON paths.part_path = parts.path -ORDER BY part_name, file_name; - -DROP TABLE IF EXISTS t_refcount SYNC; From 8bba7baeaa65548d91da4c068c6af9b583f9449c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 12:34:22 +0000 Subject: [PATCH 1222/2047] fix style --- tests/queries/0_stateless/02833_local_udf_options.reference | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02833_local_udf_options.reference diff --git a/tests/queries/0_stateless/02833_local_udf_options.reference b/tests/queries/0_stateless/02833_local_udf_options.reference old mode 100755 new mode 100644 From 33300a978e00687713d08fa786178eecc7bc15d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 15:38:53 +0300 Subject: [PATCH 1223/2047] Update CHANGELOG (#52655) --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 878edfa4add..f401b346726 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,7 +20,7 @@ * The experimental feature `hashid` is removed due to a bug. The quality of implementation was questionable at the start, and it didn't get through the experimental status. This closes [#52406](https://github.com/ClickHouse/ClickHouse/issues/52406). [#52449](https://github.com/ClickHouse/ClickHouse/pull/52449) ([Alexey Milovidov](https://github.com/alexey-milovidov)). #### New Feature -* Added support for PRQL as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Added `Overlay` database engine to combine multiple databases into one. Added `Filesystem` database engine to represent a directory in the filesystem as a set of implicitly available tables with auto-detected formats and structures. A new `S3` database engine allows to read-only interact with s3 storage by representing a prefix as a set of tables. A new `HDFS` database engine allows to interact with HDFS storage in the same way. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)). * Add support for external disks in Keeper for storing snapshots and logs. [#50098](https://github.com/ClickHouse/ClickHouse/pull/50098) ([Antonio Andelic](https://github.com/antonio2368)). * Add support for multi-directory selection (`{}`) globs. [#50559](https://github.com/ClickHouse/ClickHouse/pull/50559) ([Andrey Zvonov](https://github.com/zvonand)). * Support ZooKeeper `reconfig` command for ClickHouse Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). @@ -115,6 +115,7 @@ * Now interserver port will be closed only after tables are shut down. [#52498](https://github.com/ClickHouse/ClickHouse/pull/52498) ([alesapin](https://github.com/alesapin)). #### Experimental Feature +* Added support for [PRQL](https://prql-lang.org/) as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). * Allow to add disk name for custom disks. Previously custom disks would use an internal generated disk name. Now it will be possible with `disk = disk_(...)` (e.g. disk will have name `name`) . [#51552](https://github.com/ClickHouse/ClickHouse/pull/51552) ([Kseniia Sumarokova](https://github.com/kssenii)). This syntax can be changed in this release. * (experimental MaterializedMySQL) Fixed crash when `mysqlxx::Pool::Entry` is used after it was disconnected. [#52063](https://github.com/ClickHouse/ClickHouse/pull/52063) ([Val Doroshchuk](https://github.com/valbok)). * (experimental MaterializedMySQL) `CREATE TABLE ... AS SELECT` .. is now supported in MaterializedMySQL. [#52067](https://github.com/ClickHouse/ClickHouse/pull/52067) ([Val Doroshchuk](https://github.com/valbok)). From 541a253580b968e99c8ba9419ecf82d7459a4768 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 27 Jul 2023 12:43:38 +0000 Subject: [PATCH 1224/2047] fix crash with function tuple and one sparse argument --- src/DataTypes/DataTypeTuple.h | 1 + src/DataTypes/IDataType.h | 1 + src/Functions/IFunction.cpp | 2 +- .../02833_sparse_columns_tuple_function.reference | 4 ++++ .../02833_sparse_columns_tuple_function.sql | 14 ++++++++++++++ 5 files changed, 21 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02833_sparse_columns_tuple_function.reference create mode 100644 tests/queries/0_stateless/02833_sparse_columns_tuple_function.sql diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index ea05e6ae59b..0bf3f3ac8b3 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -37,6 +37,7 @@ public: bool canBeInsideNullable() const override { return false; } bool supportsSparseSerialization() const override { return true; } + bool canBeInsideSparseColumns() const override { return false; } MutableColumnPtr createColumn() const override; MutableColumnPtr createColumn(const ISerialization & serialization) const override; diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 4adafe5d212..54cb3d0d5c2 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -110,6 +110,7 @@ public: /// TODO: support more types. virtual bool supportsSparseSerialization() const { return !haveSubtypes(); } + virtual bool canBeInsideSparseColumns() const { return supportsSparseSerialization(); } SerializationPtr getDefaultSerialization() const; SerializationPtr getSparseSerialization() const; diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 650b54d9a37..d119b15733b 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -361,7 +361,7 @@ ColumnPtr IExecutableFunction::execute(const ColumnsWithTypeAndName & arguments, /// If default of sparse column is changed after execution of function, convert to full column. /// If there are any default in non-zero position after execution of function, convert to full column. /// Currently there is no easy way to rebuild sparse column with new offsets. - if (!result_type->supportsSparseSerialization() || !res->isDefaultAt(0) || res->getNumberOfDefaultRows() != 1) + if (!result_type->canBeInsideSparseColumns() || !res->isDefaultAt(0) || res->getNumberOfDefaultRows() != 1) { const auto & offsets_data = assert_cast &>(*sparse_offsets).getData(); return res->createWithOffsets(offsets_data, (*res)[0], input_rows_count, /*shift=*/ 1); diff --git a/tests/queries/0_stateless/02833_sparse_columns_tuple_function.reference b/tests/queries/0_stateless/02833_sparse_columns_tuple_function.reference new file mode 100644 index 00000000000..85573e2ed49 --- /dev/null +++ b/tests/queries/0_stateless/02833_sparse_columns_tuple_function.reference @@ -0,0 +1,4 @@ +(0,0) +(0,0) +(0,1) +(0,NULL) diff --git a/tests/queries/0_stateless/02833_sparse_columns_tuple_function.sql b/tests/queries/0_stateless/02833_sparse_columns_tuple_function.sql new file mode 100644 index 00000000000..776dd35ddba --- /dev/null +++ b/tests/queries/0_stateless/02833_sparse_columns_tuple_function.sql @@ -0,0 +1,14 @@ +drop table if exists t_tuple_sparse; + +create table t_tuple_sparse (a UInt64, b UInt64) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.0; + +insert into t_tuple_sparse values (0, 0); + +select (a, b) from t_tuple_sparse; +select (a, 0) from t_tuple_sparse; +select (a, 1) from t_tuple_sparse; +select (a, NULL) from t_tuple_sparse; + +drop table if exists t_tuple_sparse; From bae407e9581aab765aa24f3cdcc18f823c665af5 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 27 Jul 2023 12:48:36 +0000 Subject: [PATCH 1225/2047] fixed checks --- src/DataTypes/DataTypeArray.h | 8 +++++--- src/Functions/FunctionBinaryArithmetic.h | 4 ++-- .../0_stateless/02812_pointwise_array_operations.sql | 11 ++++------- 3 files changed, 11 insertions(+), 12 deletions(-) diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index f1cbaeb0001..52d9a8b4e01 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -44,7 +44,7 @@ public: } MutableColumnPtr createColumn() const override; - + MutableColumnPtr createColumnConst(size_t size, const Field & field) const; Field getDefault() const override; @@ -72,11 +72,13 @@ public: size_t getNumberOfDimensions() const; }; -template inline constexpr bool IsDataTypeArray() { +template inline constexpr bool IsDataTypeArray() +{ return false; } -template <> inline constexpr bool IsDataTypeArray() { +template <> inline constexpr bool IsDataTypeArray() +{ return std::is_same_v; } diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 1dc8e51060c..8a5e1149e05 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1175,7 +1175,7 @@ class FunctionBinaryArithmetic : public IFunction ColumnPtr executeArray(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { if constexpr (is_multiply || is_division) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot use multiplication or division on arrays"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot use multiplication or division on arrays"); auto num_args = arguments.size(); DataTypes data_types; @@ -1214,7 +1214,7 @@ class FunctionBinaryArithmetic : public IFunction else { if (*typeid_cast(arguments[i].column.get())->getOffsets().data() != data) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Arguments must be one size"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Arguments must be one size"); } t = typeid_cast(arguments[i].type.get())->getNestedType(); diff --git a/tests/queries/0_stateless/02812_pointwise_array_operations.sql b/tests/queries/0_stateless/02812_pointwise_array_operations.sql index f12306da519..545b2cfcdd6 100644 --- a/tests/queries/0_stateless/02812_pointwise_array_operations.sql +++ b/tests/queries/0_stateless/02812_pointwise_array_operations.sql @@ -1,8 +1,5 @@ -SELECT (materialize([1,1]) + materialize([1,4])) - -SELECT ([1,2] + [1,4]) - -SELECT ([2.5, 1, 3, 10.1] + [2, 4, 9, 0]) - -SELECT ([(1,3), (2,9)] + [(10.1, 2.4), (4,12)]) +SELECT (materialize([1,1]) + materialize([1,4])); +SELECT ([1,2] + [1,4]); +SELECT ([2.5, 1, 3, 10.1] + [2, 4, 9, 0]); +SELECT ([(1,3), (2,9)] + [(10.1, 2.4), (4,12)]); From 78f3a575f9ddbfd47e46e8169b63979e3d2aa72f Mon Sep 17 00:00:00 2001 From: Jiebin Sun Date: Thu, 27 Jul 2023 21:06:34 +0800 Subject: [PATCH 1226/2047] Convert hashSets in parallel before merge (#50748) * Convert hashSets in parallel before merge Before merge, if one of the lhs and rhs is singleLevelSet and the other is twoLevelSet, then the SingleLevelSet will call convertToTwoLevel(). The convert process is not in parallel and it will cost lots of cycle if it cosume all the singleLevelSet. The idea of the patch is to convert all the singleLevelSets to twoLevelSets in parallel if the hashsets are not all singleLevel or not all twoLevel. I have tested the patch on Intel 2 x 112 vCPUs SPR server with clickbench and latest upstream ClickHouse. Q5 has got a big 264% performance improvement and 24 queries have got at least 5% performance gain. The overall geomean of 43 queries has gained 7.4% more than the base code. Signed-off-by: Jiebin Sun * add resize() for the data_vec in parallelizeMergePrepare() Signed-off-by: Jiebin Sun * Add the performance test prepare_hash_before_merge.xml Signed-off-by: Jiebin Sun * Fit the CI to rename the data set from hits_v1 to test.hits. Signed-off-by: Jiebin Sun * remove the redundant branch in UniqExactSet Co-authored-by: Nikita Taranov * Remove the empty methods and add throw exception in parallelizeMergePrepare() Signed-off-by: Jiebin Sun --------- Signed-off-by: Jiebin Sun Co-authored-by: Nikita Taranov --- .../AggregateFunctionUniq.h | 39 ++++++++++++++ src/AggregateFunctions/IAggregateFunction.h | 8 +++ src/AggregateFunctions/UniqExactSet.h | 51 +++++++++++++++++++ src/Interpreters/Aggregator.cpp | 14 +++++ .../performance/prepare_hash_before_merge.xml | 4 ++ 5 files changed, 116 insertions(+) create mode 100644 tests/performance/prepare_hash_before_merge.xml diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index de68e9076a0..2810051a82f 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -29,6 +29,10 @@ #include #include +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} namespace DB { @@ -42,6 +46,7 @@ struct AggregateFunctionUniqUniquesHashSetData Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniq"; } @@ -55,6 +60,7 @@ struct AggregateFunctionUniqUniquesHashSetDataForVariadic Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -72,6 +78,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -84,6 +91,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -96,6 +104,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -108,6 +117,7 @@ struct AggregateFunctionUniqHLL12Data Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } @@ -120,6 +130,7 @@ struct AggregateFunctionUniqHLL12DataForVariadic Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -143,6 +154,7 @@ struct AggregateFunctionUniqExactData Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = false; static String getName() { return "uniqExact"; } @@ -162,6 +174,7 @@ struct AggregateFunctionUniqExactData Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = false; static String getName() { return "uniqExact"; } @@ -181,6 +194,7 @@ struct AggregateFunctionUniqExactData Set set; constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = false; static String getName() { return "uniqExact"; } @@ -190,6 +204,7 @@ template { constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_parallelize_merge_prepare_needed = true; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -204,6 +219,7 @@ struct AggregateFunctionUniqThetaData Set set; constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = false; static String getName() { return "uniqTheta"; } @@ -213,6 +229,7 @@ template struct AggregateFunctionUniqThetaDataForVariadic : AggregateFunctionUniqThetaData { constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_parallelize_merge_prepare_needed = false; constexpr static bool is_variadic = true; constexpr static bool is_exact = is_exact_; constexpr static bool argument_is_tuple = argument_is_tuple_; @@ -384,8 +401,10 @@ template class AggregateFunctionUniq final : public IAggregateFunctionDataHelper> { private: + using DataSet = typename Data::Set; static constexpr size_t num_args = 1; static constexpr bool is_able_to_parallelize_merge = Data::is_able_to_parallelize_merge; + static constexpr bool is_parallelize_merge_prepare_needed = Data::is_parallelize_merge_prepare_needed; public: explicit AggregateFunctionUniq(const DataTypes & argument_types_) @@ -439,6 +458,26 @@ public: detail::Adder::add(this->data(place), columns, num_args, row_begin, row_end, flags, null_map); } + bool isParallelizeMergePrepareNeeded() const override { return is_parallelize_merge_prepare_needed;} + + void parallelizeMergePrepare(AggregateDataPtrs & places, ThreadPool & thread_pool) const override + { + if constexpr (is_parallelize_merge_prepare_needed) + { + std::vector data_vec; + data_vec.resize(places.size()); + + for (unsigned long i = 0; i < data_vec.size(); i++) + data_vec[i] = &this->data(places[i]).set; + + DataSet::parallelizeMergePrepare(data_vec, thread_pool); + } + else + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "parallelizeMergePrepare() is only implemented when is_parallelize_merge_prepare_needed is true for {} ", getName()); + } + } + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { this->data(place).set.merge(this->data(rhs).set); diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index df08b6f2109..b460a66ea22 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -47,6 +47,7 @@ using DataTypePtr = std::shared_ptr; using DataTypes = std::vector; using AggregateDataPtr = char *; +using AggregateDataPtrs = std::vector; using ConstAggregateDataPtr = const char *; class IAggregateFunction; @@ -148,6 +149,13 @@ public: /// Default values must be a the 0-th positions in columns. virtual void addManyDefaults(AggregateDataPtr __restrict place, const IColumn ** columns, size_t length, Arena * arena) const = 0; + virtual bool isParallelizeMergePrepareNeeded() const { return false; } + + virtual void parallelizeMergePrepare(AggregateDataPtrs & /*places*/, ThreadPool & /*thread_pool*/) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "parallelizeMergePrepare() with thread pool parameter isn't implemented for {} ", getName()); + } + /// Merges state (on which place points to) with other state of current aggregation function. virtual void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const = 0; diff --git a/src/AggregateFunctions/UniqExactSet.h b/src/AggregateFunctions/UniqExactSet.h index 90cfe700179..0d99b29686f 100644 --- a/src/AggregateFunctions/UniqExactSet.h +++ b/src/AggregateFunctions/UniqExactSet.h @@ -28,6 +28,57 @@ public: asTwoLevel().insert(std::forward(arg)); } + /// In merge, if one of the lhs and rhs is twolevelset and the other is singlelevelset, then the singlelevelset will need to convertToTwoLevel(). + /// It's not in parallel and will cost extra large time if the thread_num is large. + /// This method will convert all the SingleLevelSet to TwoLevelSet in parallel if the hashsets are not all singlelevel or not all twolevel. + static void parallelizeMergePrepare(const std::vector & data_vec, ThreadPool & thread_pool) + { + unsigned long single_level_set_num = 0; + + for (auto ele : data_vec) + { + if (ele->isSingleLevel()) + single_level_set_num ++; + } + + if (single_level_set_num > 0 && single_level_set_num < data_vec.size()) + { + try + { + auto data_vec_atomic_index = std::make_shared(0); + auto thread_func = [data_vec, data_vec_atomic_index, thread_group = CurrentThread::getGroup()]() + { + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachFromGroupIfNotDetached(); + ); + if (thread_group) + CurrentThread::attachToGroupIfDetached(thread_group); + + setThreadName("UniqExaConvert"); + + while (true) + { + const auto i = data_vec_atomic_index->fetch_add(1); + if (i >= data_vec.size()) + return; + if (data_vec[i]->isSingleLevel()) + data_vec[i]->convertToTwoLevel(); + } + }; + for (size_t i = 0; i < std::min(thread_pool.getMaxThreads(), single_level_set_num); ++i) + thread_pool.scheduleOrThrowOnError(thread_func); + + thread_pool.wait(); + } + catch (...) + { + thread_pool.wait(); + throw; + } + } + } + auto merge(const UniqExactSet & other, ThreadPool * thread_pool = nullptr) { if (isSingleLevel() && other.isTwoLevel()) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 36cd32910b5..c2914c938b5 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2603,6 +2603,20 @@ void NO_INLINE Aggregator::mergeWithoutKeyDataImpl( AggregatedDataVariantsPtr & res = non_empty_data[0]; + for (size_t i = 0; i < params.aggregates_size; ++i) + { + if (aggregate_functions[i]->isParallelizeMergePrepareNeeded()) + { + size_t size = non_empty_data.size(); + std::vector data_vec; + + for (size_t result_num = 0; result_num < size; ++result_num) + data_vec.emplace_back(non_empty_data[result_num]->without_key + offsets_of_aggregate_states[i]); + + aggregate_functions[i]->parallelizeMergePrepare(data_vec, thread_pool); + } + } + /// We merge all aggregation results to the first. for (size_t result_num = 1, size = non_empty_data.size(); result_num < size; ++result_num) { diff --git a/tests/performance/prepare_hash_before_merge.xml b/tests/performance/prepare_hash_before_merge.xml new file mode 100644 index 00000000000..e99f762927f --- /dev/null +++ b/tests/performance/prepare_hash_before_merge.xml @@ -0,0 +1,4 @@ + + SELECT COUNT(DISTINCT Title) FROM test.hits SETTINGS max_threads = 24 + SELECT COUNT(DISTINCT Referer) FROM test.hits SETTINGS max_threads = 22 + From 671128140dc6672349421c84643076410d46ce0f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 15:34:05 +0200 Subject: [PATCH 1227/2047] Update autogenerated version to 23.8.1.1 and contributors --- cmake/autogenerated_versions.txt | 10 ++++---- .../StorageSystemContributors.generated.cpp | 24 +++++++++++++++++++ 2 files changed, 29 insertions(+), 5 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 821b7b46855..9919d018046 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54476) +SET(VERSION_REVISION 54477) SET(VERSION_MAJOR 23) -SET(VERSION_MINOR 7) +SET(VERSION_MINOR 8) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH d1c7e13d08868cb04d3562dcced704dd577cb1df) -SET(VERSION_DESCRIBE v23.7.1.1-testing) -SET(VERSION_STRING 23.7.1.1) +SET(VERSION_GITHASH a70127baecc451f1f7073bad7b6198f6703441d8) +SET(VERSION_DESCRIBE v23.8.1.1-testing) +SET(VERSION_STRING 23.8.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index f84c554afc0..031c7454ab6 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -42,10 +42,12 @@ const char * auto_contributors[] { "Alex", "Alex Bocharov", "Alex Cao", + "Alex Cheng", "Alex Karo", "Alex Krash", "Alex Ryndin", "Alex Zatelepin", + "AlexBykovski", "Alexander Avdonkin", "Alexander Bezpiatov", "Alexander Burmak", @@ -232,6 +234,7 @@ const char * auto_contributors[] { "CheSema", "Chebarykov Pavel", "Chen Yufei", + "Chen768959", "Cheng Pan", "Chienlung Cheung", "Christian", @@ -485,6 +488,7 @@ const char * auto_contributors[] { "John", "John Hummel", "John Skopis", + "John Spurlock", "Jonatas Freitas", "Jonathan-Ackerman", "Jordi", @@ -659,6 +663,7 @@ const char * auto_contributors[] { "Mikhail Gaidamaka", "Mikhail Guzov", "Mikhail Korotov", + "Mikhail Koviazin", "Mikhail Malafeev", "Mikhail Nacharov", "Mikhail Salosin", @@ -815,6 +820,7 @@ const char * auto_contributors[] { "Roman Vasin", "Roman Vlasenko", "Roman Zhukov", + "Rory Crispin", "Roy Bellingan", "Ruslan", "Ruslan Savchenko", @@ -832,7 +838,9 @@ const char * auto_contributors[] { "Salvatore Mesoraca", "Sami Kerola", "Samuel Chou", + "Samuel Colvin", "San", + "Sanjam Panda", "Saulius Valatka", "Sean Haynes", "Sean Lafferty", @@ -883,6 +891,7 @@ const char * auto_contributors[] { "SmitaRKulkarni", "Snow", "Sofia Antipushina", + "Song Liyong", "Sorck", "Stanislav Dobrovolschii", "Stanislav Pavlovichev", @@ -893,6 +902,7 @@ const char * auto_contributors[] { "Stepan Herold", "Stephan", "Steve-金勇", + "StianBerger", "Stig Bakken", "Storozhuk Kostiantyn", "Stupnikov Andrey", @@ -977,6 +987,7 @@ const char * auto_contributors[] { "Vitaliy Karnienko", "Vitaliy Kozlovskiy", "Vitaliy Lyudvichenko", + "Vitaliy Pashkov", "Vitaliy Zakaznikov", "Vitaly", "Vitaly Artemyev", @@ -1029,6 +1040,7 @@ const char * auto_contributors[] { "Yakov Olkhovskiy", "YalalovSM", "Yangkuan Liu", + "Yarik Briukhovetskyi", "Yatian Xu", "Yatsishin Ilya", "Yağızcan Değirmenci", @@ -1053,6 +1065,7 @@ const char * auto_contributors[] { "Yury Karpovich", "Yury Stankevich", "Yusuke Tanaka", + "Zach Naimon", "ZhiYong Wang", "Zhichang Yu", "Zhichun Wu", @@ -1143,6 +1156,7 @@ const char * auto_contributors[] { "changvvb", "chasingegg", "chen", + "chen768959", "chen9t", "chengy8934", "chenjian", @@ -1179,6 +1193,7 @@ const char * auto_contributors[] { "detailyang", "dfenelonov", "dgrr", + "dheerajathrey", "dimarub2000", "dinosaur", "divanorama", @@ -1329,6 +1344,7 @@ const char * auto_contributors[] { "lanfz", "larryluogit", "laurieliyang", + "lcjh", "lehasm", "leosunli", "leozhang", @@ -1455,6 +1471,7 @@ const char * auto_contributors[] { "pawelsz-rb", "pdai", "pdv-ru", + "pedro.riera", "pengxiangcai", "peshkurov", "peter279k", @@ -1548,8 +1565,10 @@ const char * auto_contributors[] { "teng.ma", "terrylin", "tesw yew isal", + "therealnick233", "tianzhou", "tiger.yan", + "timfursov", "tison", "topvisor", "tpanetti", @@ -1563,6 +1582,7 @@ const char * auto_contributors[] { "usurai", "vahid-sohrabloo", "vdimir", + "velavokr", "velom", "vesslanjin", "vgocoder", @@ -1587,17 +1607,21 @@ const char * auto_contributors[] { "wuxiaobai24", "wzl", "xPoSx", + "xiao", + "xiaolei565", "xiedeyantu", "xieyichen", "xinhuitian", "xlwh", "xmy", + "xuelei", "yakkomajuri", "yakov-olkhovskiy", "yandd", "yang", "yangshuai", "yaqi-zhao", + "yariks5s", "yeer", "ygrek", "yhgcn", From 28c49e3f20923917c46872b15dce21a72ab47b4a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Jul 2023 13:47:11 +0000 Subject: [PATCH 1228/2047] Update version_date.tsv and changelogs after v23.7.1.2470-stable --- SECURITY.md | 3 +- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.7.1.2470-stable.md | 452 +++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 6 files changed, 458 insertions(+), 4 deletions(-) create mode 100644 docs/changelogs/v23.7.1.2470-stable.md diff --git a/SECURITY.md b/SECURITY.md index 4ba5f13d09c..d61533b44b9 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -13,9 +13,10 @@ The following versions of ClickHouse server are currently being supported with s | Version | Supported | |:-|:-| +| 23.7 | ✔️ | | 23.6 | ✔️ | | 23.5 | ✔️ | -| 23.4 | ✔️ | +| 23.4 | ❌ | | 23.3 | ✔️ | | 23.2 | ❌ | | 23.1 | ❌ | diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 8a6324aef88..c9800e4e66d 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.6.2.18" +ARG VERSION="23.7.1.2470" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 7f453627601..f558338b23c 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.6.2.18" +ARG VERSION="23.7.1.2470" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 1fa7b83ae16..156de034a7f 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -23,7 +23,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.6.2.18" +ARG VERSION="23.7.1.2470" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.7.1.2470-stable.md b/docs/changelogs/v23.7.1.2470-stable.md new file mode 100644 index 00000000000..a77078cb653 --- /dev/null +++ b/docs/changelogs/v23.7.1.2470-stable.md @@ -0,0 +1,452 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.7.1.2470-stable (a70127baecc) FIXME as compared to v23.6.1.1524-stable (d1c7e13d088) + +#### Backward Incompatible Change +* Add ` NAMED COLLECTION` access type (aliases `USE NAMED COLLECTION`, `NAMED COLLECTION USAGE`). This PR is backward incompatible because this access type is disabled by default (because a parent access type `NAMED COLLECTION ADMIN` is disabled by default as well). Proposed in [#50277](https://github.com/ClickHouse/ClickHouse/issues/50277). To grant use `GRANT NAMED COLLECTION ON collection_name TO user` or `GRANT NAMED COLLECTION ON * TO user`, to be able to give these grants `named_collection_admin` is required in config (previously it was named `named_collection_control`, so will remain as an alias). [#50625](https://github.com/ClickHouse/ClickHouse/pull/50625) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixing a typo in the `system.parts` column name `last_removal_attemp_time`. Now it is named `last_removal_attempt_time`. [#52104](https://github.com/ClickHouse/ClickHouse/pull/52104) ([filimonov](https://github.com/filimonov)). +* Bump version of the distributed_ddl_entry_format_version to 5 by default (enables opentelemetry and initial_query_idd pass through). This will not allow to process existing entries for distributed DDL after **downgrade** (but note, that usually there should be no such unprocessed entries). [#52128](https://github.com/ClickHouse/ClickHouse/pull/52128) ([Azat Khuzhin](https://github.com/azat)). +* Check projection metadata the same way we check ordinary metadata. This change may prevent the server from starting in case there was a table with an invalid projection. An example is a projection that created positional columns in PK (e.g. `projection p (select * order by 1, 4)` which is not allowed in table PK and can cause a crash during insert/merge). Drop such projections before the update. Fixes [#52353](https://github.com/ClickHouse/ClickHouse/issues/52353). [#52361](https://github.com/ClickHouse/ClickHouse/pull/52361) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* The experimental feature `hashid` is removed due to a bug. The quality of implementation was questionable at the start, and it didn't get through the experimental status. This closes [#52406](https://github.com/ClickHouse/ClickHouse/issues/52406). [#52449](https://github.com/ClickHouse/ClickHouse/pull/52449) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The function `toDecimalString` is removed due to subpar implementation quality. This closes [#52407](https://github.com/ClickHouse/ClickHouse/issues/52407). [#52450](https://github.com/ClickHouse/ClickHouse/pull/52450) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### New Feature +* Implement KQL-style formatting for Interval. [#45671](https://github.com/ClickHouse/ClickHouse/pull/45671) ([ltrk2](https://github.com/ltrk2)). +* Support ZooKeeper `reconfig` command for CH Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). +* Kafka connector can fetch avro schema from schema registry with basic authentication using url-encoded credentials. [#49664](https://github.com/ClickHouse/ClickHouse/pull/49664) ([Ilya Golshtein](https://github.com/ilejn)). +* Add function `arrayJaccardIndex` which computes the Jaccard similarity between two arrays. [#50076](https://github.com/ClickHouse/ClickHouse/pull/50076) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Added support for prql as a query language. [#50686](https://github.com/ClickHouse/ClickHouse/pull/50686) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Add a column is_obsolete to system.settings and similar tables. Closes [#50819](https://github.com/ClickHouse/ClickHouse/issues/50819). [#50826](https://github.com/ClickHouse/ClickHouse/pull/50826) ([flynn](https://github.com/ucasfl)). +* Implement support of encrypted elements in configuration file Added possibility to use encrypted text in leaf elements of configuration file. The text is encrypted using encryption codecs from section. [#50986](https://github.com/ClickHouse/ClickHouse/pull/50986) ([Roman Vasin](https://github.com/rvasin)). +* Just a new request of [#49483](https://github.com/ClickHouse/ClickHouse/issues/49483). [#51013](https://github.com/ClickHouse/ClickHouse/pull/51013) ([lgbo](https://github.com/lgbo-ustc)). +* Add SYSTEM STOP LISTEN query. Closes [#47972](https://github.com/ClickHouse/ClickHouse/issues/47972). [#51016](https://github.com/ClickHouse/ClickHouse/pull/51016) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add input_format_csv_allow_variable_number_of_columns options. [#51273](https://github.com/ClickHouse/ClickHouse/pull/51273) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Another boring feature: add function substring_index, as in spark or mysql. [#51472](https://github.com/ClickHouse/ClickHouse/pull/51472) ([李扬](https://github.com/taiyang-li)). +* Show stats for jemalloc bins. Example ``` SELECT *, size * (nmalloc - ndalloc) AS allocated_bytes FROM system.jemalloc_bins WHERE allocated_bytes > 0 ORDER BY allocated_bytes DESC LIMIT 10. [#51674](https://github.com/ClickHouse/ClickHouse/pull/51674) ([Alexander Gololobov](https://github.com/davenger)). +* Add RowBinaryWithDefaults format with extra byte before each column for using column default value. Closes [#50854](https://github.com/ClickHouse/ClickHouse/issues/50854). [#51695](https://github.com/ClickHouse/ClickHouse/pull/51695) ([Kruglov Pavel](https://github.com/Avogar)). +* Added `default_temporary_table_engine` setting. Same as `default_table_engine` but for temporary tables. [#51292](https://github.com/ClickHouse/ClickHouse/issues/51292). [#51708](https://github.com/ClickHouse/ClickHouse/pull/51708) ([velavokr](https://github.com/velavokr)). +* Added new initcap / initcapUTF8 functions which convert the first letter of each word to upper case and the rest to lower case. [#51735](https://github.com/ClickHouse/ClickHouse/pull/51735) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Create table now supports `PRIMARY KEY` syntax in column definition. Columns are added to primary index in the same order columns are defined. [#51881](https://github.com/ClickHouse/ClickHouse/pull/51881) ([Ilya Yatsishin](https://github.com/qoega)). +* Added the possibility to use date and time format specifiers in log and error log file names, either in config files (`log` and `errorlog` tags) or command line arguments (`--log-file` and `--errorlog-file`). [#51945](https://github.com/ClickHouse/ClickHouse/pull/51945) ([Victor Krasnov](https://github.com/sirvickr)). +* Added Peak Memory Usage (for query) to client final statistics, and to http header. [#51946](https://github.com/ClickHouse/ClickHouse/pull/51946) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added new hasSubsequence() (+CaseInsensitive + UTF8 versions) functions. [#52050](https://github.com/ClickHouse/ClickHouse/pull/52050) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Add `array_agg` as alias of `groupArray` for PostgreSQL compatibility. Closes [#52100](https://github.com/ClickHouse/ClickHouse/issues/52100). ### Documentation entry for user-facing changes. [#52135](https://github.com/ClickHouse/ClickHouse/pull/52135) ([flynn](https://github.com/ucasfl)). +* Add `any_value` as a compatibility alias for `any` aggregate function. Closes [#52140](https://github.com/ClickHouse/ClickHouse/issues/52140). [#52147](https://github.com/ClickHouse/ClickHouse/pull/52147) ([flynn](https://github.com/ucasfl)). +* Add aggregate function `array_concat_agg` for compatibility with BigQuery, it's alias of `groupArrayArray`. Closes [#52139](https://github.com/ClickHouse/ClickHouse/issues/52139). [#52149](https://github.com/ClickHouse/ClickHouse/pull/52149) ([flynn](https://github.com/ucasfl)). +* Add `OCTET_LENGTH` as an alias to `length`. Closes [#52153](https://github.com/ClickHouse/ClickHouse/issues/52153). [#52176](https://github.com/ClickHouse/ClickHouse/pull/52176) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Re-add SipHash keyed functions. [#52206](https://github.com/ClickHouse/ClickHouse/pull/52206) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Added `firstLine` function to extract the first line from the multi-line string. This closes [#51172](https://github.com/ClickHouse/ClickHouse/issues/51172). [#52209](https://github.com/ClickHouse/ClickHouse/pull/52209) ([Mikhail Koviazin](https://github.com/mkmkme)). + +#### Performance Improvement +* Enable `move_all_conditions_to_prewhere` and `enable_multiple_prewhere_read_steps` settings by default. [#46365](https://github.com/ClickHouse/ClickHouse/pull/46365) ([Alexander Gololobov](https://github.com/davenger)). +* Improves performance of some queries by tuning allocator. [#46416](https://github.com/ClickHouse/ClickHouse/pull/46416) ([Azat Khuzhin](https://github.com/azat)). +* Writing parquet files is 10x faster, it's multi-threaded now. Almost the same speed as reading. [#49367](https://github.com/ClickHouse/ClickHouse/pull/49367) ([Michael Kolupaev](https://github.com/al13n321)). +* Enable automatic selection of the sparse serialization format by default. It improves performance. The format is supported since version 22.1. After this change, downgrading to versions older than 22.1 might not be possible. You can turn off the usage of the sparse serialization format by providing the `ratio_of_defaults_for_sparse_serialization = 1` setting for your MergeTree tables. [#49631](https://github.com/ClickHouse/ClickHouse/pull/49631) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Now we use fixed-size tasks in `MergeTreePrefetchedReadPool` as in `MergeTreeReadPool`. Also from now we use connection pool for S3 requests. [#49732](https://github.com/ClickHouse/ClickHouse/pull/49732) ([Nikita Taranov](https://github.com/nickitat)). +* More pushdown to the right side of join. [#50532](https://github.com/ClickHouse/ClickHouse/pull/50532) ([Nikita Taranov](https://github.com/nickitat)). +* Improve grace_hash join by reserving hash table's size (resubmit). [#50875](https://github.com/ClickHouse/ClickHouse/pull/50875) ([lgbo](https://github.com/lgbo-ustc)). +* Waiting on lock in `OpenedFileCache` could be noticeable sometimes. We sharded it into multiple sub-maps (each with its own lock) to avoid contention. [#51341](https://github.com/ClickHouse/ClickHouse/pull/51341) ([Nikita Taranov](https://github.com/nickitat)). +* Remove duplicate condition in functionunixtimestamp64.h. [#51857](https://github.com/ClickHouse/ClickHouse/pull/51857) ([lcjh](https://github.com/ljhcage)). +* The idea is that conditions with PK columns are likely to be used in PK analysis and will not contribute much more to PREWHERE filtering. [#51958](https://github.com/ClickHouse/ClickHouse/pull/51958) ([Alexander Gololobov](https://github.com/davenger)). +* 1. Add rewriter for both old and new analyzer. 2. Add settings `optimize_uniq_to_count` which default is 0. [#52004](https://github.com/ClickHouse/ClickHouse/pull/52004) ([JackyWoo](https://github.com/JackyWoo)). +* The performance experiments of **OnTime** on the ICX device (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads) show that this change could bring an improvement of **11.6%** to the QPS of the query **Q8** while having no impact on others. [#52036](https://github.com/ClickHouse/ClickHouse/pull/52036) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Enable `allow_vertical_merges_from_compact_to_wide_parts` by default. It will save memory usage during merges. [#52295](https://github.com/ClickHouse/ClickHouse/pull/52295) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis which invalidates primary keys. This issue only exists when `query_plan_optimize_primary_key = 1, query_plan_optimize_projection = 1` . This fixes [#48823](https://github.com/ClickHouse/ClickHouse/issues/48823) . This fixes [#51173](https://github.com/ClickHouse/ClickHouse/issues/51173) . [#52308](https://github.com/ClickHouse/ClickHouse/pull/52308) ([Amos Bird](https://github.com/amosbird)). +* Reduce the number of syscalls in FileCache::loadMetadata. [#52435](https://github.com/ClickHouse/ClickHouse/pull/52435) ([Raúl Marín](https://github.com/Algunenano)). + +#### Improvement +* Added query `SYSTEM FLUSH ASYNC INSERT QUEUE` which flushes all pending asynchronous inserts to the destination tables. Added a server-side setting `async_insert_queue_flush_on_shutdown` (`true` by default) which determines whether to flush queue of asynchronous inserts on graceful shutdown. Setting `async_insert_threads` is now a server-side setting. [#49160](https://github.com/ClickHouse/ClickHouse/pull/49160) ([Anton Popov](https://github.com/CurtizJ)). +* Don't show messages about `16 EiB` free space in logs, as they don't make sense. This closes [#49320](https://github.com/ClickHouse/ClickHouse/issues/49320). [#49342](https://github.com/ClickHouse/ClickHouse/pull/49342) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Properly check the limit for the `sleepEachRow` function. Add a setting `function_sleep_max_microseconds_per_block`. This is needed for generic query fuzzer. [#49343](https://github.com/ClickHouse/ClickHouse/pull/49343) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix two issues: ``` select geohashEncode(120.2, number::Float64) from numbers(10);. [#50066](https://github.com/ClickHouse/ClickHouse/pull/50066) ([李扬](https://github.com/taiyang-li)). +* Add support for external disks in Keeper for storing snapshots and logs. [#50098](https://github.com/ClickHouse/ClickHouse/pull/50098) ([Antonio Andelic](https://github.com/antonio2368)). +* Add support for multi-directory selection (`{}`) globs. [#50559](https://github.com/ClickHouse/ClickHouse/pull/50559) ([Andrey Zvonov](https://github.com/zvonand)). +* Allow to have strict lower boundary for file segment size by downloading remaining data in the background. Minimum size of file segment (if actual file size is bigger) is configured as cache configuration setting `boundary_alignment`, by default `4Mi`. Number of background threads are configured as cache configuration setting `background_download_threads`, by default `2`. Also `max_file_segment_size` was increased from `8Mi` to `32Mi` in this PR. [#51000](https://github.com/ClickHouse/ClickHouse/pull/51000) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow filtering HTTP headers with `http_forbid_headers` section in config. Both exact matching and regexp filters are available. [#51038](https://github.com/ClickHouse/ClickHouse/pull/51038) ([Nikolay Degterinsky](https://github.com/evillique)). +* #50727 new alias for function current_database and added new function current_schemas. [#51076](https://github.com/ClickHouse/ClickHouse/pull/51076) ([Pedro Riera](https://github.com/priera)). +* Log async insert flush queries into to system.query_log. [#51160](https://github.com/ClickHouse/ClickHouse/pull/51160) ([Raúl Marín](https://github.com/Algunenano)). +* Decreased default timeouts for S3 from 30 seconds to 3 seconds, and for other HTTP from 180 seconds to 30 seconds. [#51171](https://github.com/ClickHouse/ClickHouse/pull/51171) ([Michael Kolupaev](https://github.com/al13n321)). +* Use read_bytes/total_bytes_to_read for progress bar in s3/file/url/... table functions for better progress indication. [#51286](https://github.com/ClickHouse/ClickHouse/pull/51286) ([Kruglov Pavel](https://github.com/Avogar)). +* Functions "date_diff() and age()" now support millisecond/microsecond unit and work with microsecond precision. [#51291](https://github.com/ClickHouse/ClickHouse/pull/51291) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Allow SQL standard `FETCH` without `OFFSET`. See https://antonz.org/sql-fetch/. [#51293](https://github.com/ClickHouse/ClickHouse/pull/51293) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improve parsing of path in clickhouse-keeper-client. [#51359](https://github.com/ClickHouse/ClickHouse/pull/51359) ([Azat Khuzhin](https://github.com/azat)). +* A third-party product depending on ClickHouse (Gluten: Plugin to Double SparkSQL's Performance) had a bug. This fix avoids heap overflow in that third-party product while reading from HDFS. [#51386](https://github.com/ClickHouse/ClickHouse/pull/51386) ([李扬](https://github.com/taiyang-li)). +* Fix checking error caused by uninitialized class members. [#51418](https://github.com/ClickHouse/ClickHouse/pull/51418) ([李扬](https://github.com/taiyang-li)). +* Add ability to disable native copy for S3 (setting for BACKUP/RESTORE `allow_s3_native_copy`, and `s3_allow_native_copy` for `s3`/`s3_plain` disks). [#51448](https://github.com/ClickHouse/ClickHouse/pull/51448) ([Azat Khuzhin](https://github.com/azat)). +* Add column `primary_key_size` to `system.parts` table to show compressed primary key size on disk. Closes [#51400](https://github.com/ClickHouse/ClickHouse/issues/51400). [#51496](https://github.com/ClickHouse/ClickHouse/pull/51496) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Allow running `clickhouse-local` without procfs, without home directory existing, and without name resolution plugins from glibc. [#51518](https://github.com/ClickHouse/ClickHouse/pull/51518) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Correcting the message of modify storage policy https://github.com/clickhouse/clickhouse/issues/51516 ### documentation entry for user-facing changes. [#51519](https://github.com/ClickHouse/ClickHouse/pull/51519) ([xiaolei565](https://github.com/xiaolei565)). +* Support `DROP FILESYSTEM CACHE KEY [ OFFSET ]`. [#51547](https://github.com/ClickHouse/ClickHouse/pull/51547) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow to add disk name for custom disks. Previously custom disks would use an internal generated disk name. Now it will be possible with `disk = disk_(...)` (e.g. disk will have name `name`) . [#51552](https://github.com/ClickHouse/ClickHouse/pull/51552) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add placeholder `%a` for rull filename in rename_files_after_processing setting. [#51603](https://github.com/ClickHouse/ClickHouse/pull/51603) ([Kruglov Pavel](https://github.com/Avogar)). +* Add column modification time into system.parts_columns. [#51685](https://github.com/ClickHouse/ClickHouse/pull/51685) ([Azat Khuzhin](https://github.com/azat)). +* Add new setting `input_format_csv_use_default_on_bad_values` to CSV format that allows to insert default value when parsing of a single field failed. [#51716](https://github.com/ClickHouse/ClickHouse/pull/51716) ([KevinyhZou](https://github.com/KevinyhZou)). +* Added a crash log flush to the disk after the unexpected crash. [#51720](https://github.com/ClickHouse/ClickHouse/pull/51720) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Fix behavior in dashboard page where errors unrelated to authentication are not shown. Also fix 'overlapping' chart behavior. [#51744](https://github.com/ClickHouse/ClickHouse/pull/51744) ([Zach Naimon](https://github.com/ArctypeZach)). +* Allow UUID to UInt128 conversion. [#51765](https://github.com/ClickHouse/ClickHouse/pull/51765) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Added support for function range of Nullable arguments. [#51767](https://github.com/ClickHouse/ClickHouse/pull/51767) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Convert condition like `toyear(x) = c` to `c1 <= x < c2`. [#51795](https://github.com/ClickHouse/ClickHouse/pull/51795) ([Han Fei](https://github.com/hanfei1991)). +* Improve MySQL compatibility of statement SHOW INDEX. [#51796](https://github.com/ClickHouse/ClickHouse/pull/51796) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix `use_structure_from_insertion_table_in_table_functions` does not work with `MATERIALIZED` and `ALIAS` columns. Closes [#51817](https://github.com/ClickHouse/ClickHouse/issues/51817). Closes [#51019](https://github.com/ClickHouse/ClickHouse/issues/51019). [#51825](https://github.com/ClickHouse/ClickHouse/pull/51825) ([flynn](https://github.com/ucasfl)). +* Introduce a table setting `wait_for_unique_parts_send_before_shutdown_ms` which specify the amount of time replica will wait before closing interserver handler for replicated sends. Also fix inconsistency with shutdown of tables and interserver handlers: now server shutdown tables first and only after it shut down interserver handlers. [#51851](https://github.com/ClickHouse/ClickHouse/pull/51851) ([alesapin](https://github.com/alesapin)). +* CacheDictionary request only unique keys from source. Closes [#51762](https://github.com/ClickHouse/ClickHouse/issues/51762). [#51853](https://github.com/ClickHouse/ClickHouse/pull/51853) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed settings not applied for explain query when format provided. [#51859](https://github.com/ClickHouse/ClickHouse/pull/51859) ([Nikita Taranov](https://github.com/nickitat)). +* Allow SETTINGS before FORMAT in DESCRIBE TABLE query for compatibility with SELECT query. Closes [#51544](https://github.com/ClickHouse/ClickHouse/issues/51544). [#51899](https://github.com/ClickHouse/ClickHouse/pull/51899) ([Nikolay Degterinsky](https://github.com/evillique)). +* Var-int encoded integers (e.g. used by the native protocol) can now use the full 64-bit range. 3rd party clients are advised to update their var-int code accordingly. [#51905](https://github.com/ClickHouse/ClickHouse/pull/51905) ([Robert Schulze](https://github.com/rschu1ze)). +* Update certificates when they change without the need to manually SYSTEM RELOAD CONFIG. [#52030](https://github.com/ClickHouse/ClickHouse/pull/52030) ([Mike Kot](https://github.com/myrrc)). +* Added `allow_create_index_without_type` setting that allow to ignore `ADD INDEX` queries without specified `TYPE`. Standard SQL queries will just succeed without changing table schema. [#52056](https://github.com/ClickHouse/ClickHouse/pull/52056) ([Ilya Yatsishin](https://github.com/qoega)). +* Fixed crash when mysqlxx::Pool::Entry is used after it was disconnected. [#52063](https://github.com/ClickHouse/ClickHouse/pull/52063) ([Val Doroshchuk](https://github.com/valbok)). +* CREATE TABLE ... AS SELECT .. is now supported in MaterializedMySQL. [#52067](https://github.com/ClickHouse/ClickHouse/pull/52067) ([Val Doroshchuk](https://github.com/valbok)). +* Introduced automatic conversion of text types to utf8 for MaterializedMySQL. [#52084](https://github.com/ClickHouse/ClickHouse/pull/52084) ([Val Doroshchuk](https://github.com/valbok)). +* Add alias for functions `today` (now available under the `curdate`/`current_date` names) and `now` (`current_timestamp`). [#52106](https://github.com/ClickHouse/ClickHouse/pull/52106) ([Lloyd-Pottiger](https://github.com/Lloyd-Pottiger)). +* Log messages are written to text_log from the beginning. [#52113](https://github.com/ClickHouse/ClickHouse/pull/52113) ([Dmitry Kardymon](https://github.com/kardymonds)). +* In cases where the HTTP endpoint has multiple IP addresses and the first of them is unreachable, a timeout exception will be thrown. Made session creation with handling all resolved endpoints. [#52116](https://github.com/ClickHouse/ClickHouse/pull/52116) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Support async_deduplication_token for async insert. [#52136](https://github.com/ClickHouse/ClickHouse/pull/52136) ([Han Fei](https://github.com/hanfei1991)). +* Avro input format support Union with single type. Closes [#52131](https://github.com/ClickHouse/ClickHouse/issues/52131). [#52137](https://github.com/ClickHouse/ClickHouse/pull/52137) ([flynn](https://github.com/ucasfl)). +* Add setting `optimize_use_implicit_projections` to disable implicit projections (currently only `min_max_count` projection). This is defaulted to false until [#52075](https://github.com/ClickHouse/ClickHouse/issues/52075) is fixed. [#52152](https://github.com/ClickHouse/ClickHouse/pull/52152) ([Amos Bird](https://github.com/amosbird)). +* It was possible to use the function `hasToken` for infinite loop. Now this possibility is removed. This closes [#52156](https://github.com/ClickHouse/ClickHouse/issues/52156). [#52160](https://github.com/ClickHouse/ClickHouse/pull/52160) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* 1. Upgrade Intel QPL from v1.1.0 to v1.2.0 2. Upgrade Intel accel-config from v3.5 to v4.0 3. Fixed issue that Device IOTLB miss has big perf. impact for IAA accelerators. [#52180](https://github.com/ClickHouse/ClickHouse/pull/52180) ([jasperzhu](https://github.com/jinjunzh)). +* Functions "date_diff() and age()" now support millisecond/microsecond unit and work with microsecond precision. [#52181](https://github.com/ClickHouse/ClickHouse/pull/52181) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Create ZK ancestors optimistically. [#52195](https://github.com/ClickHouse/ClickHouse/pull/52195) ([Raúl Marín](https://github.com/Algunenano)). +* Fix [#50582](https://github.com/ClickHouse/ClickHouse/issues/50582). Avoid the `Not found column ... in block` error in some cases of reading in-order and constants. [#52259](https://github.com/ClickHouse/ClickHouse/pull/52259) ([Chen768959](https://github.com/Chen768959)). +* Check whether S2 geo primitives are invalid as early as possible on ClickHouse side. This closes: [#27090](https://github.com/ClickHouse/ClickHouse/issues/27090). [#52260](https://github.com/ClickHouse/ClickHouse/pull/52260) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Now unquoted utf-8 strings are supported in DDL for MaterializedMySQL. [#52318](https://github.com/ClickHouse/ClickHouse/pull/52318) ([Val Doroshchuk](https://github.com/valbok)). +* Add back missing projection QueryAccessInfo when `query_plan_optimize_projection = 1`. This fixes [#50183](https://github.com/ClickHouse/ClickHouse/issues/50183) . This fixes [#50093](https://github.com/ClickHouse/ClickHouse/issues/50093) . [#52327](https://github.com/ClickHouse/ClickHouse/pull/52327) ([Amos Bird](https://github.com/amosbird)). +* Add new setting `disable_url_encoding` that allows to disable decoding/encoding path in uri in URL engine. [#52337](https://github.com/ClickHouse/ClickHouse/pull/52337) ([Kruglov Pavel](https://github.com/Avogar)). +* When `ZooKeeperRetriesControl` rethrows an error, it's more useful to see its original stack trace, not the one from `ZooKeeperRetriesControl` itself. [#52347](https://github.com/ClickHouse/ClickHouse/pull/52347) ([Vitaly Baranov](https://github.com/vitlibar)). +* Now double quoted comments are supported in MaterializedMySQL. [#52355](https://github.com/ClickHouse/ClickHouse/pull/52355) ([Val Doroshchuk](https://github.com/valbok)). +* Wait for zero copy replication lock even if some disks don't support it. [#52376](https://github.com/ClickHouse/ClickHouse/pull/52376) ([Raúl Marín](https://github.com/Algunenano)). +* Now it's possible to specify min (`memory_profiler_sample_min_allocation_size`) and max (`memory_profiler_sample_max_allocation_size`) size for allocations to be tracked with sampling memory profiler. [#52419](https://github.com/ClickHouse/ClickHouse/pull/52419) ([alesapin](https://github.com/alesapin)). +* The `session_timezone` setting is demoted to experimental. [#52445](https://github.com/ClickHouse/ClickHouse/pull/52445) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Now interserver port will be closed only after tables are shut down. [#52498](https://github.com/ClickHouse/ClickHouse/pull/52498) ([alesapin](https://github.com/alesapin)). +* Added field `refcount` to `system.remote_data_paths` table. [#52518](https://github.com/ClickHouse/ClickHouse/pull/52518) ([Anton Popov](https://github.com/CurtizJ)). +* New setting `merge_tree_determine_task_size_by_prewhere_columns` added. If set to `true` only sizes of the columns from `PREWHERE` section will be considered to determine reading task size. Otherwise all the columns from query are considered. [#52606](https://github.com/ClickHouse/ClickHouse/pull/52606) ([Nikita Taranov](https://github.com/nickitat)). + +#### Build/Testing/Packaging Improvement +* Add experimental ClickHouse builds for Linux RISC-V 64 to CI. [#31398](https://github.com/ClickHouse/ClickHouse/pull/31398) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixed CRC32(WeakHash32) issue for s390x. [#50365](https://github.com/ClickHouse/ClickHouse/pull/50365) ([Harry Lee](https://github.com/HarryLeeIBM)). +* Add integration test check with the enabled analyzer. [#50926](https://github.com/ClickHouse/ClickHouse/pull/50926) ([Dmitry Novik](https://github.com/novikd)). +* Update cargo dependencies. [#51721](https://github.com/ClickHouse/ClickHouse/pull/51721) ([Raúl Marín](https://github.com/Algunenano)). +* Fixed several issues found by OSS-Fuzz. [#51736](https://github.com/ClickHouse/ClickHouse/pull/51736) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* There were a couple of failures because of (?) S3 availability. The sccache has a feature of failing over to local compilation. [#51893](https://github.com/ClickHouse/ClickHouse/pull/51893) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* 02242_delete_user_race and 02243_drop_user_grant_race tests have been corrected. [#51923](https://github.com/ClickHouse/ClickHouse/pull/51923) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Make the function `CHColumnToArrowColumn::fillArrowArrayWithArrayColumnData` to work with nullable arrays, which are not possible in ClickHouse, but needed for Gluten. [#52112](https://github.com/ClickHouse/ClickHouse/pull/52112) ([李扬](https://github.com/taiyang-li)). +* We've updated the CCTZ library to master, but there are no user-visible changes. [#52124](https://github.com/ClickHouse/ClickHouse/pull/52124) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The `system.licenses` table now includes the hard-forked library Poco. This closes [#52066](https://github.com/ClickHouse/ClickHouse/issues/52066). [#52127](https://github.com/ClickHouse/ClickHouse/pull/52127) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Follow up [#50926](https://github.com/ClickHouse/ClickHouse/issues/50926). Add integration tests check with enabled analyzer to master. [#52210](https://github.com/ClickHouse/ClickHouse/pull/52210) ([Dmitry Novik](https://github.com/novikd)). +* Reproducible builds for Rust. [#52395](https://github.com/ClickHouse/ClickHouse/pull/52395) ([Azat Khuzhin](https://github.com/azat)). +* Improve the startup time of `clickhouse-client` and `clickhouse-local` in debug and sanitizer builds. This closes [#52228](https://github.com/ClickHouse/ClickHouse/issues/52228). [#52489](https://github.com/ClickHouse/ClickHouse/pull/52489) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check that there are no cases of bad punctuation: whitespace before a comma like `Hello ,world` instead of `Hello, world`. [#52549](https://github.com/ClickHouse/ClickHouse/pull/52549) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix materialised pg syncTables [#49698](https://github.com/ClickHouse/ClickHouse/pull/49698) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix projection with optimize_aggregators_of_group_by_keys [#49709](https://github.com/ClickHouse/ClickHouse/pull/49709) ([Amos Bird](https://github.com/amosbird)). +* Fix optimize_skip_unused_shards with JOINs [#51037](https://github.com/ClickHouse/ClickHouse/pull/51037) ([Azat Khuzhin](https://github.com/azat)). +* Fix formatDateTime() with fractional negative datetime64 [#51290](https://github.com/ClickHouse/ClickHouse/pull/51290) ([Dmitry Kardymon](https://github.com/kardymonds)). +* Functions `hasToken*` were totally wrong. Add a test for [#43358](https://github.com/ClickHouse/ClickHouse/issues/43358) [#51378](https://github.com/ClickHouse/ClickHouse/pull/51378) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix optimization to move functions before sorting. [#51481](https://github.com/ClickHouse/ClickHouse/pull/51481) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix Block structure mismatch in Pipe::unitePipes for FINAL [#51492](https://github.com/ClickHouse/ClickHouse/pull/51492) ([Nikita Taranov](https://github.com/nickitat)). +* Fix SIGSEGV for clusters with zero weight across all shards (fixes INSERT INTO FUNCTION clusterAllReplicas()) [#51545](https://github.com/ClickHouse/ClickHouse/pull/51545) ([Azat Khuzhin](https://github.com/azat)). +* Fix timeout for hedged requests [#51582](https://github.com/ClickHouse/ClickHouse/pull/51582) ([Azat Khuzhin](https://github.com/azat)). +* Fix logical error in ANTI join with NULL [#51601](https://github.com/ClickHouse/ClickHouse/pull/51601) ([vdimir](https://github.com/vdimir)). +* Fix for moving 'IN' conditions to PREWHERE [#51610](https://github.com/ClickHouse/ClickHouse/pull/51610) ([Alexander Gololobov](https://github.com/davenger)). +* Do not apply PredicateExpressionsOptimizer for ASOF/ANTI join [#51633](https://github.com/ClickHouse/ClickHouse/pull/51633) ([vdimir](https://github.com/vdimir)). +* Fix async insert with deduplication for ReplicatedMergeTree using merging algorithms [#51676](https://github.com/ClickHouse/ClickHouse/pull/51676) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix reading from empty column in `parseSipHashKey` [#51804](https://github.com/ClickHouse/ClickHouse/pull/51804) ([Nikita Taranov](https://github.com/nickitat)). +* Fix segfault when create invalid EmbeddedRocksdb table [#51847](https://github.com/ClickHouse/ClickHouse/pull/51847) ([Duc Canh Le](https://github.com/canhld94)). +* Fix inserts into MongoDB tables [#51876](https://github.com/ClickHouse/ClickHouse/pull/51876) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix deadlock on DatabaseCatalog shutdown [#51908](https://github.com/ClickHouse/ClickHouse/pull/51908) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix error in subquery operators [#51922](https://github.com/ClickHouse/ClickHouse/pull/51922) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix async connect to hosts with multiple ips [#51934](https://github.com/ClickHouse/ClickHouse/pull/51934) ([Kruglov Pavel](https://github.com/Avogar)). +* Do not remove inputs after ActionsDAG::merge [#51947](https://github.com/ClickHouse/ClickHouse/pull/51947) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Check refcount in `RemoveManyObjectStorageOperation::finalize` instead of `execute` [#51954](https://github.com/ClickHouse/ClickHouse/pull/51954) ([vdimir](https://github.com/vdimir)). +* Allow parametric UDFs [#51964](https://github.com/ClickHouse/ClickHouse/pull/51964) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Small fix for toDateTime64() for dates after 2283-12-31 [#52130](https://github.com/ClickHouse/ClickHouse/pull/52130) ([Andrey Zvonov](https://github.com/zvonand)). +* Fix ORDER BY tuple of WINDOW functions [#52145](https://github.com/ClickHouse/ClickHouse/pull/52145) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect projection analysis when aggregation expression contains monotonic functions [#52151](https://github.com/ClickHouse/ClickHouse/pull/52151) ([Amos Bird](https://github.com/amosbird)). +* Fix error in `groupArrayMoving` functions [#52161](https://github.com/ClickHouse/ClickHouse/pull/52161) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable direct join for range dictionary [#52187](https://github.com/ClickHouse/ClickHouse/pull/52187) ([Duc Canh Le](https://github.com/canhld94)). +* Fix sticky mutations test (and extremely rare race condition) [#52197](https://github.com/ClickHouse/ClickHouse/pull/52197) ([alesapin](https://github.com/alesapin)). +* Fix race in Web disk [#52211](https://github.com/ClickHouse/ClickHouse/pull/52211) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix data race in Connection::setAsyncCallback on unknown packet from server [#52219](https://github.com/ClickHouse/ClickHouse/pull/52219) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix temp data deletion on startup, add test [#52275](https://github.com/ClickHouse/ClickHouse/pull/52275) ([vdimir](https://github.com/vdimir)). +* Don't use minmax_count projections when counting nullable columns [#52297](https://github.com/ClickHouse/ClickHouse/pull/52297) ([Amos Bird](https://github.com/amosbird)). +* MergeTree/ReplicatedMergeTree should use server timezone for log entries [#52325](https://github.com/ClickHouse/ClickHouse/pull/52325) ([Azat Khuzhin](https://github.com/azat)). +* Fix parameterized view with cte and multiple usage [#52328](https://github.com/ClickHouse/ClickHouse/pull/52328) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Disable expression templates for time intervals [#52335](https://github.com/ClickHouse/ClickHouse/pull/52335) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `apply_snapshot` in Keeper [#52358](https://github.com/ClickHouse/ClickHouse/pull/52358) ([Antonio Andelic](https://github.com/antonio2368)). +* Update build-osx.md [#52377](https://github.com/ClickHouse/ClickHouse/pull/52377) ([AlexBykovski](https://github.com/AlexBykovski)). +* Fix `countSubstrings()` hang with empty needle and a column haystack [#52409](https://github.com/ClickHouse/ClickHouse/pull/52409) ([Sergei Trifonov](https://github.com/serxa)). +* Fix normal projection with merge table [#52432](https://github.com/ClickHouse/ClickHouse/pull/52432) ([Amos Bird](https://github.com/amosbird)). +* Fix possible double-free in Aggregator [#52439](https://github.com/ClickHouse/ClickHouse/pull/52439) ([Nikita Taranov](https://github.com/nickitat)). +* Fixed inserting into Buffer engine [#52440](https://github.com/ClickHouse/ClickHouse/pull/52440) ([Vasily Nemkov](https://github.com/Enmk)). +* The implementation of AnyHash was non-conformant. [#52448](https://github.com/ClickHouse/ClickHouse/pull/52448) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check recursion depth in OptimizedRegularExpression [#52451](https://github.com/ClickHouse/ClickHouse/pull/52451) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix data-race DatabaseReplicated::startupTables()/canExecuteReplicatedMetadataAlter() [#52490](https://github.com/ClickHouse/ClickHouse/pull/52490) ([Azat Khuzhin](https://github.com/azat)). +* Fix abort in function `transform` [#52513](https://github.com/ClickHouse/ClickHouse/pull/52513) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix lightweight delete after drop of projection [#52517](https://github.com/ClickHouse/ClickHouse/pull/52517) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible error "Cannot drain connections: cancel first" [#52585](https://github.com/ClickHouse/ClickHouse/pull/52585) ([Kruglov Pavel](https://github.com/Avogar)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Add documentation for building in docker"'. [#51773](https://github.com/ClickHouse/ClickHouse/pull/51773) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Fix build"'. [#51911](https://github.com/ClickHouse/ClickHouse/pull/51911) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Millisecond and microsecond support in date_diff / age functions"'. [#52129](https://github.com/ClickHouse/ClickHouse/pull/52129) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Re-add SipHash keyed functions"'. [#52466](https://github.com/ClickHouse/ClickHouse/pull/52466) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Add an ability to specify allocations size for sampling memory profiler"'. [#52496](https://github.com/ClickHouse/ClickHouse/pull/52496) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Rewrite uniq to count"'. [#52576](https://github.com/ClickHouse/ClickHouse/pull/52576) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Remove duplicate_order_by_and_distinct optimization [#47135](https://github.com/ClickHouse/ClickHouse/pull/47135) ([Igor Nikonov](https://github.com/devcrafter)). +* Update sort desc in ReadFromMergeTree after applying PREWHERE info [#48669](https://github.com/ClickHouse/ClickHouse/pull/48669) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix `BindException: Address already in use` in HDFS integration tests [#49428](https://github.com/ClickHouse/ClickHouse/pull/49428) ([Nikita Taranov](https://github.com/nickitat)). +* Force libunwind usage (removes gcc_eh support) [#49438](https://github.com/ClickHouse/ClickHouse/pull/49438) ([Azat Khuzhin](https://github.com/azat)). +* Cleanup `storage_conf.xml` [#49557](https://github.com/ClickHouse/ClickHouse/pull/49557) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky tests caused by OPTIMIZE FINAL failing memory budget check [#49764](https://github.com/ClickHouse/ClickHouse/pull/49764) ([Michael Kolupaev](https://github.com/al13n321)). +* Remove unstable queries from performance/join_set_filter [#50235](https://github.com/ClickHouse/ClickHouse/pull/50235) ([vdimir](https://github.com/vdimir)). +* More accurate DNS resolve for the keeper connection [#50738](https://github.com/ClickHouse/ClickHouse/pull/50738) ([pufit](https://github.com/pufit)). +* Try to fix some trash in Disks and part moves [#51135](https://github.com/ClickHouse/ClickHouse/pull/51135) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add jemalloc support fro s390x [#51186](https://github.com/ClickHouse/ClickHouse/pull/51186) ([Boris Kuschel](https://github.com/bkuschel)). +* Resubmit [#48821](https://github.com/ClickHouse/ClickHouse/issues/48821) [#51208](https://github.com/ClickHouse/ClickHouse/pull/51208) ([Kseniia Sumarokova](https://github.com/kssenii)). +* test for [#36894](https://github.com/ClickHouse/ClickHouse/issues/36894) [#51274](https://github.com/ClickHouse/ClickHouse/pull/51274) ([Denny Crane](https://github.com/den-crane)). +* external_aggregation_fix for big endian machines [#51280](https://github.com/ClickHouse/ClickHouse/pull/51280) ([Sanjam Panda](https://github.com/saitama951)). +* Fix: Invalid number of rows in Chunk column Object [#51296](https://github.com/ClickHouse/ClickHouse/pull/51296) ([Igor Nikonov](https://github.com/devcrafter)). +* Add a test for [#44816](https://github.com/ClickHouse/ClickHouse/issues/44816) [#51305](https://github.com/ClickHouse/ClickHouse/pull/51305) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for `calculate_text_stack_trace` setting [#51311](https://github.com/ClickHouse/ClickHouse/pull/51311) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* decrease log level, make logs shorter [#51320](https://github.com/ClickHouse/ClickHouse/pull/51320) ([Sema Checherinda](https://github.com/CheSema)). +* Collect stack traces from job's scheduling and print along with exception's stack trace. [#51349](https://github.com/ClickHouse/ClickHouse/pull/51349) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Add a test for [#42691](https://github.com/ClickHouse/ClickHouse/issues/42691) [#51352](https://github.com/ClickHouse/ClickHouse/pull/51352) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#32474](https://github.com/ClickHouse/ClickHouse/issues/32474) [#51354](https://github.com/ClickHouse/ClickHouse/pull/51354) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#41727](https://github.com/ClickHouse/ClickHouse/issues/41727) [#51355](https://github.com/ClickHouse/ClickHouse/pull/51355) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#35801](https://github.com/ClickHouse/ClickHouse/issues/35801) [#51356](https://github.com/ClickHouse/ClickHouse/pull/51356) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a test for [#34626](https://github.com/ClickHouse/ClickHouse/issues/34626) [#51357](https://github.com/ClickHouse/ClickHouse/pull/51357) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Initialize text_log earlier to capture table startup messages [#51360](https://github.com/ClickHouse/ClickHouse/pull/51360) ([Azat Khuzhin](https://github.com/azat)). +* Use separate default settings for clickhouse-local [#51363](https://github.com/ClickHouse/ClickHouse/pull/51363) ([Azat Khuzhin](https://github.com/azat)). +* Attempt to remove wrong code (catch/throw in Functions) [#51367](https://github.com/ClickHouse/ClickHouse/pull/51367) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove suspicious code [#51383](https://github.com/ClickHouse/ClickHouse/pull/51383) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable hedged requests under TSan [#51392](https://github.com/ClickHouse/ClickHouse/pull/51392) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* no finalize in d-tor WriteBufferFromOStream [#51404](https://github.com/ClickHouse/ClickHouse/pull/51404) ([Sema Checherinda](https://github.com/CheSema)). +* Better diagnostics for 01193_metadata_loading [#51414](https://github.com/ClickHouse/ClickHouse/pull/51414) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix attaching gdb in stress tests [#51445](https://github.com/ClickHouse/ClickHouse/pull/51445) ([Kruglov Pavel](https://github.com/Avogar)). +* Merging [#36384](https://github.com/ClickHouse/ClickHouse/issues/36384) [#51458](https://github.com/ClickHouse/ClickHouse/pull/51458) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix possible race on shutdown wait [#51497](https://github.com/ClickHouse/ClickHouse/pull/51497) ([Sergei Trifonov](https://github.com/serxa)). +* Fix `test_alter_moving_garbage`: lock between getActiveContainingPart and swapActivePart in parts mover [#51498](https://github.com/ClickHouse/ClickHouse/pull/51498) ([vdimir](https://github.com/vdimir)). +* Fix a logical error on mutation [#51502](https://github.com/ClickHouse/ClickHouse/pull/51502) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix running integration tests with spaces in it's names [#51514](https://github.com/ClickHouse/ClickHouse/pull/51514) ([Azat Khuzhin](https://github.com/azat)). +* Fix flaky test 00417_kill_query [#51522](https://github.com/ClickHouse/ClickHouse/pull/51522) ([Nikolay Degterinsky](https://github.com/evillique)). +* fs cache: add some checks [#51536](https://github.com/ClickHouse/ClickHouse/pull/51536) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Don't run 02782_uniq_exact_parallel_merging_bug in parallel with other tests [#51549](https://github.com/ClickHouse/ClickHouse/pull/51549) ([Nikita Taranov](https://github.com/nickitat)). +* 00900_orc_load: lift kill timeout [#51559](https://github.com/ClickHouse/ClickHouse/pull/51559) ([Robert Schulze](https://github.com/rschu1ze)). +* Add retries to 00416_pocopatch_progress_in_http_headers [#51575](https://github.com/ClickHouse/ClickHouse/pull/51575) ([Nikolay Degterinsky](https://github.com/evillique)). +* Remove the usage of Analyzer setting in the client [#51578](https://github.com/ClickHouse/ClickHouse/pull/51578) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix merge_selecting_task scheduling [#51591](https://github.com/ClickHouse/ClickHouse/pull/51591) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add hex functions for cityhash [#51595](https://github.com/ClickHouse/ClickHouse/pull/51595) ([Vitaly Baranov](https://github.com/vitlibar)). +* Remove `unset CLICKHOUSE_LOG_COMMENT` from tests [#51623](https://github.com/ClickHouse/ClickHouse/pull/51623) ([Nikita Taranov](https://github.com/nickitat)). +* Implement endianness-independent serialization [#51637](https://github.com/ClickHouse/ClickHouse/pull/51637) ([ltrk2](https://github.com/ltrk2)). +* Ignore APPEND and TRUNCATE modifiers if file does not exist. [#51640](https://github.com/ClickHouse/ClickHouse/pull/51640) ([alekar](https://github.com/alekar)). +* Try to fix flaky 02210_processors_profile_log [#51641](https://github.com/ClickHouse/ClickHouse/pull/51641) ([Igor Nikonov](https://github.com/devcrafter)). +* Make common macros extendable [#51646](https://github.com/ClickHouse/ClickHouse/pull/51646) ([Amos Bird](https://github.com/amosbird)). +* Correct an exception message in src/Functions/nested.cpp [#51651](https://github.com/ClickHouse/ClickHouse/pull/51651) ([Alex Cheng](https://github.com/Alex-Cheng)). +* tests: fix 02050_client_profile_events flakiness [#51653](https://github.com/ClickHouse/ClickHouse/pull/51653) ([Azat Khuzhin](https://github.com/azat)). +* Minor follow-up to re2 update to 2023-06-02 ([#50949](https://github.com/ClickHouse/ClickHouse/issues/50949)) [#51655](https://github.com/ClickHouse/ClickHouse/pull/51655) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix 02116_tuple_element with Analyzer [#51669](https://github.com/ClickHouse/ClickHouse/pull/51669) ([Robert Schulze](https://github.com/rschu1ze)). +* Update timeouts in tests for transactions [#51683](https://github.com/ClickHouse/ClickHouse/pull/51683) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove unused code [#51684](https://github.com/ClickHouse/ClickHouse/pull/51684) ([Sergei Trifonov](https://github.com/serxa)). +* Remove `mmap/mremap/munmap` from Allocator.h [#51686](https://github.com/ClickHouse/ClickHouse/pull/51686) ([alesapin](https://github.com/alesapin)). +* SonarCloud: Add C++23 Experimental Flag [#51687](https://github.com/ClickHouse/ClickHouse/pull/51687) ([Julio Jimenez](https://github.com/juliojimenez)). +* Wait with retries when attaching GDB in tests [#51688](https://github.com/ClickHouse/ClickHouse/pull/51688) ([Antonio Andelic](https://github.com/antonio2368)). +* Update version_date.tsv and changelogs after v23.6.1.1524-stable [#51691](https://github.com/ClickHouse/ClickHouse/pull/51691) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* fix write to finalized buffer [#51696](https://github.com/ClickHouse/ClickHouse/pull/51696) ([Sema Checherinda](https://github.com/CheSema)). +* do not log exception aborted for pending mutate/merge entries when shutdown [#51697](https://github.com/ClickHouse/ClickHouse/pull/51697) ([Sema Checherinda](https://github.com/CheSema)). +* Fix race in ContextAccess [#51704](https://github.com/ClickHouse/ClickHouse/pull/51704) ([Vitaly Baranov](https://github.com/vitlibar)). +* Make test scripts backwards compatible [#51707](https://github.com/ClickHouse/ClickHouse/pull/51707) ([Antonio Andelic](https://github.com/antonio2368)). +* test for full join and null predicate [#51709](https://github.com/ClickHouse/ClickHouse/pull/51709) ([Denny Crane](https://github.com/den-crane)). +* A cmake warning on job limits underutilizing CPU [#51710](https://github.com/ClickHouse/ClickHouse/pull/51710) ([velavokr](https://github.com/velavokr)). +* Fix SQLLogic docker images [#51719](https://github.com/ClickHouse/ClickHouse/pull/51719) ([Antonio Andelic](https://github.com/antonio2368)). +* Added ASK_PASSWORD client constant instead of hardcoded '\n' [#51723](https://github.com/ClickHouse/ClickHouse/pull/51723) ([Alexey Gerasimchuck](https://github.com/Demilivor)). +* Update README.md [#51726](https://github.com/ClickHouse/ClickHouse/pull/51726) ([Tyler Hannan](https://github.com/tylerhannan)). +* Fix source image for sqllogic [#51728](https://github.com/ClickHouse/ClickHouse/pull/51728) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Remove MemoryPool from Poco because it's useless [#51732](https://github.com/ClickHouse/ClickHouse/pull/51732) ([alesapin](https://github.com/alesapin)). +* Fix: logical error in grace hash join [#51737](https://github.com/ClickHouse/ClickHouse/pull/51737) ([Igor Nikonov](https://github.com/devcrafter)). +* Update 01320_create_sync_race_condition_zookeeper.sh [#51742](https://github.com/ClickHouse/ClickHouse/pull/51742) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Pin for docker-ce [#51743](https://github.com/ClickHouse/ClickHouse/pull/51743) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Revert "Fix: Invalid number of rows in Chunk column Object" [#51750](https://github.com/ClickHouse/ClickHouse/pull/51750) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add SonarCloud to README [#51751](https://github.com/ClickHouse/ClickHouse/pull/51751) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix test `02789_object_type_invalid_num_of_rows` [#51754](https://github.com/ClickHouse/ClickHouse/pull/51754) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix (benign) data race in `transform` [#51755](https://github.com/ClickHouse/ClickHouse/pull/51755) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky KeeperMap test [#51764](https://github.com/ClickHouse/ClickHouse/pull/51764) ([Antonio Andelic](https://github.com/antonio2368)). +* Version mypy=1.4.1 falsly reports unused ignore comment [#51769](https://github.com/ClickHouse/ClickHouse/pull/51769) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Avoid keeping lock Context::getLock() while calculating access rights [#51772](https://github.com/ClickHouse/ClickHouse/pull/51772) ([Vitaly Baranov](https://github.com/vitlibar)). +* Making stateless tests with timeout less flaky [#51774](https://github.com/ClickHouse/ClickHouse/pull/51774) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix after [#51000](https://github.com/ClickHouse/ClickHouse/issues/51000) [#51790](https://github.com/ClickHouse/ClickHouse/pull/51790) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add assert in ThreadStatus destructor for correct current_thread [#51800](https://github.com/ClickHouse/ClickHouse/pull/51800) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix broken parts handling in `ReplicatedMergeTree` [#51801](https://github.com/ClickHouse/ClickHouse/pull/51801) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix tsan signal-unsafe call [#51802](https://github.com/ClickHouse/ClickHouse/pull/51802) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix for parallel replicas not completely disabled by granule count threshold [#51805](https://github.com/ClickHouse/ClickHouse/pull/51805) ([Alexander Gololobov](https://github.com/davenger)). +* Make sure that we don't attempt to serialize/deserialize block with 0 columns and non-zero rows [#51807](https://github.com/ClickHouse/ClickHouse/pull/51807) ([Alexander Gololobov](https://github.com/davenger)). +* Fix rare bug in `DROP COLUMN` and enabled sparse columns [#51809](https://github.com/ClickHouse/ClickHouse/pull/51809) ([Anton Popov](https://github.com/CurtizJ)). +* Fix flaky `test_multiple_disks` [#51821](https://github.com/ClickHouse/ClickHouse/pull/51821) ([Antonio Andelic](https://github.com/antonio2368)). +* Follow up to [#51547](https://github.com/ClickHouse/ClickHouse/issues/51547) [#51822](https://github.com/ClickHouse/ClickHouse/pull/51822) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Correctly grep archives in stress tests [#51824](https://github.com/ClickHouse/ClickHouse/pull/51824) ([Antonio Andelic](https://github.com/antonio2368)). +* Update analyzer_tech_debt.txt [#51836](https://github.com/ClickHouse/ClickHouse/pull/51836) ([Alexander Tokmakov](https://github.com/tavplubix)). +* remove unused code [#51837](https://github.com/ClickHouse/ClickHouse/pull/51837) ([flynn](https://github.com/ucasfl)). +* Fix disk config for upgrade tests [#51839](https://github.com/ClickHouse/ClickHouse/pull/51839) ([Antonio Andelic](https://github.com/antonio2368)). +* Remove Coverity from workflows, but leave in the code [#51842](https://github.com/ClickHouse/ClickHouse/pull/51842) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Many fixes [3] [#51848](https://github.com/ClickHouse/ClickHouse/pull/51848) ([Ilya Yatsishin](https://github.com/qoega)). +* Change misleading name in joins: addJoinedBlock -> addBlockToJoin [#51852](https://github.com/ClickHouse/ClickHouse/pull/51852) ([Igor Nikonov](https://github.com/devcrafter)). +* fix: correct exception messages on policies comparison [#51854](https://github.com/ClickHouse/ClickHouse/pull/51854) ([Feng Kaiyu](https://github.com/fky2015)). +* Update 02439_merge_selecting_partitions.sql [#51862](https://github.com/ClickHouse/ClickHouse/pull/51862) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove useless packages [#51863](https://github.com/ClickHouse/ClickHouse/pull/51863) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove useless logs [#51865](https://github.com/ClickHouse/ClickHouse/pull/51865) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix incorrect log level = warning [#51867](https://github.com/ClickHouse/ClickHouse/pull/51867) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test_replicated_table_attach [#51868](https://github.com/ClickHouse/ClickHouse/pull/51868) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better usability of a test [#51869](https://github.com/ClickHouse/ClickHouse/pull/51869) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove useless code [#51873](https://github.com/ClickHouse/ClickHouse/pull/51873) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Another fix upgrade check script [#51878](https://github.com/ClickHouse/ClickHouse/pull/51878) ([Antonio Andelic](https://github.com/antonio2368)). +* Sqlloogic improvements [#51883](https://github.com/ClickHouse/ClickHouse/pull/51883) ([Ilya Yatsishin](https://github.com/qoega)). +* Disable ThinLTO on non-Linux [#51897](https://github.com/ClickHouse/ClickHouse/pull/51897) ([Robert Schulze](https://github.com/rschu1ze)). +* Pin rust nightly (to make it stable) [#51903](https://github.com/ClickHouse/ClickHouse/pull/51903) ([Azat Khuzhin](https://github.com/azat)). +* Fix build [#51909](https://github.com/ClickHouse/ClickHouse/pull/51909) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix build [#51910](https://github.com/ClickHouse/ClickHouse/pull/51910) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky test `00175_partition_by_ignore` and move it to correct location [#51913](https://github.com/ClickHouse/ClickHouse/pull/51913) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky test 02360_send_logs_level_colors: avoid usage of `file` tool [#51914](https://github.com/ClickHouse/ClickHouse/pull/51914) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Maybe better tests [#51916](https://github.com/ClickHouse/ClickHouse/pull/51916) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Revert system drop filesystem cache by key [#51917](https://github.com/ClickHouse/ClickHouse/pull/51917) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky test `detach_attach_partition_race` [#51920](https://github.com/ClickHouse/ClickHouse/pull/51920) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Another fix for `02481_async_insert_race_long` [#51925](https://github.com/ClickHouse/ClickHouse/pull/51925) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix segfault caused by `ThreadStatus` [#51931](https://github.com/ClickHouse/ClickHouse/pull/51931) ([Antonio Andelic](https://github.com/antonio2368)). +* Print short fault info only from safe fields [#51932](https://github.com/ClickHouse/ClickHouse/pull/51932) ([Alexander Gololobov](https://github.com/davenger)). +* Fix typo in integration tests [#51944](https://github.com/ClickHouse/ClickHouse/pull/51944) ([Ilya Yatsishin](https://github.com/qoega)). +* Better logs on shutdown [#51951](https://github.com/ClickHouse/ClickHouse/pull/51951) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Filter databases list before querying potentially slow fields [#51955](https://github.com/ClickHouse/ClickHouse/pull/51955) ([Alexander Gololobov](https://github.com/davenger)). +* Fix some issues with transactions [#51959](https://github.com/ClickHouse/ClickHouse/pull/51959) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix unrelated messages from LSan in clickhouse-client [#51966](https://github.com/ClickHouse/ClickHouse/pull/51966) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow OOM in AST Fuzzer with Sanitizers [#51967](https://github.com/ClickHouse/ClickHouse/pull/51967) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable one test under Analyzer [#51968](https://github.com/ClickHouse/ClickHouse/pull/51968) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix Docker [#51969](https://github.com/ClickHouse/ClickHouse/pull/51969) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `01825_type_json_from_map` [#51970](https://github.com/ClickHouse/ClickHouse/pull/51970) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test `02354_distributed_with_external_aggregation_memory_usage` [#51971](https://github.com/ClickHouse/ClickHouse/pull/51971) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix disaster in integration tests, part 2 [#51973](https://github.com/ClickHouse/ClickHouse/pull/51973) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* [RFC] Cleanup remote_servers in dist config.xml [#51985](https://github.com/ClickHouse/ClickHouse/pull/51985) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v23.6.2.18-stable [#51986](https://github.com/ClickHouse/ClickHouse/pull/51986) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v22.8.20.11-lts [#51987](https://github.com/ClickHouse/ClickHouse/pull/51987) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix performance test for regexp cache [#51988](https://github.com/ClickHouse/ClickHouse/pull/51988) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Move a test to the right place [#51989](https://github.com/ClickHouse/ClickHouse/pull/51989) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a check to validate that the stateful tests are stateful [#51990](https://github.com/ClickHouse/ClickHouse/pull/51990) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Check that functional tests cleanup their tables [#51991](https://github.com/ClickHouse/ClickHouse/pull/51991) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix test_extreme_deduplication [#51992](https://github.com/ClickHouse/ClickHouse/pull/51992) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Cleanup SymbolIndex after reload got removed [#51993](https://github.com/ClickHouse/ClickHouse/pull/51993) ([Azat Khuzhin](https://github.com/azat)). +* Update CompletedPipelineExecutor exception log name [#52028](https://github.com/ClickHouse/ClickHouse/pull/52028) ([xiao](https://github.com/nicelulu)). +* Fix `00502_custom_partitioning_replicated_zookeeper_long` [#52032](https://github.com/ClickHouse/ClickHouse/pull/52032) ([Antonio Andelic](https://github.com/antonio2368)). +* Prohibit send_metadata for s3_plain disks [#52038](https://github.com/ClickHouse/ClickHouse/pull/52038) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v23.4.6.25-stable [#52061](https://github.com/ClickHouse/ClickHouse/pull/52061) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Preparations for Trivial Support For Resharding (part1) [#52068](https://github.com/ClickHouse/ClickHouse/pull/52068) ([Azat Khuzhin](https://github.com/azat)). +* Update version_date.tsv and changelogs after v23.3.8.21-lts [#52077](https://github.com/ClickHouse/ClickHouse/pull/52077) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix flakiness of test_keeper_s3_snapshot flakiness [#52083](https://github.com/ClickHouse/ClickHouse/pull/52083) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_extreme_deduplication flakiness [#52085](https://github.com/ClickHouse/ClickHouse/pull/52085) ([Azat Khuzhin](https://github.com/azat)). +* Small docs update for toYearWeek() function [#52090](https://github.com/ClickHouse/ClickHouse/pull/52090) ([Andrey Zvonov](https://github.com/zvonand)). +* Small docs update for DateTime, DateTime64 [#52094](https://github.com/ClickHouse/ClickHouse/pull/52094) ([Andrey Zvonov](https://github.com/zvonand)). +* Add missing --force for docker network prune (otherwise it is noop on CI) [#52095](https://github.com/ClickHouse/ClickHouse/pull/52095) ([Azat Khuzhin](https://github.com/azat)). +* tests: drop existing view in test_materialized_mysql_database [#52103](https://github.com/ClickHouse/ClickHouse/pull/52103) ([Azat Khuzhin](https://github.com/azat)). +* Update README.md [#52115](https://github.com/ClickHouse/ClickHouse/pull/52115) ([Tyler Hannan](https://github.com/tylerhannan)). +* Print Zxid in keeper stat command in hex (so as ZooKeeper) [#52122](https://github.com/ClickHouse/ClickHouse/pull/52122) ([Azat Khuzhin](https://github.com/azat)). +* Skip protection from double decompression if inode from maps cannot be obtained [#52138](https://github.com/ClickHouse/ClickHouse/pull/52138) ([Azat Khuzhin](https://github.com/azat)). +* There is no point in detecting flaky tests [#52142](https://github.com/ClickHouse/ClickHouse/pull/52142) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove default argument value [#52143](https://github.com/ClickHouse/ClickHouse/pull/52143) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix the "kill_mutation" test [#52144](https://github.com/ClickHouse/ClickHouse/pull/52144) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix ORDER BY tuple of WINDOW functions (and slightly more changes) [#52146](https://github.com/ClickHouse/ClickHouse/pull/52146) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix possible EADDRINUSE ("Address already in use") in integration tests [#52148](https://github.com/ClickHouse/ClickHouse/pull/52148) ([Azat Khuzhin](https://github.com/azat)). +* Fix test 02497_storage_file_reader_selection [#52154](https://github.com/ClickHouse/ClickHouse/pull/52154) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix unexpected AST Set [#52158](https://github.com/ClickHouse/ClickHouse/pull/52158) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix crash in comparison functions due to incorrect query analysis [#52172](https://github.com/ClickHouse/ClickHouse/pull/52172) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix slow test `02317_distinct_in_order_optimization` [#52173](https://github.com/ClickHouse/ClickHouse/pull/52173) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add comments for https://github.com/ClickHouse/ClickHouse/pull/52112 [#52175](https://github.com/ClickHouse/ClickHouse/pull/52175) ([李扬](https://github.com/taiyang-li)). +* Randomize timezone in tests across non-deterministic around 1970 and default [#52184](https://github.com/ClickHouse/ClickHouse/pull/52184) ([Azat Khuzhin](https://github.com/azat)). +* Fix `test_multiple_disks/test.py::test_start_stop_moves` [#52189](https://github.com/ClickHouse/ClickHouse/pull/52189) ([Antonio Andelic](https://github.com/antonio2368)). +* CMake: Simplify job limiting [#52196](https://github.com/ClickHouse/ClickHouse/pull/52196) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix self extracting binaries under qemu linux-user (qemu-$ARCH-static) [#52198](https://github.com/ClickHouse/ClickHouse/pull/52198) ([Azat Khuzhin](https://github.com/azat)). +* Fix `Integration tests flaky check (asan)` [#52201](https://github.com/ClickHouse/ClickHouse/pull/52201) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix flaky test test_lost_part [#52202](https://github.com/ClickHouse/ClickHouse/pull/52202) ([alesapin](https://github.com/alesapin)). +* MaterializedMySQL: Replace to_string by magic_enum::enum_name [#52204](https://github.com/ClickHouse/ClickHouse/pull/52204) ([Val Doroshchuk](https://github.com/valbok)). +* MaterializedMySQL: Add tests to parse db and table names from DDL [#52208](https://github.com/ClickHouse/ClickHouse/pull/52208) ([Val Doroshchuk](https://github.com/valbok)). +* Revert "Fixed several issues found by OSS-Fuzz" [#52216](https://github.com/ClickHouse/ClickHouse/pull/52216) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Use one copy replication more agressively [#52218](https://github.com/ClickHouse/ClickHouse/pull/52218) ([alesapin](https://github.com/alesapin)). +* Fix flaky test `01076_parallel_alter_replicated_zookeeper` [#52221](https://github.com/ClickHouse/ClickHouse/pull/52221) ([alesapin](https://github.com/alesapin)). +* Fix 01889_key_condition_function_chains for analyzer. [#52223](https://github.com/ClickHouse/ClickHouse/pull/52223) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Inhibit settings randomization in the test `json_ghdata` [#52226](https://github.com/ClickHouse/ClickHouse/pull/52226) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Slightly better diagnostics in a test [#52227](https://github.com/ClickHouse/ClickHouse/pull/52227) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Enable no-upgrade-check for 02273_full_sort_join [#52235](https://github.com/ClickHouse/ClickHouse/pull/52235) ([vdimir](https://github.com/vdimir)). +* Fix network manager for integration tests [#52237](https://github.com/ClickHouse/ClickHouse/pull/52237) ([Azat Khuzhin](https://github.com/azat)). +* List replication queue only for current test database [#52238](https://github.com/ClickHouse/ClickHouse/pull/52238) ([Alexander Gololobov](https://github.com/davenger)). +* Attempt to fix assert in tsan with fibers [#52241](https://github.com/ClickHouse/ClickHouse/pull/52241) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix undefined behaviour in fuzzer [#52256](https://github.com/ClickHouse/ClickHouse/pull/52256) ([Antonio Andelic](https://github.com/antonio2368)). +* Follow-up to [#51959](https://github.com/ClickHouse/ClickHouse/issues/51959) [#52261](https://github.com/ClickHouse/ClickHouse/pull/52261) ([Alexander Tokmakov](https://github.com/tavplubix)). +* More fair queue for `drop table sync` [#52276](https://github.com/ClickHouse/ClickHouse/pull/52276) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `02497_trace_events_stress_long` [#52279](https://github.com/ClickHouse/ClickHouse/pull/52279) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix test `01111_create_drop_replicated_db_stress` [#52283](https://github.com/ClickHouse/ClickHouse/pull/52283) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix ugly code [#52284](https://github.com/ClickHouse/ClickHouse/pull/52284) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add missing replica syncs in test_backup_restore_on_cluster [#52306](https://github.com/ClickHouse/ClickHouse/pull/52306) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix test_replicated_database 'node doesn't exist' flakiness [#52307](https://github.com/ClickHouse/ClickHouse/pull/52307) ([Michael Kolupaev](https://github.com/al13n321)). +* Minor: Update description of events "QueryCacheHits/Misses" [#52309](https://github.com/ClickHouse/ClickHouse/pull/52309) ([Robert Schulze](https://github.com/rschu1ze)). +* Beautify pretty-printing of the query string in SYSTEM.QUERY_CACHE [#52312](https://github.com/ClickHouse/ClickHouse/pull/52312) ([Robert Schulze](https://github.com/rschu1ze)). +* Reduce dependencies for skim by avoid using default features [#52316](https://github.com/ClickHouse/ClickHouse/pull/52316) ([Azat Khuzhin](https://github.com/azat)). +* Fix 02725_memory-for-merges [#52317](https://github.com/ClickHouse/ClickHouse/pull/52317) ([alesapin](https://github.com/alesapin)). +* Skip unsupported disks in Keeper [#52321](https://github.com/ClickHouse/ClickHouse/pull/52321) ([Antonio Andelic](https://github.com/antonio2368)). +* Revert "Improve CSVInputFormat to check and set default value to column if deserialize failed" [#52322](https://github.com/ClickHouse/ClickHouse/pull/52322) ([Kruglov Pavel](https://github.com/Avogar)). +* Resubmit [#51716](https://github.com/ClickHouse/ClickHouse/issues/51716) [#52323](https://github.com/ClickHouse/ClickHouse/pull/52323) ([Kruglov Pavel](https://github.com/Avogar)). +* Add logging about all found workflows for merge_pr.py [#52324](https://github.com/ClickHouse/ClickHouse/pull/52324) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Minor: Less awkward IAST::FormatSettings [#52332](https://github.com/ClickHouse/ClickHouse/pull/52332) ([Robert Schulze](https://github.com/rschu1ze)). +* Mark test 02125_many_mutations_2 as no-parallel to avoid flakiness [#52338](https://github.com/ClickHouse/ClickHouse/pull/52338) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix capabilities installed via systemd service (fixes netlink/IO priorities) [#52357](https://github.com/ClickHouse/ClickHouse/pull/52357) ([Azat Khuzhin](https://github.com/azat)). +* Update 01606_git_import.sh [#52360](https://github.com/ClickHouse/ClickHouse/pull/52360) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Update ci-slack-bot.py [#52372](https://github.com/ClickHouse/ClickHouse/pull/52372) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `test_keeper_session` [#52373](https://github.com/ClickHouse/ClickHouse/pull/52373) ([Antonio Andelic](https://github.com/antonio2368)). +* Update ci-slack-bot.py [#52374](https://github.com/ClickHouse/ClickHouse/pull/52374) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Disable analyzer setting in backward_compatibility integration tests. [#52375](https://github.com/ClickHouse/ClickHouse/pull/52375) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* New metric - Filesystem cache size limit [#52378](https://github.com/ClickHouse/ClickHouse/pull/52378) ([Krzysztof Góralski](https://github.com/kgoralski)). +* Fix `test_replicated_merge_tree_encrypted_disk ` [#52379](https://github.com/ClickHouse/ClickHouse/pull/52379) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix `02122_parallel_formatting_XML ` [#52380](https://github.com/ClickHouse/ClickHouse/pull/52380) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Follow up to [#49698](https://github.com/ClickHouse/ClickHouse/issues/49698) [#52381](https://github.com/ClickHouse/ClickHouse/pull/52381) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Less replication errors [#52382](https://github.com/ClickHouse/ClickHouse/pull/52382) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Rename TaskStatsInfoGetter into NetlinkMetricsProvider [#52392](https://github.com/ClickHouse/ClickHouse/pull/52392) ([Azat Khuzhin](https://github.com/azat)). +* Fix `test_keeper_force_recovery` [#52408](https://github.com/ClickHouse/ClickHouse/pull/52408) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix flaky gtest_lru_file_cache.cpp [#52418](https://github.com/ClickHouse/ClickHouse/pull/52418) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix: remove redundant distinct with views [#52438](https://github.com/ClickHouse/ClickHouse/pull/52438) ([Igor Nikonov](https://github.com/devcrafter)). +* Add 02815_range_dict_no_direct_join to analyzer_tech_debt.txt [#52464](https://github.com/ClickHouse/ClickHouse/pull/52464) ([vdimir](https://github.com/vdimir)). +* do not throw exception in OptimizedRegularExpressionImpl::analyze [#52467](https://github.com/ClickHouse/ClickHouse/pull/52467) ([Han Fei](https://github.com/hanfei1991)). +* Remove skip_startup_tables from IDatabase::loadStoredObjects() [#52491](https://github.com/ClickHouse/ClickHouse/pull/52491) ([Azat Khuzhin](https://github.com/azat)). +* Fix test_insert_same_partition_and_merge by increasing wait time [#52497](https://github.com/ClickHouse/ClickHouse/pull/52497) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Try to fix asan wanring in HashJoin [#52499](https://github.com/ClickHouse/ClickHouse/pull/52499) ([Igor Nikonov](https://github.com/devcrafter)). +* Replace with three way comparison [#52509](https://github.com/ClickHouse/ClickHouse/pull/52509) ([flynn](https://github.com/ucasfl)). +* Fix flakiness of test_version_update_after_mutation by enabling force_remove_data_recursively_on_drop [#52514](https://github.com/ClickHouse/ClickHouse/pull/52514) ([Azat Khuzhin](https://github.com/azat)). +* Fix `test_throttling` [#52515](https://github.com/ClickHouse/ClickHouse/pull/52515) ([Antonio Andelic](https://github.com/antonio2368)). +* Improve logging macros [#52519](https://github.com/ClickHouse/ClickHouse/pull/52519) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `toDecimalString` function [#52520](https://github.com/ClickHouse/ClickHouse/pull/52520) ([Andrey Zvonov](https://github.com/zvonand)). +* Remove unused code [#52527](https://github.com/ClickHouse/ClickHouse/pull/52527) ([Raúl Marín](https://github.com/Algunenano)). +* Cancel execution in PipelineExecutor in case of exception in graph->updateNode [#52533](https://github.com/ClickHouse/ClickHouse/pull/52533) ([Kruglov Pavel](https://github.com/Avogar)). +* Make 01951_distributed_push_down_limit analyzer agnostic [#52534](https://github.com/ClickHouse/ClickHouse/pull/52534) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix disallow_concurrency test for backup and restore [#52536](https://github.com/ClickHouse/ClickHouse/pull/52536) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Update 02136_scalar_subquery_metrics.sql [#52537](https://github.com/ClickHouse/ClickHouse/pull/52537) ([Alexander Tokmakov](https://github.com/tavplubix)). +* tests: fix 01035_avg_weighted_long flakiness [#52556](https://github.com/ClickHouse/ClickHouse/pull/52556) ([Azat Khuzhin](https://github.com/azat)). +* tests: increase throttling for 01923_network_receive_time_metric_insert [#52557](https://github.com/ClickHouse/ClickHouse/pull/52557) ([Azat Khuzhin](https://github.com/azat)). +* tests: fix 00719_parallel_ddl_table flakiness in debug builds [#52558](https://github.com/ClickHouse/ClickHouse/pull/52558) ([Azat Khuzhin](https://github.com/azat)). +* tests: fix 01821_join_table_race_long flakiness [#52559](https://github.com/ClickHouse/ClickHouse/pull/52559) ([Azat Khuzhin](https://github.com/azat)). +* Fix flaky `00995_exception_while_insert` [#52568](https://github.com/ClickHouse/ClickHouse/pull/52568) ([Antonio Andelic](https://github.com/antonio2368)). +* MaterializedMySQL: Fix typos in tests [#52575](https://github.com/ClickHouse/ClickHouse/pull/52575) ([Val Doroshchuk](https://github.com/valbok)). +* Fix `02497_trace_events_stress_long` again [#52587](https://github.com/ClickHouse/ClickHouse/pull/52587) ([Antonio Andelic](https://github.com/antonio2368)). +* Revert "Remove `mmap/mremap/munmap` from Allocator.h" [#52589](https://github.com/ClickHouse/ClickHouse/pull/52589) ([Nikita Taranov](https://github.com/nickitat)). +* Remove peak memory usage from the final message in the client [#52598](https://github.com/ClickHouse/ClickHouse/pull/52598) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* GinIndexStore: fix a bug when files are finalizated after first write, [#52602](https://github.com/ClickHouse/ClickHouse/pull/52602) ([Sema Checherinda](https://github.com/CheSema)). +* Fix deadlocks in StorageTableFunctionProxy [#52626](https://github.com/ClickHouse/ClickHouse/pull/52626) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix build with clang-15 [#52627](https://github.com/ClickHouse/ClickHouse/pull/52627) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix style [#52647](https://github.com/ClickHouse/ClickHouse/pull/52647) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix logging level of a noisy message [#52648](https://github.com/ClickHouse/ClickHouse/pull/52648) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Revert "Added field `refcount` to `system.remote_data_paths` table" [#52657](https://github.com/ClickHouse/ClickHouse/pull/52657) ([Alexander Tokmakov](https://github.com/tavplubix)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8b535e3d897..1eabc65a10f 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.7.1.2470-stable 2023-07-27 v23.6.2.18-stable 2023-07-09 v23.6.1.1524-stable 2023-06-30 v23.5.4.25-stable 2023-06-29 From ce38d3c5ea45507696430e9c7f39f9ab7b9de394 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:11:08 +0200 Subject: [PATCH 1229/2047] address comment --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 53481ab06a0..dbb4f7f0d8e 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -10,11 +10,17 @@ #include #include +#include "Common/Exception.h" #include namespace DB { +namespace ErrorCodes +{ + extern const int ABORTED; +} + namespace { @@ -271,8 +277,7 @@ void MergeTreeDeduplicationLog::dropPart(const MergeTreePartInfo & drop_part_inf if (stopped) { - LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we drop this part."); - return; + throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we drop this part."); } chassert(current_writer != nullptr); From f5dfb70f5c2f4b94a54e9fdb97737a70b28362ad Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:12:20 +0200 Subject: [PATCH 1230/2047] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index dbb4f7f0d8e..80e94b2fd39 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -10,7 +10,7 @@ #include #include -#include "Common/Exception.h" +#include #include namespace DB From b5160c8072b9660e7c62b8305f56eda26313ecb0 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 27 Jul 2023 14:16:19 +0000 Subject: [PATCH 1231/2047] Fix bugs and better test for SYSTEM STOP LISTEN --- programs/server/Server.cpp | 25 ++-- src/Server/ServerType.cpp | 20 +-- src/Server/ServerType.h | 1 - .../configs/cluster.xml | 4 +- .../configs/protocols.xml | 23 ++++ .../test_system_start_stop_listen/test.py | 128 ++++++++++++++++-- 6 files changed, 157 insertions(+), 44 deletions(-) create mode 100644 tests/integration/test_system_start_stop_listen/configs/protocols.xml diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index dce52ecdb12..adec13a5199 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2031,27 +2031,26 @@ void Server::createServers( for (const auto & protocol : protocols) { - if (!server_type.shouldStart(ServerType::Type::CUSTOM, protocol)) + std::string prefix = "protocols." + protocol + "."; + std::string port_name = prefix + "port"; + std::string description {" protocol"}; + if (config.has(prefix + "description")) + description = config.getString(prefix + "description"); + + if (!config.has(prefix + "port")) + continue; + + if (!server_type.shouldStart(ServerType::Type::CUSTOM, port_name)) continue; std::vector hosts; - if (config.has("protocols." + protocol + ".host")) - hosts.push_back(config.getString("protocols." + protocol + ".host")); + if (config.has(prefix + "host")) + hosts.push_back(config.getString(prefix + "host")); else hosts = listen_hosts; for (const auto & host : hosts) { - std::string conf_name = "protocols." + protocol; - std::string prefix = conf_name + "."; - - if (!config.has(prefix + "port")) - continue; - - std::string description {" protocol"}; - if (config.has(prefix + "description")) - description = config.getString(prefix + "description"); - std::string port_name = prefix + "port"; bool is_secure = false; auto stack = buildProtocolStackFromConfig(config, protocol, http_params, async_metrics, is_secure); diff --git a/src/Server/ServerType.cpp b/src/Server/ServerType.cpp index c6916ee39d9..4952cd1bd24 100644 --- a/src/Server/ServerType.cpp +++ b/src/Server/ServerType.cpp @@ -40,7 +40,7 @@ const char * ServerType::serverTypeToString(ServerType::Type type) return type_name.data(); } -bool ServerType::shouldStart(Type server_type, const std::string & custom_name_) const +bool ServerType::shouldStart(Type server_type, const std::string & server_custom_name) const { if (type == Type::QUERIES_ALL) return true; @@ -77,13 +77,15 @@ bool ServerType::shouldStart(Type server_type, const std::string & custom_name_) } } - return type == server_type && custom_name == custom_name_; + if (type == Type::CUSTOM) + return server_type == type && server_custom_name == "protocols." + custom_name + ".port"; + + return server_type == type; } bool ServerType::shouldStop(const std::string & port_name) const { Type port_type; - std::string port_custom_name; if (port_name == "http_port") port_type = Type::HTTP; @@ -119,20 +121,12 @@ bool ServerType::shouldStop(const std::string & port_name) const port_type = Type::INTERSERVER_HTTPS; else if (port_name.starts_with("protocols.") && port_name.ends_with(".port")) - { - constexpr size_t protocols_size = std::string_view("protocols.").size(); - constexpr size_t port_size = std::string_view("protocols.").size(); - port_type = Type::CUSTOM; - port_custom_name = port_name.substr(protocols_size, port_name.size() - port_size); - } - else - port_type = Type::UNKNOWN; - if (port_type == Type::UNKNOWN) + else return false; - return shouldStart(type, port_custom_name); + return shouldStart(port_type, port_name); } } diff --git a/src/Server/ServerType.h b/src/Server/ServerType.h index 345d1a10119..8fb81622ab3 100644 --- a/src/Server/ServerType.h +++ b/src/Server/ServerType.h @@ -10,7 +10,6 @@ public: enum Type { - UNKNOWN, TCP, TCP_WITH_PROXY, TCP_SECURE, diff --git a/tests/integration/test_system_start_stop_listen/configs/cluster.xml b/tests/integration/test_system_start_stop_listen/configs/cluster.xml index 93d8f890f40..34b6c32c6d0 100644 --- a/tests/integration/test_system_start_stop_listen/configs/cluster.xml +++ b/tests/integration/test_system_start_stop_listen/configs/cluster.xml @@ -3,11 +3,11 @@ - node1 + main_node 9000 - node2 + backup_node 9000 diff --git a/tests/integration/test_system_start_stop_listen/configs/protocols.xml b/tests/integration/test_system_start_stop_listen/configs/protocols.xml new file mode 100644 index 00000000000..1d8608bcaca --- /dev/null +++ b/tests/integration/test_system_start_stop_listen/configs/protocols.xml @@ -0,0 +1,23 @@ + + 0.0.0.0 + + + 9000 + 8123 + 9004 + + + + + tcp + 0.0.0.0 + 9001 + native protocol (tcp) + + + http + 8124 + http protocol + + + diff --git a/tests/integration/test_system_start_stop_listen/test.py b/tests/integration/test_system_start_stop_listen/test.py index ec1a000c599..0db313368fd 100644 --- a/tests/integration/test_system_start_stop_listen/test.py +++ b/tests/integration/test_system_start_stop_listen/test.py @@ -2,20 +2,16 @@ import pytest -import time from helpers.cluster import ClickHouseCluster -from helpers.network import PartitionManager -from helpers.test_tools import assert_eq_with_retry -import random -import string -import json +from helpers.client import Client +import requests cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", main_configs=["configs/cluster.xml"], with_zookeeper=True +main_node = cluster.add_instance( + "main_node", main_configs=["configs/cluster.xml", "configs/protocols.xml"], with_zookeeper=True ) -node2 = cluster.add_instance( - "node2", main_configs=["configs/cluster.xml"], with_zookeeper=True +backup_node = cluster.add_instance( + "backup_node", main_configs=["configs/cluster.xml"], with_zookeeper=True ) @@ -30,11 +26,113 @@ def started_cluster(): cluster.shutdown() -def test_system_start_stop_listen_queries(started_cluster): - node1.query("SYSTEM STOP LISTEN QUERIES ALL") +def http_works(port=8123): + try: + response = requests.post(f"http://{main_node.ip_address}:{port}/ping") + if response.status_code == 400: + return True + except: + pass - assert "Connection refused" in node1.query_and_get_error("SELECT 1", timeout=3) + return False - node2.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") +def assert_everything_works(): + custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) + main_node.query(QUERY) + main_node.query(MYSQL_QUERY) + custom_client.query(QUERY) + assert http_works() + assert http_works(8124) - node1.query("SELECT 1") +QUERY = "SELECT 1" +MYSQL_QUERY = "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100)" + +def test_default_protocols(started_cluster): + # TCP + assert_everything_works() + main_node.query("SYSTEM STOP LISTEN TCP") + assert "Connection refused" in main_node.query_and_get_error(QUERY) + backup_node.query("SYSTEM START LISTEN ON CLUSTER default TCP") + + # HTTP + assert_everything_works() + main_node.query("SYSTEM STOP LISTEN HTTP") + assert http_works() == False + main_node.query("SYSTEM START LISTEN HTTP") + + # MySQL + assert_everything_works() + main_node.query("SYSTEM STOP LISTEN MYSQL") + assert "Connections to mysql failed" in main_node.query_and_get_error(MYSQL_QUERY) + main_node.query("SYSTEM START LISTEN MYSQL") + + assert_everything_works() + +def test_custom_protocols(started_cluster): + # TCP + custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) + assert_everything_works() + main_node.query("SYSTEM STOP LISTEN CUSTOM 'tcp'") + assert "Connection refused" in custom_client.query_and_get_error(QUERY) + main_node.query("SYSTEM START LISTEN CUSTOM 'tcp'") + + # HTTP + assert_everything_works() + main_node.query("SYSTEM STOP LISTEN CUSTOM 'http'") + assert http_works(8124) == False + main_node.query("SYSTEM START LISTEN CUSTOM 'http'") + + assert_everything_works() + +def test_all_protocols(started_cluster): + custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) + assert_everything_works() + + # STOP LISTEN QUERIES ALL + main_node.query("SYSTEM STOP LISTEN QUERIES ALL") + assert "Connection refused" in main_node.query_and_get_error(QUERY) + assert "Connection refused" in custom_client.query_and_get_error(QUERY) + assert http_works() == False + assert http_works(8124) == False + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") + + # STOP LISTEN QUERIES DEFAULT + assert_everything_works() + + main_node.query("SYSTEM STOP LISTEN QUERIES DEFAULT") + assert "Connection refused" in main_node.query_and_get_error(QUERY) + custom_client.query(QUERY) + assert http_works() == False + assert http_works(8124) + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES DEFAULT") + + # STOP LISTEN QUERIES CUSTOM + assert_everything_works() + + main_node.query("SYSTEM STOP LISTEN QUERIES CUSTOM") + main_node.query(QUERY) + assert "Connection refused" in custom_client.query_and_get_error(QUERY) + assert http_works() + assert http_works(8124) == False + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES CUSTOM") + + # Disable all protocols, check first START LISTEN QUERIES DEFAULT then START LISTEN QUERIES CUSTOM + assert_everything_works() + + main_node.query("SYSTEM STOP LISTEN QUERIES ALL") + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES DEFAULT") + main_node.query(QUERY) + assert "Connection refused" in custom_client.query_and_get_error(QUERY) + assert http_works() + assert http_works(8124) == False + + main_node.query("SYSTEM STOP LISTEN QUERIES ALL") + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES CUSTOM") + assert "Connection refused" in main_node.query_and_get_error(QUERY) + custom_client.query(QUERY) + assert http_works() == False + assert http_works(8124) + + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") + + assert_everything_works() From ee9bad7a3140cc3164cbf36ef45486068b4be0e7 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 27 Jul 2023 14:18:46 +0000 Subject: [PATCH 1232/2047] Fix style --- tests/integration/test_system_start_stop_listen/test.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_system_start_stop_listen/test.py b/tests/integration/test_system_start_stop_listen/test.py index 0db313368fd..1925685af03 100644 --- a/tests/integration/test_system_start_stop_listen/test.py +++ b/tests/integration/test_system_start_stop_listen/test.py @@ -8,7 +8,9 @@ import requests cluster = ClickHouseCluster(__file__) main_node = cluster.add_instance( - "main_node", main_configs=["configs/cluster.xml", "configs/protocols.xml"], with_zookeeper=True + "main_node", + main_configs=["configs/cluster.xml", "configs/protocols.xml"], + with_zookeeper=True, ) backup_node = cluster.add_instance( "backup_node", main_configs=["configs/cluster.xml"], with_zookeeper=True @@ -36,6 +38,7 @@ def http_works(port=8123): return False + def assert_everything_works(): custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) main_node.query(QUERY) @@ -44,9 +47,11 @@ def assert_everything_works(): assert http_works() assert http_works(8124) + QUERY = "SELECT 1" MYSQL_QUERY = "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100)" + def test_default_protocols(started_cluster): # TCP assert_everything_works() @@ -68,6 +73,7 @@ def test_default_protocols(started_cluster): assert_everything_works() + def test_custom_protocols(started_cluster): # TCP custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) @@ -84,6 +90,7 @@ def test_custom_protocols(started_cluster): assert_everything_works() + def test_all_protocols(started_cluster): custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) assert_everything_works() From 0d44d527ef590a5471ea577c132edb42f0c99c70 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:27:04 +0200 Subject: [PATCH 1233/2047] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 80e94b2fd39..25b93160d27 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -241,8 +241,7 @@ std::pair MergeTreeDeduplicationLog::addPart(const std: if (stopped) { - LOG_ERROR(&Poco::Logger::get("MergeTreeDeduplicationLog"), "Storage has been shutdown when we add this part."); - return {}; + throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we drop this part."); } chassert(current_writer != nullptr); From f6ca013c536d76ca6c1403db5f84d792f6bd8864 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:28:00 +0200 Subject: [PATCH 1234/2047] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 25b93160d27..22dabc43a8c 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -11,7 +11,6 @@ #include #include -#include namespace DB { From 9488567bf6be7e2b751917a179222478fcb46f5e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 27 Jul 2023 16:28:08 +0200 Subject: [PATCH 1235/2047] Update src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 22dabc43a8c..548b61ce422 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -240,7 +240,7 @@ std::pair MergeTreeDeduplicationLog::addPart(const std: if (stopped) { - throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we drop this part."); + throw Exception(ErrorCodes::ABORTED, "Storage has been shutdown when we add this part."); } chassert(current_writer != nullptr); From 5611b2fff484d74c70c8ad6b62ba8d66c0b63589 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 27 Jul 2023 15:45:55 +0200 Subject: [PATCH 1236/2047] Add a note about not working _table filter for Merge with analyzer Signed-off-by: Azat Khuzhin --- src/Storages/StorageMerge.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index b0ed242d14d..272f35303bd 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -795,6 +795,10 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( bool filter_by_database_virtual_column /* = false */, bool filter_by_table_virtual_column /* = false */) const { + /// FIXME: filtering does not work with allow_experimental_analyzer due to + /// different column names there (it has "table_name._table" not just + /// "_table") + assert(!filter_by_database_virtual_column || !filter_by_table_virtual_column || query); const Settings & settings = query_context->getSettingsRef(); From 68aed0d16e331a6ba6b592243f10ce2a816152db Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 26 Jul 2023 20:25:48 +0200 Subject: [PATCH 1237/2047] RFC: Fix filtering by virtual columns with OR expression Virtual columns did not supports queries with OR, for example query like this (here `m` is the `Merge` table, see the test): select key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1'); Will always leads to: Cannot find column `value` in source stream, there are only columns ... The reason for this is that it actually executes the following queries: SELECT key, value FROM default.d1 WHERE ((value = 10) AND ('v1' = 'v1')) OR ((value = 20) AND ('v1' = 'v1')); SELECT key FROM default.d2 WHERE 0; And this kind of filtering is used not only for `Merge` table but also: - `_table` for `Merge` (already mentioned) - `_file` for `File` - `_idx` for `S3` - and as well as filtering `system.*` tables by `database`/`table`/... Signed-off-by: Azat Khuzhin --- src/Storages/VirtualColumnUtils.cpp | 36 +++++++++++++----- .../02840_merge__table_or_filter.reference | 38 +++++++++++++++++++ .../02840_merge__table_or_filter.sql.j2 | 34 +++++++++++++++++ 3 files changed, 99 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02840_merge__table_or_filter.reference create mode 100644 tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 907fc0cd22c..79be1f98a0f 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -63,14 +64,31 @@ bool isValidFunction(const ASTPtr & expression, const std::function & is_constant, ASTs & result) { const auto * function = expression->as(); - if (function && (function->name == "and" || function->name == "indexHint")) + + if (function) { - bool ret = true; - for (const auto & child : function->arguments->children) - ret &= extractFunctions(child, is_constant, result); - return ret; + if (function->name == "and" || function->name == "indexHint") + { + bool ret = true; + for (const auto & child : function->arguments->children) + ret &= extractFunctions(child, is_constant, result); + return ret; + } + else if (function->name == "or") + { + bool ret = true; + ASTs or_args; + for (const auto & child : function->arguments->children) + ret &= extractFunctions(child, is_constant, or_args); + /// We can keep condition only if it still OR condition (i.e. we + /// have dependent conditions for columns at both sides) + if (or_args.size() == 2) + result.push_back(makeASTForLogicalOr(std::move(or_args))); + return ret; + } } - else if (isValidFunction(expression, is_constant)) + + if (isValidFunction(expression, is_constant)) { result.push_back(expression->clone()); return true; @@ -80,13 +98,13 @@ bool extractFunctions(const ASTPtr & expression, const std::function Date: Thu, 27 Jul 2023 17:07:34 +0200 Subject: [PATCH 1238/2047] Update 02812_pointwise_array_operations.reference --- .../0_stateless/02812_pointwise_array_operations.reference | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/queries/0_stateless/02812_pointwise_array_operations.reference b/tests/queries/0_stateless/02812_pointwise_array_operations.reference index 6a484ce17dc..92fb7d504e7 100644 --- a/tests/queries/0_stateless/02812_pointwise_array_operations.reference +++ b/tests/queries/0_stateless/02812_pointwise_array_operations.reference @@ -1,8 +1,5 @@ [2,5] - [2,6] - [4.5,5,12,10.1] - [(11.1,5.4),(6,21)] From 8c14c4b5e8925c6793c04cced5ebe64668ba3944 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 14:59:28 +0000 Subject: [PATCH 1239/2047] Remove unneeded readBinary() specializations + update docs The default instantiation template requires is_arithmetic_v inline void readBinary(T & x, ReadBuffer & buf) { readPODBinary(x, buf); } covers Int32, Int128, Int256, UInt32, UInt128 and UInt256 cases already. --- docs/en/sql-reference/data-types/uuid.md | 24 +++++++++++------------- src/IO/ReadHelpers.h | 6 ------ 2 files changed, 11 insertions(+), 19 deletions(-) diff --git a/docs/en/sql-reference/data-types/uuid.md b/docs/en/sql-reference/data-types/uuid.md index b0f19f0d8be..40f756b9588 100644 --- a/docs/en/sql-reference/data-types/uuid.md +++ b/docs/en/sql-reference/data-types/uuid.md @@ -6,42 +6,42 @@ sidebar_label: UUID # UUID -A universally unique identifier (UUID) is a 16-byte number used to identify records. For detailed information about the UUID, see [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier). +A Universally Unique Identifier (UUID) is a 16-byte value used to identify records. For detailed information about UUIDs, see [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier). -The example of UUID type value is represented below: +While different UUID variants exist (see [here](https://datatracker.ietf.org/doc/html/draft-ietf-uuidrev-rfc4122bis)), ClickHouse does not validate that inserted UUIDs conform to a particular variant. UUIDs are internally treated as a sequence of 16 random bytes with [8-4-4-4-12 representation](https://en.wikipedia.org/wiki/Universally_unique_identifier#Textual_representation) at SQL level. + +Example UUID value: ``` text 61f0c404-5cb3-11e7-907b-a6006ad3dba0 ``` -If you do not specify the UUID column value when inserting a new record, the UUID value is filled with zero: +The default UUID is all-zero. It is used, for example, when a new record is inserted but no value for a UUID column is specified: ``` text 00000000-0000-0000-0000-000000000000 ``` -## How to Generate +## Generating UUIDs -To generate the UUID value, ClickHouse provides the [generateUUIDv4](../../sql-reference/functions/uuid-functions.md) function. +ClickHouse provides the [generateUUIDv4](../../sql-reference/functions/uuid-functions.md) function to generate random UUID version 4 values. ## Usage Example **Example 1** -This example demonstrates creating a table with the UUID type column and inserting a value into the table. +This example demonstrates the creation of a table with a UUID column and the insertion of a value into the table. ``` sql CREATE TABLE t_uuid (x UUID, y String) ENGINE=TinyLog -``` -``` sql INSERT INTO t_uuid SELECT generateUUIDv4(), 'Example 1' -``` -``` sql SELECT * FROM t_uuid ``` +Result: + ``` text ┌────────────────────────────────────x─┬─y─────────┐ │ 417ddc5d-e556-4d27-95dd-a34d84e46a50 │ Example 1 │ @@ -50,13 +50,11 @@ SELECT * FROM t_uuid **Example 2** -In this example, the UUID column value is not specified when inserting a new record. +In this example, no UUID column value is specified when the record is inserted, i.e. the default UUID value is inserted: ``` sql INSERT INTO t_uuid (y) VALUES ('Example 2') -``` -``` sql SELECT * FROM t_uuid ``` diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 2636898c1b3..1fe08a738c2 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1096,12 +1096,6 @@ inline void readBinary(bool & x, ReadBuffer & buf) } inline void readBinary(String & x, ReadBuffer & buf) { readStringBinary(x, buf); } -inline void readBinary(Int32 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(Int128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(Int256 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(UInt32 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(UInt128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(UInt256 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal32 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal64 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } From 04ad661968f3264bd423e8c707e41612e3d4e405 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 27 Jul 2023 18:17:47 +0300 Subject: [PATCH 1240/2047] Introduce clean-up of the source storage policy by setting it to an empty string --- src/Backups/RestoreSettings.cpp | 3 ++- src/Backups/RestoreSettings.h | 2 +- src/Backups/RestorerFromBackup.cpp | 22 ++++++++++------ src/Backups/RestorerFromBackup.h | 2 +- src/Backups/SettingsFieldOptionalString.cpp | 29 +++++++++++++++++++++ src/Backups/SettingsFieldOptionalString.h | 20 ++++++++++++++ 6 files changed, 67 insertions(+), 11 deletions(-) create mode 100644 src/Backups/SettingsFieldOptionalString.cpp create mode 100644 src/Backups/SettingsFieldOptionalString.h diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 95f575b846a..3290e5112c0 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -164,7 +165,7 @@ namespace M(Bool, allow_s3_native_copy) \ M(Bool, internal) \ M(String, host_id) \ - M(String, storage_policy) \ + M(OptionalString, storage_policy) \ M(OptionalUUID, restore_uuid) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 41c66b37442..788dcdb28d3 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -118,7 +118,7 @@ struct RestoreSettings String host_id; /// Alternative storage policy that may be specified in the SETTINGS clause of RESTORE queries - String storage_policy; + std::optional storage_policy; /// Internal, should not be specified by user. /// Cluster's hosts' IDs in the format 'escaped_host_name:port' for all shards and replicas in a cluster specified in BACKUP ON CLUSTER. diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 2e598ae0486..151813db574 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -322,7 +322,7 @@ void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name read_buffer.reset(); ParserCreateQuery create_parser; ASTPtr create_table_query = parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - setCustomStoragePolicyIfAny(create_table_query); + applyCustomStoragePolicy(create_table_query); renameDatabaseAndTableNameInCreateQuery(create_table_query, renaming_map, context->getGlobalContext()); QualifiedTableName table_name = renaming_map.getNewTableName(table_name_in_backup); @@ -626,16 +626,22 @@ void RestorerFromBackup::checkDatabase(const String & database_name) } } -void RestorerFromBackup::setCustomStoragePolicyIfAny(ASTPtr query_ptr) +void RestorerFromBackup::applyCustomStoragePolicy(ASTPtr query_ptr) { - if (!restore_settings.storage_policy.empty()) + constexpr auto setting_name = "storage_policy"; + if (!query_ptr) + return; + auto storage = query_ptr->as().storage; + if (storage && storage->settings) { - auto & create_table_query = query_ptr->as(); - if (create_table_query.storage && create_table_query.storage->settings) + if (restore_settings.storage_policy.has_value()) { - auto value = create_table_query.storage->settings->changes.tryGet("storage_policy"); - if (value) - *value = restore_settings.storage_policy; + if (restore_settings.storage_policy.value().empty()) + /// it has been set to "" deliberately, so the source storage policy is erased + storage->settings->changes.removeSetting(setting_name); + else + /// it has been set to a custom value, so it either overwrites the existing value or is added as a new one + storage->settings->changes.setSetting(setting_name, restore_settings.storage_policy.value()); } } } diff --git a/src/Backups/RestorerFromBackup.h b/src/Backups/RestorerFromBackup.h index dcd1922506c..194478bd8b4 100644 --- a/src/Backups/RestorerFromBackup.h +++ b/src/Backups/RestorerFromBackup.h @@ -95,7 +95,7 @@ private: void createDatabase(const String & database_name) const; void checkDatabase(const String & database_name); - void setCustomStoragePolicyIfAny(ASTPtr query_ptr); + void applyCustomStoragePolicy(ASTPtr query_ptr); void removeUnresolvedDependencies(); void createTables(); diff --git a/src/Backups/SettingsFieldOptionalString.cpp b/src/Backups/SettingsFieldOptionalString.cpp new file mode 100644 index 00000000000..573fd1e052c --- /dev/null +++ b/src/Backups/SettingsFieldOptionalString.cpp @@ -0,0 +1,29 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_BACKUP_SETTINGS; +} + +SettingFieldOptionalString::SettingFieldOptionalString(const Field & field) +{ + if (field.getType() == Field::Types::Null) + { + value = std::nullopt; + return; + } + + if (field.getType() == Field::Types::String) + { + value = field.get(); + return; + } + + throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Cannot get string from {}", field); +} + +} diff --git a/src/Backups/SettingsFieldOptionalString.h b/src/Backups/SettingsFieldOptionalString.h new file mode 100644 index 00000000000..e76c979e4ad --- /dev/null +++ b/src/Backups/SettingsFieldOptionalString.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include + +namespace DB +{ + +struct SettingFieldOptionalString +{ + std::optional value; + + explicit SettingFieldOptionalString(const std::optional & value_) : value(value_) {} + + explicit SettingFieldOptionalString(const Field & field); + + explicit operator Field() const { return Field(value ? toString(*value) : ""); } +}; + +} From c2ee77435017ba828e9ac179030bba7cb4ef4c09 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 09:49:34 +0000 Subject: [PATCH 1241/2047] Add query cache metrics to system.asynchronous_metrics Cf. https://github.com/ClickHouse/ClickHouse/pull/52384#issuecomment-1653241216 --- docs/en/operations/query-cache.md | 11 ++++++----- .../operations/system-tables/asynchronous_metrics.md | 12 ++++++++++++ docs/en/operations/system-tables/events.md | 2 ++ docs/en/operations/system-tables/metrics.md | 2 +- src/Interpreters/Cache/QueryCache.cpp | 10 ++++++++++ src/Interpreters/Cache/QueryCache.h | 5 ++++- src/Interpreters/ServerAsynchronousMetrics.cpp | 6 ++++++ 7 files changed, 41 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index 547105c65cc..d0b785d8fda 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -61,11 +61,12 @@ use_query_cache = true`) but one should keep in mind that all `SELECT` queries i may return cached results then. The query cache can be cleared using statement `SYSTEM DROP QUERY CACHE`. The content of the query cache is displayed in system table -`system.query_cache`. The number of query cache hits and misses are shown as events "QueryCacheHits" and "QueryCacheMisses" in system table -[system.events](system-tables/events.md). Both counters are only updated for `SELECT` queries which run with setting "use_query_cache = -true". Other queries do not affect the cache miss counter. Field `query_log_usage` in system table -[system.query_log](system-tables/query_log.md) shows for each ran query whether the query result was written into or read from the query -cache. +`system.query_cache`. The number of query cache hits and misses since database start are shown as events "QueryCacheHits" and +"QueryCacheMisses" in system table [system.events](system-tables/events.md). Both counters are only updated for `SELECT` queries which run +with setting `use_query_cache = true`, other queries do not affect "QueryCacheMisses". Field `query_log_usage` in system table +[system.query_log](system-tables/query_log.md) shows for each executed query whether the query result was written into or read from the +query cache. Asynchronous metrics "QueryCacheEntries" and "QueryCacheBytes" in system table +[system.asynchronous_metrics](system-tables/asynchronous_metrics.md) show how many entries / bytes the query cache currently contains. The query cache exists once per ClickHouse server process. However, cache results are by default not shared between users. This can be changed (see below) but doing so is not recommended for security reasons. diff --git a/docs/en/operations/system-tables/asynchronous_metrics.md b/docs/en/operations/system-tables/asynchronous_metrics.md index f357341da67..e46b495239c 100644 --- a/docs/en/operations/system-tables/asynchronous_metrics.md +++ b/docs/en/operations/system-tables/asynchronous_metrics.md @@ -32,6 +32,10 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 └─────────────────────────────────────────┴────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` + + ## Metric descriptions @@ -483,6 +487,14 @@ The value is similar to `OSUserTime` but divided to the number of CPU cores to b Number of threads in the server of the PostgreSQL compatibility protocol. +### QueryCacheBytes + +Total size of the query cache cache in bytes. + +### QueryCacheEntries + +Total number of entries in the query cache. + ### ReplicasMaxAbsoluteDelay Maximum difference in seconds between the most fresh replicated part and the most fresh data part still to be replicated, across Replicated tables. A very high value indicates a replica with no data. diff --git a/docs/en/operations/system-tables/events.md b/docs/en/operations/system-tables/events.md index ba5602ee292..7846fe4be5d 100644 --- a/docs/en/operations/system-tables/events.md +++ b/docs/en/operations/system-tables/events.md @@ -11,6 +11,8 @@ Columns: - `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of events occurred. - `description` ([String](../../sql-reference/data-types/string.md)) — Event description. +You can find all supported events in source file [src/Common/ProfileEvents.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/ProfileEvents.cpp). + **Example** ``` sql diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index 5a7dfd03eb4..b1dcea5500f 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -11,7 +11,7 @@ Columns: - `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — Metric value. - `description` ([String](../../sql-reference/data-types/string.md)) — Metric description. -The list of supported metrics you can find in the [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) source file of ClickHouse. +You can find all supported metrics in source file [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp). **Example** diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index a6c509e8bb1..5982a5ade50 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -496,6 +496,16 @@ void QueryCache::reset() cache_size_in_bytes = 0; } +size_t QueryCache::weight() const +{ + return cache.weight(); +} + +size_t QueryCache::count() const +{ + return cache.count(); +} + size_t QueryCache::recordQueryRun(const Key & key) { std::lock_guard lock(mutex); diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index c2de8ca22dd..eaa54c503fa 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -186,6 +186,9 @@ public: void reset(); + size_t weight() const; + size_t count() const; + /// Record new execution of query represented by key. Returns number of executions so far. size_t recordQueryRun(const Key & key); @@ -193,7 +196,7 @@ public: std::vector dump() const; private: - Cache cache; + Cache cache; /// has its own locking --> not protected by mutex mutable std::mutex mutex; TimesExecuted times_executed TSA_GUARDED_BY(mutex); diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 0fbcfc9e6a1..68411e80755 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -92,6 +92,12 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values " The files opened with `mmap` are kept in the cache to avoid costly TLB flushes."}; } + if (auto query_cache = getContext()->getQueryCache()) + { + new_values["QueryCacheBytes"] = { query_cache->weight(), "Total size of the query cache in bytes." }; + new_values["QueryCacheEntries"] = { query_cache->count(), "Total number of entries in the query cache." }; + } + { auto caches = FileCacheFactory::instance().getAll(); size_t total_bytes = 0; From 07147ef88805ca09b5cd7120ecf60d71c1d94d55 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 15:24:28 +0000 Subject: [PATCH 1242/2047] Remove remainders of legacy setting 'allow_experimental_query_cache' --- src/Core/Settings.h | 1 - ...726_set_allow_experimental_query_cache_as_obsolete.reference | 0 .../02726_set_allow_experimental_query_cache_as_obsolete.sql | 2 -- 3 files changed, 3 deletions(-) delete mode 100644 tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference delete mode 100644 tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c69d132ea25..e7fd28476d9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -808,7 +808,6 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ - MAKE_OBSOLETE(M, Bool, allow_experimental_query_cache, true) \ /* moved to config.xml: see also src/Core/ServerSettings.h */ \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_buffer_flush_schedule_pool_size, 16) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_pool_size, 16) \ diff --git a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference b/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql b/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql deleted file mode 100644 index 244ba4e959a..00000000000 --- a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql +++ /dev/null @@ -1,2 +0,0 @@ -SET allow_experimental_query_cache = 0; -SET allow_experimental_query_cache = 1; From dfc06d27143da106adc6d5fae3b5be089f1e2d64 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 27 Jul 2023 16:13:29 +0000 Subject: [PATCH 1243/2047] fix reading of unneded column in case of multistage prewhere --- .../MergeTree/MergeTreeBlockReadUtils.cpp | 6 +++-- ...02833_multiprewhere_extra_column.reference | 2 ++ .../02833_multiprewhere_extra_column.sql | 25 +++++++++++++++++++ 3 files changed, 31 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02833_multiprewhere_extra_column.reference create mode 100644 tests/queries/0_stateless/02833_multiprewhere_extra_column.sql diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index d830ba37e71..48779aa0df7 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -340,8 +340,10 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (!columns_from_previous_steps.contains(name)) step_column_names.push_back(name); - injectRequiredColumns( - data_part_info_for_reader, storage_snapshot, with_subcolumns, step_column_names); + if (!step_column_names.empty()) + injectRequiredColumns( + data_part_info_for_reader, storage_snapshot, + with_subcolumns, step_column_names); /// More columns could have been added, filter them as well by the list of columns from previous steps. Names columns_to_read_in_step; diff --git a/tests/queries/0_stateless/02833_multiprewhere_extra_column.reference b/tests/queries/0_stateless/02833_multiprewhere_extra_column.reference new file mode 100644 index 00000000000..45571c71477 --- /dev/null +++ b/tests/queries/0_stateless/02833_multiprewhere_extra_column.reference @@ -0,0 +1,2 @@ +10496500 +4 diff --git a/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql b/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql new file mode 100644 index 00000000000..a786de454ed --- /dev/null +++ b/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql @@ -0,0 +1,25 @@ +-- Tags: no-parallel, no-random-settings, no-random-merge-tree-settings + +drop table if exists t_multi_prewhere; +drop row policy if exists policy_02834 on t_multi_prewhere; + +create table t_multi_prewhere (a UInt64, b UInt64, c UInt8) +engine = MergeTree order by tuple() +settings min_bytes_for_wide_part = 0; + +create row policy policy_02834 on t_multi_prewhere using a > 2000 as permissive to all; +insert into t_multi_prewhere select number, number, number from numbers(10000); + +system drop mark cache; +select sum(b) from t_multi_prewhere prewhere a < 5000; + +system flush logs; + +select ProfileEvents['FileOpen'] from system.query_log +where + type = 'QueryFinish' + and current_database = currentDatabase() + and query ilike '%select sum(b) from t_multi_prewhere prewhere a < 5000%'; + +drop table if exists t_multi_prewhere; +drop row policy if exists policy_02834 on t_multi_prewhere; From faca843ac0d2fd1b740d009a6cef1c9060346ff6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 27 Jul 2023 18:24:40 +0200 Subject: [PATCH 1244/2047] Add utility to parse a backup metadata file and print information about the backup. --- utils/backup/print_backup_info.py | 208 ++++++++++++++++++++++++++++++ 1 file changed, 208 insertions(+) create mode 100755 utils/backup/print_backup_info.py diff --git a/utils/backup/print_backup_info.py b/utils/backup/print_backup_info.py new file mode 100755 index 00000000000..54e5c745a8c --- /dev/null +++ b/utils/backup/print_backup_info.py @@ -0,0 +1,208 @@ +#!/usr/bin/env python3 +# -*- coding: UTF-8 -*- +""" +print_backup_info: Extract information about a backup from ".backup" file +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +Usage: print_backup_info +""" +import sys +import os +import xml.etree.ElementTree as ET + + +def main(): + if len(sys.argv) != 2: + print(__doc__) + sys.exit(1) + backup_xml = sys.argv[1] + + if not os.path.isfile(backup_xml): + print("error: {} does not exist".format(backup_xml)) + sys.exit(1) + + # Process the file line-by-line + tree = ET.parse(backup_xml) + root = tree.getroot() + contents = root.find("contents") + + version_node = root.find("version") + version = int(version_node.text) if (version_node != None) else None + + timestamp_node = root.find("timestamp") + timestamp = timestamp_node.text if (timestamp_node != None) else None + + base_backup_node = root.find("base_backup") + base_backup = base_backup_node.text if (base_backup_node != None) else None + + number_of_files = 0 + size_of_files = 0 + number_of_files_from_base_backup = 0 + size_of_files_from_base_backup = 0 + databases = set() + tables = {} + + for file in contents: + name = file.find("name").text + size = int(file.find("size").text) + + use_base_node = file.find("use_base") + use_base = (use_base_node.text == "true") if (use_base_node != None) else False + + if use_base: + base_size_node = file.find("base_size") + base_size = int(base_size_node.text) if (base_size_node != None) else size + else: + base_size = 0 + + data_file_node = file.find("data_file") + data_file = data_file_node.text if (data_file_node != None) else name + + has_data_file = name == data_file + + if has_data_file: + if size > base_size: + number_of_files += 1 + size_of_files += size - base_size + if base_size > 0: + number_of_files_from_base_backup += 1 + size_of_files_from_base_backup += base_size + + table_name = extract_table_name_from_path(name) + if table_name: + if table_name not in tables: + tables[table_name] = [0, 0, 0, 0] + if not name.endswith(".sql") and has_data_file: + table_info = tables[table_name] + if size > base_size: + table_info[0] += 1 + table_info[1] += size - base_size + if base_size > 0: + table_info[2] += 1 + table_info[3] += base_size + tables[table_name] = table_info + + database_name = extract_database_name_from_path(name) + if database_name: + databases.add(database_name) + + size_of_backup = size_of_files + os.path.getsize(backup_xml) + + print(f"version={version}") + print(f"timestamp={timestamp}") + print(f"base_backup={base_backup}") + print(f"size_of_backup={size_of_backup}") + print(f"number_of_files={number_of_files}") + print(f"size_of_files={size_of_files}") + print(f"number_of_files_from_base_backup={number_of_files_from_base_backup}") + print(f"size_of_files_from_base_backup={size_of_files_from_base_backup}") + print(f"number_of_databases={len(databases)}") + print(f"number_of_tables={len(tables)}") + + print() + + print(f"{len(databases)} database(s):") + for database_name in sorted(databases): + print(database_name) + + print() + + print(f"{len(tables)} table(s):") + table_info_format = "{:>70} | {:>20} | {:>20} | {:>26} | {:>30}" + table_info_separator_line = ( + "{:->70}-+-{:->20}-+-{:->20}-+-{:->26}-+-{:->30}".format("", "", "", "", "") + ) + table_info_title_line = table_info_format.format( + "table name", + "num_files", + "size_of_files", + "num_files_from_base_backup", + "size_of_files_from_base_backup", + ) + print(table_info_title_line) + print(table_info_separator_line) + for table_name in sorted(tables): + table_info = tables[table_name] + print( + table_info_format.format( + table_name, table_info[0], table_info[1], table_info[2], table_info[3] + ) + ) + + +# Extracts a table name from a path inside a backup. +# For example, extracts 'default.tbl' from 'shards/1/replicas/1/data/default/tbl/all_0_0_0/data.bin'. +def extract_table_name_from_path(path): + path = strip_shards_replicas_from_path(path) + if not path: + return None + if path.startswith("metadata/"): + path = path[len("metadata/") :] + sep = path.find("/") + if sep == -1: + return None + database_name = path[:sep] + path = path[sep + 1 :] + sep = path.find(".sql") + if sep == -1: + return None + table_name = path[:sep] + return database_name + "." + table_name + if path.startswith("data/"): + path = path[len("data/") :] + sep = path.find("/") + if sep == -1: + return None + database_name = path[:sep] + path = path[sep + 1 :] + sep = path.find("/") + if sep == -1: + return None + table_name = path[:sep] + return database_name + "." + table_name + return None + + +# Extracts a database name from a path inside a backup. +# For example, extracts 'default' from 'shards/1/replicas/1/data/default/tbl/all_0_0_0/data.bin'. +def extract_database_name_from_path(path): + path = strip_shards_replicas_from_path(path) + if not path: + return None + if path.startswith("metadata/"): + path = path[len("metadata/") :] + sep = path.find(".sql") + if sep == -1 or path.find("/") != -1: + return None + return path[:sep] + if path.startswith("data/"): + path = path[len("data/") :] + sep = path.find("/") + if sep == -1: + return None + return path[:sep] + return None + + +# Removes a prefix "shards//replicas//" from a path. +def strip_shards_replicas_from_path(path): + if path.startswith("shards"): + sep = path.find("/") + if sep == -1: + return None + sep = path.find("/", sep + 1) + if sep == -1: + return None + path = path[sep + 1 :] + if path.startswith("replicas"): + sep = path.find("/") + if sep == -1: + return None + sep = path.find("/", sep + 1) + if sep == -1: + return None + path = path[sep + 1 :] + return path + + +if __name__ == "__main__": + main() From 03f025ddd1352c264ceb716e731efc095f9cc0c3 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 27 Jul 2023 16:38:02 +0000 Subject: [PATCH 1245/2047] Fix typo --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 5dbe9e350bf..9fdc99bc760 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6530,7 +6530,7 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif array_join_column_expressions.push_back(std::move(array_join_column_node)); }; - // Support ARRAY JOIN COLUMNS(...). COLUMNS trasformer is resolved to list of columns. + // Support ARRAY JOIN COLUMNS(...). COLUMNS transformer is resolved to list of columns. if (auto * columns_list = array_join_expression->as()) { for (auto & array_join_subexpression : columns_list->getNodes()) From 380da315121078fc3e88a1e038e5aacd296853c2 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 27 Jul 2023 19:03:44 +0200 Subject: [PATCH 1246/2047] Improvements to backup restore disallow_concurrency test --- .../test_disallow_concurrency.py | 102 +++++++++++++----- 1 file changed, 73 insertions(+), 29 deletions(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index a863a6e2047..af1b2656227 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -133,21 +133,31 @@ def test_concurrent_backups_on_same_node(): ) assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] - try: - error = nodes[0].query_and_get_error( - f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" - ) - except Exception as e: + result, error = nodes[0].query_and_get_answer_with_error( + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" + ) + + if not error: status = ( nodes[0] .query(f"SELECT status FROM system.backups WHERE id == '{id}'") .rstrip("\n") ) # It is possible that the second backup was picked up first, and then the async backup - if status == "CREATING_BACKUP" or status == "BACKUP_FAILED": + if status == "BACKUP_FAILED": + return + elif status == "CREATING_BACKUP": + assert_eq_with_retry( + nodes[0], + f"SELECT status FROM system.backups WHERE id = '{id}'", + "BACKUP_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent backups both passed, when one is expected to fail") + expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", @@ -191,20 +201,31 @@ def test_concurrent_backups_on_different_nodes(): ) assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] - try: - error = nodes[0].query_and_get_error( + result, error = nodes[0].query_and_get_answer_with_error( f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" - ) - except Exception as e: + ) + + if not error: status = ( nodes[1] .query(f"SELECT status FROM system.backups WHERE id == '{id}'") .rstrip("\n") ) - if status == "CREATING_BACKUP" or status == "BACKUP_FAILED": + # It is possible that the second backup was picked up first, and then the async backup + if status == "BACKUP_FAILED": + return + elif status == "CREATING_BACKUP": + assert_eq_with_retry( + nodes[1], + f"SELECT status FROM system.backups WHERE id = '{id}'", + "BACKUP_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent backups both passed, when one is expected to fail") + expected_errors = [ "Concurrent backups not supported", f"Backup {backup_name} already exists", @@ -247,20 +268,32 @@ def test_concurrent_restores_on_same_node(): ) assert status in ["RESTORING", "RESTORED"] - try: - error = nodes[0].query_and_get_error( - f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" - ) - except Exception as e: + result, error = nodes[0].query_and_get_answer_with_error( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" + ) + + if not error: status = ( nodes[0] - .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .query(f"SELECT status FROM system.backups WHERE id == '{restore_id}'") .rstrip("\n") ) - if status == "RESTORING" or status == "RESTORE_FAILED": + # It is possible that the second backup was picked up first, and then the async backup + if status == "RESTORE_FAILED": + return + elif status == "RESTORING": + assert_eq_with_retry( + nodes[0], + f"SELECT status FROM system.backups WHERE id == '{restore_id}'", + "RESTORE_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent restores both passed, when one is expected to fail") + + expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", @@ -303,20 +336,31 @@ def test_concurrent_restores_on_different_node(): ) assert status in ["RESTORING", "RESTORED"] - try: - error = nodes[1].query_and_get_error( - f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" - ) - except Exception as e: + result, error = nodes[1].query_and_get_answer_with_error( + f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}" + ) + + if not error: status = ( nodes[0] - .query(f"SELECT status FROM system.backups WHERE id == '{id}'") + .query(f"SELECT status FROM system.backups WHERE id == '{restore_id}'") .rstrip("\n") ) - if status == "RESTORING" or status == "RESTORE_FAILED": + # It is possible that the second backup was picked up first, and then the async backup + if status == "RESTORE_FAILED": + return + elif status == "RESTORING": + assert_eq_with_retry( + nodes[0], + f"SELECT status FROM system.backups WHERE id == '{restore_id}'", + "RESTORE_FAILED", + sleep_time=2, + retry_count=50, + ) return else: - raise e + raise Exception("Concurrent restores both passed, when one is expected to fail") + expected_errors = [ "Concurrent restores not supported", "Cannot restore the table default.tbl because it already contains some data", From a8a48af7cbd20334af531e9f71e7f6005a098db1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 27 Jul 2023 19:17:57 +0200 Subject: [PATCH 1247/2047] Fix 02417_opentelemetry_insert_on_distributed_table flakiness Looks like everything is OK with opentelemetry, and the reason of the flakiness is this: $ gg opentelemetry_start_trace_probability tests/**.xml tests/config/users.d/opentelemetry.xml: 0.1 So let's simply disable it. And also let's stop the distributed sends to increase the failure rate if there is some problem left. Signed-off-by: Azat Khuzhin --- .../02417_opentelemetry_insert_on_distributed_table.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh index edc3d06e5bf..5a1e33a8459 100755 --- a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.sh @@ -20,7 +20,9 @@ function insert() -H "tracestate: $4" \ "${CLICKHOUSE_URL}" \ --data @- - ${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH DISTRIBUTED ${CLICKHOUSE_DATABASE}.dist_opentelemetry" + + # disable probabilistic tracing to avoid stealing the trace context + ${CLICKHOUSE_CLIENT} --opentelemetry_start_trace_probability=0 -q "SYSTEM FLUSH DISTRIBUTED ${CLICKHOUSE_DATABASE}.dist_opentelemetry" } function check_span() @@ -69,6 +71,8 @@ DROP TABLE IF EXISTS ${CLICKHOUSE_DATABASE}.local_opentelemetry; CREATE TABLE ${CLICKHOUSE_DATABASE}.dist_opentelemetry (key UInt64) Engine=Distributed('test_cluster_two_shards_localhost', ${CLICKHOUSE_DATABASE}, local_opentelemetry, key % 2); CREATE TABLE ${CLICKHOUSE_DATABASE}.local_opentelemetry (key UInt64) Engine=MergeTree ORDER BY key; + +SYSTEM STOP DISTRIBUTED SENDS ${CLICKHOUSE_DATABASE}.dist_opentelemetry; " # From d40dbdee6227cb0548f65abc900b1f7f91fd9959 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 27 Jun 2023 17:10:48 +0200 Subject: [PATCH 1248/2047] Fix usage of temporary directories during RESTORE Previously during RESTORE you may get the following warning in logs: TemporaryFileOnDisk: Temporary path 'tmp/21672kaaaaa' does not exist in './disks/s3_common_disk/' The reason is that it uses not disks/s3_common_disk/tmp but instead disks/s3_common_disk/disks/s3_common_disk/tmp path. Fix this by adding TemporaryFileOnDisk::getRelativePath() and use it when appropriate. And now - `find disks`, does not shows any more temporary leftovers. v2: rename getPath to getAbsolutePath Signed-off-by: Azat Khuzhin --- src/Disks/IO/WriteBufferFromTemporaryFile.cpp | 2 +- src/Disks/TemporaryFileOnDisk.cpp | 2 +- src/Disks/TemporaryFileOnDisk.h | 5 ++++- src/Interpreters/MergeJoin.cpp | 2 +- src/Interpreters/SortedBlocksWriter.cpp | 4 ++-- src/Interpreters/TemporaryDataOnDisk.cpp | 6 +++--- .../MergeTree/DataPartStorageOnDiskBase.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageMemory.cpp | 6 +++--- src/Storages/StorageStripeLog.cpp | 2 +- .../0_stateless/02801_backup_native_copy.sh | 4 ++-- .../02803_backup_tmp_files.reference | 0 .../0_stateless/02803_backup_tmp_files.sh | 18 ++++++++++++++++++ 14 files changed, 39 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/02803_backup_tmp_files.reference create mode 100755 tests/queries/0_stateless/02803_backup_tmp_files.sh diff --git a/src/Disks/IO/WriteBufferFromTemporaryFile.cpp b/src/Disks/IO/WriteBufferFromTemporaryFile.cpp index 5bfbb2fa440..de494c03789 100644 --- a/src/Disks/IO/WriteBufferFromTemporaryFile.cpp +++ b/src/Disks/IO/WriteBufferFromTemporaryFile.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes } WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(TemporaryFileOnDiskHolder && tmp_file_) - : WriteBufferFromFile(tmp_file_->getPath(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, /* throttler= */ {}, 0600) + : WriteBufferFromFile(tmp_file_->getAbsolutePath(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, /* throttler= */ {}, 0600) , tmp_file(std::move(tmp_file_)) { } diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index 8e5c8bcebbd..b9c9e238468 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -54,7 +54,7 @@ TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, const String & p throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file name is empty"); } -String TemporaryFileOnDisk::getPath() const +String TemporaryFileOnDisk::getAbsolutePath() const { return std::filesystem::path(disk->getPath()) / relative_path; } diff --git a/src/Disks/TemporaryFileOnDisk.h b/src/Disks/TemporaryFileOnDisk.h index bd82b9744ea..cccfc82cf9e 100644 --- a/src/Disks/TemporaryFileOnDisk.h +++ b/src/Disks/TemporaryFileOnDisk.h @@ -22,7 +22,10 @@ public: ~TemporaryFileOnDisk(); DiskPtr getDisk() const { return disk; } - String getPath() const; + /// Return absolute path (disk + relative_path) + String getAbsolutePath() const; + /// Return relative path (without disk) + const String & getRelativePath() const { return relative_path; } private: DiskPtr disk; diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index ceef1371f16..6f0c8f1bff4 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -1034,7 +1034,7 @@ std::shared_ptr MergeJoin::loadRightBlock(size_t pos) const { auto load_func = [&]() -> std::shared_ptr { - TemporaryFileStreamLegacy input(flushed_right_blocks[pos]->getPath(), materializeBlock(right_sample_block)); + TemporaryFileStreamLegacy input(flushed_right_blocks[pos]->getAbsolutePath(), materializeBlock(right_sample_block)); return std::make_shared(input.block_in->read()); }; diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index e09a66a38e6..c9f48ee1be9 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -39,7 +39,7 @@ namespace TemporaryFileOnDiskHolder flushToFile(const DiskPtr & disk, const Block & header, QueryPipelineBuilder pipeline, const String & codec) { auto tmp_file = std::make_unique(disk, CurrentMetrics::TemporaryFilesForJoin); - auto write_stat = TemporaryFileStreamLegacy::write(tmp_file->getPath(), header, std::move(pipeline), codec); + auto write_stat = TemporaryFileStreamLegacy::write(tmp_file->getAbsolutePath(), header, std::move(pipeline), codec); ProfileEvents::increment(ProfileEvents::ExternalProcessingCompressedBytesTotal, write_stat.compressed_bytes); ProfileEvents::increment(ProfileEvents::ExternalProcessingUncompressedBytesTotal, write_stat.uncompressed_bytes); @@ -267,7 +267,7 @@ SortedBlocksWriter::SortedFiles SortedBlocksWriter::finishMerge(std::function(file->getPath(), materializeBlock(sample_block))); + return Pipe(std::make_shared(file->getAbsolutePath(), materializeBlock(sample_block))); } diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 69fef21dbab..0aa2a0b9f4a 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -235,9 +235,9 @@ TemporaryFileStream::TemporaryFileStream(TemporaryFileOnDiskHolder file_, const : parent(parent_) , header(header_) , file(std::move(file_)) - , out_writer(std::make_unique(std::make_unique(file->getPath()), header)) + , out_writer(std::make_unique(std::make_unique(file->getAbsolutePath()), header)) { - LOG_TEST(&Poco::Logger::get("TemporaryFileStream"), "Writing to temporary file {}", file->getPath()); + LOG_TEST(&Poco::Logger::get("TemporaryFileStream"), "Writing to temporary file {}", file->getAbsolutePath()); } TemporaryFileStream::TemporaryFileStream(FileSegmentsHolderPtr segments_, const Block & header_, TemporaryDataOnDisk * parent_) @@ -365,7 +365,7 @@ void TemporaryFileStream::release() String TemporaryFileStream::getPath() const { if (file) - return file->getPath(); + return file->getAbsolutePath(); if (segment_holder && !segment_holder->empty()) return segment_holder->front().getPathInLocalCache(); diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index e1921f45eda..a0492f5f38e 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -350,7 +350,7 @@ void DataPartStorageOnDiskBase::backup( temp_dir_it = temp_dirs->emplace(disk, std::make_shared(disk, "tmp/")).first; temp_dir_owner = temp_dir_it->second; - fs::path temp_dir = temp_dir_owner->getPath(); + fs::path temp_dir = temp_dir_owner->getRelativePath(); temp_part_dir = temp_dir / part_path_in_backup.relative_path(); disk->createDirectories(temp_part_dir); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6179c70ca57..558f5b7515e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5266,7 +5266,7 @@ public: auto it = temp_dirs.find(disk); if (it == temp_dirs.end()) it = temp_dirs.emplace(disk, std::make_shared(disk, "tmp/")).first; - return it->second->getPath(); + return it->second->getRelativePath(); } private: diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index d8065b8bb3c..d02a51fab22 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -946,7 +946,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c fs::path data_path_in_backup_fs = data_path_in_backup; auto temp_dir_owner = std::make_shared(disk, "tmp/"); - fs::path temp_dir = temp_dir_owner->getPath(); + fs::path temp_dir = temp_dir_owner->getRelativePath(); disk->createDirectories(temp_dir); bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks; diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 6ed0583bd44..c990d488969 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -314,7 +314,7 @@ namespace backup_entries.resize(file_paths.size()); temp_dir_owner.emplace(temp_disk); - fs::path temp_dir = temp_dir_owner->getPath(); + fs::path temp_dir = temp_dir_owner->getRelativePath(); temp_disk->createDirectories(temp_dir); /// Writing data.bin @@ -453,10 +453,10 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat if (!dynamic_cast(in.get())) { temp_data_file.emplace(temporary_disk); - auto out = std::make_unique(temp_data_file->getPath()); + auto out = std::make_unique(temp_data_file->getAbsolutePath()); copyData(*in, *out); out.reset(); - in = createReadBufferFromFileBase(temp_data_file->getPath(), {}); + in = createReadBufferFromFileBase(temp_data_file->getAbsolutePath(), {}); } std::unique_ptr in_from_file{static_cast(in.release())}; CompressedReadBufferFromFile compressed_in{std::move(in_from_file)}; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index d8bbd523cbf..2f3b5f25ee4 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -544,7 +544,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec fs::path data_path_in_backup_fs = data_path_in_backup; auto temp_dir_owner = std::make_shared(disk, "tmp/"); - fs::path temp_dir = temp_dir_owner->getPath(); + fs::path temp_dir = temp_dir_owner->getRelativePath(); disk->createDirectories(temp_dir); bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks; diff --git a/tests/queries/0_stateless/02801_backup_native_copy.sh b/tests/queries/0_stateless/02801_backup_native_copy.sh index 31a7cc3b410..b8ee97a7c7d 100755 --- a/tests/queries/0_stateless/02801_backup_native_copy.sh +++ b/tests/queries/0_stateless/02801_backup_native_copy.sh @@ -29,14 +29,14 @@ $CLICKHOUSE_CLIENT -nm -q " " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS allow_s3_native_copy=true" +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS allow_s3_native_copy=true" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS allow_s3_native_copy=false" +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS allow_s3_native_copy=false" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' diff --git a/tests/queries/0_stateless/02803_backup_tmp_files.reference b/tests/queries/0_stateless/02803_backup_tmp_files.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02803_backup_tmp_files.sh b/tests/queries/0_stateless/02803_backup_tmp_files.sh new file mode 100755 index 00000000000..d86beae4923 --- /dev/null +++ b/tests/queries/0_stateless/02803_backup_tmp_files.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag: no-fasttest - requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists data; + create table data (key Int) engine=MergeTree() order by tuple() settings disk='s3_disk'; + insert into data select * from numbers(10); +" + +$CLICKHOUSE_CLIENT --format Null -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data')" +$CLICKHOUSE_CLIENT --format Null -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data')" From 2717be7c476d9d7c3be841bbefdcd8688a76e7fe Mon Sep 17 00:00:00 2001 From: Anton Kozlov Date: Thu, 27 Jul 2023 17:34:53 +0000 Subject: [PATCH 1249/2047] [minor][bugfix] fix connected_zk_index column of system.zookeeper_connection table --- src/Common/ZooKeeper/IKeeper.h | 2 -- src/Common/ZooKeeper/TestKeeper.h | 3 --- src/Common/ZooKeeper/ZooKeeper.cpp | 26 ++++++-------------------- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 14 ++++++++------ src/Common/ZooKeeper/ZooKeeperImpl.h | 8 ++++---- 5 files changed, 18 insertions(+), 35 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 5240acc2616..2b2a043d389 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -490,8 +490,6 @@ public: /// Useful to check owner of ephemeral node. virtual int64_t getSessionID() const = 0; - virtual Poco::Net::SocketAddress getConnectedAddress() const = 0; - /// If the method will throw an exception, callbacks won't be called. /// /// After the method is executed successfully, you must wait for callbacks diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 8615ed0fb77..69840cbeff6 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -40,7 +40,6 @@ public: bool isExpired() const override { return expired; } int64_t getSessionID() const override { return 0; } - Poco::Net::SocketAddress getConnectedAddress() const override { return connected_zk_address; } void create( @@ -135,8 +134,6 @@ private: zkutil::ZooKeeperArgs args; - Poco::Net::SocketAddress connected_zk_address; - std::mutex push_request_mutex; std::atomic expired{false}; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 7a8088c960b..0fe536b1a08 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -112,31 +112,17 @@ void ZooKeeper::init(ZooKeeperArgs args_) throw KeeperException("Cannot use any of provided ZooKeeper nodes", Coordination::Error::ZCONNECTIONLOSS); } - impl = std::make_unique(nodes, args, zk_log); + impl = std::make_unique(nodes, args, zk_log, [this](size_t node_idx, const Coordination::ZooKeeper::Node & node) + { + connected_zk_host = node.address.host().toString(); + connected_zk_port = node.address.port(); + connected_zk_index = node_idx; + }); if (args.chroot.empty()) LOG_TRACE(log, "Initialized, hosts: {}", fmt::join(args.hosts, ",")); else LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(args.hosts, ","), args.chroot); - - Poco::Net::SocketAddress address = impl->getConnectedAddress(); - - connected_zk_host = address.host().toString(); - connected_zk_port = address.port(); - - connected_zk_index = 0; - - if (args.hosts.size() > 1) - { - for (size_t i = 0; i < args.hosts.size(); i++) - { - if (args.hosts[i] == address.toString()) - { - connected_zk_index = i; - break; - } - } - } } else if (args.implementation == "testkeeper") { diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 5e16a437be3..74b0b039ca9 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -313,8 +313,8 @@ ZooKeeper::~ZooKeeper() ZooKeeper::ZooKeeper( const Nodes & nodes, const zkutil::ZooKeeperArgs & args_, - std::shared_ptr zk_log_) - : args(args_) + std::shared_ptr zk_log_, std::optional && connected_callback_) + : args(args_), connected_callback(std::move(connected_callback_)) { log = &Poco::Logger::get("ZooKeeperClient"); std::atomic_store(&zk_log, std::move(zk_log_)); @@ -395,8 +395,9 @@ void ZooKeeper::connect( WriteBufferFromOwnString fail_reasons; for (size_t try_no = 0; try_no < num_tries; ++try_no) { - for (const auto & node : nodes) + for (size_t i = 0; i < nodes.size(); ++i) { + const auto & node = nodes[i]; try { /// Reset the state of previous attempt. @@ -443,9 +444,11 @@ void ZooKeeper::connect( e.addMessage("while receiving handshake from ZooKeeper"); throw; } - connected = true; - connected_zk_address = node.address; + + if (connected_callback.has_value()) + (*connected_callback)(i, node); + break; } @@ -462,7 +465,6 @@ void ZooKeeper::connect( if (!connected) { WriteBufferFromOwnString message; - connected_zk_address = Poco::Net::SocketAddress(); message << "All connection tries failed while connecting to ZooKeeper. nodes: "; bool first = true; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 7e27608d0a1..3684b215144 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -107,6 +107,7 @@ public: }; using Nodes = std::vector; + using ConnectedCallback = std::function; /** Connection to nodes is performed in order. If you want, shuffle them manually. * Operation timeout couldn't be greater than session timeout. @@ -115,7 +116,8 @@ public: ZooKeeper( const Nodes & nodes, const zkutil::ZooKeeperArgs & args_, - std::shared_ptr zk_log_); + std::shared_ptr zk_log_, + std::optional && connected_callback_ = {}); ~ZooKeeper() override; @@ -126,8 +128,6 @@ public: /// Useful to check owner of ephemeral node. int64_t getSessionID() const override { return session_id; } - Poco::Net::SocketAddress getConnectedAddress() const override { return connected_zk_address; } - void executeGenericRequest( const ZooKeeperRequestPtr & request, ResponseCallback callback); @@ -213,9 +213,9 @@ public: private: ACLs default_acls; - Poco::Net::SocketAddress connected_zk_address; zkutil::ZooKeeperArgs args; + std::optional connected_callback = {}; /// Fault injection void maybeInjectSendFault(); From 9e993f7011c9837ea4c3c52ee7f92243f940dc7f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 27 Jul 2023 19:37:29 +0200 Subject: [PATCH 1250/2047] Update 02812_pointwise_array_operations.reference --- .../0_stateless/02812_pointwise_array_operations.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02812_pointwise_array_operations.reference b/tests/queries/0_stateless/02812_pointwise_array_operations.reference index 92fb7d504e7..54274d0380d 100644 --- a/tests/queries/0_stateless/02812_pointwise_array_operations.reference +++ b/tests/queries/0_stateless/02812_pointwise_array_operations.reference @@ -2,4 +2,3 @@ [2,6] [4.5,5,12,10.1] [(11.1,5.4),(6,21)] - From b98cb7fa145d1a92c2c78421be1eeb8fe8353d53 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 17:50:39 +0000 Subject: [PATCH 1251/2047] Fix build --- src/IO/ReadHelpers.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 1fe08a738c2..bbb1c517d7c 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1101,6 +1101,9 @@ inline void readBinary(Decimal64 & x, ReadBuffer & buf) { readPODBinary(x, buf); inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value, buf); } inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(UUID & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(IPv4 & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(IPv6 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(StackTrace::FramePointers & x, ReadBuffer & buf) { readPODBinary(x, buf); } From b95745d916330abb3306016d512b40d3d24616dd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 17:52:44 +0000 Subject: [PATCH 1252/2047] fix: check positional options --- src/Client/ClientBase.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 496fc8fce0a..06dabf96c28 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2624,6 +2624,10 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); } + /// Check positional options. + if (std::ranges::count_if(parsed.options, [](const auto & op){ return !op.unregistered && op.string_key.empty() && !op.original_tokens[0].starts_with("--"); }) > 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional options are not supported."); + po::store(parsed, options); } From 6687f37329318c6995d795f8069ca5123e7bcf61 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 27 Jul 2023 18:11:52 +0000 Subject: [PATCH 1253/2047] added test --- .../System/attachInformationSchemaTables.cpp | 8 ++++++++ .../test_from_system_tables.py | 12 ++++++++++++ 2 files changed, 20 insertions(+) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 12cef89b553..9f8056e35a5 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -49,6 +49,14 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d database.getTableDataPath(ast_create), context, true).second; database.createTable(context, ast_create.getTable(), view, ast); + if (ast_create.getTable() == "tables") + { + database.createTable(context, Poco::toUpper(ast_create.getTable()), view, ast); + } + else if (ast_create.getTable() == "TABLES") + { + database.createTable(context, Poco::toLower(ast_create.getTable()), view, ast); + } } catch (...) { diff --git a/tests/integration/test_select_access_rights/test_from_system_tables.py b/tests/integration/test_select_access_rights/test_from_system_tables.py index ac938a9694a..ccdd09c67ef 100644 --- a/tests/integration/test_select_access_rights/test_from_system_tables.py +++ b/tests/integration/test_select_access_rights/test_from_system_tables.py @@ -190,3 +190,15 @@ def test_information_schema(): ) == "1\n" ) + assert ( + node.query( + "SELECT count() FROM information_schema.TABLES WHERE table_name='TABLES'" + ) + == "3\n" + ) + assert ( + node.query( + "SELECT count() FROM INFORMATION_SCHEMA.tables WHERE table_name='tables'" + ) + == "3\n" + ) From 017f4ea07ebf8a030973ac2b306ed3a4442c7517 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 27 Jul 2023 18:13:15 +0000 Subject: [PATCH 1254/2047] Revert "added test" This reverts commit 6687f37329318c6995d795f8069ca5123e7bcf61. --- .../System/attachInformationSchemaTables.cpp | 8 -------- .../test_from_system_tables.py | 12 ------------ 2 files changed, 20 deletions(-) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 9f8056e35a5..12cef89b553 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -49,14 +49,6 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d database.getTableDataPath(ast_create), context, true).second; database.createTable(context, ast_create.getTable(), view, ast); - if (ast_create.getTable() == "tables") - { - database.createTable(context, Poco::toUpper(ast_create.getTable()), view, ast); - } - else if (ast_create.getTable() == "TABLES") - { - database.createTable(context, Poco::toLower(ast_create.getTable()), view, ast); - } } catch (...) { diff --git a/tests/integration/test_select_access_rights/test_from_system_tables.py b/tests/integration/test_select_access_rights/test_from_system_tables.py index ccdd09c67ef..ac938a9694a 100644 --- a/tests/integration/test_select_access_rights/test_from_system_tables.py +++ b/tests/integration/test_select_access_rights/test_from_system_tables.py @@ -190,15 +190,3 @@ def test_information_schema(): ) == "1\n" ) - assert ( - node.query( - "SELECT count() FROM information_schema.TABLES WHERE table_name='TABLES'" - ) - == "3\n" - ) - assert ( - node.query( - "SELECT count() FROM INFORMATION_SCHEMA.tables WHERE table_name='tables'" - ) - == "3\n" - ) From da53bca6e4650b6d67737e6efd1b95232bf4ffc7 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 27 Jul 2023 18:29:50 +0000 Subject: [PATCH 1255/2047] added tests --- .../System/attachInformationSchemaTables.cpp | 8 ++++++++ .../test_from_system_tables.py | 12 ++++++++++++ 2 files changed, 20 insertions(+) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 61a91685324..6f378671104 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -41,6 +41,14 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d database.getTableDataPath(ast_create), context, true).second; database.createTable(context, ast_create.getTable(), view, ast); + if (ast_create.getTable() == "tables") + { + database.createTable(context, Poco::toUpper(ast_create.getTable()), view, ast); + } + else if (ast_create.getTable() == "TABLES") + { + database.createTable(context, Poco::toLower(ast_create.getTable()), view, ast); + } } catch (...) { diff --git a/tests/integration/test_select_access_rights/test_from_system_tables.py b/tests/integration/test_select_access_rights/test_from_system_tables.py index ac938a9694a..ccdd09c67ef 100644 --- a/tests/integration/test_select_access_rights/test_from_system_tables.py +++ b/tests/integration/test_select_access_rights/test_from_system_tables.py @@ -190,3 +190,15 @@ def test_information_schema(): ) == "1\n" ) + assert ( + node.query( + "SELECT count() FROM information_schema.TABLES WHERE table_name='TABLES'" + ) + == "3\n" + ) + assert ( + node.query( + "SELECT count() FROM INFORMATION_SCHEMA.tables WHERE table_name='tables'" + ) + == "3\n" + ) From 9340f02d26ae7f170611ea9b19a11e720b41b765 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 18:33:07 +0000 Subject: [PATCH 1256/2047] Silence spell check --- .../aspell-ignore/en/aspell-dict.txt | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a314815e2c4..80aeadd8738 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -211,7 +211,6 @@ Decrypted Deduplicate Deduplication DelayedInserts -delim DeliveryTag DeltaLake Denormalize @@ -699,6 +698,8 @@ PyCharm QEMU QTCreator Quantile +QueryCacheBytes +QueryCacheEntries QueryCacheHits QueryCacheMisses QueryPreempted @@ -761,9 +762,9 @@ RoaringBitmap RocksDB Rollup RowBinary +RowBinaryWithDefaults RowBinaryWithNames RowBinaryWithNamesAndTypes -RowBinaryWithDefaults Runtime SATA SELECTs @@ -776,7 +777,6 @@ SMALLINT SPNEGO SQEs SQLAlchemy -SquaredDistance SQLConsoleDetail SQLInsert SQLSTATE @@ -811,6 +811,7 @@ Smirnov'test Soundex SpanKind Spearman's +SquaredDistance StartTLS StartTime StartupSystemTables @@ -838,8 +839,6 @@ Subexpression Submodules Subqueries Substrings -substringIndex -substringIndexUTF SummingMergeTree SuperSet Superset @@ -1272,6 +1271,7 @@ cryptographic csv csvwithnames csvwithnamesandtypes +curdate currentDatabase currentProfiles currentRoles @@ -1331,6 +1331,7 @@ defaultProfiles defaultRoles defaultValueOfArgumentType defaultValueOfTypeName +delim deltaLake deltaSum deltaSumTimestamp @@ -1542,13 +1543,13 @@ hadoop halfMD halfday hardlinks +hasAll +hasAny +hasColumnInTable hasSubsequence hasSubsequenceCaseInsensitive hasSubsequenceCaseInsensitiveUTF hasSubsequenceUTF -hasAll -hasAny -hasColumnInTable hasSubstr hasToken hasTokenCaseInsensitive @@ -1590,10 +1591,10 @@ incrementing indexHint indexOf infi -initialQueryID -initializeAggregation initcap initcapUTF +initialQueryID +initializeAggregation injective innogames inodes @@ -2131,9 +2132,9 @@ routineley rowNumberInAllBlocks rowNumberInBlock rowbinary +rowbinarywithdefaults rowbinarywithnames rowbinarywithnamesandtypes -rowbinarywithdefaults rsync rsyslog runnable @@ -2185,8 +2186,8 @@ sleepEachRow snowflakeToDateTime socketcache soundex -sparkbar sparkBar +sparkbar sparsehash speedscope splitByChar @@ -2256,6 +2257,8 @@ subreddits subseconds subsequence substring +substringIndex +substringIndexUTF substringUTF substrings subtitiles @@ -2556,4 +2559,3 @@ znode znodes zookeeperSessionUptime zstd -curdate From 67b0993bdf8a5de1dd3a68db5c5082ba0a3ff759 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 27 Jul 2023 18:54:41 +0000 Subject: [PATCH 1257/2047] Add documentation --- docs/en/interfaces/formats.md | 55 ++++++ .../operations/settings/settings-formats.md | 14 ++ .../functions/other-functions.md | 184 ++++++++++++++++++ 3 files changed, 253 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ed2f010a632..f80dac1d110 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1704,6 +1704,34 @@ You can select data from a ClickHouse table and save them into some file in the ``` bash $ clickhouse-client --query = "SELECT * FROM test.hits FORMAT CapnProto SETTINGS format_schema = 'schema:Message'" ``` + +### Using autogenerated schema {#using-autogenerated-capn-proto-schema} + +If you don't have an external CapnProto schema for your data, you can still output/input data in CapnProto format by using autogenerated schema. +For example: + +```sql +SELECT * FROM test.hits format CapnProto SETTINGS format_capn_proto_use_autogenerated_schema=1 +``` + +In this case ClickHouse will autogenerate CapnProto schema according to the table structure using function [structureToCapnProtoSchema](../sql-reference/functions/other-functions.md#structure_to_capn_proto_schema) and will use this schema to serialize data in CapnProto format. + +You can also read CapnProto file with autogenerated schema (in this case the file must be created using the same schema): + +```bash +$ cat hits.bin | clickhouse-client --query "INSERT INTO test.hits SETTINGS format_capn_proto_use_autogenerated_schema=1 FORMAT CapnProto" +``` + +The setting [format_capn_proto_use_autogenerated_schema](../operations/settings/settings-formats.md#format_capn_proto_use_autogenerated_schema) is enabled by default and applies if [format_schema](../operations/settings/settings-formats.md#formatschema-format-schema) is not set. + +You can also save autogenerated schema in the file during input/output using setting [output_format_schema](../operations/settings/settings-formats.md#outputformatschema-output-format-schema). For example: + +```sql +SELECT * FROM test.hits format CapnProto SETTINGS format_capn_proto_use_autogenerated_schema=1, output_format_schema='path/to/schema/schema.capnp' +``` + +In this case autogenerated CapnProto schema will be saved in file `path/to/schema/schema.capnp`. + ## Prometheus {#prometheus} Expose metrics in [Prometheus text-based exposition format](https://prometheus.io/docs/instrumenting/exposition_formats/#text-based-format). @@ -1842,6 +1870,33 @@ ClickHouse inputs and outputs protobuf messages in the `length-delimited` format It means before every message should be written its length as a [varint](https://developers.google.com/protocol-buffers/docs/encoding#varints). See also [how to read/write length-delimited protobuf messages in popular languages](https://cwiki.apache.org/confluence/display/GEODE/Delimiting+Protobuf+Messages). +### Using autogenerated schema {#using-autogenerated-protobuf-schema} + +If you don't have an external Protobuf schema for your data, you can still output/input data in Protobuf format by using autogenerated schema. +For example: + +```sql +SELECT * FROM test.hits format Protobuf SETTINGS format_protobuf_use_autogenerated_schema=1 +``` + +In this case ClickHouse will autogenerate Protobuf schema according to the table structure using function [structureToProtobufSchema](../sql-reference/functions/other-functions.md#structure_to_protobuf_schema) and will use this schema to serialize data in Protobuf format. + +You can also read Protobuf file with autogenerated schema (in this case the file must be created using the same schema): + +```bash +$ cat hits.bin | clickhouse-client --query "INSERT INTO test.hits SETTINGS format_protobuf_use_autogenerated_schema=1 FORMAT Protobuf" +``` + +The setting [format_protobuf_use_autogenerated_schema](../operations/settings/settings-formats.md#format_protobuf_use_autogenerated_schema) is enabled by default and applies if [format_schema](../operations/settings/settings-formats.md#formatschema-format-schema) is not set. + +You can also save autogenerated schema in the file during input/output using setting [output_format_schema](../operations/settings/settings-formats.md#outputformatschema-output-format-schema). For example: + +```sql +SELECT * FROM test.hits format Protobuf SETTINGS format_protobuf_use_autogenerated_schema=1, output_format_schema='path/to/schema/schema.proto' +``` + +In this case autogenerated Protobuf schema will be saved in file `path/to/schema/schema.capnp`. + ## ProtobufSingle {#protobufsingle} Same as [Protobuf](#protobuf) but for storing/parsing single Protobuf message without length delimiters. diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 0915c51806a..b54669cf8a8 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -321,6 +321,10 @@ If both `input_format_allow_errors_num` and `input_format_allow_errors_ratio` ar This parameter is useful when you are using formats that require a schema definition, such as [Cap’n Proto](https://capnproto.org/) or [Protobuf](https://developers.google.com/protocol-buffers/). The value depends on the format. +## output_format_schema {#output-format-schema} + +The path to the file where the automatically generated schema will be saved in [Cap’n Proto](../../interfaces/formats.md#capnproto-capnproto) or [Protobuf](../../interfaces/formats.md#protobuf-protobuf) formats. + ## output_format_enable_streaming {#output_format_enable_streaming} Enable streaming in output formats that support it. @@ -1308,6 +1312,11 @@ When serializing Nullable columns with Google wrappers, serialize default values Disabled by default. +### format_protobuf_use_autogenerated_schema {#format_capn_proto_use_autogenerated_schema} + +Use autogenerated Protobuf schema when [format_schema](#formatschema-format-schema) is not set. +The schema is generated from ClickHouse table structure using function [structureToProtobufSchema](../../sql-reference/functions/other-functions.md#structure_to_protobuf_schema) + ## Avro format settings {#avro-format-settings} ### input_format_avro_allow_missing_fields {#input_format_avro_allow_missing_fields} @@ -1593,6 +1602,11 @@ Possible values: Default value: `'by_values'`. +### format_capn_proto_use_autogenerated_schema {#format_capn_proto_use_autogenerated_schema} + +Use autogenerated CapnProto schema when [format_schema](#formatschema-format-schema) is not set. +The schema is generated from ClickHouse table structure using function [structureToCapnProtoSchema](../../sql-reference/functions/other-functions.md#structure_to_capnproto_schema) + ## MySQLDump format settings {#musqldump-format-settings} ### input_format_mysql_dump_table_name (#input_format_mysql_dump_table_name) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 26dcccfd42e..bfbd26551d3 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2552,3 +2552,187 @@ Result: This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables. +## structureToCapnProtoSchema {#structure_to_capn_proto_schema} + +Converts ClickHouse table structure to CapnProto schema. + +**Syntax** + +``` sql +structureToCapnProtoSchema(structure) +``` + +**Arguments** + +- `structure` — Table structure in a format `column1_name column1_type, column2_name column2_type, ...`. +- `root_struct_name` — Name for root struct in CapnProto schema. Default value - `Message`; + +**Returned value** + +- CapnProto schema + +Type: [String](../../sql-reference/data-types/string.md). + +**Examples** + +Query: + +``` sql +SELECT structureToCapnProtoSchema('column1 String, column2 UInt32, column3 Array(String)') FORMAT RawBLOB +``` + +Result: + +``` text +@0xf96402dd754d0eb7; + +struct Message +{ + column1 @0 : Data; + column2 @1 : UInt32; + column3 @2 : List(Data); +} +``` + +Query: + +``` sql +SELECT structureToCapnProtoSchema('column1 Nullable(String), column2 Tuple(element1 UInt32, element2 Array(String)), column3 Map(String, String)') FORMAT RawBLOB +``` + +Result: + +``` text +@0xd1c8320fecad2b7f; + +struct Message +{ + struct Column1 + { + union + { + value @0 : Data; + null @1 : Void; + } + } + column1 @0 : Column1; + struct Column2 + { + element1 @0 : UInt32; + element2 @1 : List(Data); + } + column2 @1 : Column2; + struct Column3 + { + struct Entry + { + key @0 : Data; + value @1 : Data; + } + entries @0 : List(Entry); + } + column3 @2 : Column3; +} +``` + +Query: + +``` sql +SELECT structureToCapnProtoSchema('column1 String, column2 UInt32', 'Root') FORMAT RawBLOB +``` + +Result: + +``` text +@0x96ab2d4ab133c6e1; + +struct Root +{ + column1 @0 : Data; + column2 @1 : UInt32; +} +``` + +## structureToProtobufSchema {#structure_to_protobuf_schema} + +Converts ClickHouse table structure to Protobuf schema. + +**Syntax** + +``` sql +structureToProtobufSchema(structure) +``` + +**Arguments** + +- `structure` — Table structure in a format `column1_name column1_type, column2_name column2_type, ...`. +- `root_message_name` — Name for root message in Protobuf schema. Default value - `Message`; + +**Returned value** + +- Protobuf schema + +Type: [String](../../sql-reference/data-types/string.md). + +**Examples** + +Query: + +``` sql +SELECT structureToProtobufSchema('column1 String, column2 UInt32, column3 Array(String)') FORMAT RawBLOB +``` + +Result: + +``` text +syntax = "proto3"; + +message Message +{ + bytes column1 = 1; + uint32 column2 = 2; + repeated bytes column3 = 3; +} +``` + +Query: + +``` sql +SELECT structureToProtobufSchema('column1 Nullable(String), column2 Tuple(element1 UInt32, element2 Array(String)), column3 Map(String, String)') FORMAT RawBLOB +``` + +Result: + +``` text +syntax = "proto3"; + +message Message +{ + bytes column1 = 1; + message Column2 + { + uint32 element1 = 1; + repeated bytes element2 = 2; + } + Column2 column2 = 2; + map column3 = 3; +} +``` + +Query: + +``` sql +SELECT structureToProtobufSchema('column1 String, column2 UInt32', 'Root') FORMAT RawBLOB +``` + +Result: + +``` text +syntax = "proto3"; + +message Root +{ + bytes column1 = 1; + uint32 column2 = 2; +} +``` From b9c9933cc9eec7024e2af9085fff0e365ae6bad8 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 27 Jul 2023 18:56:23 +0000 Subject: [PATCH 1258/2047] Fix typo --- docs/en/interfaces/formats.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index f80dac1d110..a618de9e12b 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1707,7 +1707,7 @@ $ clickhouse-client --query = "SELECT * FROM test.hits FORMAT CapnProto SETTINGS ### Using autogenerated schema {#using-autogenerated-capn-proto-schema} -If you don't have an external CapnProto schema for your data, you can still output/input data in CapnProto format by using autogenerated schema. +If you don't have an external CapnProto schema for your data, you can still output/input data in CapnProto format using autogenerated schema. For example: ```sql @@ -1872,7 +1872,7 @@ See also [how to read/write length-delimited protobuf messages in popular langua ### Using autogenerated schema {#using-autogenerated-protobuf-schema} -If you don't have an external Protobuf schema for your data, you can still output/input data in Protobuf format by using autogenerated schema. +If you don't have an external Protobuf schema for your data, you can still output/input data in Protobuf format using autogenerated schema. For example: ```sql From d1f3849d991e3115c3ded4d97ea841cc724a0e22 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Thu, 27 Jul 2023 22:08:36 +0300 Subject: [PATCH 1259/2047] Fix errata at named collections sql-reference --- docs/en/sql-reference/statements/create/named-collection.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/named-collection.md b/docs/en/sql-reference/statements/create/named-collection.md index e30a8cd6df2..1fc7b11c554 100644 --- a/docs/en/sql-reference/statements/create/named-collection.md +++ b/docs/en/sql-reference/statements/create/named-collection.md @@ -23,7 +23,7 @@ key_name3 = 'some value', CREATE NAMED COLLECTION foobar AS a = '1', b = '2'; ``` -**Related satements** +**Related statements** - [CREATE NAMED COLLECTION](https://clickhouse.com/docs/en/sql-reference/statements/alter/named-collection) - [DROP NAMED COLLECTION](https://clickhouse.com/docs/en/sql-reference/statements/drop#drop-function) From ae4406cd807f4a94a8072d2872366d60f40a2056 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Thu, 27 Jul 2023 22:11:07 +0300 Subject: [PATCH 1260/2047] Fix named collections AST formatters (if [not] exists) --- src/Parsers/ASTAlterNamedCollectionQuery.cpp | 2 ++ src/Parsers/ASTCreateNamedCollectionQuery.cpp | 2 ++ src/Parsers/ASTDropNamedCollectionQuery.cpp | 2 ++ 3 files changed, 6 insertions(+) diff --git a/src/Parsers/ASTAlterNamedCollectionQuery.cpp b/src/Parsers/ASTAlterNamedCollectionQuery.cpp index 7e95147ad75..6363a7306bd 100644 --- a/src/Parsers/ASTAlterNamedCollectionQuery.cpp +++ b/src/Parsers/ASTAlterNamedCollectionQuery.cpp @@ -15,6 +15,8 @@ ASTPtr ASTAlterNamedCollectionQuery::clone() const void ASTAlterNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const { settings.ostr << (settings.hilite ? hilite_keyword : "") << "Alter NAMED COLLECTION "; + if (if_exists) + settings.ostr << "IF EXISTS "; settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : ""); formatOnCluster(settings); if (!changes.empty()) diff --git a/src/Parsers/ASTCreateNamedCollectionQuery.cpp b/src/Parsers/ASTCreateNamedCollectionQuery.cpp index 97e83541f05..45ef8565148 100644 --- a/src/Parsers/ASTCreateNamedCollectionQuery.cpp +++ b/src/Parsers/ASTCreateNamedCollectionQuery.cpp @@ -18,6 +18,8 @@ ASTPtr ASTCreateNamedCollectionQuery::clone() const void ASTCreateNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const { settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE NAMED COLLECTION "; + if (if_not_exists) + settings.ostr << "IF NOT EXISTS "; settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : ""); formatOnCluster(settings); diff --git a/src/Parsers/ASTDropNamedCollectionQuery.cpp b/src/Parsers/ASTDropNamedCollectionQuery.cpp index 3b8568cfd70..e317681d33d 100644 --- a/src/Parsers/ASTDropNamedCollectionQuery.cpp +++ b/src/Parsers/ASTDropNamedCollectionQuery.cpp @@ -13,6 +13,8 @@ ASTPtr ASTDropNamedCollectionQuery::clone() const void ASTDropNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const { settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP NAMED COLLECTION "; + if (if_exists) + settings.ostr << "IF EXISTS "; settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : ""); formatOnCluster(settings); } From 5942c80faed38febea5394526b5e5c670b03bd4d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 27 Jul 2023 19:11:44 +0000 Subject: [PATCH 1261/2047] fix test: '--option' now is allowed after terminating '--' --- .../02096_bad_options_in_client_and_local.reference | 2 -- .../0_stateless/02096_bad_options_in_client_and_local.sh | 4 ---- 2 files changed, 6 deletions(-) diff --git a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference index c4c0901b9df..432299e9556 100644 --- a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference +++ b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference @@ -8,5 +8,3 @@ OK OK OK OK -OK -OK diff --git a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh index d37155e8506..753d56fb424 100755 --- a/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh +++ b/tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh @@ -9,8 +9,6 @@ ${CLICKHOUSE_LOCAL} --unknown-option 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" ${CLICKHOUSE_LOCAL} --unknown-option-1 --unknown-option-2 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" && echo "OK" || echo "FAIL" -${CLICKHOUSE_LOCAL} -- --unknown-option 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" - ${CLICKHOUSE_LOCAL} -- 'positional-argument' 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" ${CLICKHOUSE_LOCAL} -f 2>&1 | grep -F -q "Bad arguments" && echo "OK" || echo "FAIL" @@ -22,8 +20,6 @@ ${CLICKHOUSE_CLIENT} --unknown-option 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" ${CLICKHOUSE_CLIENT} --unknown-option-1 --unknown-option-2 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" && echo "OK" || echo "FAIL" -${CLICKHOUSE_CLIENT} -- --unknown-option 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" - ${CLICKHOUSE_CLIENT} -- 'positional-argument' 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" ${CLICKHOUSE_CLIENT} --j 2>&1 | grep -F -q "Bad arguments" && echo "OK" || echo "FAIL" From eb74e658b7dac9cbfbd4027d2281ab4fc5c173a1 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 27 Jul 2023 19:12:06 +0000 Subject: [PATCH 1262/2047] Fix build --- src/Server/ServerType.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Server/ServerType.h b/src/Server/ServerType.h index 8fb81622ab3..1fab492222a 100644 --- a/src/Server/ServerType.h +++ b/src/Server/ServerType.h @@ -33,7 +33,8 @@ public: static const char * serverTypeToString(Type type); - bool shouldStart(Type server_type, const std::string & custom_name_ = "") const; + /// Checks whether provided in the arguments type should be started or stopped based on current server type. + bool shouldStart(Type server_type, const std::string & server_custom_name = "") const; bool shouldStop(const std::string & port_name) const; Type type; From 32d23b09049429512b7a20d32e7bda6a80685fcb Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 27 Jul 2023 21:28:28 +0200 Subject: [PATCH 1263/2047] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 6c88d63be49..3737f2264bf 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1081,6 +1081,7 @@ autocompletion autodetect autodetected autogenerated +autogenerate autogeneration autostart avgWeighted @@ -2217,6 +2218,8 @@ strtoll strtoull struct structs +structureToCapnProtoSchema +structureToProtobufSchema studentTTest studentttest subBitmap From dba8b445bd37b2fb9fb4983e0a3f740649dcbb5b Mon Sep 17 00:00:00 2001 From: Jai Jhala Date: Thu, 27 Jul 2023 12:32:53 -0700 Subject: [PATCH 1264/2047] Update default output_format_arrow_compression.md Updates the default parameter of output_format_arrow_compression_method from "none" to "lz4_frame". --- docs/en/operations/settings/settings-formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index ee8e0d547b8..fb10ff7f61b 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1164,7 +1164,7 @@ Enabled by default. Compression method used in output Arrow format. Supported codecs: `lz4_frame`, `zstd`, `none` (uncompressed) -Default value: `none`. +Default value: `lz4_frame`. ## ORC format settings {#orc-format-settings} From 4629ab1df1cf6e5de3577ff94f7bd600207f24b6 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 27 Jul 2023 23:44:32 +0400 Subject: [PATCH 1265/2047] add test with broken pipe --- .../integration/helpers/s3_mocks/broken_s3.py | 80 +++++++++++++------ .../test_checking_s3_blobs_paranoid/test.py | 77 ++++++++++++++++-- 2 files changed, 126 insertions(+), 31 deletions(-) diff --git a/tests/integration/helpers/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py index 6e1572af262..206f960293f 100644 --- a/tests/integration/helpers/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -37,9 +37,7 @@ class MockControl: ) assert response == "OK", response - def setup_action( - self, when, count=None, after=None, action="error_500", action_args=None - ): + def setup_action(self, when, count=None, after=None, action=None, action_args=None): url = f"http://localhost:{self._port}/mock_settings/{when}?nothing=1" if count is not None: @@ -128,8 +126,14 @@ class MockControl: class _ServerRuntime: class SlowPut: def __init__( - self, probability_=None, timeout_=None, minimal_length_=None, count_=None + self, + lock, + probability_=None, + timeout_=None, + minimal_length_=None, + count_=None, ): + self.lock = lock self.probability = probability_ if probability_ is not None else 1 self.timeout = timeout_ if timeout_ is not None else 0.1 self.minimal_length = minimal_length_ if minimal_length_ is not None else 0 @@ -144,14 +148,15 @@ class _ServerRuntime: ) def get_timeout(self, content_length): - if content_length > self.minimal_length: - if self.count > 0: - if ( - _runtime.slow_put.probability == 1 - or random.random() <= _runtime.slow_put.probability - ): - self.count -= 1 - return _runtime.slow_put.timeout + with self.lock: + if content_length > self.minimal_length: + if self.count > 0: + if ( + _runtime.slow_put.probability == 1 + or random.random() <= _runtime.slow_put.probability + ): + self.count -= 1 + return _runtime.slow_put.timeout return None class Expected500ErrorAction: @@ -199,29 +204,48 @@ class _ServerRuntime: ) request_handler.connection.close() + class BrokenPipeAction: + def inject_error(self, request_handler): + # partial read + self.rfile.read(50) + + time.sleep(1) + request_handler.connection.setsockopt( + socket.SOL_SOCKET, socket.SO_LINGER, struct.pack("ii", 1, 0) + ) + request_handler.connection.close() + class ConnectionRefusedAction(RedirectAction): pass class CountAfter: - def __init__(self, count_=None, after_=None, action_=None, action_args_=[]): + def __init__( + self, lock, count_=None, after_=None, action_=None, action_args_=[] + ): + self.lock = lock + self.count = count_ if count_ is not None else INF_COUNT self.after = after_ if after_ is not None else 0 self.action = action_ self.action_args = action_args_ + if self.action == "connection_refused": self.error_handler = _ServerRuntime.ConnectionRefusedAction() elif self.action == "connection_reset_by_peer": self.error_handler = _ServerRuntime.ConnectionResetByPeerAction( *self.action_args ) + elif self.action == "broken_pipe": + self.error_handler = _ServerRuntime.BrokenPipeAction() elif self.action == "redirect_to": self.error_handler = _ServerRuntime.RedirectAction(*self.action_args) else: self.error_handler = _ServerRuntime.Expected500ErrorAction() @staticmethod - def from_cgi_params(params): + def from_cgi_params(lock, params): return _ServerRuntime.CountAfter( + lock=lock, count_=_and_then(params.get("count", [None])[0], int), after_=_and_then(params.get("after", [None])[0], int), action_=params.get("action", [None])[0], @@ -232,13 +256,14 @@ class _ServerRuntime: return f"count:{self.count} after:{self.after} action:{self.action} action_args:{self.action_args}" def has_effect(self): - if self.after: - self.after -= 1 - if self.after == 0: - if self.count: - self.count -= 1 - return True - return False + with self.lock: + if self.after: + self.after -= 1 + if self.after == 0: + if self.count: + self.count -= 1 + return True + return False def inject_error(self, request_handler): self.error_handler.inject_error(request_handler) @@ -397,14 +422,16 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "at_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) - _runtime.at_part_upload = _ServerRuntime.CountAfter.from_cgi_params(params) + _runtime.at_part_upload = _ServerRuntime.CountAfter.from_cgi_params( + _runtime.lock, params + ) self.log_message("set at_part_upload %s", _runtime.at_part_upload) return self._ok() if path[1] == "at_object_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) _runtime.at_object_upload = _ServerRuntime.CountAfter.from_cgi_params( - params + _runtime.lock, params ) self.log_message("set at_object_upload %s", _runtime.at_object_upload) return self._ok() @@ -420,6 +447,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "slow_put": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) _runtime.slow_put = _ServerRuntime.SlowPut( + lock=_runtime.lock, minimal_length_=_and_then(params.get("minimal_length", [None])[0], int), probability_=_and_then(params.get("probability", [None])[0], float), timeout_=_and_then(params.get("timeout", [None])[0], float), @@ -436,7 +464,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if path[1] == "at_create_multi_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) _runtime.at_create_multi_part_upload = ( - _ServerRuntime.CountAfter.from_cgi_params(params) + _ServerRuntime.CountAfter.from_cgi_params(_runtime.lock, params) ) self.log_message( "set at_create_multi_part_upload %s", @@ -477,7 +505,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if upload_id is not None: if _runtime.at_part_upload is not None: self.log_message( - "put error_at_object_upload %s, %s, %s", + "put at_part_upload %s, %s, %s", _runtime.at_part_upload, upload_id, parts, @@ -492,7 +520,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): if _runtime.at_object_upload is not None: if _runtime.at_object_upload.has_effect(): self.log_message( - "put error_at_object_upload %s, %s, %s", + "put error_at_object_upload %s, %s", _runtime.at_object_upload, parts, ) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 28b0c9beeaa..c40e2a31a8b 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -41,11 +41,6 @@ def broken_s3(init_broken_s3): yield init_broken_s3 -@pytest.fixture(scope="module") -def init_connection_reset_by_peer(cluster): - yield start_s3_mock(cluster, "connection_reset_by_peer", "8084") - - def test_upload_after_check_works(cluster, broken_s3): node = cluster.instances["node"] @@ -397,3 +392,75 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( or "DB::Exception: Poco::Exception. Code: 1000, e.code() = 104, Connection reset by peer" in error ), error + + +def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_part_upload( + count=3, + after=2, + action="broken_pipe", + ) + + insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=1000000, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + count_create_multi_part_uploads, count_upload_parts, count_s3_errors = get_counters( + node, insert_query_id, log_type="QueryFinish" + ) + + assert count_create_multi_part_uploads == 1 + assert count_upload_parts == 7 + assert count_s3_errors == 3 + + broken_s3.setup_at_part_upload( + count=1000, + after=2, + action="broken_pipe", + ) + insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD_1" + error = node.query_and_get_error( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=1000000, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + assert "Code: 1000" in error, error + assert ( + "DB::Exception: Poco::Exception. Code: 1000, e.code() = 32, I/O error: Broken pipe" + in error + ), error From 686ed14e714771f92186e31ede4ae78ab96e2565 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Thu, 27 Jul 2023 23:30:21 +0300 Subject: [PATCH 1266/2047] Fix alter named collection if exists --- .../NamedCollections/NamedCollectionUtils.cpp | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index 12bf100feae..cab844d6213 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -428,9 +428,21 @@ void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr contex { auto lock = lockNamedCollectionsTransaction(); loadIfNotUnlocked(lock); + auto & instance = NamedCollectionFactory::instance(); + if (!instance.exists(query.collection_name)) + { + if (!query.if_exists) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "Cannot remove collection `{}`, because it doesn't exist", + query.collection_name); + } + return; + } LoadFromSQL(context).update(query); - auto collection = NamedCollectionFactory::instance().getMutable(query.collection_name); + auto collection = instance.getMutable(query.collection_name); auto collection_lock = collection->lock(); for (const auto & [name, value] : query.changes) From 407fb58d4bd82d73886f77545d9053ea904b0e65 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 27 Jul 2023 20:53:08 +0000 Subject: [PATCH 1267/2047] Fix style check --- src/Processors/Transforms/DistinctSortedChunkTransform.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Transforms/DistinctSortedChunkTransform.cpp b/src/Processors/Transforms/DistinctSortedChunkTransform.cpp index 720543991c5..91806a90083 100644 --- a/src/Processors/Transforms/DistinctSortedChunkTransform.cpp +++ b/src/Processors/Transforms/DistinctSortedChunkTransform.cpp @@ -5,6 +5,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int SET_SIZE_LIMIT_EXCEEDED; } From 7d8dc92ed0522e309760037720f6fd8fb3f2542d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 27 Jul 2023 21:07:54 +0000 Subject: [PATCH 1268/2047] Automatic style fix --- .../test_disallow_concurrency.py | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py index af1b2656227..5c3f06a9d9d 100644 --- a/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py +++ b/tests/integration/test_backup_restore_on_cluster/test_disallow_concurrency.py @@ -156,7 +156,9 @@ def test_concurrent_backups_on_same_node(): ) return else: - raise Exception("Concurrent backups both passed, when one is expected to fail") + raise Exception( + "Concurrent backups both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent backups not supported", @@ -202,7 +204,7 @@ def test_concurrent_backups_on_different_nodes(): assert status in ["CREATING_BACKUP", "BACKUP_CREATED"] result, error = nodes[0].query_and_get_answer_with_error( - f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" + f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name}" ) if not error: @@ -224,7 +226,9 @@ def test_concurrent_backups_on_different_nodes(): ) return else: - raise Exception("Concurrent backups both passed, when one is expected to fail") + raise Exception( + "Concurrent backups both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent backups not supported", @@ -291,8 +295,9 @@ def test_concurrent_restores_on_same_node(): ) return else: - raise Exception("Concurrent restores both passed, when one is expected to fail") - + raise Exception( + "Concurrent restores both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent restores not supported", @@ -359,7 +364,9 @@ def test_concurrent_restores_on_different_node(): ) return else: - raise Exception("Concurrent restores both passed, when one is expected to fail") + raise Exception( + "Concurrent restores both passed, when one is expected to fail" + ) expected_errors = [ "Concurrent restores not supported", From 18c1fd6f08cc2be964ed15604c26a70d7d168561 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 27 Jul 2023 21:24:39 +0000 Subject: [PATCH 1269/2047] Refactor InDepthQueryTreeVisitorWithContext --- src/Analyzer/InDepthQueryTreeVisitor.h | 158 +++--------------- ...egateFunctionsArithmericOperationsPass.cpp | 8 +- src/Analyzer/Passes/ArrayExistsToHasPass.cpp | 2 +- src/Analyzer/Passes/AutoFinalOnQueryPass.cpp | 2 +- .../Passes/ConvertOrLikeChainPass.cpp | 2 +- src/Analyzer/Passes/ConvertQueryToCNFPass.cpp | 2 +- src/Analyzer/Passes/CountDistinctPass.cpp | 34 ++-- src/Analyzer/Passes/CrossToInnerJoinPass.cpp | 2 +- .../Passes/FunctionToSubcolumnsPass.cpp | 2 +- src/Analyzer/Passes/FuseFunctionsPass.cpp | 2 +- .../Passes/GroupingFunctionsResolvePass.cpp | 2 +- src/Analyzer/Passes/IfChainToMultiIfPass.cpp | 2 +- .../Passes/IfTransformStringsToEnumPass.cpp | 2 +- .../Passes/LogicalExpressionOptimizerPass.cpp | 2 +- src/Analyzer/Passes/MultiIfToIfPass.cpp | 2 +- .../Passes/NormalizeCountVariantsPass.cpp | 2 +- .../OptimizeGroupByFunctionKeysPass.cpp | 2 +- ...ptimizeRedundantFunctionsInOrderByPass.cpp | 2 +- .../RewriteAggregateFunctionWithIfPass.cpp | 2 +- .../Passes/ShardNumColumnToFunctionPass.cpp | 2 +- src/Analyzer/Passes/SumIfToCountIfPass.cpp | 2 +- .../UniqInjectiveFunctionsEliminationPass.cpp | 2 +- src/Storages/buildQueryTreeForShard.cpp | 2 +- 23 files changed, 55 insertions(+), 185 deletions(-) diff --git a/src/Analyzer/InDepthQueryTreeVisitor.h b/src/Analyzer/InDepthQueryTreeVisitor.h index be3a760d4e6..59ee57996c4 100644 --- a/src/Analyzer/InDepthQueryTreeVisitor.h +++ b/src/Analyzer/InDepthQueryTreeVisitor.h @@ -91,26 +91,25 @@ private: template using ConstInDepthQueryTreeVisitor = InDepthQueryTreeVisitor; -/** 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 class InDepthQueryTreeVisitorWithContext { public: - using VisitQueryTreeNodeType = std::conditional_t; + 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]]) { @@ -147,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); + visitChildren(query_tree_node); getDerived().leaveImpl(query_tree_node); } + void enterImpl(VisitQueryTreeNodeType & node [[maybe_unused]]) + {} + void leaveImpl(VisitQueryTreeNodeType & node [[maybe_unused]]) {} private: @@ -172,85 +169,15 @@ private: return *static_cast(this); } - void visitChildren(VisitQueryTreeNodeType & expression) + bool shouldSkipSubtree( + VisitQueryTreeNodeType & parent, + VisitQueryTreeNodeType & child, + size_t subtree_index) { - for (auto & child : expression->getChildren()) - { - if (!child) - continue; + bool need_visit_child = getDerived().needChildVisit(parent, child); + if (!need_visit_child) + return true; - bool need_visit_child = getDerived().needChildVisit(expression, child); - - if (need_visit_child) - visit(child); - } - } - - ContextPtr current_context; - size_t subquery_depth = 0; -}; - -template -using ConstInDepthQueryTreeVisitorWithContext = InDepthQueryTreeVisitorWithContext; - -/** 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 -class InDepthQueryTreeConditionalVisitor : public InDepthQueryTreeVisitor, const_visitor> -{ -public: - using Base = InDepthQueryTreeVisitor, 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 -using ConstInDepthQueryTreeConditionalVisitor = InDepthQueryTreeConditionalVisitor; - -template -class QueryTreeVisitor -{ -public: - explicit QueryTreeVisitor(ContextPtr context_) - : current_context(std::move(context_)) - {} - - bool needApply(QueryTreeNodePtr & node) - { - return getImpl().needApply(node); - } - - bool shouldSkipSubtree(QueryTreeNodePtr & parent, size_t subtree_index) - { if (auto * table_function_node = parent->as()) { const auto & unresolved_indexes = table_function_node->getUnresolvedArgumentIndexes(); @@ -259,58 +186,19 @@ public: return false; } - void visit(QueryTreeNodePtr & node) - { - auto current_scope_context_ptr = current_context; - SCOPE_EXIT( - current_context = std::move(current_scope_context_ptr); - ); - - if (auto * query_node = node->template as()) - current_context = query_node->getContext(); - else if (auto * union_node = node->template as()) - current_context = union_node->getContext(); - - if (!TOP_TO_BOTTOM) - visitChildren(node); - - if (needApply(node)) - getImpl().apply(node); - - if (TOP_TO_BOTTOM) - visitChildren(node); - } - - const ContextPtr & getContext() const - { - return current_context; - } - - const Settings & getSettings() const - { - return current_context->getSettingsRef(); - } -private: - - Impl & getImpl() - { - return *static_cast(this); - } - - void visitChildren(QueryTreeNodePtr & node) + void visitChildren(VisitQueryTreeNodeType & expression) { size_t index = 0; - for (auto & child : node->getChildren()) + for (auto & child : expression->getChildren()) { - if (child && !shouldSkipSubtree(node, index)) + if (child && !shouldSkipSubtree(expression, child, index)) visit(child); ++index; } } - static constexpr bool TOP_TO_BOTTOM = Impl::TOP_TO_BOTTOM; - ContextPtr current_context; + size_t subquery_depth = 0; }; } diff --git a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp index 1476a66c892..3615a632374 100644 --- a/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp +++ b/src/Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.cpp @@ -51,13 +51,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; 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; diff --git a/src/Analyzer/Passes/ArrayExistsToHasPass.cpp b/src/Analyzer/Passes/ArrayExistsToHasPass.cpp index c0f958588f1..a95bcea4fac 100644 --- a/src/Analyzer/Passes/ArrayExistsToHasPass.cpp +++ b/src/Analyzer/Passes/ArrayExistsToHasPass.cpp @@ -22,7 +22,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_rewrite_array_exists_to_has) return; diff --git a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp index 15326ca1dc8..2c89ec9dc20 100644 --- a/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp +++ b/src/Analyzer/Passes/AutoFinalOnQueryPass.cpp @@ -20,7 +20,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().final) return; diff --git a/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp b/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp index 7d7362fb742..1fada88a21c 100644 --- a/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp +++ b/src/Analyzer/Passes/ConvertOrLikeChainPass.cpp @@ -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(); if (!function_node || function_node->getFunctionName() != "or") diff --git a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp index 4d32c96b845..724448ad742 100644 --- a/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp +++ b/src/Analyzer/Passes/ConvertQueryToCNFPass.cpp @@ -688,7 +688,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * query_node = node->as(); if (!query_node) diff --git a/src/Analyzer/Passes/CountDistinctPass.cpp b/src/Analyzer/Passes/CountDistinctPass.cpp index 38f7d07d052..dc58747221e 100644 --- a/src/Analyzer/Passes/CountDistinctPass.cpp +++ b/src/Analyzer/Passes/CountDistinctPass.cpp @@ -16,17 +16,16 @@ namespace DB namespace { -class CountDistinctVisitor : public QueryTreeVisitor +class CountDistinctVisitor : public InDepthQueryTreeVisitorWithContext { public: - using QueryTreeVisitor::QueryTreeVisitor; + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; - static constexpr bool TOP_TO_BOTTOM = true; - - bool needApply(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().count_distinct_optimization) - return false; + return; auto * query_node = node->as(); @@ -34,43 +33,32 @@ public: if (!query_node || (query_node->hasWith() || query_node->hasPrewhere() || query_node->hasWhere() || query_node->hasGroupBy() || query_node->hasHaving() || query_node->hasWindow() || query_node->hasOrderBy() || query_node->hasLimitByLimit() || query_node->hasLimitByOffset() || query_node->hasLimitBy() || query_node->hasLimit() || query_node->hasOffset())) - return false; + return; /// Check that query has only single table expression auto join_tree_node_type = query_node->getJoinTree()->getNodeType(); if (join_tree_node_type == QueryTreeNodeType::JOIN || join_tree_node_type == QueryTreeNodeType::ARRAY_JOIN) - return false; + return; /// Check that query has only single node in projection auto & projection_nodes = query_node->getProjection().getNodes(); if (projection_nodes.size() != 1) - return false; + return; /// Check that query single projection node is `countDistinct` function auto & projection_node = projection_nodes[0]; auto * function_node = projection_node->as(); if (!function_node) - return false; + return; auto lower_function_name = Poco::toLower(function_node->getFunctionName()); if (lower_function_name != "countdistinct" && lower_function_name != "uniqexact") - return false; + return; /// Check that `countDistinct` function has single COLUMN argument auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); if (count_distinct_arguments_nodes.size() != 1 && count_distinct_arguments_nodes[0]->getNodeType() != QueryTreeNodeType::COLUMN) - return false; - - return true; - } - - void apply(QueryTreeNodePtr & node) - { - auto * query_node = node->as(); - auto & projection_nodes = query_node->getProjection().getNodes(); - auto * function_node = projection_nodes[0]->as(); - - auto & count_distinct_arguments_nodes = function_node->getArguments().getNodes(); + return; auto & count_distinct_argument_column = count_distinct_arguments_nodes[0]; auto & count_distinct_argument_column_typed = count_distinct_argument_column->as(); diff --git a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp index d4877d23f28..b5ece1a4c49 100644 --- a/src/Analyzer/Passes/CrossToInnerJoinPass.cpp +++ b/src/Analyzer/Passes/CrossToInnerJoinPass.cpp @@ -193,7 +193,7 @@ public: return true; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!isEnabled()) return; diff --git a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp index 696483862e0..cd635f87e0e 100644 --- a/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp +++ b/src/Analyzer/Passes/FunctionToSubcolumnsPass.cpp @@ -29,7 +29,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) const + void enterImpl(QueryTreeNodePtr & node) const { if (!getSettings().optimize_functions_to_subcolumns) return; diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp index 14082697955..2cb7afa4ad6 100644 --- a/src/Analyzer/Passes/FuseFunctionsPass.cpp +++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp @@ -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; diff --git a/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp b/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp index 0cf5310a3ad..577bca8d1ae 100644 --- a/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp +++ b/src/Analyzer/Passes/GroupingFunctionsResolvePass.cpp @@ -46,7 +46,7 @@ public: { } - void visitImpl(const QueryTreeNodePtr & node) + void enterImpl(const QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || function_node->getFunctionName() != "grouping") diff --git a/src/Analyzer/Passes/IfChainToMultiIfPass.cpp b/src/Analyzer/Passes/IfChainToMultiIfPass.cpp index 1f97e012331..b0018d474d5 100644 --- a/src/Analyzer/Passes/IfChainToMultiIfPass.cpp +++ b/src/Analyzer/Passes/IfChainToMultiIfPass.cpp @@ -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; diff --git a/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp b/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp index 562aff4cf05..901867b8889 100644 --- a/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp +++ b/src/Analyzer/Passes/IfTransformStringsToEnumPass.cpp @@ -113,7 +113,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_if_transform_strings_to_enum) return; diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 13f8025f5ea..46056aeaf6f 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -19,7 +19,7 @@ public: : Base(std::move(context)) {} - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); diff --git a/src/Analyzer/Passes/MultiIfToIfPass.cpp b/src/Analyzer/Passes/MultiIfToIfPass.cpp index 4672351bcfb..85dd33af8bb 100644 --- a/src/Analyzer/Passes/MultiIfToIfPass.cpp +++ b/src/Analyzer/Passes/MultiIfToIfPass.cpp @@ -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; diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp index d36be98751c..c85b863a203 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp @@ -20,7 +20,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_normalize_count_variants) return; diff --git a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp index 5ed52f1210b..2e3f207fdeb 100644 --- a/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp +++ b/src/Analyzer/Passes/OptimizeGroupByFunctionKeysPass.cpp @@ -26,7 +26,7 @@ public: return !child->as(); } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_group_by_function_keys) return; diff --git a/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp b/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp index c6d312d0ecf..875d0c8b5fb 100644 --- a/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp +++ b/src/Analyzer/Passes/OptimizeRedundantFunctionsInOrderByPass.cpp @@ -28,7 +28,7 @@ public: return true; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_redundant_functions_in_order_by) return; diff --git a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp index de264948d4c..38f2fbfa274 100644 --- a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp +++ b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp @@ -26,7 +26,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_rewrite_aggregate_function_with_if) return; diff --git a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp index b28816e8ff3..52c30b7b35d 100644 --- a/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp +++ b/src/Analyzer/Passes/ShardNumColumnToFunctionPass.cpp @@ -24,7 +24,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) const + void enterImpl(QueryTreeNodePtr & node) const { auto * column_node = node->as(); if (!column_node) diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.cpp b/src/Analyzer/Passes/SumIfToCountIfPass.cpp index d55af278152..cff9ba1111c 100644 --- a/src/Analyzer/Passes/SumIfToCountIfPass.cpp +++ b/src/Analyzer/Passes/SumIfToCountIfPass.cpp @@ -26,7 +26,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_rewrite_sum_if_to_count_if) return; diff --git a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp index 5c4484457e8..179bd1c38e4 100644 --- a/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp +++ b/src/Analyzer/Passes/UniqInjectiveFunctionsEliminationPass.cpp @@ -31,7 +31,7 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { if (!getSettings().optimize_injective_functions_inside_uniq) return; diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 1ee7d747fcc..9929b5bb39b 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -130,7 +130,7 @@ public: return true; } - void visitImpl(QueryTreeNodePtr & node) + void enterImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); auto * join_node = node->as(); From 5bd670d8c6ef266ce59625250a0861ed55ba4c47 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 27 Jul 2023 15:54:33 -0700 Subject: [PATCH 1270/2047] Improve endianness-independent support for hash functions --- contrib/libmetrohash/src/platform.h | 18 +- src/Functions/FunctionsHashing.h | 156 ++++++------------ .../0_stateless/00678_murmurhash.reference | 4 +- .../queries/0_stateless/00678_murmurhash.sql | 6 +- .../00746_hashing_tuples.reference | 14 +- .../0_stateless/00746_hashing_tuples.sql | 14 +- .../0_stateless/02534_keyed_siphash.reference | 4 +- .../0_stateless/02534_keyed_siphash.sql | 4 +- .../02552_siphash128_reference.reference | 2 +- .../02552_siphash128_reference.sql | 2 +- 10 files changed, 91 insertions(+), 133 deletions(-) diff --git a/contrib/libmetrohash/src/platform.h b/contrib/libmetrohash/src/platform.h index bc00e5a286b..9e83d11cb7c 100644 --- a/contrib/libmetrohash/src/platform.h +++ b/contrib/libmetrohash/src/platform.h @@ -17,7 +17,8 @@ #ifndef METROHASH_PLATFORM_H #define METROHASH_PLATFORM_H -#include +#include +#include #include // rotate right idiom recognized by most compilers @@ -33,6 +34,11 @@ inline static uint64_t read_u64(const void * const ptr) // so we use memcpy() which is the most portable. clang & gcc usually translates `memcpy()` into a single `load` instruction // when hardware supports it, so using memcpy() is efficient too. memcpy(&result, ptr, sizeof(result)); + +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + result = std::byteswap(result); +#endif + return result; } @@ -40,6 +46,11 @@ inline static uint64_t read_u32(const void * const ptr) { uint32_t result; memcpy(&result, ptr, sizeof(result)); + +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + result = std::byteswap(result); +#endif + return result; } @@ -47,6 +58,11 @@ inline static uint64_t read_u16(const void * const ptr) { uint16_t result; memcpy(&result, ptr, sizeof(result)); + +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + result = std::byteswap(result); +#endif + return result; } diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 279294b367c..929cb2f77f4 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -153,15 +153,10 @@ struct IntHash64Impl template T combineHashesFunc(T t1, T t2) { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - T tmp; - reverseMemcpy(&tmp, &t1, sizeof(T)); - t1 = tmp; - reverseMemcpy(&tmp, &t2, sizeof(T)); - t2 = tmp; -#endif - T hashes[] = {t1, t2}; - return HashFunction::apply(reinterpret_cast(hashes), 2 * sizeof(T)); + transformEndianness(t1); + transformEndianness(t2); + const T hashes[] {t1, t2}; + return HashFunction::apply(reinterpret_cast(hashes), sizeof(hashes)); } @@ -184,21 +179,14 @@ struct HalfMD5Impl MD5_Update(&ctx, reinterpret_cast(begin), size); MD5_Final(buf.char_data, &ctx); -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - return buf.uint64_data; /// No need to flip bytes on big endian machines -#else - return std::byteswap(buf.uint64_data); /// Compatibility with existing code. Cast need for old poco AND macos where UInt64 != uint64_t -#endif + /// Compatibility with existing code. Cast need for old poco AND macos where UInt64 != uint64_t + transformEndianness(buf.uint64_data); + return buf.uint64_data; } static UInt64 combineHashes(UInt64 h1, UInt64 h2) { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - h1 = std::byteswap(h1); - h2 = std::byteswap(h2); -#endif - UInt64 hashes[] = {h1, h2}; - return apply(reinterpret_cast(hashes), 16); + return combineHashesFunc(h1, h2); } /// If true, it will use intHash32 or intHash64 to hash POD types. This behaviour is intended for better performance of some functions. @@ -311,15 +299,8 @@ struct SipHash64Impl static constexpr auto name = "sipHash64"; using ReturnType = UInt64; - static UInt64 apply(const char * begin, size_t size) - { - return sipHash64(begin, size); - } - - static UInt64 combineHashes(UInt64 h1, UInt64 h2) - { - return combineHashesFunc(h1, h2); - } + static UInt64 apply(const char * begin, size_t size) { return sipHash64(begin, size); } + static UInt64 combineHashes(UInt64 h1, UInt64 h2) { return combineHashesFunc(h1, h2); } static constexpr bool use_int_hash_for_pods = false; }; @@ -336,12 +317,10 @@ struct SipHash64KeyedImpl static UInt64 combineHashesKeyed(const Key & key, UInt64 h1, UInt64 h2) { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - h1 = std::byteswap(h1); - h2 = std::byteswap(h2); -#endif - UInt64 hashes[] = {h1, h2}; - return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt64)); + transformEndianness(h1); + transformEndianness(h2); + const UInt64 hashes[]{h1, h2}; + return applyKeyed(key, reinterpret_cast(hashes), sizeof(hashes)); } static constexpr bool use_int_hash_for_pods = false; @@ -353,15 +332,8 @@ struct SipHash128Impl using ReturnType = UInt128; - static UInt128 combineHashes(UInt128 h1, UInt128 h2) - { - return combineHashesFunc(h1, h2); - } - - static UInt128 apply(const char * data, const size_t size) - { - return sipHash128(data, size); - } + static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc(h1, h2); } + static UInt128 apply(const char * data, const size_t size) { return sipHash128(data, size); } static constexpr bool use_int_hash_for_pods = false; }; @@ -378,15 +350,10 @@ struct SipHash128KeyedImpl static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2) { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - UInt128 tmp; - reverseMemcpy(&tmp, &h1, sizeof(UInt128)); - h1 = tmp; - reverseMemcpy(&tmp, &h2, sizeof(UInt128)); - h2 = tmp; -#endif - UInt128 hashes[] = {h1, h2}; - return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt128)); + transformEndianness(h1); + transformEndianness(h2); + const UInt128 hashes[]{h1, h2}; + return applyKeyed(key, reinterpret_cast(hashes), sizeof(hashes)); } static constexpr bool use_int_hash_for_pods = false; @@ -531,10 +498,7 @@ struct MurmurHash3Impl64 return h[0] ^ h[1]; } - static UInt64 combineHashes(UInt64 h1, UInt64 h2) - { - return IntHash64Impl::apply(h1) ^ h2; - } + static UInt64 combineHashes(UInt64 h1, UInt64 h2) { return IntHash64Impl::apply(h1) ^ h2; } static constexpr bool use_int_hash_for_pods = false; }; @@ -552,10 +516,7 @@ struct MurmurHash3Impl128 return *reinterpret_cast(bytes); } - static UInt128 combineHashes(UInt128 h1, UInt128 h2) - { - return combineHashesFunc(h1, h2); - } + static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc(h1, h2); } static constexpr bool use_int_hash_for_pods = false; }; @@ -1040,11 +1001,10 @@ private: if (const ColVecType * col_from = checkAndGetColumn(column)) { const typename ColVecType::Container & vec_from = col_from->getData(); - size_t size = vec_from.size(); + const size_t size = vec_from.size(); for (size_t i = 0; i < size; ++i) { ToType hash; - if constexpr (Impl::use_int_hash_for_pods) { if constexpr (std::is_same_v) @@ -1058,13 +1018,8 @@ private: hash = JavaHashImpl::apply(vec_from[i]); else { - FromType value = vec_from[i]; - if constexpr (std::endian::native == std::endian::big) - { - FromType value_reversed; - reverseMemcpy(&value_reversed, &value, sizeof(value)); - value = value_reversed; - } + auto value = vec_from[i]; + transformEndianness(value); hash = apply(key, reinterpret_cast(&value), sizeof(value)); } } @@ -1078,8 +1033,8 @@ private: else if (auto col_from_const = checkAndGetColumnConst(column)) { auto value = col_from_const->template getValue(); - ToType hash; + ToType hash; if constexpr (Impl::use_int_hash_for_pods) { if constexpr (std::is_same_v) @@ -1093,17 +1048,12 @@ private: hash = JavaHashImpl::apply(value); else { - if constexpr (std::endian::native == std::endian::big) - { - FromType value_reversed; - reverseMemcpy(&value_reversed, &value, sizeof(value)); - value = value_reversed; - } + transformEndianness(value); hash = apply(key, reinterpret_cast(&value), sizeof(value)); } } - size_t size = vec_to.size(); + const size_t size = vec_to.size(); if constexpr (first) vec_to.assign(size, hash); else @@ -1120,6 +1070,16 @@ private: { using ColVecType = ColumnVectorOrDecimal; + static const auto to_little_endian = [](auto & value) + { + // IPv6 addresses are parsed into four 32-bit components in big-endian ordering on both platforms, so no change is necessary. + // Reference: `parseIPv6orIPv4` in src/Common/formatIPv6.h. + if constexpr (std::endian::native == std::endian::big && std::is_same_v) + return; + + transformEndianness(value); + }; + if (const ColVecType * col_from = checkAndGetColumn(column)) { const typename ColVecType::Container & vec_from = col_from->getData(); @@ -1131,9 +1091,10 @@ private: hash = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); else { - char tmp_buffer[sizeof(vec_from[i])]; - reverseMemcpy(tmp_buffer, &vec_from[i], sizeof(vec_from[i])); - hash = apply(key, reinterpret_cast(tmp_buffer), sizeof(vec_from[i])); + auto value = vec_from[i]; + to_little_endian(value); + + hash = apply(key, reinterpret_cast(&value), sizeof(value)); } if constexpr (first) vec_to[i] = hash; @@ -1144,17 +1105,10 @@ private: else if (auto col_from_const = checkAndGetColumnConst(column)) { auto value = col_from_const->template getValue(); + to_little_endian(value); - ToType hash; - if constexpr (std::endian::native == std::endian::little) - hash = apply(key, reinterpret_cast(&value), sizeof(value)); - else - { - char tmp_buffer[sizeof(value)]; - reverseMemcpy(tmp_buffer, &value, sizeof(value)); - hash = apply(key, reinterpret_cast(tmp_buffer), sizeof(value)); - } - size_t size = vec_to.size(); + const auto hash = apply(key, reinterpret_cast(&value), sizeof(value)); + const size_t size = vec_to.size(); if constexpr (first) vec_to.assign(size, hash); else @@ -1423,6 +1377,9 @@ public: if constexpr (std::is_same_v) /// backward-compatible { + if (std::endian::native == std::endian::big) + std::ranges::for_each(col_to->getData(), transformEndianness); + auto col_to_fixed_string = ColumnFixedString::create(sizeof(UInt128)); col_to_fixed_string->getChars() = std::move(*reinterpret_cast(&col_to->getData())); return col_to_fixed_string; @@ -1673,21 +1630,8 @@ struct ImplWyHash64 static constexpr auto name = "wyHash64"; using ReturnType = UInt64; - static UInt64 apply(const char * s, const size_t len) - { - return wyhash(s, len, 0, _wyp); - } - static UInt64 combineHashes(UInt64 h1, UInt64 h2) - { - union - { - UInt64 u64[2]; - char chars[16]; - }; - u64[0] = h1; - u64[1] = h2; - return apply(chars, 16); - } + static UInt64 apply(const char * s, const size_t len) { return wyhash(s, len, 0, _wyp); } + static UInt64 combineHashes(UInt64 h1, UInt64 h2) { return combineHashesFunc(h1, h2); } static constexpr bool use_int_hash_for_pods = false; }; diff --git a/tests/queries/0_stateless/00678_murmurhash.reference b/tests/queries/0_stateless/00678_murmurhash.reference index fb4a00ba046..988c022f1bf 100644 --- a/tests/queries/0_stateless/00678_murmurhash.reference +++ b/tests/queries/0_stateless/00678_murmurhash.reference @@ -25,5 +25,5 @@ 9631199822919835226 4334672815104069193 4334672815104069193 -1 -1 +6145F501578671E2877DBA2BE487AF7E +16FE7483905CCE7A85670E43E4678877 diff --git a/tests/queries/0_stateless/00678_murmurhash.sql b/tests/queries/0_stateless/00678_murmurhash.sql index eda29fd17cd..705c62480a0 100644 --- a/tests/queries/0_stateless/00678_murmurhash.sql +++ b/tests/queries/0_stateless/00678_murmurhash.sql @@ -32,7 +32,5 @@ SELECT gccMurmurHash('foo'); SELECT gccMurmurHash('\x01'); SELECT gccMurmurHash(1); --- Comparison with reverse for big endian -SELECT hex(murmurHash3_128('foo')) = hex(reverse(unhex('6145F501578671E2877DBA2BE487AF7E'))) or hex(murmurHash3_128('foo')) = '6145F501578671E2877DBA2BE487AF7E'; --- Comparison with reverse for big endian -SELECT hex(murmurHash3_128('\x01')) = hex(reverse(unhex('16FE7483905CCE7A85670E43E4678877'))) or hex(murmurHash3_128('\x01')) = '16FE7483905CCE7A85670E43E4678877'; +SELECT hex(murmurHash3_128('foo')); +SELECT hex(murmurHash3_128('\x01')); diff --git a/tests/queries/0_stateless/00746_hashing_tuples.reference b/tests/queries/0_stateless/00746_hashing_tuples.reference index e3b896f6077..71d45be5a54 100644 --- a/tests/queries/0_stateless/00746_hashing_tuples.reference +++ b/tests/queries/0_stateless/00746_hashing_tuples.reference @@ -1,11 +1,11 @@ 12940785793559895259 17926972817233444501 7456555839952096623 -1 -1 -1 -1 -1 +CC45107CC4B79F62D831BEF2103C7CBF +DF2EC2F0669B000EDFF6ADEE264E7D68 +4CD1C30C38AB935D418B5269EF197B9E +9D78134EE48654D753CCA1B76185CF8E +389D16428D2AADEC9713905572F42864 955237314186186656 8175794665478042155 9325786087413524176 @@ -18,8 +18,8 @@ 8163029322371165472 8788309436660676487 236561483980029756 -1 -1 +8DD5527CC43D76F4760D26BE0F641F7E +F8F7AD9B6CD4CF117A71E277E2EC2931 12384823029245979431 4507350192761038840 1188926775431157506 diff --git a/tests/queries/0_stateless/00746_hashing_tuples.sql b/tests/queries/0_stateless/00746_hashing_tuples.sql index f17ad6fa77f..466a2184c65 100644 --- a/tests/queries/0_stateless/00746_hashing_tuples.sql +++ b/tests/queries/0_stateless/00746_hashing_tuples.sql @@ -4,11 +4,11 @@ SELECT sipHash64(1, 2, 3); SELECT sipHash64(1, 3, 2); SELECT sipHash64(('a', [1, 2, 3], 4, (4, ['foo', 'bar'], 1, (1, 2)))); -SELECT hex(sipHash128('foo')) = hex(reverse(unhex('CC45107CC4B79F62D831BEF2103C7CBF'))) or hex(sipHash128('foo')) = 'CC45107CC4B79F62D831BEF2103C7CBF'; -SELECT hex(sipHash128('\x01')) = hex(reverse(unhex('DF2EC2F0669B000EDFF6ADEE264E7D68'))) or hex(sipHash128('\x01')) = 'DF2EC2F0669B000EDFF6ADEE264E7D68'; -SELECT hex(sipHash128('foo', 'foo')) = hex(reverse(unhex('4CD1C30C38AB935D418B5269EF197B9E'))) or hex(sipHash128('foo', 'foo')) = '4CD1C30C38AB935D418B5269EF197B9E'; -SELECT hex(sipHash128('foo', 'foo', 'foo')) = hex(reverse(unhex('9D78134EE48654D753CCA1B76185CF8E'))) or hex(sipHash128('foo', 'foo', 'foo')) = '9D78134EE48654D753CCA1B76185CF8E'; -SELECT hex(sipHash128(1, 2, 3)) = hex(reverse(unhex('389D16428D2AADEC9713905572F42864'))) or hex(sipHash128(1, 2, 3)) = '389D16428D2AADEC9713905572F42864'; +SELECT hex(sipHash128('foo')); +SELECT hex(sipHash128('\x01')); +SELECT hex(sipHash128('foo', 'foo')); +SELECT hex(sipHash128('foo', 'foo', 'foo')); +SELECT hex(sipHash128(1, 2, 3)); SELECT halfMD5(1, 2, 3); SELECT halfMD5(1, 3, 2); @@ -26,8 +26,8 @@ SELECT murmurHash3_64(1, 2, 3); SELECT murmurHash3_64(1, 3, 2); SELECT murmurHash3_64(('a', [1, 2, 3], 4, (4, ['foo', 'bar'], 1, (1, 2)))); -SELECT hex(murmurHash3_128('foo', 'foo')) = hex(reverse(unhex('8DD5527CC43D76F4760D26BE0F641F7E'))) or hex(murmurHash3_128('foo', 'foo')) = '8DD5527CC43D76F4760D26BE0F641F7E'; -SELECT hex(murmurHash3_128('foo', 'foo', 'foo')) = hex(reverse(unhex('F8F7AD9B6CD4CF117A71E277E2EC2931'))) or hex(murmurHash3_128('foo', 'foo', 'foo')) = 'F8F7AD9B6CD4CF117A71E277E2EC2931'; +SELECT hex(murmurHash3_128('foo', 'foo')); +SELECT hex(murmurHash3_128('foo', 'foo', 'foo')); SELECT gccMurmurHash(1, 2, 3); SELECT gccMurmurHash(1, 3, 2); diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index ccc514e7ea2..33838d5298c 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -191,9 +191,9 @@ E51B38608EF25F57 1 1 E28DBDE7FE22E41C -1 +1CE422FEE7BD8DE20000000000000000 E28DBDE7FE22E41C -1 +1CE422FEE7BD8DE20000000000000000 Check bug with hashing of const integer values 11862823756610506724 11862823756610506724 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 900b99f548a..80180cc568f 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -269,9 +269,9 @@ select sipHash64Keyed(toUInt64(0), '1'); -- { serverError 48 } select sipHash128Keyed(toUInt64(0), '1'); -- { serverError 48 } select hex(sipHash64()); -SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; +SELECT hex(sipHash128()); select hex(sipHash64Keyed()); -SELECT hex(sipHash128Keyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; +SELECT hex(sipHash128Keyed()); SELECT 'Check bug with hashing of const integer values'; DROP TABLE IF EXISTS tab; diff --git a/tests/queries/0_stateless/02552_siphash128_reference.reference b/tests/queries/0_stateless/02552_siphash128_reference.reference index d00491fd7e5..6aa6c7ae075 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.reference +++ b/tests/queries/0_stateless/02552_siphash128_reference.reference @@ -1 +1 @@ -1 +1CE422FEE7BD8DE20000000000000000 diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql index 200954c3b57..f3cfa6093dc 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.sql +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -1 +1 @@ -SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; +SELECT hex(sipHash128Reference()); From 6573ba537819ce03dd644ff02bdf7341bcc26d58 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Jul 2023 19:37:28 -0400 Subject: [PATCH 1271/2047] Temporary returning metadata_cache.xml into tests config --- tests/config/install.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/install.sh b/tests/config/install.sh index 9aaadbc74a5..50f2627d37c 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -34,6 +34,7 @@ ln -sf $SRC_PATH/config.d/keeper_port.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/merge_tree.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/lost_forever_check.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/metadata_cache.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/prometheus.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/ From 9cc87b642b5c9077d563ee006f85065a158927ae Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 27 Jul 2023 16:58:35 -0700 Subject: [PATCH 1272/2047] Fix compilation error --- src/Common/TransformEndianness.hpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 0a9055dde15..fe43861f66f 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -3,6 +3,8 @@ #include #include +#include + #include namespace DB @@ -65,4 +67,11 @@ inline void transformEndianness(StrongTypedef & x) { transformEndianness(x.toUnderType()); } + +template +inline void transformEndianness(CityHash_v1_0_2::uint128 & x) +{ + transformEndianness(x.low64); + transformEndianness(x.high64); +} } From b3351bb547b8753b405d820925f8f4270be6132d Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 28 Jul 2023 03:36:23 +0000 Subject: [PATCH 1273/2047] partially fixed 01747_system_session_log_long test --- src/Core/PostgreSQLProtocol.h | 49 +++-- tests/config/users.d/session_log_test.xml | 2 +- .../01747_system_session_log_long.reference | 198 ++++++++++++------ .../01747_system_session_log_long.sh | 119 ++++++----- 4 files changed, 224 insertions(+), 144 deletions(-) rename tests/queries/{bugs => 0_stateless}/01747_system_session_log_long.reference (73%) rename tests/queries/{bugs => 0_stateless}/01747_system_session_log_long.sh (78%) diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index 8c0654b559f..b0d7646a5f7 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -805,20 +805,9 @@ protected: const String & user_name, const String & password, Session & session, - Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) { - try - { - session.authenticate(user_name, password, address); - } - catch (const Exception &) - { - mt.send( - Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "28P01", "Invalid user or password"), - true); - throw; - } + session.authenticate(user_name, password, address); } public: @@ -839,10 +828,10 @@ public: void authenticate( const String & user_name, Session & session, - Messaging::MessageTransport & mt, + [[maybe_unused]] Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) override { - return setPassword(user_name, "", session, mt, address); + return setPassword(user_name, "", session, address); } AuthenticationType getType() const override @@ -866,7 +855,7 @@ public: if (type == Messaging::FrontMessageType::PASSWORD_MESSAGE) { std::unique_ptr password = mt.receive(); - return setPassword(user_name, password->password, session, mt, address); + return setPassword(user_name, password->password, session, address); } else throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, @@ -901,20 +890,30 @@ public: Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) { - const AuthenticationType user_auth_type = session.getAuthenticationTypeOrLogInFailure(user_name); - if (type_to_method.find(user_auth_type) != type_to_method.end()) + AuthenticationType user_auth_type; + try { - type_to_method[user_auth_type]->authenticate(user_name, session, mt, address); - mt.send(Messaging::AuthenticationOk(), true); - LOG_DEBUG(log, "Authentication for user {} was successful.", user_name); - return; + user_auth_type = session.getAuthenticationTypeOrLogInFailure(user_name); + if (type_to_method.find(user_auth_type) != type_to_method.end()) + { + type_to_method[user_auth_type]->authenticate(user_name, session, mt, address); + mt.send(Messaging::AuthenticationOk(), true); + LOG_DEBUG(log, "Authentication for user {} was successful.", user_name); + return; + } + } + catch (const Exception&) + { + mt.send(Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "28P01", "Invalid user or password"), + true); + + throw; } - mt.send( - Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "0A000", "Authentication method is not supported"), - true); + mt.send(Messaging::ErrorOrNoticeResponse(Messaging::ErrorOrNoticeResponse::ERROR, "0A000", "Authentication method is not supported"), + true); - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Authentication type {} is not supported.", user_auth_type); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Authentication method is not supported: {}", user_auth_type); } }; } diff --git a/tests/config/users.d/session_log_test.xml b/tests/config/users.d/session_log_test.xml index daddaa6e4b9..cc2c2c5fcde 100644 --- a/tests/config/users.d/session_log_test.xml +++ b/tests/config/users.d/session_log_test.xml @@ -17,7 +17,7 @@ - + ::1 127.0.0.1 diff --git a/tests/queries/bugs/01747_system_session_log_long.reference b/tests/queries/0_stateless/01747_system_session_log_long.reference similarity index 73% rename from tests/queries/bugs/01747_system_session_log_long.reference rename to tests/queries/0_stateless/01747_system_session_log_long.reference index 9ecf7e05421..e4f0b6f6076 100644 --- a/tests/queries/bugs/01747_system_session_log_long.reference +++ b/tests/queries/0_stateless/01747_system_session_log_long.reference @@ -4,215 +4,291 @@ TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # no_password - No profiles no roles TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # no_password - Two profiles, no roles TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # no_password - Two profiles and two simple roles TCP endpoint TCP 'wrong password' case is skipped for no_password. HTTP endpoint HTTP 'wrong password' case is skipped for no_password. -MySQL endpoint +HTTP endpoint with named session +HTTP 'wrong password' case is skipped for no_password. +MySQL endpoint no_password +Wrong username +Wrong password MySQL 'wrong password' case is skipped for no_password. +PostrgreSQL endpoint +PostgreSQL 'wrong password' case is skipped for no_password. # plaintext_password - No profiles no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint plaintext_password +Wrong username +Wrong password +PostrgreSQL endpoint # plaintext_password - Two profiles, no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint plaintext_password +Wrong username +Wrong password +PostrgreSQL endpoint # plaintext_password - Two profiles and two simple roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint plaintext_password +Wrong username +Wrong password +PostrgreSQL endpoint # sha256_password - No profiles no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint sha256_password MySQL 'successful login' case is skipped for sha256_password. +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for sha256_password # sha256_password - Two profiles, no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint sha256_password MySQL 'successful login' case is skipped for sha256_password. +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for sha256_password # sha256_password - Two profiles and two simple roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint sha256_password MySQL 'successful login' case is skipped for sha256_password. +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for sha256_password # double_sha1_password - No profiles no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint double_sha1_password +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for double_sha1_password # double_sha1_password - Two profiles, no roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint double_sha1_password +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for double_sha1_password # double_sha1_password - Two profiles and two simple roles TCP endpoint HTTP endpoint -MySQL endpoint +HTTP endpoint with named session +MySQL endpoint double_sha1_password +Wrong username +Wrong password +PostrgreSQL endpoint +PostgreSQL tests are skipped for double_sha1_password ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP LoginFailure 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure many +${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL LoginFailure many ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL Logout 1 ${BASE_USERNAME}_no_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_no_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_no_password_no_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_no_password_no_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_no_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_no_password_two_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_no_roles MySQL Logout 1 ${BASE_USERNAME}_no_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_no_password_two_profiles_two_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_no_password_two_profiles_two_roles MySQL Logout 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL Logout 1 +${BASE_USERNAME}_plaintext_password_no_profiles_no_roles PostgreSQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_no_roles PostgreSQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP LoginFailure 1 ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure 1 -${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure many +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL LoginFailure many ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL LoginSuccess 1 ${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL Logout 1 +${BASE_USERNAME}_plaintext_password_two_profiles_two_roles PostgreSQL LoginFailure many ${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_sha256_password_no_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP LoginFailure 1 ${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP LoginSuccess 1 ${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP Logout 1 -${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure 1 -${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP Logout 1 +${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure many +${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginSuccess many +${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP Logout many ${BASE_USERNAME}_sha256_password_two_profiles_no_roles MySQL LoginFailure many ${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP LoginFailure 1 ${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP LoginSuccess 1 ${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP Logout 1 -${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure 1 -${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginSuccess 1 -${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP Logout 1 +${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure many +${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginSuccess many +${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP Logout many ${BASE_USERNAME}_sha256_password_two_profiles_two_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_no_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_double_sha1_password_two_profiles_two_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_no_password_no_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_no_password_two_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_no_password_two_profiles_two_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_plaintext_password_no_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_no_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles MySQL LoginFailure many +invalid_${BASE_USERNAME}_plaintext_password_two_profiles_two_roles PostgreSQL LoginFailure many invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_sha256_password_no_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_no_roles MySQL LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles TCP LoginFailure 1 -invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure 1 +invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles HTTP LoginFailure many invalid_${BASE_USERNAME}_sha256_password_two_profiles_two_roles MySQL LoginFailure many invalid_session_log_test_xml_user TCP LoginFailure 1 -invalid_session_log_test_xml_user HTTP LoginFailure 1 +invalid_session_log_test_xml_user HTTP LoginFailure many invalid_session_log_test_xml_user MySQL LoginFailure many +invalid_session_log_test_xml_user PostgreSQL LoginFailure many session_log_test_xml_user TCP LoginSuccess 1 session_log_test_xml_user TCP Logout 1 -session_log_test_xml_user HTTP LoginSuccess 1 -session_log_test_xml_user HTTP Logout 1 +session_log_test_xml_user HTTP LoginSuccess many +session_log_test_xml_user HTTP Logout many session_log_test_xml_user MySQL LoginSuccess 1 session_log_test_xml_user MySQL Logout 1 diff --git a/tests/queries/bugs/01747_system_session_log_long.sh b/tests/queries/0_stateless/01747_system_session_log_long.sh similarity index 78% rename from tests/queries/bugs/01747_system_session_log_long.sh rename to tests/queries/0_stateless/01747_system_session_log_long.sh index 9b127e0b48d..c6e93f4abd7 100755 --- a/tests/queries/bugs/01747_system_session_log_long.sh +++ b/tests/queries/0_stateless/01747_system_session_log_long.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash # Tags: long, no-parallel, no-fasttest -# Tag no-fasttest: Accesses CH via mysql table function (which is unavailable) ################################################################################################## # Verify that login, logout, and login failure events are properly stored in system.session_log @@ -11,9 +10,8 @@ # Using multiple protocols # * native TCP protocol with CH client # * HTTP with CURL -# * MySQL - CH server accesses itself via mysql table function, query typically fails (unrelated) -# but auth should be performed properly. -# * PostgreSQL - CH server accesses itself via postgresql table function (currently out of order). +# * MySQL - CH server accesses itself via mysql table function. +# * PostgreSQL - CH server accesses itself via postgresql table function, but can't execute query (No LOGIN SUCCESS entry). # * gRPC - not done yet # # There is way to control how many time a query (e.g. via mysql table function) is retried @@ -53,7 +51,7 @@ function reportError() function executeQuery() { - ## Execute query (provided via heredoc or herestring) and print query in case of error. + # Execute query (provided via heredoc or herestring) and print query in case of error. trap 'rm -f ${TMP_QUERY_FILE}; trap - ERR RETURN' RETURN # Since we want to report with current values supplied to this function call # shellcheck disable=SC2064 @@ -82,7 +80,7 @@ trap "cleanup" EXIT function executeQueryExpectError() { cat - > "${TMP_QUERY_FILE}" - ! ${CLICKHOUSE_CLIENT} "${@}" --multiquery --queries-file "${TMP_QUERY_FILE}" 2>&1 | tee -a ${TMP_QUERY_FILE} + ! ${CLICKHOUSE_CLIENT} --multiquery --queries-file "${TMP_QUERY_FILE}" "${@}" 2>&1 | tee -a ${TMP_QUERY_FILE} } function createUser() @@ -121,6 +119,8 @@ function createUser() executeQuery < Date: Thu, 27 Jul 2023 23:37:09 -0400 Subject: [PATCH 1274/2047] Fix tests, docs --- .../sql-reference/statements/create/quota.md | 1 + .../sql-reference/statements/create/role.md | 1 + .../statements/create/row-policy.md | 1 + .../statements/create/settings-profile.md | 1 + .../sql-reference/statements/create/user.md | 1 + docs/en/sql-reference/statements/drop.md | 10 +++--- docs/en/sql-reference/statements/move.md | 32 +++++++++++++++++ src/Access/IAccessStorage.cpp | 15 +++++--- src/Access/MultipleAccessStorage.cpp | 4 +-- .../InterpreterMoveAccessEntityQuery.cpp | 7 ++-- .../test.py | 11 +++--- tests/integration/test_quota/test.py | 36 +++++++++---------- 12 files changed, 81 insertions(+), 39 deletions(-) create mode 100644 docs/en/sql-reference/statements/move.md diff --git a/docs/en/sql-reference/statements/create/quota.md b/docs/en/sql-reference/statements/create/quota.md index c69285171ab..a6ced870c18 100644 --- a/docs/en/sql-reference/statements/create/quota.md +++ b/docs/en/sql-reference/statements/create/quota.md @@ -11,6 +11,7 @@ Syntax: ``` sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] + [IN access_storage_type] [KEYED BY {user_name | ip_address | client_key | client_key,user_name | client_key,ip_address} | NOT KEYED] [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year} {MAX { {queries | query_selects | query_inserts | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number } [,...] | diff --git a/docs/en/sql-reference/statements/create/role.md b/docs/en/sql-reference/statements/create/role.md index 9b14e220e1f..4b6fffe4f60 100644 --- a/docs/en/sql-reference/statements/create/role.md +++ b/docs/en/sql-reference/statements/create/role.md @@ -11,6 +11,7 @@ Syntax: ``` sql CREATE ROLE [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...] + [IN access_storage_type] [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [CONST|READONLY|WRITABLE|CHANGEABLE_IN_READONLY] | PROFILE 'profile_name'] [,...] ``` diff --git a/docs/en/sql-reference/statements/create/row-policy.md b/docs/en/sql-reference/statements/create/row-policy.md index 83bb2e6bb9a..cd7718793bd 100644 --- a/docs/en/sql-reference/statements/create/row-policy.md +++ b/docs/en/sql-reference/statements/create/row-policy.md @@ -16,6 +16,7 @@ Syntax: ``` sql CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name1 [ON CLUSTER cluster_name1] ON [db1.]table1|db1.* [, policy_name2 [ON CLUSTER cluster_name2] ON [db2.]table2|db2.* ...] + [IN access_storage_type] [FOR SELECT] USING condition [AS {PERMISSIVE | RESTRICTIVE}] [TO {role1 [, role2 ...] | ALL | ALL EXCEPT role1 [, role2 ...]}] diff --git a/docs/en/sql-reference/statements/create/settings-profile.md b/docs/en/sql-reference/statements/create/settings-profile.md index 8e221a4d82f..d8afce9d6de 100644 --- a/docs/en/sql-reference/statements/create/settings-profile.md +++ b/docs/en/sql-reference/statements/create/settings-profile.md @@ -12,6 +12,7 @@ Syntax: ``` sql CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...] + [IN access_storage_type] [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [CONST|READONLY|WRITABLE|CHANGEABLE_IN_READONLY] | INHERIT 'profile_name'] [,...] ``` diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index d168be63c36..11d4eae7bc8 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -14,6 +14,7 @@ CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...] [NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name'}] [HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] + [IN access_storage_type] [DEFAULT ROLE role [,...]] [DEFAULT DATABASE database | NONE] [GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]] diff --git a/docs/en/sql-reference/statements/drop.md b/docs/en/sql-reference/statements/drop.md index b6208c2fd52..245fd22e57c 100644 --- a/docs/en/sql-reference/statements/drop.md +++ b/docs/en/sql-reference/statements/drop.md @@ -49,7 +49,7 @@ Deletes a user. Syntax: ``` sql -DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP ROLE @@ -59,7 +59,7 @@ Deletes a role. The deleted role is revoked from all the entities where it was a Syntax: ``` sql -DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP ROW POLICY @@ -69,7 +69,7 @@ Deletes a row policy. Deleted row policy is revoked from all the entities where Syntax: ``` sql -DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name] +DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP QUOTA @@ -79,7 +79,7 @@ Deletes a quota. The deleted quota is revoked from all the entities where it was Syntax: ``` sql -DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP SETTINGS PROFILE @@ -89,7 +89,7 @@ Deletes a settings profile. The deleted settings profile is revoked from all the Syntax: ``` sql -DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP VIEW diff --git a/docs/en/sql-reference/statements/move.md b/docs/en/sql-reference/statements/move.md new file mode 100644 index 00000000000..fac738ff711 --- /dev/null +++ b/docs/en/sql-reference/statements/move.md @@ -0,0 +1,32 @@ +--- +slug: /en/sql-reference/statements/move +sidebar_position: 54 +sidebar_label: MOVE +--- + +# MOVE access entity statement + +This statement allows to move an access entity from one access storage to another. + +Syntax: + +```sql +MOVE {USER, ROLE, QUOTA, SETTINGS PROFILE, ROW POLICY} name1 [, name2, ...] TO access_storage_type +``` + +Currently, there are five access storages in ClickHouse: + - `local_directory` + - `memory` + - `replicated` + - `users_xml` (ro) + - `ldap` (ro) + +Examples: + +```sql +MOVE USER test TO local_directory +``` + +```sql +MOVE ROLE test TO memory +``` diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 468c93c1116..ec25fd505f0 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -202,16 +202,21 @@ std::vector IAccessStorage::insert(const std::vector & mu std::vector IAccessStorage::insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists, bool throw_if_exists) { - if (!ids.empty()) - assert(multiple_entities.size() == ids.size()); + assert(ids.empty() || (multiple_entities.size() == ids.size())); if (multiple_entities.empty()) return {}; if (multiple_entities.size() == 1) { - if (auto id = insert(multiple_entities[0], replace_if_exists, throw_if_exists)) - return {*id}; + UUID id; + if (!ids.empty()) + id = ids[0]; + else + id = generateRandomID(); + + if (insert(id, multiple_entities[0], replace_if_exists, throw_if_exists)) + return {id}; return {}; } @@ -229,7 +234,7 @@ std::vector IAccessStorage::insert(const std::vector & mu else id = generateRandomID(); - if (insertImpl(id, entity, replace_if_exists, throw_if_exists)) + if (insert(id, entity, replace_if_exists, throw_if_exists)) { successfully_inserted.push_back(entity); new_ids.push_back(id); diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index e6665349ae6..c6c9ede9e1c 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -230,8 +230,8 @@ StoragePtr MultipleAccessStorage::findExcludingStorage(AccessEntityType type, co void MultipleAccessStorage::moveAccessEntities(const std::vector & ids, const String & source_storage_name, const String & destination_storage_name) { - auto source_storage = findStorageByName(source_storage_name); - auto destination_storage = findStorageByName(destination_storage_name); + auto source_storage = getStorageByName(source_storage_name); + auto destination_storage = getStorageByName(destination_storage_name); auto to_move = source_storage->read(ids); source_storage->remove(ids); diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp index ac7ff4efcb6..49e90783a59 100644 --- a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp @@ -28,12 +28,9 @@ BlockIO InterpreterMoveAccessEntityQuery::execute() std::vector ids; if (query.type == AccessEntityType::ROW_POLICY) - ids = access_control.find(query.type, query.row_policy_names->toStrings()); + ids = access_control.getIDs(query.type, query.row_policy_names->toStrings()); else - ids = access_control.find(query.type, query.names); - - if (ids.empty()) - return {}; + ids = access_control.getIDs(query.type, query.names); /// Validate that all entities are from the same storage. const auto source_storage = access_control.findStorage(ids.front()); diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index 4c2696158ec..77aea411394 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -76,6 +76,13 @@ def execute_test_for_access_type(access_type: str, system_table_name: str): with pytest.raises(QueryRuntimeException): node.query(f"MOVE {access_type} test6 TO users_xml") + node.query(f"DROP {access_type} test1") + node.query(f"DROP {access_type} test2") + node.query(f"DROP {access_type} test3") + node.query(f"DROP {access_type} test4") + node.query(f"DROP {access_type} test5") + node.query(f"DROP {access_type} test6") + def test_roles(): execute_test_for_access_type("ROLE", "roles") @@ -93,10 +100,6 @@ def test_quotas(): execute_test_for_access_type("QUOTA", "quotas") -def test_row_policies(): - execute_test_for_access_type("ROW POLICY", "row_policies") - - def test_role_from_different_storages(): node.query("CREATE ROLE default_role") node.query("GRANT SELECT ON system.* TO default_role") diff --git a/tests/integration/test_quota/test.py b/tests/integration/test_quota/test.py index 4f98b9a0d0d..cec14b0af73 100644 --- a/tests/integration/test_quota/test.py +++ b/tests/integration/test_quota/test.py @@ -105,7 +105,7 @@ def test_quota_from_users_xml(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -252,7 +252,7 @@ def test_simpliest_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[]", 0, @@ -326,7 +326,7 @@ def test_tracking_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -446,7 +446,7 @@ def test_exceed_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -523,7 +523,7 @@ def test_exceed_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -613,7 +613,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -675,7 +675,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952,63113904]", 0, @@ -824,7 +824,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -914,7 +914,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[]", 0, @@ -986,7 +986,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -1048,7 +1048,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -1111,7 +1111,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1121,7 +1121,7 @@ def test_add_remove_quota(): [ "myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", - "users.xml", + "users_xml", "['client_key','user_name']", "[3600,2629746]", 0, @@ -1214,7 +1214,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1283,7 +1283,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1346,7 +1346,7 @@ def test_reload_users_xml_by_timer(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1385,7 +1385,7 @@ def test_reload_users_xml_by_timer(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", ["user_name"], "[31556952]", 0, @@ -1554,7 +1554,7 @@ def test_query_inserts(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, From 0e15f098d19577845197cd025a53390c26f001e6 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 02:09:57 -0400 Subject: [PATCH 1275/2047] Yet another fix test --- programs/keeper-client/Commands.cpp | 17 +++-- tests/integration/test_keeper_client/test.py | 80 +++++++++----------- 2 files changed, 46 insertions(+), 51 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 985e844afdf..cbdcde4f89a 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -27,8 +27,15 @@ void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con auto children = client->zookeeper->getChildren(path); std::sort(children.begin(), children.end()); + + bool need_space = false; for (const auto & child : children) - std::cout << child << " "; + { + if (std::exchange(need_space, true)) + std::cout << " "; + + std::cout << child; + } std::cout << "\n"; } @@ -194,7 +201,7 @@ void FindSuperNodes::execute(const ASTKeeperQuery * query, KeeperClient * client if (stat.numChildren >= static_cast(threshold)) { - std::cout << path << "\t" << stat.numChildren << "\n"; + std::cout << static_cast(path) << "\t" << stat.numChildren << "\n"; return; } @@ -219,15 +226,15 @@ void DeleteStableBackups::execute(const ASTKeeperQuery * /* query */, KeeperClie "You are going to delete all inactive backups in /clickhouse/backups.", [client] { - String backup_root = "/clickhouse/backups"; + fs::path backup_root = "/clickhouse/backups"; auto backups = client->zookeeper->getChildren(backup_root); for (const auto & child : backups) { - String backup_path = backup_root + "/" + child; + auto backup_path = backup_root / child; std::cout << "Found backup " << backup_path << ", checking if it's active\n"; - String stage_path = backup_path + "/stage"; + String stage_path = backup_path / "stage"; auto stages = client->zookeeper->getChildren(stage_path); bool is_active = false; diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 5115c7d3789..3a54d0d175d 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -14,7 +14,7 @@ node = cluster.add_instance( ) -@pytest.fixture(scope="module") +@pytest.fixture(scope="module", autouse=True) def started_cluster(): try: cluster.start() @@ -24,30 +24,24 @@ def started_cluster(): cluster.shutdown() -class KeeperClient: - def __init__(self, started_cluster: ClickHouseCluster): - self.cluster = started_cluster - - def query(self, query: str): - return CommandRequest( - [ - self.cluster.server_bin_path, - "keeper-client", - "--host", - str(cluster.get_instance_ip("zoo1")), - "--port", - str(cluster.zookeeper_port), - "-q", - query, - ], - stdin="", - ) +def keeper_query(query: str): + return CommandRequest( + [ + cluster.server_bin_path, + "keeper-client", + "--host", + str(cluster.get_instance_ip("zoo1")), + "--port", + str(cluster.zookeeper_port), + "-q", + query, + ], + stdin="", + ) -def test_big_family(started_cluster: ClickHouseCluster): - client = KeeperClient(started_cluster) - - command = client.query( +def test_big_family(): + command = keeper_query( "create test_big_family foo;" "create test_big_family/1 foo;" "create test_big_family/1/1 foo;" @@ -55,6 +49,7 @@ def test_big_family(started_cluster: ClickHouseCluster): "create test_big_family/1/3 foo;" "create test_big_family/1/4 foo;" "create test_big_family/1/5 foo;" + "create test_big_family/2 foo;" "create test_big_family/2/1 foo;" "create test_big_family/2/2 foo;" "create test_big_family/2/3 foo;" @@ -76,7 +71,7 @@ def test_big_family(started_cluster: ClickHouseCluster): ] ) - command = client.query("find_big_family test_big_family 1;") + command = keeper_query("find_big_family test_big_family 1;") assert command.get_answer() == TSV( [ @@ -85,16 +80,16 @@ def test_big_family(started_cluster: ClickHouseCluster): ) -def test_find_super_nodes(started_cluster: ClickHouseCluster): - client = KeeperClient(started_cluster) - - command = client.query( +def test_find_super_nodes(): + command = keeper_query( + "create test_find_super_nodes foo;" "create test_find_super_nodes/1 foo;" "create test_find_super_nodes/1/1 foo;" "create test_find_super_nodes/1/2 foo;" "create test_find_super_nodes/1/3 foo;" "create test_find_super_nodes/1/4 foo;" "create test_find_super_nodes/1/5 foo;" + "create test_find_super_nodes/2 foo;" "create test_find_super_nodes/2/1 foo;" "create test_find_super_nodes/2/2 foo;" "create test_find_super_nodes/2/3 foo;" @@ -111,11 +106,8 @@ def test_find_super_nodes(started_cluster: ClickHouseCluster): ) -def test_delete_stable_backups(started_cluster: ClickHouseCluster): - client = KeeperClient(started_cluster) - - command = client.query( - "create /clickhouse foo;" +def test_delete_stable_backups(): + command = keeper_query( "create /clickhouse/backups foo;" "create /clickhouse/backups/1 foo;" "create /clickhouse/backups/1/stage foo;" @@ -130,18 +122,16 @@ def test_delete_stable_backups(started_cluster: ClickHouseCluster): assert command.get_answer() == ( "You are going to delete all inactive backups in /clickhouse/backups. Continue?\n" - "Found backup /clickhouse/backups/1, checking if it's active\n" - "Backup /clickhouse/backups/1 is active, not going to delete\n" - "Found backup /clickhouse/backups/2, checking if it's active\n" - "Backup /clickhouse/backups/2 is not active, deleting it\n" - "1" + "Found backup \"/clickhouse/backups/1\", checking if it's active\n" + "Backup \"/clickhouse/backups/1\" is active, not going to delete\n" + "Found backup \"/clickhouse/backups/2\", checking if it's active\n" + "Backup \"/clickhouse/backups/2\" is not active, deleting it\n" + "1\n" ) -def test_base_commands(started_cluster: ClickHouseCluster): - client = KeeperClient(started_cluster) - - command = client.query( +def test_base_commands(): + command = keeper_query( "create test_create_zk_node1 testvalue1;" "create test_create_zk_node_2 testvalue2;" "get test_create_zk_node1;" @@ -150,8 +140,6 @@ def test_base_commands(started_cluster: ClickHouseCluster): assert command.get_answer() == "testvalue1\n" -def test_four_letter_word_commands(started_cluster: ClickHouseCluster): - client = KeeperClient(started_cluster) - - command = client.query("ruok") +def test_four_letter_word_commands(): + command = keeper_query("ruok") assert command.get_answer() == "imok\n" From 9fd99ce81496867aab1499bf59ce2e6121065c5e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Jul 2023 06:34:05 +0000 Subject: [PATCH 1276/2047] Automatic style fix --- tests/integration/test_keeper_client/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 3a54d0d175d..3187ce10d2a 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -122,10 +122,10 @@ def test_delete_stable_backups(): assert command.get_answer() == ( "You are going to delete all inactive backups in /clickhouse/backups. Continue?\n" - "Found backup \"/clickhouse/backups/1\", checking if it's active\n" - "Backup \"/clickhouse/backups/1\" is active, not going to delete\n" - "Found backup \"/clickhouse/backups/2\", checking if it's active\n" - "Backup \"/clickhouse/backups/2\" is not active, deleting it\n" + 'Found backup "/clickhouse/backups/1", checking if it\'s active\n' + 'Backup "/clickhouse/backups/1" is active, not going to delete\n' + 'Found backup "/clickhouse/backups/2", checking if it\'s active\n' + 'Backup "/clickhouse/backups/2" is not active, deleting it\n' "1\n" ) From 97e63d523fa25053e9d141b6566e55a4e2feb86c Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 28 Jul 2023 07:08:11 +0000 Subject: [PATCH 1277/2047] use same executor for GET_PART and ATTACH_PART --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d62a1d960e6..2c2cea0af2b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -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( [this, selected_entry] () mutable From 63b05da1f2da6cee086d1154ddc670329aba667d Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 28 Jul 2023 07:23:34 +0000 Subject: [PATCH 1278/2047] System logs improvements --- .../settings.md | 128 ++++++++++- docs/en/operations/system-tables/index.md | 4 + .../settings.md | 212 +++++++++++++++--- docs/ru/operations/system-tables/index.md | 4 + programs/server/config.xml | 53 +++++ src/Common/SystemLogBase.cpp | 81 ++++--- src/Common/SystemLogBase.h | 43 +++- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- src/Daemon/BaseDaemon.cpp | 4 + .../IO/AsynchronousBoundedReadBuffer.cpp | 2 +- .../IO/CachedOnDiskReadBufferFromFile.cpp | 2 +- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 4 +- src/Interpreters/Context.cpp | 6 + src/Interpreters/Context.h | 3 + src/Interpreters/CrashLog.cpp | 5 +- src/Interpreters/CrashLog.h | 5 + src/Interpreters/MetricLog.cpp | 2 +- src/Interpreters/PartLog.cpp | 2 +- src/Interpreters/ProcessorsProfileLog.cpp | 7 - src/Interpreters/ProcessorsProfileLog.h | 7 +- src/Interpreters/Session.cpp | 2 +- src/Interpreters/SessionLog.cpp | 6 +- src/Interpreters/SystemLog.cpp | 92 +++++--- src/Interpreters/SystemLog.h | 17 +- src/Interpreters/TextLog.cpp | 11 +- src/Interpreters/TextLog.h | 13 +- src/Interpreters/ThreadStatusExt.cpp | 4 +- src/Interpreters/TraceCollector.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 2 +- src/Interpreters/TransactionsInfoLog.cpp | 2 +- src/Loggers/Loggers.cpp | 44 +++- src/Loggers/OwnSplitChannel.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- tests/integration/parallel_skip.json | 9 +- .../test_crash_log/configs/crash_log.xml | 16 ++ tests/integration/test_crash_log/test.py | 19 +- .../test_system_flush_logs/test.py | 99 +++++++- .../test_system_logs/test_system_logs.py | 50 +++++ 40 files changed, 794 insertions(+), 178 deletions(-) create mode 100644 tests/integration/test_crash_log/configs/crash_log.xml diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index a6ae517e401..e9f0f0dae00 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -512,7 +512,7 @@ Both the cache for `local_disk`, and temporary data will be stored in `/tiny_loc cache local_disk /tiny_local_cache/ - 10M + 10M 1M 1 0 @@ -1592,6 +1592,10 @@ To manually turn on metrics history collection [`system.metric_log`](../../opera metric_log
7500 1000 + 1048576 + 8192 + 524288 + false ``` @@ -1695,6 +1699,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1706,6 +1718,10 @@ Use the following parameters to configure logging: part_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1773,6 +1789,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1786,6 +1810,10 @@ If the table does not exist, ClickHouse will create it. If the structure of the query_log
Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1831,6 +1859,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size_rows, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1844,6 +1880,10 @@ If the table does not exist, ClickHouse will create it. If the structure of the query_thread_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1861,6 +1901,14 @@ Use the following parameters to configure logging: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1874,6 +1922,10 @@ If the table does not exist, ClickHouse will create it. If the structure of the query_views_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1890,6 +1942,14 @@ Parameters: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1901,13 +1961,16 @@ Parameters: system text_log
7500 + 1048576 + 8192 + 524288 + false Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day ``` - ## trace_log {#server_configuration_parameters-trace_log} Settings for the [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table operation. @@ -1920,6 +1983,12 @@ Parameters: - `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` or `order_by` defined. - `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. - `storage_policy` – Name of storage policy to use for the table (optional) - `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). @@ -1931,6 +2000,10 @@ The default server configuration file `config.xml` contains the following settin trace_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1945,9 +2018,18 @@ Parameters: - `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. - `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) for a system table. Can't be used if `partition_by` defined. - `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. - `storage_policy` – Name of storage policy to use for the table (optional) **Example** + ```xml @@ -1955,11 +2037,53 @@ Parameters: asynchronous_insert_log
7500 toYYYYMM(event_date) + 1048576 + 8192 + 524288 + false
``` +## crash_log {#server_configuration_parameters-crash_log} + +Settings for the [crash_log](../../operations/system-tables/crash-log.md) system table operation. + +Parameters: + +- `database` — Database for storing a table. +- `table` — Table name. +- `partition_by` — [Custom partitioning key](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) for a system table. Can't be used if `engine` defined. +- `order_by` - [Custom sorting key](../../engines/table-engines/mergetree-family/mergetree.md#order_by) for a system table. Can't be used if `engine` defined. +- `engine` - [MergeTree Engine Definition](../../engines/table-engines/mergetree-family/index.md) for a system table. Can't be used if `partition_by` or `order_by` defined. +- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `max_size_rows` – Maximal size in lines for the logs. When non-flushed logs amount reaches max_size, logs dumped to the disk. +Default: 1048576. +- `reserved_size_rows` – Pre-allocated memory size in lines for the logs. +Default: 8192. +- `buffer_size_rows_flush_threshold` – Lines amount threshold, reaching it launches flushing logs to the disk in background. +Default: `max_size_rows / 2`. +- `flush_on_crash` - Indication whether logs should be dumped to the disk in case of a crash. +Default: false. +- `storage_policy` – Name of storage policy to use for the table (optional) +- `settings` - [Additional parameters](../../engines/table-engines/mergetree-family/mergetree.md/#settings) that control the behavior of the MergeTree (optional). + +The default server configuration file `config.xml` contains the following settings section: + +``` xml + + system + crash_log
+ toYYYYMM(event_date) + 7500 + 1024 + 1024 + 512 + false +
+``` + ## query_masking_rules {#query-masking-rules} Regexp-based rules, which will be applied to queries as well as all log messages before storing them in server logs, diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index 1b720098fc7..a46f306f677 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -47,6 +47,10 @@ An example: ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024 --> 7500 + 1048576 + 8192 + 524288 + false ``` diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 421df3fe3eb..81a696bcfc1 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1058,6 +1058,10 @@ ClickHouse использует потоки из глобального пул metric_log
7500 1000 + 1048576 + 8192 + 524288 + false ``` @@ -1155,12 +1159,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` — имя базы данных; -- `table` — имя таблицы; -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. - +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. **Пример** ``` xml @@ -1169,6 +1180,10 @@ ClickHouse использует потоки из глобального пул part_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1218,11 +1233,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` — имя базы данных; -- `table` — имя таблицы, куда будет записываться лог; -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. @@ -1234,6 +1257,10 @@ ClickHouse использует потоки из глобального пул query_log
Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1245,11 +1272,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` — имя базы данных; -- `table` — имя таблицы, куда будет записываться лог; -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. @@ -1261,6 +1296,10 @@ ClickHouse использует потоки из глобального пул query_thread_log
toMonday(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1272,11 +1311,19 @@ ClickHouse использует потоки из глобального пул При настройке логирования используются следующие параметры: -- `database` – имя базы данных. -- `table` – имя системной таблицы, где будут логироваться запросы. -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать, если задан параметр `engine`. -- `engine` — устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать, если задан параметр `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. @@ -1288,6 +1335,10 @@ ClickHouse использует потоки из глобального пул query_views_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false ``` @@ -1297,12 +1348,20 @@ ClickHouse использует потоки из глобального пул Параметры: -- `level` — Максимальный уровень сообщения (по умолчанию `Trace`) которое будет сохранено в таблице. -- `database` — имя базы данных для хранения таблицы. -- `table` — имя таблицы, куда будут записываться текстовые сообщения. -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `level` — Максимальный уровень сообщения (по умолчанию `Trace`) которое будет сохранено в таблице. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. **Пример** ```xml @@ -1312,6 +1371,10 @@ ClickHouse использует потоки из глобального пул system text_log
7500 + 1048576 + 8192 + 524288 + false Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day @@ -1323,13 +1386,21 @@ ClickHouse использует потоки из глобального пул Настройки для [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) system table operation. -Parameters: +Параметры: -- `database` — Database for storing a table. -- `table` — Table name. -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. -- `flush_interval_milliseconds` — Interval for flushing data from the buffer in memory to the table. +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. По умолчанию файл настроек сервера `config.xml` содержит следующие настройки: @@ -1339,9 +1410,84 @@ Parameters: trace_log
toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 ``` +## asynchronous_insert_log {#server_configuration_parameters-asynchronous_insert_log} + +Настройки для asynchronous_insert_log Система для логирования ассинхронных вставок. + +Параметры: + +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1048576. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 8192. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: false. + +**Пример** + +```xml + + + system + asynchronous_insert_log
+ 7500 + toYYYYMM(event_date) + 1048576 + 8192 + 524288 + +
+
+``` + +## crash_log {#server_configuration_parameters-crash_log} + +Настройки для таблицы [crash_log](../../operations/system-tables/crash-log.md). + +Параметры: + +- `database` — имя базы данных; +- `table` — имя таблицы; +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../operations/server-configuration-parameters/settings.md). Нельзя использовать если используется `engine` +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. +- `max_size_rows` – максимальный размер в строках для буфера с логами. Когда буфер будет заполнен полностью, сбрасывает логи на диск. +Значение по умолчанию: 1024. +- `reserved_size_rows` – преаллоцированный размер в строках для буфера с логами. +Значение по умолчанию: 1024. +- `buffer_size_bytes_flush_threshold` – количество линий в логе при достижении которого логи начнут скидываться на диск в неблокирующем режиме. +Значение по умолчанию: `max_size / 2`. +- `flush_on_crash` - должны ли логи быть сброшены на диск в случае неожиданной остановки программы. +Значение по умолчанию: true. + +**Пример** + +``` xml + + system + crash_log
+ toYYYYMM(event_date) + 7500 + 1024 + 1024 + 512 + true +
+``` + ## query_masking_rules {#query-masking-rules} Правила, основанные на регулярных выражениях, которые будут применены для всех запросов, а также для всех сообщений перед сохранением их в лог на сервере, diff --git a/docs/ru/operations/system-tables/index.md b/docs/ru/operations/system-tables/index.md index 7ff368b1910..24f79cae212 100644 --- a/docs/ru/operations/system-tables/index.md +++ b/docs/ru/operations/system-tables/index.md @@ -45,6 +45,10 @@ sidebar_label: "Системные таблицы" ENGINE = MergeTree PARTITION BY toYYYYMM(event_date) ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024 --> 7500 + 1048576 + 8192 + 524288 + false ``` diff --git a/programs/server/config.xml b/programs/server/config.xml index 2a7dc1e576a..153cb728bb4 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1026,6 +1026,14 @@ 7500 + + 1048576 + + 8192 + + 524288 + + false @@ -1039,6 +1047,11 @@ toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + + false @@ -1084,7 +1109,11 @@ system metric_log
7500 + 1048576 + 8192 + 524288 1000 + false @@ -1151,6 +1196,10 @@ toYYYYMM(event_date) 7500 + 1048576 + 8192 + 524288 + false + in_order + + 2 + 4 + + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + 500 + + diff --git a/tests/integration/test_zookeeper_fallback_session/test.py b/tests/integration/test_zookeeper_fallback_session/test.py new file mode 100644 index 00000000000..570eca4f0a6 --- /dev/null +++ b/tests/integration/test_zookeeper_fallback_session/test.py @@ -0,0 +1,101 @@ +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.network import PartitionManager + + +cluster = ClickHouseCluster( + __file__, zookeeper_config_path="configs/zookeeper_load_balancing.xml" +) + +node1 = cluster.add_instance( + "node1", + with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_load_balancing.xml"], +) +node2 = cluster.add_instance( + "node2", + with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_load_balancing.xml"], +) +node3 = cluster.add_instance( + "node3", + with_zookeeper=True, + main_configs=["configs/remote_servers.xml", "configs/zookeeper_load_balancing.xml"], +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + for node in [node1, node2, node3]: + node.query("DROP TABLE IF EXISTS simple SYNC") + node.query( + """ + CREATE TABLE simple (date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; + """.format( + replica=node.name + ) + ) + yield cluster + finally: + cluster.shutdown() + + +def assert_uses_zk_node(node: ClickHouseInstance, zk_node): + def check_callback(host): + return host.strip() == expected_zk_ip_addr + + expected_zk_ip_addr = node.cluster.get_instance_ip(zk_node) + + host = node.query_with_retry( + "select host from system.zookeeper_connection", check_callback=check_callback + ) + assert host.strip() == expected_zk_ip_addr + + +def test_fallback_session(started_cluster: ClickHouseCluster): + # only leave connecting to zoo3 possible + with PartitionManager() as pm: + for node in started_cluster.instances.values(): + for zk in ["zoo1", "zoo2"]: + pm._add_rule( + { + "source": node.ip_address, + "destination": cluster.get_instance_ip(zk), + "action": "REJECT --reject-with tcp-reset", + } + ) + + for node in [node1, node2, node3]: + # all nodes will have to switch to zoo3 + assert_uses_zk_node(node, "zoo3") + + node1.query_with_retry("INSERT INTO simple VALUES ({0}, {0})".format(1)) + + # and replication still works + for node in [node2, node3]: + assert ( + node.query_with_retry( + "SELECT count() from simple", + check_callback=lambda count: count.strip() == "1", + ) + == "1\n" + ) + + # at this point network partitioning has been reverted. + # the nodes should switch to zoo1 automatically because of `in_order` load-balancing. + # otherwise they would connect to a random replica + for node in [node1, node2, node3]: + assert_uses_zk_node(node, "zoo1") + + node1.query_with_retry("INSERT INTO simple VALUES ({0}, {0})".format(2)) + for node in [node2, node3]: + assert ( + node.query_with_retry( + "SELECT count() from simple", + check_callback=lambda count: count.strip() == "2", + ) + == "2\n" + ) From e83e0ec2cd78a8bc09f34a7cde849531a59eafdb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Jul 2023 11:55:23 +0000 Subject: [PATCH 1283/2047] Fix build --- contrib/libarchive-cmake/CMakeLists.txt | 4 +- contrib/libarchive-cmake/config.h | 76 ++++++++++++++----- src/IO/Archives/ArchiveUtils.h | 7 ++ src/IO/Archives/SevenZipArchiveReader.cpp | 11 +-- src/IO/Archives/SevenZipArchiveReader.h | 3 - src/IO/Archives/TarArchiveReader.cpp | 12 +-- src/IO/Archives/TarArchiveReader.h | 3 - src/IO/Archives/util/archive_writers.h | 52 ------------- .../tests/gtest_archive_reader_and_writer.cpp | 66 +++++++++++++--- src/Storages/StorageFile.cpp | 2 + src/Storages/StorageFile.h | 2 +- src/TableFunctions/TableFunctionFile.cpp | 2 +- 12 files changed, 141 insertions(+), 99 deletions(-) create mode 100644 src/IO/Archives/ArchiveUtils.h delete mode 100644 src/IO/Archives/util/archive_writers.h diff --git a/contrib/libarchive-cmake/CMakeLists.txt b/contrib/libarchive-cmake/CMakeLists.txt index 4593f7f96c8..400bded56dc 100644 --- a/contrib/libarchive-cmake/CMakeLists.txt +++ b/contrib/libarchive-cmake/CMakeLists.txt @@ -1,7 +1,5 @@ set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/libarchive") -set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-reserved-id-macro") - set(SRCS "${LIBRARY_DIR}/libarchive/archive_acl.c" "${LIBRARY_DIR}/libarchive/archive_blake2sp_ref.c" @@ -146,4 +144,6 @@ target_compile_definitions(_libarchive PUBLIC HAVE_CONFIG_H ) +target_compile_options(_libarchive PRIVATE "-Wno-reserved-macro-identifier") + add_library(ch_contrib::libarchive ALIAS _libarchive) \ No newline at end of file diff --git a/contrib/libarchive-cmake/config.h b/contrib/libarchive-cmake/config.h index 2fa1bb8945d..9861e88359d 100644 --- a/contrib/libarchive-cmake/config.h +++ b/contrib/libarchive-cmake/config.h @@ -207,6 +207,9 @@ typedef uint64_t uintmax_t; /* MD5 via ARCHIVE_CRYPTO_MD5_LIBSYSTEM supported. */ /* #undef ARCHIVE_CRYPTO_MD5_LIBSYSTEM */ +/* MD5 via ARCHIVE_CRYPTO_MD5_MBEDTLS supported. */ +/* #undef ARCHIVE_CRYPTO_MD5_MBEDTLS */ + /* MD5 via ARCHIVE_CRYPTO_MD5_NETTLE supported. */ /* #undef ARCHIVE_CRYPTO_MD5_NETTLE */ @@ -222,6 +225,9 @@ typedef uint64_t uintmax_t; /* RMD160 via ARCHIVE_CRYPTO_RMD160_NETTLE supported. */ /* #undef ARCHIVE_CRYPTO_RMD160_NETTLE */ +/* RMD160 via ARCHIVE_CRYPTO_RMD160_MBEDTLS supported. */ +/* #undef ARCHIVE_CRYPTO_RMD160_MBEDTLS */ + /* RMD160 via ARCHIVE_CRYPTO_RMD160_OPENSSL supported. */ /* #undef ARCHIVE_CRYPTO_RMD160_OPENSSL */ @@ -231,6 +237,9 @@ typedef uint64_t uintmax_t; /* SHA1 via ARCHIVE_CRYPTO_SHA1_LIBSYSTEM supported. */ /* #undef ARCHIVE_CRYPTO_SHA1_LIBSYSTEM */ +/* SHA1 via ARCHIVE_CRYPTO_SHA1_MBEDTLS supported. */ +/* #undef ARCHIVE_CRYPTO_SHA1_MBEDTLS */ + /* SHA1 via ARCHIVE_CRYPTO_SHA1_NETTLE supported. */ /* #undef ARCHIVE_CRYPTO_SHA1_NETTLE */ @@ -252,6 +261,9 @@ typedef uint64_t uintmax_t; /* SHA256 via ARCHIVE_CRYPTO_SHA256_LIBSYSTEM supported. */ /* #undef ARCHIVE_CRYPTO_SHA256_LIBSYSTEM */ +/* SHA256 via ARCHIVE_CRYPTO_SHA256_MBEDTLS supported. */ +/* #undef ARCHIVE_CRYPTO_SHA256_MBEDTLS */ + /* SHA256 via ARCHIVE_CRYPTO_SHA256_NETTLE supported. */ /* #undef ARCHIVE_CRYPTO_SHA256_NETTLE */ @@ -273,6 +285,9 @@ typedef uint64_t uintmax_t; /* SHA384 via ARCHIVE_CRYPTO_SHA384_LIBSYSTEM supported. */ /* #undef ARCHIVE_CRYPTO_SHA384_LIBSYSTEM */ +/* SHA384 via ARCHIVE_CRYPTO_SHA384_MBEDTLS supported. */ +/* #undef ARCHIVE_CRYPTO_SHA384_MBEDTLS */ + /* SHA384 via ARCHIVE_CRYPTO_SHA384_NETTLE supported. */ /* #undef ARCHIVE_CRYPTO_SHA384_NETTLE */ @@ -294,6 +309,9 @@ typedef uint64_t uintmax_t; /* SHA512 via ARCHIVE_CRYPTO_SHA512_LIBSYSTEM supported. */ /* #undef ARCHIVE_CRYPTO_SHA512_LIBSYSTEM */ +/* SHA512 via ARCHIVE_CRYPTO_SHA512_MBEDTLS supported. */ +/* #undef ARCHIVE_CRYPTO_SHA512_MBEDTLS */ + /* SHA512 via ARCHIVE_CRYPTO_SHA512_NETTLE supported. */ /* #undef ARCHIVE_CRYPTO_SHA512_NETTLE */ @@ -313,16 +331,16 @@ typedef uint64_t uintmax_t; /* #undef ARCHIVE_XATTR_FREEBSD */ /* Linux xattr support */ -#define ARCHIVE_XATTR_LINUX 1 +/* #undef ARCHIVE_XATTR_LINUX */ /* Version number of bsdcpio */ -#define BSDCPIO_VERSION_STRING "3.6.3" +#define BSDCPIO_VERSION_STRING "3.7.0" /* Version number of bsdtar */ -#define BSDTAR_VERSION_STRING "3.6.3" +#define BSDTAR_VERSION_STRING "3.7.0" /* Version number of bsdcat */ -#define BSDCAT_VERSION_STRING "3.6.3" +#define BSDCAT_VERSION_STRING "3.7.0" /* Define to 1 if you have the `acl_create_entry' function. */ /* #undef HAVE_ACL_CREATE_ENTRY */ @@ -366,7 +384,7 @@ typedef uint64_t uintmax_t; /* Define to 1 if you have the header file. */ /* #undef HAVE_ATTR_XATTR_H */ -/* Define to 1 if you have the header file. */ +/* Define to 1 if you have the header file. */ /* #undef HAVE_BCRYPT_H */ /* Define to 1 if you have the header file. */ @@ -562,7 +580,7 @@ typedef uint64_t uintmax_t; /* #undef HAVE_FGETEA */ /* Define to 1 if you have the `fgetxattr' function. */ -#define HAVE_FGETXATTR 1 +/* #undef HAVE_FGETXATTR */ /* Define to 1 if you have the `flistea' function. */ /* #undef HAVE_FLISTEA */ @@ -570,6 +588,12 @@ typedef uint64_t uintmax_t; /* Define to 1 if you have the `flistxattr' function. */ #define HAVE_FLISTXATTR 1 +/* Define to 1 if you have the `fnmatch' function. */ +#define HAVE_FNMATCH 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_FNMATCH_H 1 + /* Define to 1 if you have the `fork' function. */ #define HAVE_FORK 1 @@ -580,7 +604,7 @@ typedef uint64_t uintmax_t; /* #undef HAVE_FSETEA */ /* Define to 1 if you have the `fsetxattr' function. */ -#define HAVE_FSETXATTR 1 +/* #undef HAVE_FSETXATTR */ /* Define to 1 if you have the `fstat' function. */ #define HAVE_FSTAT 1 @@ -618,6 +642,9 @@ typedef uint64_t uintmax_t; /* Define to 1 if you have the `getgrnam_r' function. */ #define HAVE_GETGRNAM_R 1 +/* Define to 1 if platform uses `optreset` to reset `getopt` */ +#define HAVE_GETOPT_OPTRESET 1 + /* Define to 1 if you have the `getpid' function. */ #define HAVE_GETPID 1 @@ -727,10 +754,10 @@ typedef uint64_t uintmax_t; #define HAVE_LIBXML2 1 /* Define to 1 if you have the header file. */ -#define HAVE_LIBXML_XMLREADER_H 0 +/* #undef HAVE_LIBXML_XMLREADER_H */ /* Define to 1 if you have the header file. */ -// #define HAVE_LIBXML_XMLWRITER_H 1 +/* #undef HAVE_LIBXML_XMLWRITER_H */ /* Define to 1 if you have the `z' library (-lz). */ /* #undef HAVE_LIBZ */ @@ -752,13 +779,13 @@ typedef uint64_t uintmax_t; #define HAVE_LINKAT 1 /* Define to 1 if you have the header file. */ -// #define HAVE_LINUX_FIEMAP_H 1 +/* #undef HAVE_LINUX_FIEMAP_H */ /* Define to 1 if you have the header file. */ #define HAVE_LINUX_FS_H 1 /* Define to 1 if you have the header file. */ -// #define HAVE_LINUX_MAGIC_H 0 +/* #undef HAVE_LINUX_MAGIC_H */ /* Define to 1 if you have the header file. */ #define HAVE_LINUX_TYPES_H 1 @@ -827,8 +854,17 @@ typedef uint64_t uintmax_t; /* Define to 1 if you have the header file. */ /* #undef HAVE_LZO_LZOCONF_H */ +/* Define to 1 if you have the header file. */ +/* #undef HAVE_MBEDTLS_AES_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_MBEDTLS_MD_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_MBEDTLS_PKCS5_H */ + /* Define to 1 if you have the `mbrtowc' function. */ -// #define HAVE_MBRTOWC 1 +/* #undef HAVE_MBRTOWC */ /* Define to 1 if you have the header file. */ /* #undef HAVE_MEMBERSHIP_H */ @@ -878,6 +914,9 @@ typedef uint64_t uintmax_t; /* Define to 1 if you have the `openat' function. */ #define HAVE_OPENAT 1 +/* Define to 1 if you have the header file. */ +/* #undef HAVE_OPENSSL_EVP_H */ + /* Define to 1 if you have the header file. */ #define HAVE_PATHS_H 1 @@ -909,7 +948,7 @@ typedef uint64_t uintmax_t; #define HAVE_PWD_H 1 /* Define to 1 if you have the `readdir_r' function. */ -/* #undef HAVE_READDIR_R */ +#define HAVE_READDIR_R 1 /* Define to 1 if you have the `readlink' function. */ #define HAVE_READLINK 1 @@ -1073,6 +1112,9 @@ typedef uint64_t uintmax_t; /* Define to 1 if you have the header file. */ #define HAVE_SYS_POLL_H 1 +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_QUEUE_H 1 + /* Define to 1 if you have the header file. */ /* #undef HAVE_SYS_RICHACL_H */ @@ -1111,7 +1153,7 @@ typedef uint64_t uintmax_t; #define HAVE_SYS_WAIT_H 1 /* Define to 1 if you have the header file. */ -// #define HAVE_SYS_XATTR_H 0 +/* #undef HAVE_SYS_XATTR_H */ /* Define to 1 if you have the `timegm' function. */ #define HAVE_TIMEGM 1 @@ -1234,10 +1276,10 @@ typedef uint64_t uintmax_t; #define ICONV_CONST /* Version number of libarchive as a single integer */ -#define LIBARCHIVE_VERSION_NUMBER "3006003" +#define LIBARCHIVE_VERSION_NUMBER "3007000" /* Version number of libarchive */ -#define LIBARCHIVE_VERSION_STRING "3.6.3" +#define LIBARCHIVE_VERSION_STRING "3.7.0" /* Define to 1 if `lstat' dereferences a symlink specified with a trailing slash. */ @@ -1291,7 +1333,7 @@ typedef uint64_t uintmax_t; #endif /* SAFE_TO_DEFINE_EXTENSIONS */ /* Version number of package */ -#define VERSION "3.6.3" +#define VERSION "3.7.0" /* Number of bits in a file offset, on hosts where this is settable. */ /* #undef _FILE_OFFSET_BITS */ diff --git a/src/IO/Archives/ArchiveUtils.h b/src/IO/Archives/ArchiveUtils.h new file mode 100644 index 00000000000..4c1f62ab0fb --- /dev/null +++ b/src/IO/Archives/ArchiveUtils.h @@ -0,0 +1,7 @@ +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wreserved-macro-identifier" + +#include +#include +#endif diff --git a/src/IO/Archives/SevenZipArchiveReader.cpp b/src/IO/Archives/SevenZipArchiveReader.cpp index dc3daa4cccc..2e334e53c45 100644 --- a/src/IO/Archives/SevenZipArchiveReader.cpp +++ b/src/IO/Archives/SevenZipArchiveReader.cpp @@ -1,7 +1,9 @@ -#include "SevenZipArchiveReader.h" +#include #include #include +#include + namespace DB { @@ -14,7 +16,7 @@ namespace ErrorCodes class SevenZipArchiveReader::Handle { public: - Handle(const String & path_to_archive_) : path_to_archive(path_to_archive_) + explicit Handle(const String & path_to_archive_) : path_to_archive(path_to_archive_) { archive = archive_read_new(); archive_read_support_filter_all(archive); @@ -127,9 +129,8 @@ SevenZipArchiveReader::SevenZipArchiveReader( { } -SevenZipArchiveReader::~SevenZipArchiveReader() -{ -} +SevenZipArchiveReader::~SevenZipArchiveReader() = default; + bool SevenZipArchiveReader::fileExists(const String & filename) { Handle handle(path_to_archive); diff --git a/src/IO/Archives/SevenZipArchiveReader.h b/src/IO/Archives/SevenZipArchiveReader.h index 62ea4daff9e..0d6c54b9051 100644 --- a/src/IO/Archives/SevenZipArchiveReader.h +++ b/src/IO/Archives/SevenZipArchiveReader.h @@ -3,9 +3,6 @@ #include #include -#include -#include - namespace DB { diff --git a/src/IO/Archives/TarArchiveReader.cpp b/src/IO/Archives/TarArchiveReader.cpp index b47b90b04aa..01400cfd863 100644 --- a/src/IO/Archives/TarArchiveReader.cpp +++ b/src/IO/Archives/TarArchiveReader.cpp @@ -1,7 +1,8 @@ -#include "TarArchiveReader.h" -#include +#include #include +#include +#include namespace DB { @@ -14,7 +15,7 @@ namespace ErrorCodes class TarArchiveReader::Handle { public: - Handle(const String & path_to_archive_) : path_to_archive(path_to_archive_) + explicit Handle(const String & path_to_archive_) : path_to_archive(path_to_archive_) { archive = archive_read_new(); archive_read_support_filter_all(archive); @@ -127,9 +128,8 @@ TarArchiveReader::TarArchiveReader( { } -TarArchiveReader::~TarArchiveReader() -{ -} +TarArchiveReader::~TarArchiveReader() = default; + bool TarArchiveReader::fileExists(const String & filename) { Handle handle(path_to_archive); diff --git a/src/IO/Archives/TarArchiveReader.h b/src/IO/Archives/TarArchiveReader.h index 644ae806d75..6223af47159 100644 --- a/src/IO/Archives/TarArchiveReader.h +++ b/src/IO/Archives/TarArchiveReader.h @@ -2,9 +2,6 @@ #include -#include -#include - namespace DB { diff --git a/src/IO/Archives/util/archive_writers.h b/src/IO/Archives/util/archive_writers.h deleted file mode 100644 index a340565756f..00000000000 --- a/src/IO/Archives/util/archive_writers.h +++ /dev/null @@ -1,52 +0,0 @@ -#include -#include - -bool create_tar_with_file(const std::string &archivename, std::map files) { - struct archive *a; - struct archive_entry *entry; - - a = archive_write_new(); - archive_write_set_format_pax_restricted(a); - archive_write_open_filename(a, archivename.c_str()); - - for (auto &[filename, content] : files) { - entry = archive_entry_new(); - archive_entry_set_pathname(entry, filename.c_str()); - archive_entry_set_size(entry, content.size()); - archive_entry_set_mode(entry, S_IFREG | 0644); // regular file with rw-r--r-- permissions - archive_entry_set_mtime(entry, time(nullptr), 0); - archive_write_header(a, entry); - archive_write_data(a, content.c_str(), content.size()); - archive_entry_free(entry); - } - - archive_write_close(a); - archive_write_free(a); - - return true; -} - -bool create_7z_with_file(const std::string &archivename, std::map files) { - struct archive *a; - struct archive_entry *entry; - - a = archive_write_new(); - archive_write_set_format_7zip(a); - archive_write_open_filename(a, archivename.c_str()); - - for (auto &[filename, content] : files) { - entry = archive_entry_new(); - archive_entry_set_pathname(entry, filename.c_str()); - archive_entry_set_size(entry, content.size()); - archive_entry_set_mode(entry, S_IFREG | 0644); // regular file with rw-r--r-- permissions - archive_entry_set_mtime(entry, time(nullptr), 0); - archive_write_header(a, entry); - archive_write_data(a, content.c_str(), content.size()); - archive_entry_free(entry); - } - - archive_write_close(a); - archive_write_free(a); - - return true; -} diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index 4d05a31af1f..8586adbbd94 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -1,11 +1,18 @@ #include #include "config.h" +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wreserved-macro-identifier" + +#include +#include +#endif + #include #include #include #include -#include #include #include #include @@ -20,11 +27,52 @@ namespace DB::ErrorCodes { extern const int CANNOT_UNPACK_ARCHIVE; + extern const int LOGICAL_ERROR; } namespace fs = std::filesystem; using namespace DB; +enum class ArchiveType : uint8_t +{ + Tar, + SevenZip +}; + +template +bool createArchiveWithFiles(const std::string & archivename, std::map files) +{ + struct archive * a; + struct archive_entry * entry; + + a = archive_write_new(); + + if constexpr (archive_type == ArchiveType::Tar) + archive_write_set_format_pax_restricted(a); + else if constexpr (archive_type == ArchiveType::SevenZip) + archive_write_set_format_7zip(a); + else + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Invalid archive type requested: {}", static_cast(archive_type)); + + archive_write_open_filename(a, archivename.c_str()); + + for (auto & [filename, content] : files) { + entry = archive_entry_new(); + archive_entry_set_pathname(entry, filename.c_str()); + archive_entry_set_size(entry, content.size()); + archive_entry_set_mode(entry, S_IFREG | 0644); // regular file with rw-r--r-- permissions + archive_entry_set_mtime(entry, time(nullptr), 0); + archive_write_header(a, entry); + archive_write_data(a, content.c_str(), content.size()); + archive_entry_free(entry); + } + + archive_write_close(a); + archive_write_free(a); + + return true; + +} class ArchiveReaderAndWriterTest : public ::testing::TestWithParam { @@ -332,7 +380,7 @@ TEST(TarArchiveReaderTest, FileExists) { String archive_path = "archive.tar"; String filename = "file.txt"; String contents = "test"; - bool created = create_tar_with_file(archive_path, {{filename, contents}}); + bool created = createArchiveWithFiles(archive_path, {{filename, contents}}); EXPECT_EQ(created, true); auto reader = createArchiveReader(archive_path); EXPECT_EQ(reader->fileExists(filename), true); @@ -343,7 +391,7 @@ TEST(TarArchiveReaderTest, ReadFile) { String archive_path = "archive.tar"; String filename = "file.txt"; String contents = "test"; - bool created = create_tar_with_file(archive_path, {{filename, contents}}); + bool created = createArchiveWithFiles(archive_path, {{filename, contents}}); EXPECT_EQ(created, true); auto reader = createArchiveReader(archive_path); auto in = reader->readFile(filename); @@ -359,7 +407,7 @@ TEST(TarArchiveReaderTest, ReadTwoFiles) { String contents1 = "test1"; String file2 = "file2.txt"; String contents2 = "test2"; - bool created = create_tar_with_file(archive_path, {{file1, contents1}, {file2, contents2}}); + bool created = createArchiveWithFiles(archive_path, {{file1, contents1}, {file2, contents2}}); EXPECT_EQ(created, true); auto reader = createArchiveReader(archive_path); EXPECT_EQ(reader->fileExists(file1), true); @@ -380,7 +428,7 @@ TEST(TarArchiveReaderTest, CheckFileInfo) { String archive_path = "archive.tar"; String filename = "file.txt"; String contents = "test"; - bool created = create_tar_with_file(archive_path, {{filename, contents}}); + bool created = createArchiveWithFiles(archive_path, {{filename, contents}}); EXPECT_EQ(created, true); auto reader = createArchiveReader(archive_path); auto info = reader->getFileInfo(filename); @@ -393,7 +441,7 @@ TEST(SevenZipArchiveReaderTest, FileExists) { String archive_path = "archive.7z"; String filename = "file.txt"; String contents = "test"; - bool created = create_7z_with_file(archive_path, {{filename, contents}}); + bool created = createArchiveWithFiles(archive_path, {{filename, contents}}); EXPECT_EQ(created, true); auto reader = createArchiveReader(archive_path); EXPECT_EQ(reader->fileExists(filename), true); @@ -404,7 +452,7 @@ TEST(SevenZipArchiveReaderTest, ReadFile) { String archive_path = "archive.7z"; String filename = "file.txt"; String contents = "test"; - bool created = create_7z_with_file(archive_path, {{filename, contents}}); + bool created = createArchiveWithFiles(archive_path, {{filename, contents}}); EXPECT_EQ(created, true); auto reader = createArchiveReader(archive_path); auto in = reader->readFile(filename); @@ -418,7 +466,7 @@ TEST(SevenZipArchiveReaderTest, CheckFileInfo) { String archive_path = "archive.7z"; String filename = "file.txt"; String contents = "test"; - bool created = create_7z_with_file(archive_path, {{filename, contents}}); + bool created = createArchiveWithFiles(archive_path, {{filename, contents}}); EXPECT_EQ(created, true); auto reader = createArchiveReader(archive_path); auto info = reader->getFileInfo(filename); @@ -433,7 +481,7 @@ TEST(SevenZipArchiveReaderTest, ReadTwoFiles) { String contents1 = "test1"; String file2 = "file2.txt"; String contents2 = "test2"; - bool created = create_7z_with_file(archive_path, {{file1, contents1}, {file2, contents2}}); + bool created = createArchiveWithFiles(archive_path, {{file1, contents1}, {file2, contents2}}); EXPECT_EQ(created, true); auto reader = createArchiveReader(archive_path); EXPECT_EQ(reader->fileExists(file1), true); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 6410b880c04..30905f20aeb 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -21,6 +21,8 @@ #include #include #include +#include +#include #include #include diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 55a21a17b7d..10c234811ff 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -22,8 +22,8 @@ public: const ColumnsDescription & columns; const ConstraintsDescription & constraints; const String & comment; - std::string path_to_archive = "auto"; const std::string rename_after_processing; + std::string path_to_archive = "auto"; }; /// From file descriptor diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 63af52d7de9..c39723fdf82 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -75,8 +75,8 @@ StoragePtr TableFunctionFile::getStorage(const String & source, columns, ConstraintsDescription{}, String{}, - path_to_archive, global_context->getSettingsRef().rename_files_after_processing, + path_to_archive, }; if (fd >= 0) return std::make_shared(fd, args); From caef499541d2ecdc69354b7444291aa5f44f3482 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 28 Jul 2023 12:37:51 +0000 Subject: [PATCH 1284/2047] fix test --- tests/queries/0_stateless/02833_multiprewhere_extra_column.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql b/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql index a786de454ed..3a751294cba 100644 --- a/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql +++ b/tests/queries/0_stateless/02833_multiprewhere_extra_column.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-random-settings, no-random-merge-tree-settings +-- Tags: no-parallel, no-random-settings, no-random-merge-tree-settings, no-s3-storage drop table if exists t_multi_prewhere; drop row policy if exists policy_02834 on t_multi_prewhere; From 1fc1b6aae45800e3879245ea1abc3884c0ce0231 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Jul 2023 13:00:35 +0000 Subject: [PATCH 1285/2047] More fixes --- docs/en/sql-reference/table-functions/file.md | 4 +-- src/CMakeLists.txt | 4 ++- src/Common/config.h.in | 1 + src/IO/Archives/ArchiveUtils.h | 7 ++++++ src/IO/Archives/SevenZipArchiveReader.cpp | 25 +++++++++++-------- src/IO/Archives/SevenZipArchiveReader.h | 16 ++++++++---- src/IO/Archives/TarArchiveReader.cpp | 24 ++++++++++-------- src/IO/Archives/TarArchiveReader.h | 16 ++++++++---- src/IO/Archives/createArchiveReader.cpp | 22 +++++++++++++--- .../tests/gtest_archive_reader_and_writer.cpp | 13 +++------- src/TableFunctions/ITableFunctionFileLike.cpp | 7 ++++-- src/TableFunctions/TableFunctionFile.cpp | 2 +- src/configure_config.cmake | 3 +++ 13 files changed, 94 insertions(+), 50 deletions(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 8d21e53b209..4db9494502e 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -19,7 +19,7 @@ file([path_to_archive ::] path [,format] [,structure] [,compression]) **Parameters** - `path` — The relative path to the file from [user_files_path](/docs/en/operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. -- `path_to_archive` - The relative path to zip/tar/7z archive. Path to archive support the same globs. +- `path_to_archive` - The relative path to zip/tar/7z archive. Path to archive support the same globs as `path`. - `format` — The [format](/docs/en/interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format: `'column1_name column1_type, column2_name column2_type, ...'`. - `compression` — The existing compression type when used in a `SELECT` query, or the desired compression type when used in an `INSERT` query. The supported compression types are `gz`, `br`, `xz`, `zst`, `lz4`, and `bz2`. @@ -130,7 +130,7 @@ file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32'); └─────────┴─────────┴─────────┘ ``` -Getting data from table in table.csv, located in archive1.zip or(and) archive2.zip +Getting data from table in table.csv, located in archive1.zip or/and archive2.zip ``` sql SELECT * FROM file('user_files/archives/archive{1..2}.zip :: table.csv'); ``` diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 3c4caa4359a..c508638f972 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -575,7 +575,9 @@ if (TARGET ch_contrib::bzip2) target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::bzip2) endif() -target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::libarchive) +if (TARGET ch_contrib::libarchive) + target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::libarchive) +endif() if (TARGET ch_contrib::minizip) target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::minizip) diff --git a/src/Common/config.h.in b/src/Common/config.h.in index 628f0847d65..72932dda55d 100644 --- a/src/Common/config.h.in +++ b/src/Common/config.h.in @@ -59,6 +59,7 @@ #cmakedefine01 USE_ULID #cmakedefine01 FIU_ENABLE #cmakedefine01 USE_BCRYPT +#cmakedefine01 USE_LIBARCHIVE /// This is needed for .incbin in assembly. For some reason, include paths don't work there in presence of LTO. /// That's why we use absolute paths. diff --git a/src/IO/Archives/ArchiveUtils.h b/src/IO/Archives/ArchiveUtils.h index 4c1f62ab0fb..810b9d8d730 100644 --- a/src/IO/Archives/ArchiveUtils.h +++ b/src/IO/Archives/ArchiveUtils.h @@ -1,3 +1,9 @@ +#pragma once + +#include "config.h" + +#if USE_LIBARCHIVE + #ifdef __clang__ #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wreserved-macro-identifier" @@ -5,3 +11,4 @@ #include #include #endif +#endif diff --git a/src/IO/Archives/SevenZipArchiveReader.cpp b/src/IO/Archives/SevenZipArchiveReader.cpp index 2e334e53c45..2daef777e94 100644 --- a/src/IO/Archives/SevenZipArchiveReader.cpp +++ b/src/IO/Archives/SevenZipArchiveReader.cpp @@ -7,11 +7,15 @@ namespace DB { + +#if USE_LIBARCHIVE + namespace ErrorCodes { extern const int CANNOT_UNPACK_ARCHIVE; extern const int LOGICAL_ERROR; extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int NOT_IMPLEMENTED; } class SevenZipArchiveReader::Handle { @@ -123,9 +127,8 @@ SevenZipArchiveReader::SevenZipArchiveReader(const String & path_to_archive_) : { } -SevenZipArchiveReader::SevenZipArchiveReader( - const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_) - : path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), archive_size(archive_size_) +SevenZipArchiveReader::SevenZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_) + : path_to_archive(path_to_archive_), archive_read_function(archive_read_function_) { } @@ -152,7 +155,7 @@ SevenZipArchiveReader::FileInfo SevenZipArchiveReader::getFileInfo(const String std::unique_ptr SevenZipArchiveReader::firstFile() { - return nullptr; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for 7z archives"); } std::unique_ptr SevenZipArchiveReader::readFile(const String & filename) @@ -163,21 +166,23 @@ std::unique_ptr SevenZipArchiveReader::readFile(const St return std::make_unique(path_to_archive, filename); } -std::unique_ptr SevenZipArchiveReader::readFile([[maybe_unused]] std::unique_ptr enumerator) +std::unique_ptr SevenZipArchiveReader::readFile(std::unique_ptr /*enumerator*/) { - return nullptr; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for 7z archives"); } std::unique_ptr -SevenZipArchiveReader::nextFile([[maybe_unused]] std::unique_ptr read_buffer) +SevenZipArchiveReader::nextFile(std::unique_ptr /*read_buffer*/) { - return nullptr; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for 7z archives"); } -void SevenZipArchiveReader::setPassword([[maybe_unused]] const String & password_) +void SevenZipArchiveReader::setPassword(const String & /*password_*/) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to .7z archive"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to 7z archive"); } +#endif + } diff --git a/src/IO/Archives/SevenZipArchiveReader.h b/src/IO/Archives/SevenZipArchiveReader.h index 0d6c54b9051..de7150f5a10 100644 --- a/src/IO/Archives/SevenZipArchiveReader.h +++ b/src/IO/Archives/SevenZipArchiveReader.h @@ -1,11 +1,16 @@ #pragma once +#include "config.h" + #include #include namespace DB { + +#if USE_LIBARCHIVE + class ReadBuffer; class ReadBufferFromFileBase; class SeekableReadBuffer; @@ -19,7 +24,7 @@ public: /// Constructs an archive's reader that will read by making a read buffer by using /// a specified function. - SevenZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_); + SevenZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_); ~SevenZipArchiveReader() override; @@ -38,11 +43,11 @@ public: std::unique_ptr readFile(const String & filename) override; /// It's possible to convert a file enumerator to a read buffer and vice versa. - [[maybe_unused]] std::unique_ptr readFile(std::unique_ptr enumerator) override; - [[maybe_unused]] std::unique_ptr nextFile(std::unique_ptr read_buffer) override; + std::unique_ptr readFile(std::unique_ptr enumerator) override; + std::unique_ptr nextFile(std::unique_ptr read_buffer) override; /// Sets password used to decrypt the contents of the files in the archive. - void setPassword([[maybe_unused]] const String & password_) override; + void setPassword(const String & password_) override; private: class ReadBufferFromSevenZipArchive; @@ -51,7 +56,8 @@ private: const String path_to_archive; String password; const ReadArchiveFunction archive_read_function; - [[maybe_unused]] const UInt64 archive_size = 0; }; +#endif + } diff --git a/src/IO/Archives/TarArchiveReader.cpp b/src/IO/Archives/TarArchiveReader.cpp index 01400cfd863..5d88e0e5cd4 100644 --- a/src/IO/Archives/TarArchiveReader.cpp +++ b/src/IO/Archives/TarArchiveReader.cpp @@ -6,6 +6,9 @@ namespace DB { + +#if USE_LIBARCHIVE + namespace ErrorCodes { extern const int CANNOT_UNPACK_ARCHIVE; @@ -122,9 +125,8 @@ TarArchiveReader::TarArchiveReader(const String & path_to_archive_) : path_to_ar { } -TarArchiveReader::TarArchiveReader( - const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_) - : path_to_archive(path_to_archive_), archive_read_function(archive_read_function_), archive_size(archive_size_) +TarArchiveReader::TarArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_) + : path_to_archive(path_to_archive_), archive_read_function(archive_read_function_) { } @@ -151,7 +153,7 @@ TarArchiveReader::FileInfo TarArchiveReader::getFileInfo(const String & filename std::unique_ptr TarArchiveReader::firstFile() { - return nullptr; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for tar archives"); } std::unique_ptr TarArchiveReader::readFile(const String & filename) @@ -162,20 +164,22 @@ std::unique_ptr TarArchiveReader::readFile(const String return std::make_unique(path_to_archive, filename); } -std::unique_ptr TarArchiveReader::readFile([[maybe_unused]] std::unique_ptr enumerator) +std::unique_ptr TarArchiveReader::readFile(std::unique_ptr /*enumerator*/) { - return nullptr; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for tar archives"); } -std::unique_ptr TarArchiveReader::nextFile([[maybe_unused]] std::unique_ptr read_buffer) +std::unique_ptr TarArchiveReader::nextFile(std::unique_ptr /*read_buffer*/) { - return nullptr; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for tar archives"); } -void TarArchiveReader::setPassword([[maybe_unused]] const String & password_) +void TarArchiveReader::setPassword(const String & /*password_*/) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to .tar archive"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to tar archive"); } +#endif + } diff --git a/src/IO/Archives/TarArchiveReader.h b/src/IO/Archives/TarArchiveReader.h index 6223af47159..8968a2b53dc 100644 --- a/src/IO/Archives/TarArchiveReader.h +++ b/src/IO/Archives/TarArchiveReader.h @@ -1,10 +1,15 @@ #pragma once +#include "config.h" + #include namespace DB { + +#if USE_LIBARCHIVE + class ReadBuffer; class ReadBufferFromFileBase; class SeekableReadBuffer; @@ -18,7 +23,7 @@ public: /// Constructs an archive's reader that will read by making a read buffer by using /// a specified function. - TarArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_, UInt64 archive_size_); + TarArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_); ~TarArchiveReader() override; @@ -37,11 +42,11 @@ public: std::unique_ptr readFile(const String & filename) override; /// It's possible to convert a file enumerator to a read buffer and vice versa. - [[maybe_unused]] std::unique_ptr readFile(std::unique_ptr enumerator) override; - [[maybe_unused]] std::unique_ptr nextFile(std::unique_ptr read_buffer) override; + std::unique_ptr readFile(std::unique_ptr enumerator) override; + std::unique_ptr nextFile(std::unique_ptr read_buffer) override; /// Sets password used to decrypt the contents of the files in the archive. - void setPassword([[maybe_unused]] const String & password_) override; + void setPassword(const String & password_) override; private: class ReadBufferFromTarArchive; @@ -49,7 +54,8 @@ private: const String path_to_archive; const ReadArchiveFunction archive_read_function; - [[maybe_unused]] const UInt64 archive_size = 0; }; +#endif + } diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index df6b0d15ce4..b84f69ceb41 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -32,13 +32,27 @@ std::shared_ptr createArchiveReader( #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "minizip library is disabled"); #endif - } else if (path_to_archive.ends_with(".tar")) { - return std::make_shared(path_to_archive, archive_read_function, archive_size); - } else if (path_to_archive.ends_with(".7z")) { - return std::make_shared(path_to_archive, archive_read_function, archive_size); + } + else if (path_to_archive.ends_with(".tar")) + { +#if USE_LIBARCHIVE + return std::make_shared(path_to_archive, archive_read_function); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "libarchive library is disabled"); +#endif + } + else if (path_to_archive.ends_with(".7z")) + { +#if USE_LIBARCHIVE + return std::make_shared(path_to_archive, archive_read_function); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "libarchive library is disabled"); +#endif } else + { throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Cannot determine the type of archive {}", path_to_archive); + } } } diff --git a/src/IO/tests/gtest_archive_reader_and_writer.cpp b/src/IO/tests/gtest_archive_reader_and_writer.cpp index 8586adbbd94..8eeccbcdf75 100644 --- a/src/IO/tests/gtest_archive_reader_and_writer.cpp +++ b/src/IO/tests/gtest_archive_reader_and_writer.cpp @@ -1,14 +1,7 @@ #include #include "config.h" -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wreserved-macro-identifier" - -#include -#include -#endif - +#include #include #include #include @@ -40,7 +33,7 @@ enum class ArchiveType : uint8_t }; template -bool createArchiveWithFiles(const std::string & archivename, std::map files) +bool createArchiveWithFiles(const std::string & archivename, const std::map & files) { struct archive * a; struct archive_entry * entry; @@ -56,7 +49,7 @@ bool createArchiveWithFiles(const std::string & archivename, std::map(arg, "source"); size_t pos = path.find(" :: "); - if (pos == String::npos) { + if (pos == String::npos) + { filename = path; - } else { + } + else + { path_to_archive = path.substr(0, pos); filename = path.substr(pos + 4, path.size() - pos - 3); } diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index c39723fdf82..1992f06c398 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -80,7 +80,7 @@ StoragePtr TableFunctionFile::getStorage(const String & source, }; if (fd >= 0) return std::make_shared(fd, args); - + return std::make_shared(source, global_context->getUserFilesPath(), args); } diff --git a/src/configure_config.cmake b/src/configure_config.cmake index 5529e2f2f39..9f038f10fcd 100644 --- a/src/configure_config.cmake +++ b/src/configure_config.cmake @@ -162,5 +162,8 @@ endif () if (TARGET ch_contrib::fiu) set(FIU_ENABLE 1) endif() +if (TARGET ch_contrib::libarchive) + set(USE_LIBARCHIVE 1) +endif() set(SOURCE_DIR ${CMAKE_SOURCE_DIR}) From 96df21bbf58efbf5c1a914632702472f74f67490 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 28 Jul 2023 13:35:13 +0000 Subject: [PATCH 1286/2047] implemented for all tables, fixed style and fast test --- src/Storages/System/attachInformationSchemaTables.cpp | 6 +++--- .../0_stateless/01161_information_schema.reference | 8 ++++++++ .../02206_information_schema_show_database.reference | 2 ++ 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 6f378671104..efa24daeffe 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -41,13 +41,13 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d database.getTableDataPath(ast_create), context, true).second; database.createTable(context, ast_create.getTable(), view, ast); - if (ast_create.getTable() == "tables") + if (database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA) { database.createTable(context, Poco::toUpper(ast_create.getTable()), view, ast); } - else if (ast_create.getTable() == "TABLES") + else { - database.createTable(context, Poco::toLower(ast_create.getTable()), view, ast); + database.createTable(context, Poco::toLower(ast_create.getTable()), view, ast); } } catch (...) diff --git a/tests/queries/0_stateless/01161_information_schema.reference b/tests/queries/0_stateless/01161_information_schema.reference index 5331e30b899..4ec33a70be0 100644 --- a/tests/queries/0_stateless/01161_information_schema.reference +++ b/tests/queries/0_stateless/01161_information_schema.reference @@ -1,3 +1,7 @@ +COLUMNS +SCHEMATA +TABLES +VIEWS columns schemata tables @@ -6,6 +10,10 @@ COLUMNS SCHEMATA TABLES VIEWS +columns +schemata +tables +views INFORMATION_SCHEMA INFORMATION_SCHEMA default \N \N \N \N information_schema information_schema default \N \N \N \N default default mv VIEW diff --git a/tests/queries/0_stateless/02206_information_schema_show_database.reference b/tests/queries/0_stateless/02206_information_schema_show_database.reference index 821fddbb933..f5fd221a689 100644 --- a/tests/queries/0_stateless/02206_information_schema_show_database.reference +++ b/tests/queries/0_stateless/02206_information_schema_show_database.reference @@ -1,4 +1,6 @@ CREATE DATABASE INFORMATION_SCHEMA\nENGINE = Memory CREATE VIEW INFORMATION_SCHEMA.COLUMNS\n(\n `table_catalog` String,\n `table_schema` String,\n `table_name` String,\n `TABLE_SCHEMA` String,\n `TABLE_NAME` String,\n `column_name` String,\n `ordinal_position` UInt64,\n `column_default` String,\n `is_nullable` String,\n `data_type` String,\n `character_maximum_length` Nullable(UInt64),\n `character_octet_length` Nullable(UInt64),\n `numeric_precision` Nullable(UInt64),\n `numeric_precision_radix` Nullable(UInt64),\n `numeric_scale` Nullable(UInt64),\n `datetime_precision` Nullable(UInt64),\n `character_set_catalog` Nullable(String),\n `character_set_schema` Nullable(String),\n `character_set_name` Nullable(String),\n `collation_catalog` Nullable(String),\n `collation_schema` Nullable(String),\n `collation_name` Nullable(String),\n `domain_catalog` Nullable(String),\n `domain_schema` Nullable(String),\n `domain_name` Nullable(String),\n `column_comment` String,\n `column_type` String,\n `TABLE_CATALOG` String ALIAS table_catalog,\n `COLUMN_NAME` String ALIAS column_name,\n `ORDINAL_POSITION` UInt64 ALIAS ordinal_position,\n `COLUMN_DEFAULT` String ALIAS column_default,\n `IS_NULLABLE` String ALIAS is_nullable,\n `DATA_TYPE` String ALIAS data_type,\n `CHARACTER_MAXIMUM_LENGTH` Nullable(UInt64) ALIAS character_maximum_length,\n `CHARACTER_OCTET_LENGTH` Nullable(UInt64) ALIAS character_octet_length,\n `NUMERIC_PRECISION` Nullable(UInt64) ALIAS numeric_precision,\n `NUMERIC_PRECISION_RADIX` Nullable(UInt64) ALIAS numeric_precision_radix,\n `NUMERIC_SCALE` Nullable(UInt64) ALIAS numeric_scale,\n `DATETIME_PRECISION` Nullable(UInt64) ALIAS datetime_precision,\n `CHARACTER_SET_CATALOG` Nullable(String) ALIAS character_set_catalog,\n `CHARACTER_SET_SCHEMA` Nullable(String) ALIAS character_set_schema,\n `CHARACTER_SET_NAME` Nullable(String) ALIAS character_set_name,\n `COLLATION_CATALOG` Nullable(String) ALIAS collation_catalog,\n `COLLATION_SCHEMA` Nullable(String) ALIAS collation_schema,\n `COLLATION_NAME` Nullable(String) ALIAS collation_name,\n `DOMAIN_CATALOG` Nullable(String) ALIAS domain_catalog,\n `DOMAIN_SCHEMA` Nullable(String) ALIAS domain_schema,\n `DOMAIN_NAME` Nullable(String) ALIAS domain_name,\n `COLUMN_COMMENT` String ALIAS column_comment,\n `COLUMN_TYPE` String ALIAS column_type\n) AS\nSELECT\n database AS table_catalog,\n database AS table_schema,\n database AS TABLE_SCHEMA,\n table AS table_name,\n table AS TABLE_NAME,\n name AS column_name,\n position AS ordinal_position,\n default_expression AS column_default,\n type LIKE \'Nullable(%)\' AS is_nullable,\n type AS data_type,\n character_octet_length AS character_maximum_length,\n character_octet_length,\n numeric_precision,\n numeric_precision_radix,\n numeric_scale,\n datetime_precision,\n NULL AS character_set_catalog,\n NULL AS character_set_schema,\n NULL AS character_set_name,\n NULL AS collation_catalog,\n NULL AS collation_schema,\n NULL AS collation_name,\n NULL AS domain_catalog,\n NULL AS domain_schema,\n NULL AS domain_name,\n comment AS column_comment,\n type AS column_type\nFROM system.columns CREATE VIEW INFORMATION_SCHEMA.TABLES (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables +CREATE VIEW INFORMATION_SCHEMA.TABLES (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables +CREATE VIEW information_schema.tables (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables CREATE VIEW information_schema.tables (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables From ff9e85529ad638f5717670294cfc00eb6aa19140 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 28 Jul 2023 12:20:48 +0000 Subject: [PATCH 1287/2047] Allow reading zero objects in CachedObjectStorage::readObjects() --- .../Cached/CachedObjectStorage.cpp | 2 - .../02731_zero_objects_in_metadata.reference | 3 ++ .../02731_zero_objects_in_metadata.sh | 37 ++++++++++--------- 3 files changed, 23 insertions(+), 19 deletions(-) diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index 3e7c4d12c42..717acb08f62 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -79,8 +79,6 @@ std::unique_ptr CachedObjectStorage::readObjects( /// NO std::optional read_hint, std::optional 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); } diff --git a/tests/queries/0_stateless/02731_zero_objects_in_metadata.reference b/tests/queries/0_stateless/02731_zero_objects_in_metadata.reference index 125915f4f65..1898b2c543b 100644 --- a/tests/queries/0_stateless/02731_zero_objects_in_metadata.reference +++ b/tests/queries/0_stateless/02731_zero_objects_in_metadata.reference @@ -1,3 +1,6 @@ 1 [] 1 [] [] +1 [] +1 [] +[] diff --git a/tests/queries/0_stateless/02731_zero_objects_in_metadata.sh b/tests/queries/0_stateless/02731_zero_objects_in_metadata.sh index 690cf977d08..eef52002e36 100755 --- a/tests/queries/0_stateless/02731_zero_objects_in_metadata.sh +++ b/tests/queries/0_stateless/02731_zero_objects_in_metadata.sh @@ -5,23 +5,26 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${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'; +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='$DISK'; -INSERT INTO test (id) VALUES (1); -SELECT * FROM test; -" + INSERT INTO test (id) VALUES (1); + SELECT * FROM test; + " -${CLICKHOUSE_CLIENT} -n --query " -BACKUP TABLE test TO Disk('backups', 'test_s3_backup'); -DROP TABLE test; -RESTORE TABLE test FROM Disk('backups', 'test_s3_backup'); -" &>/dev/null + ${CLICKHOUSE_CLIENT} -n --query " + BACKUP TABLE test TO Disk('backups', 'test_s3_backup'); + DROP TABLE test; + RESTORE TABLE test FROM Disk('backups', 'test_s3_backup'); + " &>/dev/null -${CLICKHOUSE_CLIENT} -n --query " -SELECT * FROM test; -SELECT empty FROM test; -" + ${CLICKHOUSE_CLIENT} -n --query " + SELECT * FROM test; + SELECT empty FROM test; + " +done From 139033247cad342dbdf1f58428921328d5c191aa Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 28 Jul 2023 15:46:45 +0200 Subject: [PATCH 1288/2047] bypass a limitation of github From bf9504d95d65f23552ac5708c2d0365125269085 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 28 Jul 2023 15:49:55 +0200 Subject: [PATCH 1289/2047] Move UnlinkMetadataFileOperationOutcome to common header (#52710) --- src/Disks/ObjectStorages/IMetadataStorage.h | 9 ++++++++- .../MetadataStorageFromDiskTransactionOperations.h | 9 --------- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Disks/ObjectStorages/IMetadataStorage.h b/src/Disks/ObjectStorages/IMetadataStorage.h index 264c481ee08..6b75e157dee 100644 --- a/src/Disks/ObjectStorages/IMetadataStorage.h +++ b/src/Disks/ObjectStorages/IMetadataStorage.h @@ -22,7 +22,14 @@ namespace ErrorCodes } class IMetadataStorage; -struct UnlinkMetadataFileOperationOutcome; + +/// Return the result of operation to the caller. +/// It is used in `IDiskObjectStorageOperation::finalize` after metadata transaction executed to make decision on blob removal. +struct UnlinkMetadataFileOperationOutcome +{ + UInt32 num_hardlinks = std::numeric_limits::max(); +}; + using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; /// Tries to provide some "transactions" interface, which allow diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h index 4662ebc3423..ccb77f6ae7b 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h @@ -244,15 +244,6 @@ private: std::unique_ptr write_operation; }; -/// Return the result of operation to the caller. -/// It is used in `IDiskObjectStorageOperation::finalize` after metadata transaction executed to make decision on blob removal. -struct UnlinkMetadataFileOperationOutcome -{ - UInt32 num_hardlinks = std::numeric_limits::max(); -}; - -using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr; - struct UnlinkMetadataFileOperation final : public IMetadataOperation { const UnlinkMetadataFileOperationOutcomePtr outcome = std::make_shared(); From 7cb853bcd89746c454ac46095bd4e75b155ce1e1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 28 Jul 2023 14:22:12 +0000 Subject: [PATCH 1290/2047] Fix TableFunctionNode::cloneImpl --- src/Analyzer/TableFunctionNode.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Analyzer/TableFunctionNode.cpp b/src/Analyzer/TableFunctionNode.cpp index 30644ad4ec4..e5158a06373 100644 --- a/src/Analyzer/TableFunctionNode.cpp +++ b/src/Analyzer/TableFunctionNode.cpp @@ -133,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; } From 1e77c7c73f3dc7db2e52d618999f909fcf3e9d97 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 10:48:22 -0400 Subject: [PATCH 1291/2047] Deterministic delete_stable_backups --- programs/keeper-client/Commands.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index cbdcde4f89a..fd0a00d59db 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -228,6 +228,7 @@ void DeleteStableBackups::execute(const ASTKeeperQuery * /* query */, KeeperClie { fs::path backup_root = "/clickhouse/backups"; auto backups = client->zookeeper->getChildren(backup_root); + std::sort(backups.begin(), backups.end()); for (const auto & child : backups) { From f3a31c20fe8fd4a0ba57586bd36828af8581cf5d Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 10:53:02 -0400 Subject: [PATCH 1292/2047] Update test_row_policy.py --- .../test_row_policy.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py index 509b4de1a37..b620e88e7eb 100644 --- a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py +++ b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py @@ -76,7 +76,7 @@ def test_introspection(): "mydb", "filtered_table1", "6068883a-0e9d-f802-7e22-0144f8e66d3c", - "users.xml", + "users_xml", "1", 0, 0, @@ -89,7 +89,7 @@ def test_introspection(): "mydb", "filtered_table2", "c019e957-c60b-d54e-cc52-7c90dac5fb01", - "users.xml", + "users_xml", "1", 0, 0, @@ -102,7 +102,7 @@ def test_introspection(): "mydb", "filtered_table3", "4cb080d0-44e8-dbef-6026-346655143628", - "users.xml", + "users_xml", "1", 0, 0, @@ -115,7 +115,7 @@ def test_introspection(): "mydb", "local", "5b23c389-7e18-06bf-a6bc-dd1afbbc0a97", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -128,7 +128,7 @@ def test_introspection(): "mydb", "filtered_table1", "9e8a8f62-4965-2b5e-8599-57c7b99b3549", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -141,7 +141,7 @@ def test_introspection(): "mydb", "filtered_table2", "cffae79d-b9bf-a2ef-b798-019c18470b25", - "users.xml", + "users_xml", "a + b < 1 or c - d > 5", 0, 0, @@ -154,7 +154,7 @@ def test_introspection(): "mydb", "filtered_table3", "12fc5cef-e3da-3940-ec79-d8be3911f42b", - "users.xml", + "users_xml", "c = 1", 0, 0, @@ -167,7 +167,7 @@ def test_introspection(): "mydb", "local", "cdacaeb5-1d97-f99d-2bb0-4574f290629c", - "users.xml", + "users_xml", "1", 0, 0, From e0e6472195f87607a87da7207e20ea8609a1482f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 28 Jul 2023 15:50:22 +0000 Subject: [PATCH 1293/2047] changed tests, imporved code and exceptions --- src/DataTypes/DataTypeArray.cpp | 7 --- src/DataTypes/DataTypeArray.h | 6 -- src/Functions/FunctionBinaryArithmetic.h | 60 ++++++++++--------- ...02812_pointwise_array_operations.reference | 12 ++++ .../02812_pointwise_array_operations.sql | 5 +- 5 files changed, 48 insertions(+), 42 deletions(-) diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index effea7bfcd7..e31f10046b7 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -35,13 +35,6 @@ MutableColumnPtr DataTypeArray::createColumn() const return ColumnArray::create(nested->createColumn(), ColumnArray::ColumnOffsets::create()); } -MutableColumnPtr DataTypeArray::createColumnConst(size_t size, const Field & field) const -{ - auto column = createColumn(); - column->insert(field); - return ColumnConst::create(std::move(column), size); -} - Field DataTypeArray::getDefault() const { return Array(); diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index 52d9a8b4e01..82af498ab13 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -45,7 +45,6 @@ public: MutableColumnPtr createColumn() const override; - MutableColumnPtr createColumnConst(size_t size, const Field & field) const; Field getDefault() const override; @@ -77,9 +76,4 @@ template inline constexpr bool IsDataTypeArray() return false; } -template <> inline constexpr bool IsDataTypeArray() -{ - return std::is_same_v; -} - } diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 8a5e1149e05..b4806d24a8e 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -71,6 +71,9 @@ namespace ErrorCodes extern const int DECIMAL_OVERFLOW; extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int UNSUPPORTED_METHOD; + extern const int BAD_ARGUMENTS; } namespace traits_ @@ -1154,28 +1157,22 @@ class FunctionBinaryArithmetic : public IFunction ColumnPtr executeArrayImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { - bool is_const = false; const auto * return_type_array = checkAndGetDataType(result_type.get()); if (!return_type_array) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Return type for function {} must be array.", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Return type for function {} must be array.", getName()); - ColumnPtr result_column = executeArray(arguments, result_type, input_rows_count); + return executeArray(arguments, result_type, input_rows_count); - if (arguments[0].dumpStructure().contains("Const")) - is_const = true; - - if (is_const) - return result_column; - else - return ColumnArray::create(result_column, typeid_cast(arguments[0].column.get())->getOffsetsPtr()); + // if (typeid_cast(arguments[0].column.get())) + // return result_column; + // else } - template ColumnPtr executeArray(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { if constexpr (is_multiply || is_division) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot use multiplication or division on arrays"); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use multiplication or division on arrays"); auto num_args = arguments.size(); DataTypes data_types; @@ -1204,23 +1201,25 @@ class FunctionBinaryArithmetic : public IFunction } /// Unpacking non-const arrays and checking sizes of them. - UInt64 data = 0; - for (size_t i = 0; i < num_args; ++i) + if (*typeid_cast(arguments[0].column.get())->getOffsets().data() != + *typeid_cast(arguments[1].column.get())->getOffsets().data()) { - auto a = typeid_cast(arguments[i].column.get())->getData().getPtr(); - - if (i == 0) - data = *typeid_cast(arguments[i].column.get())->getOffsets().data(); - else - { - if (*typeid_cast(arguments[i].column.get())->getOffsets().data() != data) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Arguments must be one size"); - } - - t = typeid_cast(arguments[i].type.get())->getNestedType(); - new_arguments[i] = {a, t, arguments[i].name}; + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, + "Cannot apply operation for arguments of different sizes. Size of the first argument: {}, size of the second argument: {}", + *typeid_cast(arguments[0].column.get())->getOffsets().data(), + *typeid_cast(arguments[1].column.get())->getOffsets().data()); } - return executeImpl(new_arguments, t, input_rows_count); + + auto a = typeid_cast(arguments[0].column.get())->getData().getPtr(); + t = typeid_cast(arguments[0].type.get())->getNestedType(); + new_arguments[0] = {a, t, arguments[0].name}; + + a = typeid_cast(arguments[1].column.get())->getData().getPtr(); + t = typeid_cast(arguments[1].type.get())->getNestedType(); + new_arguments[1] = {a, t, arguments[1].name}; + + auto res = executeImpl(new_arguments, t, input_rows_count); + return ColumnArray::create(res, typeid_cast(arguments[0].column.get())->getOffsetsPtr()); } ColumnPtr executeTupleNumberOperator(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, @@ -1424,7 +1423,7 @@ public: return getReturnTypeImplStatic(new_arguments, context); } - if (isArray(arguments[0]) || isArray(arguments[1])) + if (isArray(arguments[0]) && isArray(arguments[1])) { DataTypes new_arguments { static_cast(*arguments[0]).getNestedType(), @@ -1434,6 +1433,11 @@ public: return std::make_shared(getReturnTypeImplStatic(new_arguments, context)); } + if (isArray(arguments[0]) || isArray(arguments[1])) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot execute arguments of different type. Type of the first argument: {}, type of the second argument: {}", + arguments[0]->getName(), arguments[1]->getName()); + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1], context)) { diff --git a/tests/queries/0_stateless/02812_pointwise_array_operations.reference b/tests/queries/0_stateless/02812_pointwise_array_operations.reference index 54274d0380d..5c9702d910a 100644 --- a/tests/queries/0_stateless/02812_pointwise_array_operations.reference +++ b/tests/queries/0_stateless/02812_pointwise_array_operations.reference @@ -2,3 +2,15 @@ [2,6] [4.5,5,12,10.1] [(11.1,5.4),(6,21)] +[[13,2],[3]] +[2,2] +[2,3] +[2,4] +[2,5] +[2,6] +[2,2] +[2,3] +[2,4] +[2,5] +[2,6] +[0,0,0] diff --git a/tests/queries/0_stateless/02812_pointwise_array_operations.sql b/tests/queries/0_stateless/02812_pointwise_array_operations.sql index 545b2cfcdd6..d72ced4b6fa 100644 --- a/tests/queries/0_stateless/02812_pointwise_array_operations.sql +++ b/tests/queries/0_stateless/02812_pointwise_array_operations.sql @@ -2,4 +2,7 @@ SELECT (materialize([1,1]) + materialize([1,4])); SELECT ([1,2] + [1,4]); SELECT ([2.5, 1, 3, 10.1] + [2, 4, 9, 0]); SELECT ([(1,3), (2,9)] + [(10.1, 2.4), (4,12)]); - +SELECT ([[1,1],[2]]+[[12,1],[1]]); +SELECT ([1,2]+[1,number]) from numbers(5); +SELECT ([1,2::UInt64]+[1,number]) from numbers(5); +SELECT ([materialize(1),materialize(2),materialize(3)]-[1,2,3]); From b2ee4505fb1feee2e3581d438544e491bfa7c9a1 Mon Sep 17 00:00:00 2001 From: Yury Bogomolov Date: Fri, 28 Jul 2023 16:06:03 +0000 Subject: [PATCH 1294/2047] review fixes --- src/Storages/Kafka/StorageKafka.cpp | 6 +----- tests/integration/test_storage_kafka/test.py | 22 ++++---------------- 2 files changed, 5 insertions(+), 23 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 8cc4fd90f8d..a7315eb51ea 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -250,11 +250,7 @@ StorageKafka::StorageKafka( : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , kafka_settings(std::move(kafka_settings_)) - , macros_info([&table_id_](){ - Macros::MacroExpansionInfo info; - info.table_id = table_id_; - return info; - }()) + , macros_info{.table_id = table_id_} , topics(parseTopics(getContext()->getMacros()->expand(kafka_settings->kafka_topic_list.value, macros_info))) , brokers(getContext()->getMacros()->expand(kafka_settings->kafka_broker_list.value, macros_info)) , group(getContext()->getMacros()->expand(kafka_settings->kafka_group_name.value, macros_info)) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 621fc72a607..aa227c08196 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -461,28 +461,14 @@ def test_kafka_settings_predefined_macros(kafka_cluster): ) messages = [] - for i in range(25): + for i in range(50): messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "test_kafka", messages) - - # Insert couple of malformed messages. - kafka_produce(kafka_cluster, "test_kafka", ["}{very_broken_message,"]) - kafka_produce(kafka_cluster, "test_kafka", ["}another{very_broken_message,"]) - - messages = [] - for i in range(25, 50): - messages.append(json.dumps({"key": i, "value": i})) - kafka_produce(kafka_cluster, "test_kafka", messages) - - result = "" - while True: - result += instance.query("SELECT * FROM test.kafka", ignore_error=True) - if kafka_check_result(result): - break + kafka_produce(kafka_cluster, "test_kafka_topic", messages) + result = instance.query("SELECT * FROM test.kafka", ignore_error=True) kafka_check_result(result, True) - members = describe_consumer_group(kafka_cluster, "new") + members = describe_consumer_group(kafka_cluster, "test_kafka_group") assert members[0]["client_id"] == "test_kafka test 1234" From cecc6364496333d52a3d6a7aa6bd1ba55678c2f4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 28 Jul 2023 19:35:22 +0200 Subject: [PATCH 1295/2047] Update CachedObjectStorage.cpp --- src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index 717acb08f62..298cffaf9b0 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -16,11 +16,6 @@ namespace fs = std::filesystem; namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - CachedObjectStorage::CachedObjectStorage( ObjectStoragePtr object_storage_, FileCachePtr cache_, From 2a40bc9055d446d0d9f04d25b47c96d19f8d37fe Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 28 Jul 2023 23:29:59 +0000 Subject: [PATCH 1296/2047] Fix: __bitSwapLastTwo was applied to non BoolMask - ActionsDAG was missing convertions to BoolMask - __bitWrapperFunc --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 9c34a149128..56550ab7f3a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -276,7 +277,9 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); filter_actions_dag->removeUnusedActions(); + LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Filter actions DAG:\n{}", filter_actions_dag->dumpDAG()); actions = std::make_shared(filter_actions_dag); + LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Dump actions:\n{}", actions->dumpActions()); } else { @@ -290,7 +293,9 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( traverseAST(expression_ast); auto syntax_analyzer_result = TreeRewriter(context).analyze(expression_ast, index_sample_block.getNamesAndTypesList()); + LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "expression AST:\n{}", expression_ast->dumpTree()); actions = ExpressionAnalyzer(expression_ast, syntax_analyzer_result, context).getActions(true); + LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Dump actions:\n{}", actions->dumpActions()); } } @@ -351,6 +356,8 @@ const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDA const ContextPtr & context, std::unordered_map & node_to_result_node) const { + // LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "Traversing node:\n{}", node.dumpDAG()); + auto result_node_it = node_to_result_node.find(&node); if (result_node_it != node_to_result_node.end()) return *result_node_it->second; @@ -457,8 +464,11 @@ const ActionsDAG::Node * MergeTreeIndexConditionSet::operatorFromDAG(const Actio if (arguments_size != 1) return nullptr; + auto bit_wrapper_function = FunctionFactory::instance().get("__bitWrapperFunc", context); + const auto & bit_wrapper_func_node = result_dag->addFunction(bit_wrapper_function, {arguments[0]}, {}); + auto bit_swap_last_two_function = FunctionFactory::instance().get("__bitSwapLastTwo", context); - return &result_dag->addFunction(bit_swap_last_two_function, {arguments[0]}, {}); + return &result_dag->addFunction(bit_swap_last_two_function, {&bit_wrapper_func_node}, {}); } else if (function_name == "and" || function_name == "indexHint" || function_name == "or") { From 45b96142249c02f71b140b14784969b9a228760c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 28 Jul 2023 23:41:18 +0000 Subject: [PATCH 1297/2047] Remove debug logging --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 56550ab7f3a..05ea7d15603 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -277,9 +276,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( filter_actions_dag->getOutputs()[0] = &traverseDAG(*filter_actions_dag_node, filter_actions_dag, context, node_to_result_node); filter_actions_dag->removeUnusedActions(); - LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Filter actions DAG:\n{}", filter_actions_dag->dumpDAG()); actions = std::make_shared(filter_actions_dag); - LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Dump actions:\n{}", actions->dumpActions()); } else { @@ -293,9 +290,7 @@ MergeTreeIndexConditionSet::MergeTreeIndexConditionSet( traverseAST(expression_ast); auto syntax_analyzer_result = TreeRewriter(context).analyze(expression_ast, index_sample_block.getNamesAndTypesList()); - LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "expression AST:\n{}", expression_ast->dumpTree()); actions = ExpressionAnalyzer(expression_ast, syntax_analyzer_result, context).getActions(true); - LOG_DEBUG(&Poco::Logger::get("MergeTreeIndexConditionSet"), "Dump actions:\n{}", actions->dumpActions()); } } @@ -356,8 +351,6 @@ const ActionsDAG::Node & MergeTreeIndexConditionSet::traverseDAG(const ActionsDA const ContextPtr & context, std::unordered_map & node_to_result_node) const { - // LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "Traversing node:\n{}", node.dumpDAG()); - auto result_node_it = node_to_result_node.find(&node); if (result_node_it != node_to_result_node.end()) return *result_node_it->second; From a6fc632aa18f45ad141d3f524bf1492d0450060a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 28 Jul 2023 23:57:56 +0000 Subject: [PATCH 1298/2047] More test queries + update file with broken tests --- tests/analyzer_tech_debt.txt | 1 - tests/queries/0_stateless/00979_set_index_not.reference | 2 ++ tests/queries/0_stateless/00979_set_index_not.sql | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 737e0e0a5e4..c8f2bb9f43d 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -11,7 +11,6 @@ 00927_asof_joins 00940_order_by_read_in_order_query_plan 00945_bloom_filter_index -00979_set_index_not 00981_in_subquery_with_tuple 01049_join_low_card_bug_long 01062_pm_all_join_with_block_continuation diff --git a/tests/queries/0_stateless/00979_set_index_not.reference b/tests/queries/0_stateless/00979_set_index_not.reference index 455708dfe99..033fcb0467b 100644 --- a/tests/queries/0_stateless/00979_set_index_not.reference +++ b/tests/queries/0_stateless/00979_set_index_not.reference @@ -1,2 +1,4 @@ Jon alive Jon alive +Ramsey rip +Ramsey rip diff --git a/tests/queries/0_stateless/00979_set_index_not.sql b/tests/queries/0_stateless/00979_set_index_not.sql index 2ad27e337f9..13a0b4cbb09 100644 --- a/tests/queries/0_stateless/00979_set_index_not.sql +++ b/tests/queries/0_stateless/00979_set_index_not.sql @@ -11,5 +11,7 @@ insert into set_index_not values ('Jon','alive'),('Ramsey','rip'); select * from set_index_not where status!='rip'; select * from set_index_not where NOT (status ='rip'); +select * from set_index_not where NOT (status!='rip'); +select * from set_index_not where NOT (NOT (status ='rip')); DROP TABLE set_index_not; From f42cfddee7966496836da44e26d27fe208d43ac9 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 20:28:56 -0400 Subject: [PATCH 1299/2047] Update test.py --- tests/integration/test_row_policy/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 1933823f5d2..acf544c98aa 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -420,7 +420,7 @@ def test_introspection(): "mydb", "local", "5b23c389-7e18-06bf-a6bc-dd1afbbc0a97", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -433,7 +433,7 @@ def test_introspection(): "mydb", "filtered_table1", "9e8a8f62-4965-2b5e-8599-57c7b99b3549", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -446,7 +446,7 @@ def test_introspection(): "mydb", "filtered_table2", "cffae79d-b9bf-a2ef-b798-019c18470b25", - "users.xml", + "users_xml", "a + b < 1 or c - d > 5", 0, 0, @@ -459,7 +459,7 @@ def test_introspection(): "mydb", "filtered_table3", "12fc5cef-e3da-3940-ec79-d8be3911f42b", - "users.xml", + "users_xml", "c = 1", 0, 0, From 1ee1ae120e921a355a752b332aae25b18b1cf89d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 29 Jul 2023 02:03:21 +0000 Subject: [PATCH 1300/2047] add modulo, intDiv, intDivOrZero for tuple --- src/Functions/FunctionBinaryArithmetic.h | 24 ++++++++++++++++--- src/Functions/IsOperation.h | 2 +- src/Functions/vectorFunctions.cpp | 21 ++++++++++++++++ .../0_stateless/02841_tuple_modulo.reference | 4 ++++ .../0_stateless/02841_tuple_modulo.sql | 4 ++++ 5 files changed, 51 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02841_tuple_modulo.reference create mode 100644 tests/queries/0_stateless/02841_tuple_modulo.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index bf3b33d13ff..408a16236e8 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -742,6 +742,9 @@ class FunctionBinaryArithmetic : public IFunction static constexpr bool is_multiply = IsOperation::multiply; static constexpr bool is_division = IsOperation::division; static constexpr bool is_bit_hamming_distance = IsOperation::bit_hamming_distance; + static constexpr bool is_modulo = IsOperation::modulo; + static constexpr bool is_div_int = IsOperation::div_int; + static constexpr bool is_div_int_or_zero = IsOperation::div_int_or_zero; ContextPtr context; bool check_decimal_overflow = true; @@ -951,13 +954,28 @@ class FunctionBinaryArithmetic : public IFunction "argument of numeric type cannot be first", name); std::string function_name; - if (is_multiply) + if constexpr (is_multiply) { function_name = "tupleMultiplyByNumber"; } - else + else // is_division { - function_name = "tupleDivideByNumber"; + if constexpr (is_modulo) + { + function_name = "tupleModuloByNumber"; + } + else if constexpr (is_div_int) + { + function_name = "tupleIntDivByNumber"; + } + else if constexpr (is_div_int_or_zero) + { + function_name = "tupleIntDivOrZeroByNumber"; + } + else + { + function_name = "tupleDivideByNumber"; + } } return FunctionFactory::instance().get(function_name, context); diff --git a/src/Functions/IsOperation.h b/src/Functions/IsOperation.h index 0c54901579e..8ea53c865ce 100644 --- a/src/Functions/IsOperation.h +++ b/src/Functions/IsOperation.h @@ -60,7 +60,7 @@ struct IsOperation static constexpr bool bit_hamming_distance = IsSameOperation::value; - static constexpr bool division = div_floating || div_int || div_int_or_zero; + static constexpr bool division = div_floating || div_int || div_int_or_zero || modulo; static constexpr bool allow_decimal = plus || minus || multiply || division || least || greatest; }; diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index d53d39e2f3b..35ba49e4545 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -23,6 +23,9 @@ struct PlusName { static constexpr auto name = "plus"; }; struct MinusName { static constexpr auto name = "minus"; }; struct MultiplyName { static constexpr auto name = "multiply"; }; struct DivideName { static constexpr auto name = "divide"; }; +struct ModuloName { static constexpr auto name = "modulo"; }; +struct IntDivName { static constexpr auto name = "intDiv"; }; +struct IntDivOrZeroName { static constexpr auto name = "intDivOrZero"; }; struct L1Label { static constexpr auto name = "1"; }; struct L2Label { static constexpr auto name = "2"; }; @@ -141,6 +144,12 @@ using FunctionTupleMultiply = FunctionTupleOperator; using FunctionTupleDivide = FunctionTupleOperator; +using FunctionTupleModulo = FunctionTupleOperator; + +using FunctionTupleIntDiv = FunctionTupleOperator; + +using FunctionTupleIntDivOrZero = FunctionTupleOperator; + class FunctionTupleNegate : public ITupleFunction { public: @@ -297,6 +306,12 @@ using FunctionTupleMultiplyByNumber = FunctionTupleOperatorByNumber; +using FunctionTupleModuloByNumber = FunctionTupleOperatorByNumber; + +using FunctionTupleIntDivByNumber = FunctionTupleOperatorByNumber; + +using FunctionTupleIntDivOrZeroByNumber = FunctionTupleOperatorByNumber; + class FunctionDotProduct : public ITupleFunction { public: @@ -1563,6 +1578,9 @@ REGISTER_FUNCTION(VectorFunctions) factory.registerAlias("vectorDifference", FunctionTupleMinus::name, FunctionFactory::CaseInsensitive); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(FunctionDocumentation @@ -1626,6 +1644,9 @@ If the types of the first interval (or the interval in the tuple) and the second factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerAlias("scalarProduct", TupleOrArrayFunctionDotProduct::name, FunctionFactory::CaseInsensitive); diff --git a/tests/queries/0_stateless/02841_tuple_modulo.reference b/tests/queries/0_stateless/02841_tuple_modulo.reference new file mode 100644 index 00000000000..6e6f07d0683 --- /dev/null +++ b/tests/queries/0_stateless/02841_tuple_modulo.reference @@ -0,0 +1,4 @@ +(1,0) +(2,2) +(2,2) +(0,0) diff --git a/tests/queries/0_stateless/02841_tuple_modulo.sql b/tests/queries/0_stateless/02841_tuple_modulo.sql new file mode 100644 index 00000000000..56bacf87967 --- /dev/null +++ b/tests/queries/0_stateless/02841_tuple_modulo.sql @@ -0,0 +1,4 @@ +SELECT (5,4) % 2; +SELECT intDiv((5,4), 2); +SELECT intDivOrZero((5,4), 2); +SELECT intDivOrZero((5,4), 0); From 6f2404b79209bf332e54e83b86eff2f677bbd2bc Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 22:18:53 -0400 Subject: [PATCH 1301/2047] Additional info in exception --- src/Access/MultipleAccessStorage.cpp | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index c6c9ede9e1c..0550c140c17 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -234,17 +234,32 @@ void MultipleAccessStorage::moveAccessEntities(const std::vector & ids, co auto destination_storage = getStorageByName(destination_storage_name); auto to_move = source_storage->read(ids); - source_storage->remove(ids); + bool need_rollback = false; try { + source_storage->remove(ids); + need_rollback = true; destination_storage->insert(to_move, ids); } catch (Exception & e) { - e.addMessage("while moving access entities"); + String message; + + bool need_comma = false; + for (const auto & entity : to_move) + { + if (std::exchange(need_comma, true)) + message += ", "; + + message += entity->formatTypeWithName(); + } + + e.addMessage("while moving {} from {} to {}", message, source_storage_name, destination_storage_name); + + if (need_rollback) + source_storage->insert(to_move, ids); - source_storage->insert(to_move, ids); throw; } } From 10a5230df4d8c352657327fdd75947acc269377a Mon Sep 17 00:00:00 2001 From: copperybean Date: Sat, 29 Jul 2023 11:20:12 +0800 Subject: [PATCH 1302/2047] fix unexpected sort result on multi columns with nulls first direction Change-Id: I94494019312d6ccfe25695f3614353be08800f5f --- src/Columns/ColumnNullable.cpp | 13 ++++++++++--- .../0_stateless/02832_nulls_first_sort.reference | 5 +++++ .../queries/0_stateless/02832_nulls_first_sort.sql | 7 +++++++ 3 files changed, 22 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02832_nulls_first_sort.reference create mode 100644 tests/queries/0_stateless/02832_nulls_first_sort.sql diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index aa04fc910a5..2eb2ff0bf69 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -564,15 +564,22 @@ void ColumnNullable::updatePermutationImpl(IColumn::PermutationSortDirection dir else getNestedColumn().updatePermutation(direction, stability, limit, null_direction_hint, res, new_ranges); - equal_ranges = std::move(new_ranges); - if (unlikely(stability == PermutationSortStability::Stable)) { for (auto & null_range : null_ranges) ::sort(res.begin() + null_range.first, res.begin() + null_range.second); } - std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges)); + if (is_nulls_last || null_ranges.empty()) + { + equal_ranges = std::move(new_ranges); + std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges)); + } + else + { + equal_ranges = std::move(null_ranges); + std::move(new_ranges.begin(), new_ranges.end(), std::back_inserter(equal_ranges)); + } } void ColumnNullable::getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, diff --git a/tests/queries/0_stateless/02832_nulls_first_sort.reference b/tests/queries/0_stateless/02832_nulls_first_sort.reference new file mode 100644 index 00000000000..c16f69ac3c1 --- /dev/null +++ b/tests/queries/0_stateless/02832_nulls_first_sort.reference @@ -0,0 +1,5 @@ +5 \N 1 +5 \N 2 +5 \N 3 +5 \N 7 +5 1 1 diff --git a/tests/queries/0_stateless/02832_nulls_first_sort.sql b/tests/queries/0_stateless/02832_nulls_first_sort.sql new file mode 100644 index 00000000000..e17a49baf24 --- /dev/null +++ b/tests/queries/0_stateless/02832_nulls_first_sort.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS nulls_first_sort_test; +CREATE TABLE nulls_first_sort_test (a Nullable(Int32), b Nullable(Int32), c Nullable(Int32)) ENGINE = Memory; + +INSERT INTO nulls_first_sort_test VALUES (5,null,2), (5,null,1), (5,null,7), (5,null,3), (5,7,4), (5,7,6), (5,7,2), (5,7,1), (5,7,3), (5,7,9), (5,1,4), (5,1,6), (5,1,2), (5,1,1), (5,1,3), (5,1,9); + +SELECT * FROM nulls_first_sort_test ORDER BY a NULLS FIRST,b NULLS FIRST,c NULLS FIRST LIMIT 5; +DROP TABLE nulls_first_sort_test; From a64d6b11f29c3eac5eedf4e48fc118510d4e2a01 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 29 Jul 2023 04:55:19 +0000 Subject: [PATCH 1303/2047] add functions to undocumented --- .../02415_all_new_functions_must_be_documented.reference | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 61a2e4e9f02..b57372ffa8d 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -885,7 +885,13 @@ tupleDivide tupleDivideByNumber tupleElement tupleHammingDistance +tupleIntDiv +tupleIntDivByNumber +tupleIntDivOrZero +tupleIntDivOrZeroByNumber tupleMinus +tupleModulo +tupleModuloByNumber tupleMultiply tupleMultiplyByNumber tupleNegate From 553304a81ac0063f4d60d178ef9b1763fd2b083c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 29 Jul 2023 11:15:56 +0200 Subject: [PATCH 1304/2047] Fixed clang tidy build by removing unued variable --- src/Storages/StorageSnapshot.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index 6abca59268f..c0e85900794 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -120,9 +120,8 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) cons const auto & columns = getMetadataForQuery()->getColumns(); for (const auto & column_name : column_names) { - std::string substituted_column_name = column_name; - auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name); - auto object_column = object_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, substituted_column_name); + auto column = columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, column_name); + auto object_column = object_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::All, column_name); if (column && !object_column) { res.insert({column->type->createColumn(), column->type, column_name}); @@ -141,7 +140,7 @@ Block StorageSnapshot::getSampleBlockForColumns(const Names & column_names) cons else { throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, - "Column {} not found in table {}", backQuote(substituted_column_name), storage.getStorageID().getNameForLogs()); + "Column {} not found in table {}", backQuote(column_name), storage.getStorageID().getNameForLogs()); } } return res; From c270393495c1622c1e085c1abf750c18bfeb27d4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 29 Jul 2023 14:56:43 +0000 Subject: [PATCH 1305/2047] Add abortOnFailedAssertion() declaration in defines.h --- base/base/defines.h | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/base/base/defines.h b/base/base/defines.h index 6abf8155b95..ee29ecf6118 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -115,8 +115,15 @@ /// because SIGABRT is easier to debug than SIGTRAP (the second one makes gdb crazy) #if !defined(chassert) #if defined(ABORT_ON_LOGICAL_ERROR) + // clang-format off + #include + namespace DB + { + void abortOnFailedAssertion(const String & description); + } #define chassert(x) static_cast(x) ? void(0) : ::DB::abortOnFailedAssertion(#x) #define UNREACHABLE() abort() + // clang-format off #else /// Here sizeof() trick is used to suppress unused warning for result, /// since simple "(void)x" will evaluate the expression, while From 84ab4238426b96fb126be8c59055657ed104c1c2 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 28 Jul 2023 18:18:38 +0200 Subject: [PATCH 1306/2047] Allow reading empty file with no blobs --- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 762151b3808..6bb6982065a 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -485,8 +486,15 @@ std::unique_ptr DiskObjectStorage::readFile( std::optional read_hint, std::optional file_size) const { + auto storage_objects = metadata_storage->getStorageObjects(path); + + const bool file_can_be_empty = !file_size.has_value() || *file_size == 0; + + if (storage_objects.empty() && file_can_be_empty) + return std::make_unique(); + return object_storage->readObjects( - metadata_storage->getStorageObjects(path), + storage_objects, object_storage->getAdjustedSettingsFromMetadataFile(settings, path), read_hint, file_size); From cbb546a1a97b4d9819efa82547b48103a40d575d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 00:18:08 +0300 Subject: [PATCH 1307/2047] Update easy_tasks_sorted_ru.md --- tests/instructions/easy_tasks_sorted_ru.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/instructions/easy_tasks_sorted_ru.md b/tests/instructions/easy_tasks_sorted_ru.md index 09ea48d0bd9..17e9708eef5 100644 --- a/tests/instructions/easy_tasks_sorted_ru.md +++ b/tests/instructions/easy_tasks_sorted_ru.md @@ -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 и оператор <=>. From 12069b2cff6c4ae77cc674f02ce4e04987c9515b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Jul 2023 23:25:44 +0200 Subject: [PATCH 1308/2047] Allow yaml configs in clickhouse-client --- src/Common/Config/configReadClient.cpp | 51 ++++++++++++++++++-------- 1 file changed, 36 insertions(+), 15 deletions(-) diff --git a/src/Common/Config/configReadClient.cpp b/src/Common/Config/configReadClient.cpp index 44d338c07af..fea055618bb 100644 --- a/src/Common/Config/configReadClient.cpp +++ b/src/Common/Config/configReadClient.cpp @@ -6,36 +6,57 @@ #include #include + namespace fs = std::filesystem; namespace DB { -bool safeFsExists(const String & path) -{ - std::error_code ec; - return fs::exists(path, ec); -} - bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::string & home_path) { std::string config_path; - if (config.has("config-file")) - config_path = config.getString("config-file"); - else if (safeFsExists("./clickhouse-client.xml")) - config_path = "./clickhouse-client.xml"; - else if (!home_path.empty() && safeFsExists(home_path + "/.clickhouse-client/config.xml")) - config_path = home_path + "/.clickhouse-client/config.xml"; - else if (safeFsExists("/etc/clickhouse-client/config.xml")) - config_path = "/etc/clickhouse-client/config.xml"; - if (!config_path.empty()) + bool found = false; + if (config.has("config-file")) + { + found = true; + config_path = config.getString("config-file"); + } + else + { + std::vector names; + names.emplace_back("./clickhouse-client"); + if (!home_path.empty()) + names.emplace_back(home_path + "/.clickhouse-client/config"); + names.emplace_back("/etc/clickhouse-client/config"); + + for (const auto & name : names) + { + for (const auto & extension : {".xml", ".yaml", ".yml"}) + { + config_path = name + extension; + + std::error_code ec; + if (fs::exists(config_path, ec)) + { + found = true; + break; + } + } + if (found) + break; + } + } + + if (found) { ConfigProcessor config_processor(config_path); auto loaded_config = config_processor.loadConfig(); config.add(loaded_config.configuration); return true; } + return false; } + } From 918571a5eecd08362265fb0bd9543a5920fcd96a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 29 Jul 2023 22:04:21 +0000 Subject: [PATCH 1309/2047] Fix: check correctly frame bounds for RANGE --- src/Interpreters/WindowDescription.cpp | 48 +++++++++---------- src/Processors/Transforms/WindowTransform.cpp | 2 +- .../02833_window_func_range_offset.reference | 0 .../02833_window_func_range_offset.sql | 6 +++ 4 files changed, 29 insertions(+), 27 deletions(-) create mode 100644 tests/queries/0_stateless/02833_window_func_range_offset.reference create mode 100644 tests/queries/0_stateless/02833_window_func_range_offset.sql diff --git a/src/Interpreters/WindowDescription.cpp b/src/Interpreters/WindowDescription.cpp index 7ed7788cf1d..8a7a5024d69 100644 --- a/src/Interpreters/WindowDescription.cpp +++ b/src/Interpreters/WindowDescription.cpp @@ -91,34 +91,30 @@ void WindowFrame::toString(WriteBuffer & buf) const void WindowFrame::checkValid() const { // Check the validity of offsets. - if (type == WindowFrame::FrameType::ROWS - || type == WindowFrame::FrameType::GROUPS) + if (begin_type == BoundaryType::Offset + && !((begin_offset.getType() == Field::Types::UInt64 + || begin_offset.getType() == Field::Types::Int64) + && begin_offset.get() >= 0 + && begin_offset.get() < INT_MAX)) { - if (begin_type == BoundaryType::Offset - && !((begin_offset.getType() == Field::Types::UInt64 - || begin_offset.getType() == Field::Types::Int64) - && begin_offset.get() >= 0 - && begin_offset.get() < INT_MAX)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", - type, - applyVisitor(FieldVisitorToString(), begin_offset), - begin_offset.getType()); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", + type, + applyVisitor(FieldVisitorToString(), begin_offset), + begin_offset.getType()); + } - if (end_type == BoundaryType::Offset - && !((end_offset.getType() == Field::Types::UInt64 - || end_offset.getType() == Field::Types::Int64) - && end_offset.get() >= 0 - && end_offset.get() < INT_MAX)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", - type, - applyVisitor(FieldVisitorToString(), end_offset), - end_offset.getType()); - } + if (end_type == BoundaryType::Offset + && !((end_offset.getType() == Field::Types::UInt64 + || end_offset.getType() == Field::Types::Int64) + && end_offset.get() >= 0 + && end_offset.get() < INT_MAX)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", + type, + applyVisitor(FieldVisitorToString(), end_offset), + end_offset.getType()); } // Check relative positioning of offsets. diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index a785d52bf65..be76971ddcd 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -148,7 +148,7 @@ static int compareValuesWithOffsetFloat(const IColumn * _compared_column, const auto * reference_column = assert_cast( _reference_column); const auto offset = _offset.get(); - assert(offset >= 0); + chassert(offset >= 0); const auto compared_value_data = compared_column->getDataAt(compared_row); assert(compared_value_data.size == sizeof(typename ColumnType::ValueType)); diff --git a/tests/queries/0_stateless/02833_window_func_range_offset.reference b/tests/queries/0_stateless/02833_window_func_range_offset.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02833_window_func_range_offset.sql b/tests/queries/0_stateless/02833_window_func_range_offset.sql new file mode 100644 index 00000000000..f1d26c5cbaf --- /dev/null +++ b/tests/queries/0_stateless/02833_window_func_range_offset.sql @@ -0,0 +1,6 @@ +-- invalid start offset with RANGE +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN 0.0 PRECEDING AND UNBOUNDED FOLLOWING); -- { serverError BAD_ARGUMENTS } +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN nan PRECEDING AND UNBOUNDED FOLLOWING); -- { serverError BAD_ARGUMENTS } +-- invalid end offset with RANGE +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.0 FOLLOWING); -- { serverError BAD_ARGUMENTS } +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND nan FOLLOWING); -- { serverError BAD_ARGUMENTS } From 63b155a74111e7f2b1b94808e8f9aa47374bc71f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 29 Jul 2023 23:12:54 +0000 Subject: [PATCH 1310/2047] bypass merging attempt for non 'clickhouse' configurations with different root node names --- src/Common/Config/ConfigProcessor.cpp | 13 +++++++++++-- src/Common/Config/ConfigProcessor.h | 4 +++- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index a55183782d8..04f55600b40 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -328,7 +328,7 @@ void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root, } } -void ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with) +bool ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with) { Node * config_root = getRootNode(config.get()); Node * with_root = getRootNode(with.get()); @@ -343,11 +343,15 @@ void ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with) && !((config_root_node_name == "yandex" || config_root_node_name == "clickhouse") && (merged_root_node_name == "yandex" || merged_root_node_name == "clickhouse"))) { + if (config_root_node_name != "clickhouse" && config_root_node_name != "yandex") + return false; + throw Poco::Exception("Root element doesn't have the corresponding root element as the config file." " It must be <" + config_root->nodeName() + ">"); } mergeRecursive(config, config_root, with_root); + return true; } void ConfigProcessor::doIncludesRecursive( @@ -645,7 +649,12 @@ XMLDocumentPtr ConfigProcessor::processConfig( with = dom_parser.parse(merge_file); } - merge(config, with); + if (!merge(config, with)) + { + LOG_DEBUG(log, "Merging bypassed - configuration file '{}' doesn't belong to configuration '{}' - merging root node name '{}' doesn't match '{}'", + merge_file, path, getRootNode(with.get())->nodeName(), getRootNode(config.get())->nodeName()); + continue; + } contributing_files.push_back(merge_file); } diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 060ef49d36a..b4f85b10526 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -144,7 +144,9 @@ private: void mergeRecursive(XMLDocumentPtr config, Poco::XML::Node * config_root, const Poco::XML::Node * with_root); - void merge(XMLDocumentPtr config, XMLDocumentPtr with); + /// If config root node name is not 'clickhouse' and merging config's root node names doesn't match, bypasses merging and returns false. + /// For compatibility root node 'yandex' considered equal to 'clickhouse'. + bool merge(XMLDocumentPtr config, XMLDocumentPtr with); void doIncludesRecursive( XMLDocumentPtr config, From 6bcc421c0262fce9e2eade3cbc436b8241e4ae61 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 01:27:14 +0200 Subject: [PATCH 1311/2047] Numerical stability of the test for Polygons --- .../01304_polygons_sym_difference.reference | 6 ++-- .../01304_polygons_sym_difference.sql | 31 +++++++++++++++++-- 2 files changed, 31 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01304_polygons_sym_difference.reference b/tests/queries/0_stateless/01304_polygons_sym_difference.reference index 9344410f192..e03b357aab5 100644 --- a/tests/queries/0_stateless/01304_polygons_sym_difference.reference +++ b/tests/queries/0_stateless/01304_polygons_sym_difference.reference @@ -1,7 +1,7 @@ [[[(1,2.9),(1,1),(2.9,1),(3,0),(0,0),(0,3),(1,2.9)]],[[(1,2.9),(1,4),(4,4),(4,1),(2.9,1),(2.6,2),(2,2.6),(1,2.9)]]] -------- MultiPolygon with Polygon -MULTIPOLYGON(((-20 -10.3067,-20 -20,-10 -20.8791,-10 -40,-40 -40,-40 -10,-20 -10.3067)),((20 10.3067,20 -20,-10 -20.8791,-10 -10,-20 -10.3067,-20 20,10 20.8791,10 10,20 10.3067)),((20 10.3067,20 20,10 20.8791,10 40,40 40,40 10,20 10.3067))) +MULTIPOLYGON(((-40 -40,-40 -10,-20 -10.3067,-20 -20,-10 -20.8791,-10 -40,-40 -40)),((-20 20,10 20.8791,10 10,20 10.3067,20 -20,-10 -20.8791,-10 -10,-20 -10.3067,-20 20)),((10 40,40 40,40 10,20 10.3067,20 20,10 20.8791,10 40))) -------- MultiPolygon with Polygon with Holes -MULTIPOLYGON(((-10 -20.8791,-20 -20,-20 -10.3067,-10 -10,-10 -20.8791)),((10 20.8791,20 20,20 10.3067,10 10,10 20.8791)),((50 50,50 -50,-50 -50,-50 50,50 50),(20 10.3067,40 10,40 40,10 40,10 20.8791,-20 20,-20 -10.3067,-40 -10,-40 -40,-10 -40,-10 -20.8791,20 -20,20 10.3067))) +MULTIPOLYGON(((-50 -50,-50 50,50 50,50 -50,-50 -50),(-40 -40,-10 -40,-10 -20.8791,20 -20,20 10.3067,40 10,40 40,10 40,10 20.8791,-20 20,-20 -10.3067,-40 -10,-40 -40)),((-20 -20,-20 -10.3067,-10 -10,-10 -20.8791,-20 -20)),((10 10,10 20.8791,20 20,20 10.3067,10 10))) -------- Polygon with Polygon with Holes -MULTIPOLYGON(((-20 -10.3067,-10 -10,-10 -20.8791,-20 -20,-20 -10.3067)),((10 20.8791,20 20,20 10.3067,10 10,10 20.8791)),((50 50,50 -50,-50 -50,-50 50,50 50),(20 10.3067,40 10,40 40,10 40,10 20.8791,-20 20,-20 -10.3067,-40 -10,-40 -40,-10 -40,-10 -20.8791,20 -20,20 10.3067))) +MULTIPOLYGON(((-50 -50,-50 50,50 50,50 -50,-50 -50),(-40 -40,-10 -40,-10 -20.8791,20 -20,20 10.3067,40 10,40 40,10 40,10 20.8791,-20 20,-20 -10.3067,-40 -10,-40 -40)),((-20 -20,-20 -10.3067,-10 -10,-10 -20.8791,-20 -20)),((10 10,10 20.8791,20 20,20 10.3067,10 10))) diff --git a/tests/queries/0_stateless/01304_polygons_sym_difference.sql b/tests/queries/0_stateless/01304_polygons_sym_difference.sql index c4129676b26..b2dc24149d3 100644 --- a/tests/queries/0_stateless/01304_polygons_sym_difference.sql +++ b/tests/queries/0_stateless/01304_polygons_sym_difference.sql @@ -1,15 +1,40 @@ +-- Tags: no-parallel + +DROP FUNCTION IF EXISTS sortPolygon; +DROP FUNCTION IF EXISTS rotate; +DROP FUNCTION IF EXISTS sortRing; +DROP FUNCTION IF EXISTS sortMultiPolygon; + +-- in every polygon, the first array is the main part, the rest are - the holes: +CREATE FUNCTION sortPolygon AS arr -> arrayPushFront(arraySort(arrayPopFront(arr)), arr[1]); + +-- Rotate array, putting minimum element to the front: +CREATE FUNCTION rotate AS arr -> arraySort((e, i) -> (i < arrayFirstIndex(e -> e = arrayReduce('min', arr), arr), i), arr, arrayEnumerate(arr)); + +-- in every ring, the first and the last points are identical, but they are selected randomly in a ring: +CREATE FUNCTION sortRing AS arr -> arrayPushBack(rotate(arrayPopBack(arr)) AS rotated, rotated[1]); + +-- normalize the whole multipolygon: +CREATE FUNCTION sortMultiPolygon AS arr -> arraySort(arrayMap(polygon -> sortPolygon(arrayMap(ring -> sortRing(ring), polygon)), arr)); + + select polygonsSymDifferenceCartesian([[[(0, 0),(0, 3),(1, 2.9),(2, 2.6),(2.6, 2),(2.9, 1),(3, 0),(0, 0)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]); -- Google "draw wkt online" select '-------- MultiPolygon with Polygon'; select wkt(polygonsSymDifferenceSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]])) format Null; -SELECT wkt(arraySort(polygonsSymDifferenceSpherical([[[(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)]], [[(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]]], [[[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]]))); +SELECT wkt(sortMultiPolygon(polygonsSymDifferenceSpherical([[[(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)]], [[(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]]], [[[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]]))); select '-------- MultiPolygon with Polygon with Holes'; select wkt(polygonsSymDifferenceSpherical([[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]])) format Null; -SELECT wkt(arraySort(polygonsSymDifferenceSpherical([[(50.,50.),(50.,-50.),(-50.,-50.),(-50.,50.),(50.,50.)],[(10.,10.),(10.,40.),(40.,40.),(40.,10.),(10.,10.)],[(-10.,-10.),(-10.,-40.),(-40.,-40.),(-40.,-10.),(-10.,-10.)]], [[[(-20.,-20.),(-20.,20.),(20.,20.),(20.,-20.),(-20.,-20.)]]]))); +SELECT wkt(sortMultiPolygon(polygonsSymDifferenceSpherical([[(50.,50.),(50.,-50.),(-50.,-50.),(-50.,50.),(50.,50.)],[(10.,10.),(10.,40.),(40.,40.),(40.,10.),(10.,10.)],[(-10.,-10.),(-10.,-40.),(-40.,-40.),(-40.,-10.),(-10.,-10.)]], [[[(-20.,-20.),(-20.,20.),(20.,20.),(20.,-20.),(-20.,-20.)]]]))); select '-------- Polygon with Polygon with Holes'; select wkt(polygonsSymDifferenceSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]])) format Null; -SELECT wkt(arraySort(polygonsSymDifferenceSpherical([[(50., 50.), (50., -50.), (-50., -50.), (-50., 50.), (50., 50.)], [(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)], [(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]], [[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]))); +SELECT wkt(sortMultiPolygon(polygonsSymDifferenceSpherical([[(50., 50.), (50., -50.), (-50., -50.), (-50., 50.), (50., 50.)], [(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)], [(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]], [[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]))); + +DROP FUNCTION sortPolygon; +DROP FUNCTION rotate; +DROP FUNCTION sortRing; +DROP FUNCTION sortMultiPolygon; From 4d27cf4386e1b3b3d0b1516723fbef0f9143ebcc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 02:11:32 +0200 Subject: [PATCH 1312/2047] Proper stripping --- cmake/split_debug_symbols.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/split_debug_symbols.cmake b/cmake/split_debug_symbols.cmake index 8ba7669d9e2..d6821eb6c48 100644 --- a/cmake/split_debug_symbols.cmake +++ b/cmake/split_debug_symbols.cmake @@ -24,7 +24,7 @@ macro(clickhouse_split_debug_symbols) COMMAND chmod 0644 "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" # Strips binary, sections '.note' & '.comment' are removed in line with Debian's stripping policy: www.debian.org/doc/debian-policy/ch-files.html, section '.clickhouse.hash' is needed for integrity check. # Also, after we disabled the export of symbols for dynamic linking, we still to keep a static symbol table for good stack traces. - COMMAND "${STRIP_PATH}" --remove-section=.comment --remove-section=.note --keep-section=.clickhouse.hash --keep-section=.symtab --keep-section=.strtab --keep-section=.shstrtab "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" + COMMAND "${STRIP_PATH}" --strip-debug --remove-section=.comment --remove-section=.note "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" # Associate stripped binary with debug symbols: COMMAND "${OBJCOPY_PATH}" --add-gnu-debuglink "${STRIP_DESTINATION_DIR}/lib/debug/bin/${STRIP_TARGET}.debug" "${STRIP_DESTINATION_DIR}/bin/${STRIP_TARGET}" COMMENT "Stripping clickhouse binary" VERBATIM From a4f2eea51603bb98340e8c6add215ca8b7fc676d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 02:19:39 +0200 Subject: [PATCH 1313/2047] Maybe better --- utils/check-style/check-style | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 67c185a0b54..b728602ef40 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -12,6 +12,7 @@ # (simple regexps) to check if the code is likely to have basic style violations. # and then to run formatter only for the specified files. +LC_ALL="en_US.UTF-8" ROOT_PATH=$(git rev-parse --show-toplevel) EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|poco/|memcpy/|consistent-hashing|benchmark|tests/.*.cpp|utils/keeper-bench/example.yaml' From 9e0fb6b81da2a72c41a40f16dea245d1f6a16f35 Mon Sep 17 00:00:00 2001 From: copperybean Date: Sun, 30 Jul 2023 09:48:05 +0800 Subject: [PATCH 1314/2047] rename test files Change-Id: I2388f72af133e4496f957f8361fd23a5cf79469f --- ...ulls_first_sort.reference => 02834_nulls_first_sort.reference} | 0 .../{02832_nulls_first_sort.sql => 02834_nulls_first_sort.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02832_nulls_first_sort.reference => 02834_nulls_first_sort.reference} (100%) rename tests/queries/0_stateless/{02832_nulls_first_sort.sql => 02834_nulls_first_sort.sql} (100%) diff --git a/tests/queries/0_stateless/02832_nulls_first_sort.reference b/tests/queries/0_stateless/02834_nulls_first_sort.reference similarity index 100% rename from tests/queries/0_stateless/02832_nulls_first_sort.reference rename to tests/queries/0_stateless/02834_nulls_first_sort.reference diff --git a/tests/queries/0_stateless/02832_nulls_first_sort.sql b/tests/queries/0_stateless/02834_nulls_first_sort.sql similarity index 100% rename from tests/queries/0_stateless/02832_nulls_first_sort.sql rename to tests/queries/0_stateless/02834_nulls_first_sort.sql From 1deebd3ae624232e2b9ae05e7b1350816ef490ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 04:22:36 +0200 Subject: [PATCH 1315/2047] Fix Docker --- docker/test/base/Dockerfile | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index f6836804454..bfbe9461c1e 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -11,6 +11,7 @@ RUN apt-get update \ pv \ ripgrep \ zstd \ + locales \ --yes --no-install-recommends # Sanitizer options for services (clickhouse-server) @@ -28,7 +29,13 @@ ENV TSAN_OPTIONS='halt_on_error=1 history_size=7 memory_limit_mb=46080 second_de ENV UBSAN_OPTIONS='print_stacktrace=1' ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1' +RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8 +ENV LC_ALL en_US.UTF-8 + ENV TZ=Europe/Moscow RUN ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone CMD sleep 1 + + +RUN apt-get update && apt-get install -y locales From d050a8fbd90c80dbb6be6ff20dc1e1844fd023ac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 04:29:11 +0200 Subject: [PATCH 1316/2047] Fix test --- .../01304_polygons_sym_difference.reference | 6 +++--- .../0_stateless/01304_polygons_sym_difference.sql | 11 ++++++++--- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01304_polygons_sym_difference.reference b/tests/queries/0_stateless/01304_polygons_sym_difference.reference index e03b357aab5..f10be156d6f 100644 --- a/tests/queries/0_stateless/01304_polygons_sym_difference.reference +++ b/tests/queries/0_stateless/01304_polygons_sym_difference.reference @@ -1,7 +1,7 @@ [[[(1,2.9),(1,1),(2.9,1),(3,0),(0,0),(0,3),(1,2.9)]],[[(1,2.9),(1,4),(4,4),(4,1),(2.9,1),(2.6,2),(2,2.6),(1,2.9)]]] -------- MultiPolygon with Polygon -MULTIPOLYGON(((-40 -40,-40 -10,-20 -10.3067,-20 -20,-10 -20.8791,-10 -40,-40 -40)),((-20 20,10 20.8791,10 10,20 10.3067,20 -20,-10 -20.8791,-10 -10,-20 -10.3067,-20 20)),((10 40,40 40,40 10,20 10.3067,20 20,10 20.8791,10 40))) +[[[(-40,-40),(-40,-10),(-20,-10.307),(-20,-20),(-10,-20.879),(-10,-40),(-40,-40)]],[[(-20,20),(10,20.879),(10,10),(20,10.307),(20,-20),(-10,-20.879),(-10,-10),(-20,-10.307),(-20,20)]],[[(10,40),(40,40),(40,10),(20,10.307),(20,20),(10,20.879),(10,40)]]] -------- MultiPolygon with Polygon with Holes -MULTIPOLYGON(((-50 -50,-50 50,50 50,50 -50,-50 -50),(-40 -40,-10 -40,-10 -20.8791,20 -20,20 10.3067,40 10,40 40,10 40,10 20.8791,-20 20,-20 -10.3067,-40 -10,-40 -40)),((-20 -20,-20 -10.3067,-10 -10,-10 -20.8791,-20 -20)),((10 10,10 20.8791,20 20,20 10.3067,10 10))) +[[[(-50,-50),(-50,50),(50,50),(50,-50),(-50,-50)],[(-40,-40),(-10,-40),(-10,-20.879),(20,-20),(20,10.307),(40,10),(40,40),(10,40),(10,20.879),(-20,20),(-20,-10.307),(-40,-10),(-40,-40)]],[[(-20,-20),(-20,-10.307),(-10,-10),(-10,-20.879),(-20,-20)]],[[(10,10),(10,20.879),(20,20),(20,10.307),(10,10)]]] -------- Polygon with Polygon with Holes -MULTIPOLYGON(((-50 -50,-50 50,50 50,50 -50,-50 -50),(-40 -40,-10 -40,-10 -20.8791,20 -20,20 10.3067,40 10,40 40,10 40,10 20.8791,-20 20,-20 -10.3067,-40 -10,-40 -40)),((-20 -20,-20 -10.3067,-10 -10,-10 -20.8791,-20 -20)),((10 10,10 20.8791,20 20,20 10.3067,10 10))) +[[[(-50,-50),(-50,50),(50,50),(50,-50),(-50,-50)],[(-40,-40),(-10,-40),(-10,-20.879),(20,-20),(20,10.307),(40,10),(40,40),(10,40),(10,20.879),(-20,20),(-20,-10.307),(-40,-10),(-40,-40)]],[[(-20,-20),(-20,-10.307),(-10,-10),(-10,-20.879),(-20,-20)]],[[(10,10),(10,20.879),(20,20),(20,10.307),(10,10)]]] diff --git a/tests/queries/0_stateless/01304_polygons_sym_difference.sql b/tests/queries/0_stateless/01304_polygons_sym_difference.sql index b2dc24149d3..f8d7a487980 100644 --- a/tests/queries/0_stateless/01304_polygons_sym_difference.sql +++ b/tests/queries/0_stateless/01304_polygons_sym_difference.sql @@ -4,6 +4,7 @@ DROP FUNCTION IF EXISTS sortPolygon; DROP FUNCTION IF EXISTS rotate; DROP FUNCTION IF EXISTS sortRing; DROP FUNCTION IF EXISTS sortMultiPolygon; +DROP FUNCTION IF EXISTS normalizeMultiPolygon; -- in every polygon, the first array is the main part, the rest are - the holes: CREATE FUNCTION sortPolygon AS arr -> arrayPushFront(arraySort(arrayPopFront(arr)), arr[1]); @@ -17,6 +18,9 @@ CREATE FUNCTION sortRing AS arr -> arrayPushBack(rotate(arrayPopBack(arr)) AS ro -- normalize the whole multipolygon: CREATE FUNCTION sortMultiPolygon AS arr -> arraySort(arrayMap(polygon -> sortPolygon(arrayMap(ring -> sortRing(ring), polygon)), arr)); +-- also, cut the floating point numbers: +CREATE FUNCTION normalizeMultiPolygon AS arr -> arrayMap(x -> arrayMap(y -> arrayMap(z -> (round(z.1, 3), round(z.2, 3)), y), x), sortMultiPolygon(arr)); + select polygonsSymDifferenceCartesian([[[(0, 0),(0, 3),(1, 2.9),(2, 2.6),(2.6, 2),(2.9, 1),(3, 0),(0, 0)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]); @@ -24,17 +28,18 @@ select polygonsSymDifferenceCartesian([[[(0, 0),(0, 3),(1, 2.9),(2, 2.6),(2.6, 2 select '-------- MultiPolygon with Polygon'; select wkt(polygonsSymDifferenceSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]])) format Null; -SELECT wkt(sortMultiPolygon(polygonsSymDifferenceSpherical([[[(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)]], [[(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]]], [[[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]]))); +SELECT normalizeMultiPolygon(polygonsSymDifferenceSpherical([[[(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)]], [[(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]]], [[[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]])); select '-------- MultiPolygon with Polygon with Holes'; select wkt(polygonsSymDifferenceSpherical([[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]])) format Null; -SELECT wkt(sortMultiPolygon(polygonsSymDifferenceSpherical([[(50.,50.),(50.,-50.),(-50.,-50.),(-50.,50.),(50.,50.)],[(10.,10.),(10.,40.),(40.,40.),(40.,10.),(10.,10.)],[(-10.,-10.),(-10.,-40.),(-40.,-40.),(-40.,-10.),(-10.,-10.)]], [[[(-20.,-20.),(-20.,20.),(20.,20.),(20.,-20.),(-20.,-20.)]]]))); +SELECT normalizeMultiPolygon(polygonsSymDifferenceSpherical([[(50.,50.),(50.,-50.),(-50.,-50.),(-50.,50.),(50.,50.)],[(10.,10.),(10.,40.),(40.,40.),(40.,10.),(10.,10.)],[(-10.,-10.),(-10.,-40.),(-40.,-40.),(-40.,-10.),(-10.,-10.)]], [[[(-20.,-20.),(-20.,20.),(20.,20.),(20.,-20.),(-20.,-20.)]]])); select '-------- Polygon with Polygon with Holes'; select wkt(polygonsSymDifferenceSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]])) format Null; -SELECT wkt(sortMultiPolygon(polygonsSymDifferenceSpherical([[(50., 50.), (50., -50.), (-50., -50.), (-50., 50.), (50., 50.)], [(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)], [(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]], [[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]))); +SELECT normalizeMultiPolygon(polygonsSymDifferenceSpherical([[(50., 50.), (50., -50.), (-50., -50.), (-50., 50.), (50., 50.)], [(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)], [(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]], [[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]])); DROP FUNCTION sortPolygon; DROP FUNCTION rotate; DROP FUNCTION sortRing; DROP FUNCTION sortMultiPolygon; +DROP FUNCTION normalizeMultiPolygon; From a336f06f57542015ba63043a4bdf7261ee1f0ac3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 05:46:05 +0300 Subject: [PATCH 1317/2047] Update Context.cpp --- src/Interpreters/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 123c2ab8f85..8b8333524b8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1581,7 +1581,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const { auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression); - StorageView::replaceQueryParametersIfParametrizedView(query,parameterized_view_values); + StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values); ASTCreateQuery create; create.select = query->as(); From 972653cde3045d309c2bff180630cedefa9b10b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 05:18:52 +0200 Subject: [PATCH 1318/2047] Change the default timezones in Docker test images --- docker/test/base/Dockerfile | 2 +- docker/test/fasttest/Dockerfile | 2 +- docker/test/fuzzer/Dockerfile | 2 +- docker/test/performance-comparison/Dockerfile | 2 +- docker/test/stateless/Dockerfile | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index f6836804454..17aed0d4402 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -28,7 +28,7 @@ ENV TSAN_OPTIONS='halt_on_error=1 history_size=7 memory_limit_mb=46080 second_de ENV UBSAN_OPTIONS='print_stacktrace=1' ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1' -ENV TZ=Europe/Moscow +ENV TZ=Europe/Amsterdam RUN ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone CMD sleep 1 diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index da4baa8c687..ad24e662a6c 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -32,7 +32,7 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \ && rm -rf /tmp/clickhouse-odbc-tmp -ENV TZ=Europe/Moscow +ENV TZ=Europe/Amsterdam RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ENV COMMIT_SHA='' diff --git a/docker/test/fuzzer/Dockerfile b/docker/test/fuzzer/Dockerfile index aa71074c02a..0bc0fb06633 100644 --- a/docker/test/fuzzer/Dockerfile +++ b/docker/test/fuzzer/Dockerfile @@ -8,7 +8,7 @@ ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list ENV LANG=C.UTF-8 -ENV TZ=Europe/Moscow +ENV TZ=Europe/Amsterdam RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone RUN apt-get update \ diff --git a/docker/test/performance-comparison/Dockerfile b/docker/test/performance-comparison/Dockerfile index ab9f1f8a2e3..cfd7c613868 100644 --- a/docker/test/performance-comparison/Dockerfile +++ b/docker/test/performance-comparison/Dockerfile @@ -11,7 +11,7 @@ ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list ENV LANG=C.UTF-8 -ENV TZ=Europe/Moscow +ENV TZ=Europe/Amsterdam RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone RUN apt-get update \ diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index e1e84c427ba..6f44c7cc88b 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -52,7 +52,7 @@ RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && odbcinst -i -s -l -f /tmp/clickhouse-odbc-tmp/share/doc/clickhouse-odbc/config/odbc.ini.sample \ && rm -rf /tmp/clickhouse-odbc-tmp -ENV TZ=Europe/Moscow +ENV TZ=Europe/Amsterdam RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ENV NUM_TRIES=1 From ecfde7edc6ad3292739ada60ec36c8f9413f9ffe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 05:20:51 +0200 Subject: [PATCH 1319/2047] Fix Docker --- docker/test/style/Dockerfile | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 2aa0b1a62d6..bd1c26855b7 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -18,9 +18,13 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ shellcheck \ yamllint \ + locales \ && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 \ && apt-get clean \ - && rm -rf /root/.cache/pip + && rm -rf /root/.cache/pip + +RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8 +ENV LC_ALL en_US.UTF-8 # Architecture of the image when BuildKit/buildx is used ARG TARGETARCH From 424fe43090fce65d564997452a85e5d617271b52 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 06:57:11 +0300 Subject: [PATCH 1320/2047] Update Dockerfile --- docker/test/base/Dockerfile | 3 --- 1 file changed, 3 deletions(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index bfbe9461c1e..6ba44662af9 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -36,6 +36,3 @@ ENV TZ=Europe/Moscow RUN ln -snf "/usr/share/zoneinfo/$TZ" /etc/localtime && echo "$TZ" > /etc/timezone CMD sleep 1 - - -RUN apt-get update && apt-get install -y locales From 85387912ee3c1f5345b8c5b95181e1325124eb6d Mon Sep 17 00:00:00 2001 From: xiebin Date: Sun, 30 Jul 2023 12:06:30 +0800 Subject: [PATCH 1321/2047] just remove the first bad query, because automatically converting the layout simple to complex with two keys now. --- .../01018_ddl_dictionaries_bad_queries.reference | 1 - .../01018_ddl_dictionaries_bad_queries.sh | 15 --------------- 2 files changed, 16 deletions(-) diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.reference b/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.reference index abc3218ce6c..5ee4e7592f6 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.reference +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.reference @@ -4,4 +4,3 @@ 2 2 2 -2 diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh b/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh index 6826ed677db..8a40129ad4f 100755 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh @@ -9,21 +9,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP DICTIONARY IF EXISTS dict1" -# Simple layout, but with two keys -$CLICKHOUSE_CLIENT -q " - CREATE DICTIONARY dict1 - ( - key1 UInt64, - key2 UInt64, - value String - ) - PRIMARY KEY key1, key2 - LAYOUT(HASHED()) - SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE')) - LIFETIME(MIN 1 MAX 10) -" 2>&1 | grep -c 'Primary key for simple dictionary must contain exactly one element' - - # Simple layout, but with non existing key $CLICKHOUSE_CLIENT -q " CREATE DICTIONARY dict1 From c921748d6199c763d794b44f3ec3c26eae03fbba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Sun, 30 Jul 2023 08:58:16 +0300 Subject: [PATCH 1322/2047] Add named collections ddl on cluster integration test --- .../__init__.py | 0 .../configs/config.d/cluster.xml | 22 +++ .../configs/users.d/default.xml | 12 ++ .../test.py | 135 ++++++++++++++++++ 4 files changed, 169 insertions(+) create mode 100644 tests/integration/test_named_collections_if_exists_on_cluster/__init__.py create mode 100644 tests/integration/test_named_collections_if_exists_on_cluster/configs/config.d/cluster.xml create mode 100644 tests/integration/test_named_collections_if_exists_on_cluster/configs/users.d/default.xml create mode 100644 tests/integration/test_named_collections_if_exists_on_cluster/test.py diff --git a/tests/integration/test_named_collections_if_exists_on_cluster/__init__.py b/tests/integration/test_named_collections_if_exists_on_cluster/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_named_collections_if_exists_on_cluster/configs/config.d/cluster.xml b/tests/integration/test_named_collections_if_exists_on_cluster/configs/config.d/cluster.xml new file mode 100644 index 00000000000..95336350c1e --- /dev/null +++ b/tests/integration/test_named_collections_if_exists_on_cluster/configs/config.d/cluster.xml @@ -0,0 +1,22 @@ + + + + + true + + clickhouse1 + 9000 + + + clickhouse2 + 9000 + + + clickhouse3 + 9000 + + + true + + + diff --git a/tests/integration/test_named_collections_if_exists_on_cluster/configs/users.d/default.xml b/tests/integration/test_named_collections_if_exists_on_cluster/configs/users.d/default.xml new file mode 100644 index 00000000000..15da914f666 --- /dev/null +++ b/tests/integration/test_named_collections_if_exists_on_cluster/configs/users.d/default.xml @@ -0,0 +1,12 @@ + + + + + default + default + 1 + 1 + 1 + + + diff --git a/tests/integration/test_named_collections_if_exists_on_cluster/test.py b/tests/integration/test_named_collections_if_exists_on_cluster/test.py new file mode 100644 index 00000000000..9b8159a1071 --- /dev/null +++ b/tests/integration/test_named_collections_if_exists_on_cluster/test.py @@ -0,0 +1,135 @@ +""" +Test cases: + +--- execute on the first node +create named collection foobar as a=1, b=2; +create named collection if not exists foobar on cluster '{cluster}' as a=1, b=2, c=3; +create named collection collection_present_on_first_node as a=1, b=2, s='string', x=0, y=-1; + +--- execute on any other node +alter named collection foobar on cluster '{cluster}' set a=2, c=3; +alter named collection foobar on cluster '{cluster}' delete b; +alter named collection foobar on cluster '{cluster}' set a=3 delete c; +alter named collection if exists collection_absent_ewerywhere on cluster '{cluster}' delete b; +alter named collection if exists collection_present_on_first_node on cluster '{cluster}' delete b; + +--- execute on every node +select * from system.named_collections; + +--- execute on any node +drop named collection foobar on cluster '{cluster}'; +drop named collection if exists collection_absent_ewerywhere on cluster '{cluster}'; +drop named collection if exists collection_present_on_first_node on cluster '{cluster}'; + +--- execute on every node +select * from system.named_collections; +""" + +import logging +from json import dumps, loads +from functools import partial +import pytest +from helpers.cluster import ClickHouseCluster + +dumps = partial(dumps, ensure_ascii=False) + +NODE01, NODE02, NODE03 = "clickhouse1", "clickhouse2", "clickhouse3" + +CHECK_STRING_VALUE = "Some ~`$tr!ng-_+=123@#%^&&()|?[]{}<🤡>.,\t\n:;" + +STMT_CREATE = "CREATE NAMED COLLECTION" +STMT_ALTER = "ALTER NAMED COLLECTION" +STMT_DROP = "DROP NAMED COLLECTION" + +SYSTEM_TABLE = "system.named_collections" + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + common_kwargs = dict( + main_configs=[ + "configs/config.d/cluster.xml", + ], + user_configs=[ + "configs/users.d/default.xml", + ], + with_zookeeper=True, + stay_alive=True + ) + for name in [NODE01, NODE02, NODE03]: + cluster.add_instance(name, **common_kwargs) + + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def test_create_alter_drop_on_cluster(cluster): + """ + Executes the set of queries and checks the final named collections state. + """ + q_count_collections = f"select count() from {SYSTEM_TABLE}" + + def check_collections_empty(): + for name, node in list(cluster.instances.items()): + assert "0" == node.query(q_count_collections).strip(), f"{SYSTEM_TABLE} is not empty on {name}" + + foobar_final_state = { + "name": "foobar", + "collection": {"a": "3"} + } + collection_present_on_first_node_final_state = { + "name": "collection_present_on_first_node", + "collection": {"a": "1", "s": CHECK_STRING_VALUE, "x": "0", "y": "-1"} + } + expected_state = { + NODE01: [foobar_final_state, collection_present_on_first_node_final_state], + NODE02: [foobar_final_state], + NODE03: [foobar_final_state] + } + + q_get_collections = f"select * from {SYSTEM_TABLE} order by name desc format JSON" + + def check_state(): + for name, node in list(cluster.instances.items()): + result = loads(node.query(q_get_collections))["data"] + logging.debug('%s ?= %s', dumps(result), dumps(expected_state[name])) + assert expected_state[name] == result, f"invalid {SYSTEM_TABLE} content on {name}: {result}" + + check_collections_empty() + + # create executed on the first node + node = cluster.instances[NODE01] + node.query(f"{STMT_CREATE} foobar AS a=1, b=2") + node.query(f"{STMT_CREATE} IF NOT EXISTS foobar ON CLUSTER 'cluster' AS a=1, b=2, c=3") + node.query(f"{STMT_CREATE} collection_present_on_first_node AS a=1, b=2, s='{CHECK_STRING_VALUE}', x=0, y=-1") + + # alter executed on the second node + node = cluster.instances[NODE02] + node.query(f"{STMT_ALTER} foobar ON CLUSTER 'cluster' SET a=2, c=3") + node.query(f"{STMT_ALTER} foobar ON CLUSTER 'cluster' DELETE b") + node.query(f"{STMT_ALTER} foobar ON CLUSTER 'cluster' SET a=3 DELETE c") + node.query(f"{STMT_ALTER} IF EXISTS collection_absent_ewerywhere ON CLUSTER 'cluster' DELETE b") + node.query(f"{STMT_ALTER} IF EXISTS collection_present_on_first_node ON CLUSTER 'cluster' DELETE b") + + check_state() + for node in list(cluster.instances.values()): + node.restart_clickhouse() + check_state() + + # drop executed on the third node + node = cluster.instances[NODE03] + node.query(f"{STMT_DROP} foobar ON CLUSTER 'cluster'") + node.query(f"{STMT_DROP} IF EXISTS collection_absent_ewerywhere ON CLUSTER 'cluster'") + node.query(f"{STMT_DROP} IF EXISTS collection_present_on_first_node ON CLUSTER 'cluster'") + + check_collections_empty() + for node in list(cluster.instances.values()): + node.restart_clickhouse() + check_collections_empty() From b3834a3a5b9382e1e25dded831719e2af0da0d30 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Sun, 30 Jul 2023 08:59:44 +0300 Subject: [PATCH 1323/2047] Add bugfix validate check description --- docs/en/development/continuous-integration.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index 738c5458cc3..c76ab738004 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -141,6 +141,10 @@ Runs [stateful functional tests](tests.md#functional-tests). Treat them in the s Runs [integration tests](tests.md#integration-tests). +## Bugfix validate check +Checks that either a new test (functional or integration) or there some changed tests that fail with the binary built on master branch. This check is triggered when pull request has "pr-bugfix" label. + + ## Stress Test Runs stateless functional tests concurrently from several clients to detect concurrency-related errors. If it fails: From 9c5b96e26cee0fd2685ef0b8312ebf3935e34f25 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Sun, 30 Jul 2023 09:37:19 +0300 Subject: [PATCH 1324/2047] Add "bugfix" to aspell-ignore --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a314815e2c4..81415436c45 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -98,6 +98,7 @@ BlockWriteOps BlockWriteTime Bool BrokenDistributedFilesToInsert +Bugfix BuildID BuilderBinAarch BuilderBinAmd @@ -1158,6 +1159,7 @@ brotli bson bsoneachrow buffersize +bugfix buildId buildable builtins From 7725c534081d28a009bfe3b23d563b1bb2dc1471 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=98=D0=BB=D1=8C=D1=8F=20=D0=9A=D0=BE=D1=80=D0=B3=D1=83?= =?UTF-8?q?=D0=BD?= Date: Sun, 30 Jul 2023 09:42:42 +0300 Subject: [PATCH 1325/2047] Reformat the previously added test with black --- .../test.py | 45 ++++++++++++------- 1 file changed, 29 insertions(+), 16 deletions(-) diff --git a/tests/integration/test_named_collections_if_exists_on_cluster/test.py b/tests/integration/test_named_collections_if_exists_on_cluster/test.py index 9b8159a1071..5f5be9156b9 100644 --- a/tests/integration/test_named_collections_if_exists_on_cluster/test.py +++ b/tests/integration/test_named_collections_if_exists_on_cluster/test.py @@ -56,7 +56,7 @@ def cluster(): "configs/users.d/default.xml", ], with_zookeeper=True, - stay_alive=True + stay_alive=True, ) for name in [NODE01, NODE02, NODE03]: cluster.add_instance(name, **common_kwargs) @@ -78,20 +78,19 @@ def test_create_alter_drop_on_cluster(cluster): def check_collections_empty(): for name, node in list(cluster.instances.items()): - assert "0" == node.query(q_count_collections).strip(), f"{SYSTEM_TABLE} is not empty on {name}" + assert ( + "0" == node.query(q_count_collections).strip() + ), f"{SYSTEM_TABLE} is not empty on {name}" - foobar_final_state = { - "name": "foobar", - "collection": {"a": "3"} - } + foobar_final_state = {"name": "foobar", "collection": {"a": "3"}} collection_present_on_first_node_final_state = { "name": "collection_present_on_first_node", - "collection": {"a": "1", "s": CHECK_STRING_VALUE, "x": "0", "y": "-1"} + "collection": {"a": "1", "s": CHECK_STRING_VALUE, "x": "0", "y": "-1"}, } expected_state = { NODE01: [foobar_final_state, collection_present_on_first_node_final_state], NODE02: [foobar_final_state], - NODE03: [foobar_final_state] + NODE03: [foobar_final_state], } q_get_collections = f"select * from {SYSTEM_TABLE} order by name desc format JSON" @@ -99,24 +98,34 @@ def test_create_alter_drop_on_cluster(cluster): def check_state(): for name, node in list(cluster.instances.items()): result = loads(node.query(q_get_collections))["data"] - logging.debug('%s ?= %s', dumps(result), dumps(expected_state[name])) - assert expected_state[name] == result, f"invalid {SYSTEM_TABLE} content on {name}: {result}" + logging.debug("%s ?= %s", dumps(result), dumps(expected_state[name])) + assert ( + expected_state[name] == result + ), f"invalid {SYSTEM_TABLE} content on {name}: {result}" check_collections_empty() # create executed on the first node node = cluster.instances[NODE01] node.query(f"{STMT_CREATE} foobar AS a=1, b=2") - node.query(f"{STMT_CREATE} IF NOT EXISTS foobar ON CLUSTER 'cluster' AS a=1, b=2, c=3") - node.query(f"{STMT_CREATE} collection_present_on_first_node AS a=1, b=2, s='{CHECK_STRING_VALUE}', x=0, y=-1") + node.query( + f"{STMT_CREATE} IF NOT EXISTS foobar ON CLUSTER 'cluster' AS a=1, b=2, c=3" + ) + node.query( + f"{STMT_CREATE} collection_present_on_first_node AS a=1, b=2, s='{CHECK_STRING_VALUE}', x=0, y=-1" + ) # alter executed on the second node node = cluster.instances[NODE02] node.query(f"{STMT_ALTER} foobar ON CLUSTER 'cluster' SET a=2, c=3") node.query(f"{STMT_ALTER} foobar ON CLUSTER 'cluster' DELETE b") node.query(f"{STMT_ALTER} foobar ON CLUSTER 'cluster' SET a=3 DELETE c") - node.query(f"{STMT_ALTER} IF EXISTS collection_absent_ewerywhere ON CLUSTER 'cluster' DELETE b") - node.query(f"{STMT_ALTER} IF EXISTS collection_present_on_first_node ON CLUSTER 'cluster' DELETE b") + node.query( + f"{STMT_ALTER} IF EXISTS collection_absent_ewerywhere ON CLUSTER 'cluster' DELETE b" + ) + node.query( + f"{STMT_ALTER} IF EXISTS collection_present_on_first_node ON CLUSTER 'cluster' DELETE b" + ) check_state() for node in list(cluster.instances.values()): @@ -126,8 +135,12 @@ def test_create_alter_drop_on_cluster(cluster): # drop executed on the third node node = cluster.instances[NODE03] node.query(f"{STMT_DROP} foobar ON CLUSTER 'cluster'") - node.query(f"{STMT_DROP} IF EXISTS collection_absent_ewerywhere ON CLUSTER 'cluster'") - node.query(f"{STMT_DROP} IF EXISTS collection_present_on_first_node ON CLUSTER 'cluster'") + node.query( + f"{STMT_DROP} IF EXISTS collection_absent_ewerywhere ON CLUSTER 'cluster'" + ) + node.query( + f"{STMT_DROP} IF EXISTS collection_present_on_first_node ON CLUSTER 'cluster'" + ) check_collections_empty() for node in list(cluster.instances.values()): From 41ffc097711b9ce3c3a26ba0fdc38315e3a39363 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 09:18:06 +0200 Subject: [PATCH 1326/2047] Upload build statistics to the CI database --- tests/ci/build_check.py | 21 ++++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 35b98a7c3bb..9e44af5296c 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -35,6 +35,11 @@ from version_helper import ( get_version_from_repo, update_version_local, ) +from clickhouse_helper import ( + ClickHouseHelper, + prepare_tests_results_for_clickhouse, +) +from stopwatch import Stopwatch IMAGE_NAME = "clickhouse/binary-builder" BUILD_LOG_NAME = "build_log.log" @@ -268,6 +273,7 @@ def mark_failed_reports_pending(build_name: str, pr_info: PRInfo) -> None: def main(): logging.basicConfig(level=logging.INFO) + stopwatch = Stopwatch() build_name = sys.argv[1] build_config = CI_CONFIG["build_config"][build_name] @@ -394,7 +400,20 @@ def main(): ) upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path) - # Fail build job if not successeded + + ch_helper = ClickHouseHelper() + prepared_events = prepare_tests_results_for_clickhouse( + pr_info, + [], + "success" if success else "failure", + stopwatch.duration_seconds, + stopwatch.start_time_str, + log_url, + f"Build ({build_name})", + ) + ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) + + # Fail the build job if it didn't succeed if not success: sys.exit(1) From 448756d14872cc89cc08aedf80e214941d1fdd4e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 09:31:21 +0200 Subject: [PATCH 1327/2047] Add machine_type information to the CI database --- tests/ci/clickhouse_helper.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 9410b37d69f..eca5f6f399d 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -132,6 +132,21 @@ class ClickHouseHelper: return result +# Obtain the machine type from IMDS: +def get_instance_type(): + url = "http://169.254.169.254/latest/meta-data/instance-type" + for i in range(5): + try: + response = requests.get(url, timeout=1) + if response.status_code == 200: + return response.text + except Exception as e: + error = f"Received exception while sending data to {url} on {i} attempt: {e}" + logging.warning(error) + continue + return '' + + def prepare_tests_results_for_clickhouse( pr_info: PRInfo, test_results: TestResults, @@ -168,6 +183,7 @@ def prepare_tests_results_for_clickhouse( head_ref=head_ref, head_repo=head_repo, task_url=pr_info.task_url, + instance_type=get_instance_type(), ) # Always publish a total record for all checks. For checks with individual From 358773c620ad091458c63de1e37307a053f044c9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 30 Jul 2023 07:45:27 +0000 Subject: [PATCH 1328/2047] Automatic style fix --- tests/ci/clickhouse_helper.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index eca5f6f399d..7f98f6e52d2 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -141,10 +141,12 @@ def get_instance_type(): if response.status_code == 200: return response.text except Exception as e: - error = f"Received exception while sending data to {url} on {i} attempt: {e}" + error = ( + f"Received exception while sending data to {url} on {i} attempt: {e}" + ) logging.warning(error) continue - return '' + return "" def prepare_tests_results_for_clickhouse( From bc1f80dd82d2992aa108f323af4349bd2f4c9968 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 10:25:44 +0200 Subject: [PATCH 1329/2047] Remove Coverity (part 2) --- docker/packager/binary/build.sh | 17 ----------------- docker/packager/packager | 7 +------ tests/ci/ci_config.py | 11 ----------- 3 files changed, 1 insertion(+), 34 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index c0803c74147..ce3f729fe33 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -80,16 +80,6 @@ else cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. fi -if [ "coverity" == "$COMBINED_OUTPUT" ] -then - mkdir -p /workdir/cov-analysis - - wget --post-data "token=$COVERITY_TOKEN&project=ClickHouse%2FClickHouse" -qO- https://scan.coverity.com/download/linux64 | tar xz -C /workdir/cov-analysis --strip-components 1 - export PATH=$PATH:/workdir/cov-analysis/bin - cov-configure --config ./coverity.config --template --comptype clangcc --compiler "$CC" - SCAN_WRAPPER="cov-build --config ./coverity.config --dir cov-int" -fi - # No quotes because I want it to expand to nothing if empty. # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. $SCAN_WRAPPER ninja $NINJA_FLAGS $BUILD_TARGET @@ -175,13 +165,6 @@ then mv "$COMBINED_OUTPUT.tar.zst" /output fi -if [ "coverity" == "$COMBINED_OUTPUT" ] -then - # Coverity does not understand ZSTD. - tar -cvz -f "coverity-scan.tar.gz" cov-int - mv "coverity-scan.tar.gz" /output -fi - ccache_status ccache --evict-older-than 1d diff --git a/docker/packager/packager b/docker/packager/packager index e12bd55dde3..399e12e6680 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -253,11 +253,6 @@ def parse_env_variables( cmake_flags.append(f"-DCMAKE_C_COMPILER={cc}") cmake_flags.append(f"-DCMAKE_CXX_COMPILER={cxx}") - # Create combined output archive for performance tests. - if package_type == "coverity": - result.append("COMBINED_OUTPUT=coverity") - result.append('COVERITY_TOKEN="$COVERITY_TOKEN"') - if sanitizer: result.append(f"SANITIZER={sanitizer}") if build_type: @@ -356,7 +351,7 @@ def parse_args() -> argparse.Namespace: ) parser.add_argument( "--package-type", - choices=["deb", "binary", "coverity"], + choices=["deb", "binary"], required=True, ) parser.add_argument( diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index db9a7f926be..bea654ca76f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -19,16 +19,6 @@ CI_CONFIG = { "with_coverage": False, "comment": "", }, - "coverity": { - "compiler": "clang-16", - "build_type": "", - "sanitizer": "", - "package_type": "coverity", - "tidy": "disable", - "with_coverage": False, - "official": False, - "comment": "A special build for coverity", - }, "package_aarch64": { "compiler": "clang-16-aarch64", "build_type": "", @@ -187,7 +177,6 @@ CI_CONFIG = { "builds_report_config": { "ClickHouse build check": [ "package_release", - "coverity", "package_aarch64", "package_asan", "package_ubsan", From 6bff19bd824bb38d026b12b000a76e7b47c098f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 10:27:33 +0200 Subject: [PATCH 1330/2047] Remove Coverity (part 2) --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index ce3f729fe33..640654e5445 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -82,7 +82,7 @@ fi # No quotes because I want it to expand to nothing if empty. # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. -$SCAN_WRAPPER ninja $NINJA_FLAGS $BUILD_TARGET +ninja $NINJA_FLAGS $BUILD_TARGET ls -la ./programs From 6e99e5c4537c087abb735fa9312b238ad7389dda Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 30 Jul 2023 13:27:01 +0200 Subject: [PATCH 1331/2047] Fix another flaky test --- .../integration/test_storage_s3_queue/test.py | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index f06772df9be..77e32e2922c 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -797,7 +797,17 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): files_to_generate, prefix, started_cluster, bucket, row_num=1 ) - time.sleep((files_to_generate // poll_size) * 2) + def get_count(node, table_name): + return int(run_query(node, f"SELECT count() FROM {table_name}")) + + for _ in range(100): + if ( + get_count(instance, "test.s3_queue_persistent") + + get_count(instance_2, "test.s3_queue_persistent") + ) == files_to_generate: + break + time.sleep(1) + get_query = f"SELECT * FROM test.s3_queue_persistent" res1 = [ list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() @@ -810,10 +820,16 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): assert len(res1) > 0 assert len(res2) > 0 - # Checking that all files were processed only once assert len(res1) + len(res2) == files_to_generate assert {tuple(v) for v in res1 + res2} == set([tuple(i) for i in total_values]) + # Checking that all files were processed only once + time.sleep(10) + assert ( + get_count(instance, "test.s3_queue_persistent") + + get_count(instance_2, "test.s3_queue_persistent") + ) == files_to_generate + def test_max_set_size(started_cluster): files_to_generate = 10 From 08ef4d015c6b42061f6de467279f0f1207913435 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 30 Jul 2023 13:36:52 +0200 Subject: [PATCH 1332/2047] Minor change --- docs/en/operations/system-tables/events.md | 2 +- docs/en/operations/system-tables/metrics.md | 2 +- src/Storages/System/StorageSystemEvents.h | 1 + src/Storages/System/StorageSystemMetrics.h | 1 + 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/system-tables/events.md b/docs/en/operations/system-tables/events.md index b53628a8384..26f5cb2031b 100644 --- a/docs/en/operations/system-tables/events.md +++ b/docs/en/operations/system-tables/events.md @@ -10,7 +10,7 @@ Columns: - `event` ([String](../../sql-reference/data-types/string.md)) — Event name. - `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of events occurred. - `description` ([String](../../sql-reference/data-types/string.md)) — Event description. -- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for event. +- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for `event`. **Example** diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index ea36411cf3d..6771e8d7aad 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -10,7 +10,7 @@ Columns: - `metric` ([String](../../sql-reference/data-types/string.md)) — Metric name. - `value` ([Int64](../../sql-reference/data-types/int-uint.md)) — Metric value. - `description` ([String](../../sql-reference/data-types/string.md)) — Metric description. -- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for metric. +- `name` ([String](../../sql-reference/data-types/string.md)) — Alias for `metric`. The list of supported metrics you can find in the [src/Common/CurrentMetrics.cpp](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/CurrentMetrics.cpp) source file of ClickHouse. diff --git a/src/Storages/System/StorageSystemEvents.h b/src/Storages/System/StorageSystemEvents.h index ed80a441529..b2e4bac072a 100644 --- a/src/Storages/System/StorageSystemEvents.h +++ b/src/Storages/System/StorageSystemEvents.h @@ -18,6 +18,7 @@ public: static NamesAndTypesList getNamesAndTypes(); static NamesAndAliases getNamesAndAliases(); + protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/src/Storages/System/StorageSystemMetrics.h b/src/Storages/System/StorageSystemMetrics.h index bffc7e6f1fc..e3e2c07014f 100644 --- a/src/Storages/System/StorageSystemMetrics.h +++ b/src/Storages/System/StorageSystemMetrics.h @@ -19,6 +19,7 @@ public: static NamesAndTypesList getNamesAndTypes(); static NamesAndAliases getNamesAndAliases(); + protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; From 6ae6268ad0075b1a99e619df599adc58dbcba848 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 30 Jul 2023 13:39:54 +0200 Subject: [PATCH 1333/2047] Revert "Merge pull request #51917 from ClickHouse/revert-system-drop-filesystem-cache-by-key" This reverts commit 520195f7ef5aeaf84f445a85460c25db0988009e, reversing changes made to e36a638014542348285eeda1a6de650782c0e7ab. --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 13 ++-- src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 2 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 + src/Interpreters/Cache/FileCache.cpp | 34 +++++----- src/Interpreters/Cache/FileCache.h | 12 +++- src/Interpreters/Cache/FileCacheKey.cpp | 5 ++ src/Interpreters/Cache/FileCacheKey.h | 2 + src/Interpreters/Cache/Metadata.cpp | 26 ++++++- src/Interpreters/Cache/Metadata.h | 8 ++- src/Interpreters/FilesystemCacheLog.cpp | 4 ++ src/Interpreters/FilesystemCacheLog.h | 13 +--- src/Interpreters/InterpreterSystemQuery.cpp | 13 +++- src/Parsers/ASTSystemQuery.cpp | 8 +++ src/Parsers/ASTSystemQuery.h | 2 + src/Parsers/ParserSystemQuery.cpp | 8 +++ ...2808_filesystem_cache_drop_query.reference | 4 ++ .../02808_filesystem_cache_drop_query.sh | 67 +++++++++++++++++++ 17 files changed, 183 insertions(+), 40 deletions(-) create mode 100644 tests/queries/0_stateless/02808_filesystem_cache_drop_query.reference create mode 100755 tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 6f08c6ba912..7be6129a287 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -74,19 +74,22 @@ CachedOnDiskReadBufferFromFile::CachedOnDiskReadBufferFromFile( } void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog( - const FileSegment::Range & file_segment_range, CachedOnDiskReadBufferFromFile::ReadType type) + const FileSegment & file_segment, CachedOnDiskReadBufferFromFile::ReadType type) { if (!cache_log) return; + const auto range = file_segment.range(); FilesystemCacheLogElement elem { .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), .query_id = query_id, .source_file_path = source_file_path, - .file_segment_range = { file_segment_range.left, file_segment_range.right }, + .file_segment_range = { range.left, range.right }, .requested_range = { first_offset, read_until_position }, - .file_segment_size = file_segment_range.size(), + .file_segment_key = file_segment.key().toString(), + .file_segment_offset = file_segment.offset(), + .file_segment_size = range.size(), .read_from_cache_attempted = true, .read_buffer_id = current_buffer_id, .profile_counters = std::make_shared( @@ -495,7 +498,7 @@ bool CachedOnDiskReadBufferFromFile::completeFileSegmentAndGetNext() auto completed_range = current_file_segment->range(); if (cache_log) - appendFilesystemCacheLog(completed_range, read_type); + appendFilesystemCacheLog(*current_file_segment, read_type); chassert(file_offset_of_buffer_end > completed_range.right); @@ -518,7 +521,7 @@ CachedOnDiskReadBufferFromFile::~CachedOnDiskReadBufferFromFile() { if (cache_log && file_segments && !file_segments->empty()) { - appendFilesystemCacheLog(file_segments->front().range(), read_type); + appendFilesystemCacheLog(file_segments->front(), read_type); } } diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index b4e7701de75..36cf8a54183 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -90,7 +90,7 @@ private: bool completeFileSegmentAndGetNext(); - void appendFilesystemCacheLog(const FileSegment::Range & file_segment_range, ReadType read_type); + void appendFilesystemCacheLog(const FileSegment & file_segment, ReadType read_type); bool writeCache(char * data, size_t size, size_t offset, FileSegment & file_segment); diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index ada1b2cbecd..cc75f18bc0d 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -109,6 +109,8 @@ void ReadBufferFromRemoteFSGather::appendUncachedReadInfo() .source_file_path = current_object.remote_path, .file_segment_range = { 0, current_object.bytes_size }, .cache_type = FilesystemCacheLogElement::CacheType::READ_FROM_FS_BYPASSING_CACHE, + .file_segment_key = {}, + .file_segment_offset = {}, .file_segment_size = current_object.bytes_size, .read_from_cache_attempted = false, }; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 42cc7b80a66..9507622d386 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -806,6 +806,13 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) return true; } +void FileCache::removeKey(const Key & key) +{ + assertInitialized(); + auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW); + locked_key->removeAll(); +} + void FileCache::removeKeyIfExists(const Key & key) { assertInitialized(); @@ -818,7 +825,14 @@ void FileCache::removeKeyIfExists(const Key & key) /// But if we have multiple replicated zero-copy tables on the same server /// it became possible to start removing something from cache when it is used /// by other "zero-copy" tables. That is why it's not an error. - locked_key->removeAllReleasable(); + locked_key->removeAll(/* if_releasable */true); +} + +void FileCache::removeFileSegment(const Key & key, size_t offset) +{ + assertInitialized(); + auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW); + locked_key->removeFileSegment(offset); } void FileCache::removePathIfExists(const String & path) @@ -830,22 +844,12 @@ void FileCache::removeAllReleasable() { assertInitialized(); - auto lock = lockCache(); - - main_priority->iterate([&](LockedKey & locked_key, const FileSegmentMetadataPtr & segment_metadata) - { - if (segment_metadata->releasable()) - { - auto file_segment = segment_metadata->file_segment; - locked_key.removeFileSegment(file_segment->offset(), file_segment->lock()); - return PriorityIterationResult::REMOVE_AND_CONTINUE; - } - return PriorityIterationResult::CONTINUE; - }, lock); + metadata.iterate([](LockedKey & locked_key) { locked_key.removeAll(/* if_releasable */true); }); if (stash) { /// Remove all access information. + auto lock = lockCache(); stash->records.clear(); stash->queue->removeAll(lock); } @@ -914,7 +918,7 @@ void FileCache::loadMetadata() continue; } - const auto key = Key(unhexUInt(key_directory.filename().string().data())); + const auto key = Key::fromKeyString(key_directory.filename().string()); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, /* is_initial_load */true); for (fs::directory_iterator offset_it{key_directory}; offset_it != fs::directory_iterator(); ++offset_it) @@ -1069,7 +1073,7 @@ FileSegmentsHolderPtr FileCache::getSnapshot() FileSegmentsHolderPtr FileCache::getSnapshot(const Key & key) { FileSegments file_segments; - auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW); + auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW_LOGICAL); for (const auto & [_, file_segment_metadata] : *locked_key->getKeyMetadata()) file_segments.push_back(FileSegment::getSnapshot(file_segment_metadata->file_segment)); return std::make_unique(std::move(file_segments)); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 0e3b17baa2f..2e6a5094758 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -83,13 +83,19 @@ public: FileSegmentsHolderPtr set(const Key & key, size_t offset, size_t size, const CreateFileSegmentSettings & settings); - /// Remove files by `key`. Removes files which might be used at the moment. + /// Remove file segment by `key` and `offset`. Throws if file segment does not exist. + void removeFileSegment(const Key & key, size_t offset); + + /// Remove files by `key`. Throws if key does not exist. + void removeKey(const Key & key); + + /// Remove files by `key`. void removeKeyIfExists(const Key & key); - /// Removes files by `path`. Removes files which might be used at the moment. + /// Removes files by `path`. void removePathIfExists(const String & path); - /// Remove files by `key`. Will not remove files which are used at the moment. + /// Remove files by `key`. void removeAllReleasable(); std::vector tryGetCachePaths(const Key & key); diff --git a/src/Interpreters/Cache/FileCacheKey.cpp b/src/Interpreters/Cache/FileCacheKey.cpp index f97cdc058aa..772fcd600bf 100644 --- a/src/Interpreters/Cache/FileCacheKey.cpp +++ b/src/Interpreters/Cache/FileCacheKey.cpp @@ -28,4 +28,9 @@ FileCacheKey FileCacheKey::random() return FileCacheKey(UUIDHelpers::generateV4().toUnderType()); } +FileCacheKey FileCacheKey::fromKeyString(const std::string & key_str) +{ + return FileCacheKey(unhexUInt(key_str.data())); +} + } diff --git a/src/Interpreters/Cache/FileCacheKey.h b/src/Interpreters/Cache/FileCacheKey.h index bab8359732c..e788cd5e7cd 100644 --- a/src/Interpreters/Cache/FileCacheKey.h +++ b/src/Interpreters/Cache/FileCacheKey.h @@ -21,6 +21,8 @@ struct FileCacheKey static FileCacheKey random(); bool operator==(const FileCacheKey & other) const { return key == other.key; } + + static FileCacheKey fromKeyString(const std::string & key_str); }; using FileCacheKeyAndOffset = std::pair; diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 783c71448fc..ae9c87f5523 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -25,6 +25,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } FileSegmentMetadata::FileSegmentMetadata(FileSegmentPtr && file_segment_) @@ -191,6 +192,8 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata( if (it == end()) { if (key_not_found_policy == KeyNotFoundPolicy::THROW) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}` in cache", key); + else if (key_not_found_policy == KeyNotFoundPolicy::THROW_LOGICAL) throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}` in cache", key); else if (key_not_found_policy == KeyNotFoundPolicy::RETURN_NULL) return nullptr; @@ -215,6 +218,8 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata( return locked_metadata; if (key_not_found_policy == KeyNotFoundPolicy::THROW) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}` in cache", key); + else if (key_not_found_policy == KeyNotFoundPolicy::THROW_LOGICAL) throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}` in cache", key); if (key_not_found_policy == KeyNotFoundPolicy::RETURN_NULL) @@ -561,11 +566,11 @@ bool LockedKey::isLastOwnerOfFileSegment(size_t offset) const return file_segment_metadata->file_segment.use_count() == 2; } -void LockedKey::removeAllReleasable() +void LockedKey::removeAll(bool if_releasable) { for (auto it = key_metadata->begin(); it != key_metadata->end();) { - if (!it->second->releasable()) + if (if_releasable && !it->second->releasable()) { ++it; continue; @@ -586,17 +591,32 @@ void LockedKey::removeAllReleasable() } } +KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset) +{ + auto it = key_metadata->find(offset); + if (it == key_metadata->end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no offset {}", offset); + + auto file_segment = it->second->file_segment; + return removeFileSegmentImpl(it, file_segment->lock()); +} + KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegmentGuard::Lock & segment_lock) { auto it = key_metadata->find(offset); if (it == key_metadata->end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no offset {}", offset); + return removeFileSegmentImpl(it, segment_lock); +} + +KeyMetadata::iterator LockedKey::removeFileSegmentImpl(KeyMetadata::iterator it, const FileSegmentGuard::Lock & segment_lock) +{ auto file_segment = it->second->file_segment; LOG_DEBUG( key_metadata->log, "Remove from cache. Key: {}, offset: {}, size: {}", - getKey(), offset, file_segment->reserved_size); + getKey(), file_segment->offset(), file_segment->reserved_size); chassert(file_segment->assertCorrectnessUnlocked(segment_lock)); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 503c19f4150..42d74338e12 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -87,7 +87,7 @@ struct CacheMetadata : public std::unordered_map, { public: using Key = FileCacheKey; - using IterateCacheMetadataFunc = std::function; + using IterateCacheMetadataFunc = std::function; explicit CacheMetadata(const std::string & path_); @@ -106,6 +106,7 @@ public: enum class KeyNotFoundPolicy { THROW, + THROW_LOGICAL, CREATE_EMPTY, RETURN_NULL, }; @@ -169,9 +170,10 @@ struct LockedKey : private boost::noncopyable std::shared_ptr getKeyMetadata() const { return key_metadata; } std::shared_ptr getKeyMetadata() { return key_metadata; } - void removeAllReleasable(); + void removeAll(bool if_releasable = true); KeyMetadata::iterator removeFileSegment(size_t offset, const FileSegmentGuard::Lock &); + KeyMetadata::iterator removeFileSegment(size_t offset); void shrinkFileSegmentToDownloadedSize(size_t offset, const FileSegmentGuard::Lock &); @@ -188,6 +190,8 @@ struct LockedKey : private boost::noncopyable std::string toString() const; private: + KeyMetadata::iterator removeFileSegmentImpl(KeyMetadata::iterator it, const FileSegmentGuard::Lock &); + const std::shared_ptr key_metadata; KeyGuard::Lock lock; /// `lock` must be destructed before `key_metadata`. }; diff --git a/src/Interpreters/FilesystemCacheLog.cpp b/src/Interpreters/FilesystemCacheLog.cpp index 17f0fda71ec..b660db064d1 100644 --- a/src/Interpreters/FilesystemCacheLog.cpp +++ b/src/Interpreters/FilesystemCacheLog.cpp @@ -40,6 +40,8 @@ NamesAndTypesList FilesystemCacheLogElement::getNamesAndTypes() {"source_file_path", std::make_shared()}, {"file_segment_range", std::make_shared(types)}, {"total_requested_range", std::make_shared(types)}, + {"key", std::make_shared()}, + {"offset", std::make_shared()}, {"size", std::make_shared()}, {"read_type", std::make_shared()}, {"read_from_cache_attempted", std::make_shared()}, @@ -60,6 +62,8 @@ void FilesystemCacheLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(source_file_path); columns[i++]->insert(Tuple{file_segment_range.first, file_segment_range.second}); columns[i++]->insert(Tuple{requested_range.first, requested_range.second}); + columns[i++]->insert(file_segment_key); + columns[i++]->insert(file_segment_offset); columns[i++]->insert(file_segment_size); columns[i++]->insert(typeToString(cache_type)); columns[i++]->insert(read_from_cache_attempted); diff --git a/src/Interpreters/FilesystemCacheLog.h b/src/Interpreters/FilesystemCacheLog.h index 1b22d561c51..0d088a922e0 100644 --- a/src/Interpreters/FilesystemCacheLog.h +++ b/src/Interpreters/FilesystemCacheLog.h @@ -11,16 +11,7 @@ namespace DB { -/// -/// -------- Column --------- Type ------ -/// | event_date | DateTime | -/// | event_time | UInt64 | -/// | query_id | String | -/// | remote_file_path | String | -/// | segment_range | Tuple | -/// | read_type | String | -/// ------------------------------------- -/// + struct FilesystemCacheLogElement { enum class CacheType @@ -39,6 +30,8 @@ struct FilesystemCacheLogElement std::pair file_segment_range{}; std::pair requested_range{}; CacheType cache_type{}; + std::string file_segment_key; + size_t file_segment_offset; size_t file_segment_size; bool read_from_cache_attempted; String read_buffer_id; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 3207da9941a..b62840945c4 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -371,7 +371,18 @@ BlockIO InterpreterSystemQuery::execute() else { auto cache = FileCacheFactory::instance().getByName(query.filesystem_cache_name).cache; - cache->removeAllReleasable(); + if (query.key_to_drop.empty()) + { + cache->removeAllReleasable(); + } + else + { + auto key = FileCacheKey::fromKeyString(query.key_to_drop); + if (query.offset_to_drop.has_value()) + cache->removeFileSegment(key, query.offset_to_drop.value()); + else + cache->removeKey(key); + } } break; } diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 754eb825dcc..3385844cb4f 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -210,7 +210,15 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, else if (type == Type::DROP_FILESYSTEM_CACHE) { if (!filesystem_cache_name.empty()) + { settings.ostr << (settings.hilite ? hilite_none : "") << " " << filesystem_cache_name; + if (!key_to_drop.empty()) + { + settings.ostr << (settings.hilite ? hilite_none : "") << " KEY " << key_to_drop; + if (offset_to_drop.has_value()) + settings.ostr << (settings.hilite ? hilite_none : "") << " OFFSET " << offset_to_drop.value(); + } + } } else if (type == Type::UNFREEZE) { diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index ebaf357c0ab..f97f4465621 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -108,6 +108,8 @@ public: UInt64 seconds{}; String filesystem_cache_name; + std::string key_to_drop; + std::optional offset_to_drop; String backup_name; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 9aff0e8879e..3af0d62f769 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -405,7 +405,15 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & ParserLiteral path_parser; ASTPtr ast; if (path_parser.parse(pos, ast, expected)) + { res->filesystem_cache_name = ast->as()->value.safeGet(); + if (ParserKeyword{"KEY"}.ignore(pos, expected) && ParserIdentifier().parse(pos, ast, expected)) + { + res->key_to_drop = ast->as()->name(); + if (ParserKeyword{"OFFSET"}.ignore(pos, expected) && ParserLiteral().parse(pos, ast, expected)) + res->offset_to_drop = ast->as()->value.safeGet(); + } + } if (!parseQueryWithOnCluster(res, pos, expected)) return false; break; diff --git a/tests/queries/0_stateless/02808_filesystem_cache_drop_query.reference b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.reference new file mode 100644 index 00000000000..d80fc78e03d --- /dev/null +++ b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.reference @@ -0,0 +1,4 @@ +1 +0 +1 +0 diff --git a/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh new file mode 100755 index 00000000000..9d987d0ebf2 --- /dev/null +++ b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh @@ -0,0 +1,67 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings + +# set -x + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" +$CLICKHOUSE_CLIENT -nm --query """ +DROP TABLE IF EXISTS test; +CREATE TABLE test (a Int32, b String) +ENGINE = MergeTree() ORDER BY tuple() +SETTINGS disk = disk_$disk_name(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); + +INSERT INTO test SELECT 1, 'test'; +""" + +query_id=$RANDOM + +$CLICKHOUSE_CLIENT --query_id "$query_id" --query "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1" + +${CLICKHOUSE_CLIENT} -q " system flush logs" + +key=$($CLICKHOUSE_CLIENT -nm --query """ +SELECT key FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1; +""") + +offset=$($CLICKHOUSE_CLIENT -nm --query """ +SELECT offset FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1; +""") + +$CLICKHOUSE_CLIENT -nm --query """ +SELECT count() FROM system.filesystem_cache WHERE key = '$key' AND file_segment_range_begin = $offset; +""" + +$CLICKHOUSE_CLIENT -nm --query """ +SYSTEM DROP FILESYSTEM CACHE '$disk_name' KEY $key OFFSET $offset; +""" + +$CLICKHOUSE_CLIENT -nm --query """ +SELECT count() FROM system.filesystem_cache WHERE key = '$key' AND file_segment_range_begin = $offset; +""" + +query_id=$RANDOM$RANDOM + +$CLICKHOUSE_CLIENT --query_id "$query_id" --query "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1" + +${CLICKHOUSE_CLIENT} -q " system flush logs" + +key=$($CLICKHOUSE_CLIENT -nm --query """ +SELECT key FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1; +""") + +$CLICKHOUSE_CLIENT -nm --query """ +SELECT count() FROM system.filesystem_cache WHERE key = '$key'; +""" + +$CLICKHOUSE_CLIENT -nm --query """ +SYSTEM DROP FILESYSTEM CACHE '$disk_name' KEY $key +""" + +$CLICKHOUSE_CLIENT -nm --query """ +SELECT count() FROM system.filesystem_cache WHERE key = '$key'; +""" From 38f12f06646b5d65f988aa0a2e0d43f7b7b94161 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 30 Jul 2023 13:46:40 +0200 Subject: [PATCH 1334/2047] Revert some commenting --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 23 +++++++++++-------- src/Interpreters/Cache/LRUFileCachePriority.h | 11 +++++++-- 2 files changed, 23 insertions(+), 11 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index b71c8a7478a..6f08c6ba912 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -108,7 +108,7 @@ void CachedOnDiskReadBufferFromFile::appendFilesystemCacheLog( break; } - cache_log->add(elem); + cache_log->add(std::move(elem)); } void CachedOnDiskReadBufferFromFile::initialize(size_t offset, size_t size) @@ -259,6 +259,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegment & file_s } else { + LOG_TEST(log, "Bypassing cache because `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` option is used"); read_type = ReadType::REMOTE_FS_READ_BYPASS_CACHE; return getRemoteReadBuffer(file_segment, read_type); } @@ -344,6 +345,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegment & file_s /// ^ /// file_offset_of_buffer_end + LOG_TEST(log, "Predownload. File segment info: {}", file_segment.getInfoForLog()); chassert(file_offset_of_buffer_end > current_write_offset); bytes_to_predownload = file_offset_of_buffer_end - current_write_offset; chassert(bytes_to_predownload < file_segment.range().size()); @@ -391,13 +393,13 @@ CachedOnDiskReadBufferFromFile::getImplementationBuffer(FileSegment & file_segme watch.stop(); - // LOG_TEST( - // log, - // "Current read type: {}, read offset: {}, impl read range: {}, file segment: {}", - // toString(read_type), - // file_offset_of_buffer_end, - // read_buffer_for_file_segment->getFileOffsetOfBufferEnd(), - // file_segment.getInfoForLog()); + LOG_TEST( + log, + "Current read type: {}, read offset: {}, impl read range: {}, file segment: {}", + toString(read_type), + file_offset_of_buffer_end, + read_buffer_for_file_segment->getFileOffsetOfBufferEnd(), + file_segment.getInfoForLog()); current_file_segment_counters.increment( ProfileEvents::FileSegmentWaitReadBufferMicroseconds, watch.elapsedMicroseconds()); @@ -540,6 +542,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) /// downloaded because it intersects with the range he needs. /// But then first downloader fails and second must continue. In this case we need to /// download from offset a'' < a', but return buffer from offset a'. + LOG_TEST(log, "Bytes to predownload: {}, caller_id: {}", bytes_to_predownload, FileSegment::getCallerId()); /// chassert(implementation_buffer->getFileOffsetOfBufferEnd() == file_segment.getCurrentWriteOffset(false)); chassert(static_cast(implementation_buffer->getPosition()) == file_segment.getCurrentWriteOffset(false)); @@ -606,7 +609,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) bool continue_predownload = file_segment.reserve(current_predownload_size); if (continue_predownload) { - // LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size); + LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size); chassert(file_segment.getCurrentWriteOffset(false) == static_cast(implementation_buffer->getPosition())); @@ -700,6 +703,8 @@ bool CachedOnDiskReadBufferFromFile::updateImplementationBufferIfNeeded() auto current_write_offset = file_segment.getCurrentWriteOffset(true); bool cached_part_is_finished = current_write_offset == file_offset_of_buffer_end; + LOG_TEST(log, "Current write offset: {}, file offset of buffer end: {}", current_write_offset, file_offset_of_buffer_end); + if (cached_part_is_finished) { /// TODO: makes sense to reuse local file reader if we return here with CACHED read type again? diff --git a/src/Interpreters/Cache/LRUFileCachePriority.h b/src/Interpreters/Cache/LRUFileCachePriority.h index 33e5c9677aa..e041e59a91a 100644 --- a/src/Interpreters/Cache/LRUFileCachePriority.h +++ b/src/Interpreters/Cache/LRUFileCachePriority.h @@ -5,6 +5,11 @@ #include #include +namespace CurrentMetrics +{ + extern const Metric FilesystemCacheSizeLimit; +} + namespace DB { @@ -18,7 +23,10 @@ private: using LRUQueueIterator = typename LRUQueue::iterator; public: - LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) {} + LRUFileCachePriority(size_t max_size_, size_t max_elements_) : IFileCachePriority(max_size_, max_elements_) + { + CurrentMetrics::set(CurrentMetrics::FilesystemCacheSizeLimit, max_size_); + } size_t getSize(const CacheGuard::Lock &) const override { return current_size; } @@ -37,7 +45,6 @@ private: void updateSize(int64_t size); LRUQueue queue; - Poco::Logger * log = &Poco::Logger::get("LRUFileCachePriority"); std::atomic current_size = 0; From 9291e4993a155a0f73705263c1c229b086511eb6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 30 Jul 2023 13:49:37 +0200 Subject: [PATCH 1335/2047] Update .reference --- .../02241_filesystem_cache_on_write_operations.reference | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference index b3b7d12d219..c6099b96d31 100644 --- a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference +++ b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.reference @@ -24,8 +24,8 @@ state: DOWNLOADED 8 2014 8 2014 24 84045 -35 168815 -44 252113 +32 167243 +41 250541 5010500 18816 Using storage policy: local_cache @@ -54,7 +54,7 @@ state: DOWNLOADED 8 2014 8 2014 24 84045 -35 168815 -44 252113 +32 167243 +41 250541 5010500 18816 From 0cf93e71140fe30710859409bf839c1a4fc0e979 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 30 Jul 2023 13:51:10 +0200 Subject: [PATCH 1336/2047] Reduce redundant logging --- src/Interpreters/Cache/FileSegment.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 050097ddab8..6afccafe94a 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -258,7 +258,10 @@ void FileSegment::resetDownloader() void FileSegment::resetDownloaderUnlocked(const FileSegmentGuard::Lock &) { - // LOG_TEST(log, "Resetting downloader from {}", downloader_id); + if (downloader_id.empty()) + return; + + LOG_TEST(log, "Resetting downloader from {}", downloader_id); downloader_id.clear(); } From aba291ca688fa58d82efc329e7be6d1c2e638f4a Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 30 Jul 2023 14:01:34 +0200 Subject: [PATCH 1337/2047] Revert "Revert "Add an ability to specify allocations size for sampling memory profiler"" --- programs/server/Server.cpp | 21 ++++++++--- src/Common/MemoryTracker.cpp | 10 ++++- src/Common/MemoryTracker.h | 18 +++++++++ src/Core/ServerSettings.h | 8 +++- src/Core/Settings.h | 4 +- src/Interpreters/ProcessList.cpp | 3 ++ src/Interpreters/ThreadStatusExt.cpp | 2 + .../__init__.py | 1 + .../configs/max_untracked_memory.xml | 7 ++++ .../configs/memory_profiler.xml | 5 +++ .../test.py | 37 +++++++++++++++++++ ...r_sample_min_max_allocation_size.reference | 1 + ...profiler_sample_min_max_allocation_size.sh | 18 +++++++++ 13 files changed, 124 insertions(+), 11 deletions(-) create mode 100644 tests/integration/test_memory_profiler_min_max_borders/__init__.py create mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml create mode 100644 tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml create mode 100644 tests/integration/test_memory_profiler_min_max_borders/test.py create mode 100644 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference create mode 100755 tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 405ebf7fb2f..85ee48ea075 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1691,17 +1691,26 @@ try global_context->initializeTraceCollector(); /// Set up server-wide memory profiler (for total memory tracker). - UInt64 total_memory_profiler_step = config().getUInt64("total_memory_profiler_step", 0); - if (total_memory_profiler_step) + if (server_settings.total_memory_profiler_step) { - total_memory_tracker.setProfilerStep(total_memory_profiler_step); + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); } - double total_memory_tracker_sample_probability = config().getDouble("total_memory_tracker_sample_probability", 0); - if (total_memory_tracker_sample_probability > 0.0) + if (server_settings.total_memory_tracker_sample_probability > 0.0) { - total_memory_tracker.setSampleProbability(total_memory_tracker_sample_probability); + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); } + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + { + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + } + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + { + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + } #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 81cac2617c5..52cae0768dc 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -229,7 +229,7 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = size}); @@ -413,7 +413,7 @@ void MemoryTracker::free(Int64 size) } std::bernoulli_distribution sample(sample_probability); - if (unlikely(sample_probability > 0.0 && sample(thread_local_rng))) + if (unlikely(sample_probability > 0.0 && isSizeOkForSampling(size) && sample(thread_local_rng))) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); DB::TraceSender::send(DB::TraceType::MemorySample, StackTrace(), {.size = -size}); @@ -534,6 +534,12 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) ; } +bool MemoryTracker::isSizeOkForSampling(UInt64 size) const +{ + /// We can avoid comparison min_allocation_size_bytes with zero, because we cannot have 0 bytes allocation/deallocation + return ((max_allocation_size_bytes == 0 || size <= max_allocation_size_bytes) && size >= min_allocation_size_bytes); +} + bool canEnqueueBackgroundTask() { auto limit = background_memory_tracker.getSoftLimit(); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 3ea1ea8702c..6f576e53e7a 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -67,6 +67,12 @@ private: /// To randomly sample allocations and deallocations in trace_log. double sample_probability = 0; + /// Randomly sample allocations only larger or equal to this size + UInt64 min_allocation_size_bytes = 0; + + /// Randomly sample allocations only smaller or equal to this size + UInt64 max_allocation_size_bytes = 0; + /// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy). /// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker. std::atomic parent {}; @@ -88,6 +94,8 @@ private: void setOrRaiseProfilerLimit(Int64 value); + bool isSizeOkForSampling(UInt64 size) const; + /// allocImpl(...) and free(...) should not be used directly friend struct CurrentMemoryTracker; void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr); @@ -166,6 +174,16 @@ public: sample_probability = value; } + void setSampleMinAllocationSize(UInt64 value) + { + min_allocation_size_bytes = value; + } + + void setSampleMaxAllocationSize(UInt64 value) + { + max_allocation_size_bytes = value; + } + void setProfilerStep(Int64 value) { profiler_step = value; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index ca27cbdbf19..7678e8c3f24 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -83,8 +83,12 @@ namespace DB M(UInt64, background_schedule_pool_size, 128, "The maximum number of threads that will be used for constantly executing some lightweight periodic operations.", 0) \ M(UInt64, background_message_broker_schedule_pool_size, 16, "The maximum number of threads that will be used for executing background operations for message streaming.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "The maximum number of threads that will be used for executing distributed sends.", 0) \ - M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) - + M(Bool, display_secrets_in_show_and_select, false, "Allow showing secrets in SHOW and SELECT queries via a format setting and a grant", 0) \ + \ + M(UInt64, total_memory_profiler_step, 0, "Whenever server memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down server.", 0) \ + M(Double, total_memory_tracker_sample_probability, 0, "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(UInt64, total_memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(UInt64, total_memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `total_memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) DECLARE_SETTINGS_TRAITS(ServerSettingsTraits, SERVER_SETTINGS) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c69d132ea25..26e0808aaa3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -427,7 +427,9 @@ class IColumn; M(UInt64, memory_overcommit_ratio_denominator_for_user, 1_GiB, "It represents soft memory limit on the global level. This value is used to compute query overcommit ratio.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ - M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation (can be changed with `memory_profiler_sample_min_allocation_size` and `memory_profiler_sample_max_allocation_size`). Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ + M(UInt64, memory_profiler_sample_min_allocation_size, 0, "Collect random allocations of size greater or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ + M(UInt64, memory_profiler_sample_max_allocation_size, 0, "Collect random allocations of size less or equal than specified value with probability equal to `memory_profiler_sample_probability`. 0 means disabled. You may want to set 'max_untracked_memory' to 0 to make this threshold to work as expected.", 0) \ M(Bool, trace_profile_events, false, "Send to system.trace_log profile event and value of increment on each increment with 'ProfileEvent' trace_type", 0) \ \ M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \ diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 1503e396298..c299572ef41 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -223,7 +223,10 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q { /// Set up memory profiling thread_group->memory_tracker.setProfilerStep(settings.memory_profiler_step); + thread_group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); + thread_group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); + thread_group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); thread_group->performance_counters.setTraceProfileEvents(settings.trace_profile_events); } diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 7a6bc45c118..1b39bb6c0dc 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -83,6 +83,8 @@ ThreadGroupPtr ThreadGroup::createForBackgroundProcess(ContextPtr storage_contex const Settings & settings = storage_context->getSettingsRef(); group->memory_tracker.setProfilerStep(settings.memory_profiler_step); group->memory_tracker.setSampleProbability(settings.memory_profiler_sample_probability); + group->memory_tracker.setSampleMinAllocationSize(settings.memory_profiler_sample_min_allocation_size); + group->memory_tracker.setSampleMaxAllocationSize(settings.memory_profiler_sample_max_allocation_size); group->memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator); group->memory_tracker.setParent(&background_memory_tracker); if (settings.memory_tracker_fault_probability > 0.0) diff --git a/tests/integration/test_memory_profiler_min_max_borders/__init__.py b/tests/integration/test_memory_profiler_min_max_borders/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml new file mode 100644 index 00000000000..56fc5ed34ca --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/configs/max_untracked_memory.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml b/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml new file mode 100644 index 00000000000..5b3e17d145f --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/configs/memory_profiler.xml @@ -0,0 +1,5 @@ + + 1 + 4096 + 8192 + diff --git a/tests/integration/test_memory_profiler_min_max_borders/test.py b/tests/integration/test_memory_profiler_min_max_borders/test.py new file mode 100644 index 00000000000..6ab971fa9c4 --- /dev/null +++ b/tests/integration/test_memory_profiler_min_max_borders/test.py @@ -0,0 +1,37 @@ +from helpers.cluster import ClickHouseCluster +import pytest + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/memory_profiler.xml"], + user_configs=["configs/max_untracked_memory.xml"], +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_trace_boundaries_work(started_cluster): + node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null") + node.query("SYSTEM FLUSH LOGS") + + assert ( + node.query( + "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where trace_type = 'MemorySample'" + ) + == "1\n" + ) + assert ( + node.query( + "SELECT count() FROM system.trace_log where trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" + ) + == "0\n" + ) diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh new file mode 100755 index 00000000000..b1fbea26da7 --- /dev/null +++ b/tests/queries/0_stateless/02818_memory_profiler_sample_min_max_allocation_size.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash +# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-cpu-aarch64, no-random-settings +# requires TraceCollector, does not available under sanitizers and aarch64 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +query_id="${CLICKHOUSE_DATABASE}_min_max_allocation_size_$RANDOM$RANDOM" +${CLICKHOUSE_CLIENT} --query_id="$query_id" --memory_profiler_sample_min_allocation_size=4096 --memory_profiler_sample_max_allocation_size=8192 --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 --query "select randomPrintableASCII(number) from numbers(1000) FORMAT Null" + +${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" + +# at least something allocated +${CLICKHOUSE_CLIENT} --query "SELECT countDistinct(abs(size)) > 0 FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample'" + +# show wrong allocations +${CLICKHOUSE_CLIENT} --query "SELECT abs(size) FROM system.trace_log where query_id='$query_id' and trace_type = 'MemorySample' and (abs(size) > 8192 or abs(size) < 4096)" From 3e5767853d824034221b981ef7c3ad29f0e0ce77 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 30 Jul 2023 14:15:17 +0200 Subject: [PATCH 1338/2047] Disable test for sanitizers --- tests/integration/test_memory_profiler_min_max_borders/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_memory_profiler_min_max_borders/test.py b/tests/integration/test_memory_profiler_min_max_borders/test.py index 6ab971fa9c4..df9439bc2bb 100644 --- a/tests/integration/test_memory_profiler_min_max_borders/test.py +++ b/tests/integration/test_memory_profiler_min_max_borders/test.py @@ -20,6 +20,9 @@ def started_cluster(): def test_trace_boundaries_work(started_cluster): + if node.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + node.query("select randomPrintableASCII(number) from numbers(1000) FORMAT Null") node.query("SYSTEM FLUSH LOGS") From 0ff0a4e41f39479530c1ef81388dc058266201b5 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 30 Jul 2023 19:42:10 +0200 Subject: [PATCH 1339/2047] diverge parsing methods --- src/Functions/FunctionsConversion.h | 76 ++++++++++++++----- src/IO/ReadHelpers.h | 4 +- src/IO/readFloatText.cpp | 12 ++- src/IO/readFloatText.h | 16 +++- .../0_stateless/02813_float_parsing.reference | 1 + .../0_stateless/02813_float_parsing.sql | 10 +++ 6 files changed, 92 insertions(+), 27 deletions(-) create mode 100644 tests/queries/0_stateless/02813_float_parsing.reference create mode 100644 tests/queries/0_stateless/02813_float_parsing.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 00c4cfe7284..01ff47af18c 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1024,14 +1024,23 @@ inline void convertFromTime(DataTypeDateTime::FieldType & x, t /** Conversion of strings to numbers, dates, datetimes: through parsing. */ +template +void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +{ + if constexpr (std::is_floating_point_v) + readFloatText(x, rb); + else + parseType(x, rb, time_zone); +} + template -void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +void parseType(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { readText(x, rb); } template <> -inline void parseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline void parseType(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { DayNum tmp(0); readDateText(tmp, rb, *time_zone); @@ -1039,7 +1048,7 @@ inline void parseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb } template <> -inline void parseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline void parseType(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { ExtendedDayNum tmp(0); readDateText(tmp, rb, *time_zone); @@ -1049,7 +1058,7 @@ inline void parseImpl(DataTypeDate32::FieldType & x, ReadBuffer // NOTE: no need of extra overload of DateTime64, since readDateTimeText64 has different signature and that case is explicitly handled in the calling code. template <> -inline void parseImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline void parseType(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { time_t time = 0; readDateTimeText(time, rb, *time_zone); @@ -1057,7 +1066,7 @@ inline void parseImpl(DataTypeDateTime::FieldType & x, ReadBuf } template <> -inline void parseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline void parseType(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { UUID tmp; readUUIDText(tmp, rb); @@ -1065,7 +1074,7 @@ inline void parseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb } template <> -inline void parseImpl(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline void parseType(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { IPv4 tmp; readIPv4Text(tmp, rb); @@ -1073,24 +1082,30 @@ inline void parseImpl(DataTypeIPv4::FieldType & x, ReadBuffer & rb } template <> -inline void parseImpl(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline void parseType(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { IPv6 tmp; readIPv6Text(tmp, rb); x = tmp; } -template -bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +template +bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { if constexpr (std::is_floating_point_v) - return tryReadFloatText(x, rb); + return tryReadFloatText(x, rb); else /*if constexpr (is_integer_v)*/ - return tryReadIntText(x, rb); + return tryParseTypeImpl(x, rb, time_zone); +} + +template +inline bool tryParseTypeImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +{ + return tryReadIntText(x, rb); } template <> -inline bool tryParseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline bool tryParseTypeImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { DayNum tmp(0); if (!tryReadDateText(tmp, rb, *time_zone)) @@ -1100,7 +1115,7 @@ inline bool tryParseImpl(DataTypeDate::FieldType & x, ReadBuffer & } template <> -inline bool tryParseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline bool tryParseTypeImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { ExtendedDayNum tmp(0); if (!tryReadDateText(tmp, rb, *time_zone)) @@ -1110,7 +1125,7 @@ inline bool tryParseImpl(DataTypeDate32::FieldType & x, ReadBuff } template <> -inline bool tryParseImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline bool tryParseTypeImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { time_t tmp = 0; if (!tryReadDateTimeText(tmp, rb, *time_zone)) @@ -1120,7 +1135,7 @@ inline bool tryParseImpl(DataTypeDateTime::FieldType & x, Read } template <> -inline bool tryParseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline bool tryParseTypeImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { UUID tmp; if (!tryReadUUIDText(tmp, rb)) @@ -1131,7 +1146,7 @@ inline bool tryParseImpl(DataTypeUUID::FieldType & x, ReadBuffer & } template <> -inline bool tryParseImpl(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline bool tryParseTypeImpl(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { IPv4 tmp; if (!tryReadIPv4Text(tmp, rb)) @@ -1142,7 +1157,7 @@ inline bool tryParseImpl(DataTypeIPv4::FieldType & x, ReadBuffer & } template <> -inline bool tryParseImpl(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline bool tryParseTypeImpl(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { IPv6 tmp; if (!tryReadIPv6Text(tmp, rb)) @@ -1321,6 +1336,29 @@ struct ConvertThroughParsing size_t current_offset = 0; + bool precise_float_parsing = false; + + if (DB::CurrentThread::isInitialized()) + { + const DB::ContextPtr query_context = DB::CurrentThread::get().getQueryContext(); + + if (query_context) + precise_float_parsing = query_context->getSettingsRef().precise_float_parsing; + } + + std::function parseFunction; + std::function tryParseFunction; + + if constexpr (!(parsing_mode == ConvertFromStringParsingMode::BestEffort || + parsing_mode == ConvertFromStringParsingMode::BestEffortUS || + to_datetime64 || IsDataTypeDecimal)) + { + if constexpr (exception_mode == ConvertFromStringExceptionMode::Throw) + parseFunction = precise_float_parsing ? parseImpl : parseImpl; + else + tryParseFunction = precise_float_parsing ? tryParseImpl : tryParseImpl; + } + for (size_t i = 0; i < size; ++i) { size_t next_offset = std::is_same_v ? (*offsets)[i] : (current_offset + fixed_string_size); @@ -1387,7 +1425,7 @@ struct ConvertThroughParsing } } - parseImpl(vec_to[i], read_buffer, local_time_zone); + parseFunction(vec_to[i], read_buffer, local_time_zone); } while (false); } } @@ -1457,7 +1495,7 @@ struct ConvertThroughParsing } } - parsed = tryParseImpl(vec_to[i], read_buffer, local_time_zone); + parsed = tryParseFunction(vec_to[i], read_buffer, local_time_zone); } while (false); } } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index b8ce162ec91..f4d31c2d3d9 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -526,8 +526,8 @@ void tryReadIntTextUnsafe(T & x, ReadBuffer & buf) /// Look at readFloatText.h -template void readFloatText(T & x, ReadBuffer & in); -template bool tryReadFloatText(T & x, ReadBuffer & in); +template void readFloatText(T & x, ReadBuffer & in); +template bool tryReadFloatText(T & x, ReadBuffer & in); /// simple: all until '\n' or '\t' diff --git a/src/IO/readFloatText.cpp b/src/IO/readFloatText.cpp index d1143f7c62c..60505e0fb22 100644 --- a/src/IO/readFloatText.cpp +++ b/src/IO/readFloatText.cpp @@ -62,9 +62,13 @@ template void readFloatTextSimple(Float64 &, ReadBuffer &); template bool tryReadFloatTextSimple(Float32 &, ReadBuffer &); template bool tryReadFloatTextSimple(Float64 &, ReadBuffer &); -template void readFloatText(Float32 &, ReadBuffer &); -template void readFloatText(Float64 &, ReadBuffer &); -template bool tryReadFloatText(Float32 &, ReadBuffer &); -template bool tryReadFloatText(Float64 &, ReadBuffer &); +template void readFloatText(Float32 &, ReadBuffer &); +template void readFloatText(Float64 &, ReadBuffer &); +template void readFloatText(Float32 &, ReadBuffer &); +template void readFloatText(Float64 &, ReadBuffer &); +template bool tryReadFloatText(Float32 &, ReadBuffer &); +template bool tryReadFloatText(Float64 &, ReadBuffer &); +template bool tryReadFloatText(Float32 &, ReadBuffer &); +template bool tryReadFloatText(Float64 &, ReadBuffer &); } diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index da4719b8dcb..de80342687f 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -590,7 +590,19 @@ template bool tryReadFloatTextSimple(T & x, ReadBuffer & in) { retu /// Implementation that is selected as default. -template void readFloatText(T & x, ReadBuffer & in) { readFloatTextFast(x, in); } -template bool tryReadFloatText(T & x, ReadBuffer & in) { return tryReadFloatTextFast(x, in); } +template void readFloatText(T & x, ReadBuffer & in) +{ + if constexpr (precise_float_parsing) + readFloatTextPrecise(x, in); + else + readFloatTextFast(x, in); +} +template bool tryReadFloatText(T & x, ReadBuffer & in) +{ + if constexpr (precise_float_parsing) + return tryReadFloatTextPrecise(x, in); + else + return tryReadFloatTextFast(x, in); +} } diff --git a/tests/queries/0_stateless/02813_float_parsing.reference b/tests/queries/0_stateless/02813_float_parsing.reference new file mode 100644 index 00000000000..ed4797f7a37 --- /dev/null +++ b/tests/queries/0_stateless/02813_float_parsing.reference @@ -0,0 +1 @@ +1.7091 15008753 6e-9 6.000000000000001e-9 1.7091 15008753 6e-9 6e-9 diff --git a/tests/queries/0_stateless/02813_float_parsing.sql b/tests/queries/0_stateless/02813_float_parsing.sql new file mode 100644 index 00000000000..10f390f56d7 --- /dev/null +++ b/tests/queries/0_stateless/02813_float_parsing.sql @@ -0,0 +1,10 @@ +SELECT + toFloat64('1.7091'), + toFloat64('15008753.'), + toFloat64('6e-09'), + toFloat64('6.000000000000001e-9'), + toFloat32('1.7091'), + toFloat32('15008753.'), + toFloat32('6e-09'), + toFloat32('6.000000000000001e-9') +SETTINGS precise_float_parsing = 1; From 00a1a9f6bdcae72a5c115f467874b2b18f4a60e8 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 30 Jul 2023 20:59:15 +0200 Subject: [PATCH 1340/2047] added docs --- docs/en/operations/settings/settings.md | 25 ++++++++++++++++++++++++ docs/ru/operations/settings/settings.md | 26 +++++++++++++++++++++++++ 2 files changed, 51 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8b969f87a4d..886177b253b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4569,3 +4569,28 @@ Type: Int64 Default: 0 +## precise_float_parsing {#precise_float_parsing} + +Switches [Float32/Float64](../../sql-reference/data-types/float.md) parsing algorithms: +* If the value is `1`, then precise method is used. It is slower than fast method, but it always returns a number that is the closest machine representable number to the input. +* Otherwise, fast method is used (default). It usually returns the same value as precise, but in rare cases result may differ by one or two least significant bits. + +Possible values: `0`, `1`. + +Default value: `0`. + +Example: + +```sql +SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_parsing = 0; + +┌─toFloat64('1.7091')─┬─toFloat64('1.5008753E7')─┐ +│ 1.7090999999999998 │ 15008753.000000002 │ +└─────────────────────┴──────────────────────────┘ + +SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_parsing = 1; + +┌─toFloat64('1.7091')─┬─toFloat64('1.5008753E7')─┐ +│ 1.7091 │ 15008753 │ +└─────────────────────┴──────────────────────────┘ +``` diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 42e21f6140b..bb84c4225f7 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4212,3 +4212,29 @@ SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS sessi - Запрос: `SELECT * FROM file('sample.csv')` Если чтение и обработка `sample.csv` прошли успешно, файл будет переименован в `processed_sample_1683473210851438.csv`. + +## precise_float_parsing {#precise_float_parsing} + +Позволяет выбрать алгоритм, используемый при парсинге [Float32/Float64](../../sql-reference/data-types/float.md): +* Если установлено значение `1`, то используется точный метод. Он более медленный, но всегда возвращает число, наиболее близкое к входному значению. +* В противном случае используется быстрый метод (поведение по умолчанию). Обычно результат его работы совпадает с результатом, полученным точным методом, однако в редких случаях он может отличаться на 1 или 2 наименее значимых бит. + +Возможные значения: `0`, `1`. + +Значение по умолчанию: `0`. + +Пример: + +```sql +SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_parsing = 0; + +┌─toFloat64('1.7091')─┬─toFloat64('1.5008753E7')─┐ +│ 1.7090999999999998 │ 15008753.000000002 │ +└─────────────────────┴──────────────────────────┘ + +SELECT toFloat64('1.7091'), toFloat64('1.5008753E7') SETTINGS precise_float_parsing = 1; + +┌─toFloat64('1.7091')─┬─toFloat64('1.5008753E7')─┐ +│ 1.7091 │ 15008753 │ +└─────────────────────┴──────────────────────────┘ +``` From dd85c3c114c846b3a4398841e32f1d96981a9a42 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 30 Jul 2023 22:40:11 +0200 Subject: [PATCH 1341/2047] Update ReadBufferFromHDFS.cpp --- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 94780c2e290..3476defd5b1 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -58,7 +58,6 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(file_info->mSize); + if (file_size_.has_value()) + { + file_size = file_size_.value(); + } + else + { + auto * file_info = hdfsGetPathInfo(fs.get(), hdfs_file_path.c_str()); + if (!file_info) + throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size for: {}", hdfs_file_path); + file_size = static_cast(file_info->mSize); + } } ~ReadBufferFromHDFSImpl() override From d23dd7d30fde7b81273bdec99968422ec49ecabe Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 30 Jul 2023 23:02:45 +0200 Subject: [PATCH 1342/2047] upd test --- tests/queries/0_stateless/02813_float_parsing.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02813_float_parsing.sql b/tests/queries/0_stateless/02813_float_parsing.sql index 10f390f56d7..ab74f1e64e4 100644 --- a/tests/queries/0_stateless/02813_float_parsing.sql +++ b/tests/queries/0_stateless/02813_float_parsing.sql @@ -1,10 +1,10 @@ SELECT toFloat64('1.7091'), - toFloat64('15008753.'), + toFloat64('1.5008753E7'), toFloat64('6e-09'), toFloat64('6.000000000000001e-9'), toFloat32('1.7091'), - toFloat32('15008753.'), + toFloat32('1.5008753E7'), toFloat32('6e-09'), toFloat32('6.000000000000001e-9') SETTINGS precise_float_parsing = 1; From e58b3cfd3246681092908dc6e38d0b9a130b5df7 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Sun, 30 Jul 2023 22:09:03 +0000 Subject: [PATCH 1343/2047] Implemented max sessions for user --- .../operations/settings/query-complexity.md | 36 +++ .../operations/settings/settings-profiles.md | 4 +- .../operations/settings/query-complexity.md | 37 +++ .../operations/settings/settings-profiles.md | 3 +- src/Access/ContextAccess.cpp | 32 ++- src/Access/SettingsConstraints.cpp | 81 +++++-- src/Access/SettingsConstraints.h | 23 +- src/Client/Suggest.cpp | 3 + src/Common/ErrorCodes.cpp | 1 + src/Common/SettingSource.h | 43 ++++ src/Core/Settings.h | 2 + .../Access/InterpreterCreateRoleQuery.cpp | 2 +- .../InterpreterCreateSettingsProfileQuery.cpp | 2 +- .../Access/InterpreterCreateUserQuery.cpp | 2 +- src/Interpreters/Context.cpp | 37 ++- src/Interpreters/Context.h | 17 +- src/Interpreters/InterpreterSetQuery.cpp | 4 +- src/Interpreters/Session.cpp | 39 ++- src/Interpreters/Session.h | 3 + src/Interpreters/SessionTracker.cpp | 62 +++++ src/Interpreters/SessionTracker.h | 60 +++++ src/Server/GRPCServer.cpp | 4 +- src/Server/HTTPHandler.cpp | 2 +- src/Server/TCPHandler.cpp | 21 +- tests/integration/parallel_skip.json | 12 +- .../__init__.py | 0 .../configs/dhparam.pem | 8 + .../configs/log.xml | 9 + .../configs/ports.xml | 9 + .../configs/server.crt | 18 ++ .../configs/server.key | 28 +++ .../configs/ssl_conf.xml | 17 ++ .../configs/users.xml | 16 ++ .../protos/clickhouse_grpc.proto | 1 + .../test.py | 222 ++++++++++++++++++ ...2832_alter_max_sessions_for_user.reference | 12 + .../02832_alter_max_sessions_for_user.sh | 64 +++++ 37 files changed, 851 insertions(+), 85 deletions(-) create mode 100644 src/Common/SettingSource.h create mode 100644 src/Interpreters/SessionTracker.cpp create mode 100644 src/Interpreters/SessionTracker.h create mode 100755 tests/integration/test_profile_max_sessions_for_user/__init__.py create mode 100755 tests/integration/test_profile_max_sessions_for_user/configs/dhparam.pem create mode 100644 tests/integration/test_profile_max_sessions_for_user/configs/log.xml create mode 100644 tests/integration/test_profile_max_sessions_for_user/configs/ports.xml create mode 100755 tests/integration/test_profile_max_sessions_for_user/configs/server.crt create mode 100755 tests/integration/test_profile_max_sessions_for_user/configs/server.key create mode 100644 tests/integration/test_profile_max_sessions_for_user/configs/ssl_conf.xml create mode 100644 tests/integration/test_profile_max_sessions_for_user/configs/users.xml create mode 120000 tests/integration/test_profile_max_sessions_for_user/protos/clickhouse_grpc.proto create mode 100755 tests/integration/test_profile_max_sessions_for_user/test.py create mode 100644 tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference create mode 100755 tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 163ed5d5826..3af368b87a7 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -327,3 +327,39 @@ The maximum amount of data consumed by temporary files on disk in bytes for all Zero means unlimited. Default value: 0. + +## max_sessions_for_user {#max-sessions-per-user} + +Maximum number of simultaneous sessions per authenticated user to the ClickHouse server. + +Example: + +``` xml + + + 1 + + + 2 + + + 0 + + + + + + single_session_user + + + + two_sessions_profile + + + + unlimited_sessions_profile + + +``` + +Default value: 0 (Infinite count of simultaneous sessions). diff --git a/docs/en/operations/settings/settings-profiles.md b/docs/en/operations/settings/settings-profiles.md index 2f39a75453c..d08266b8ef8 100644 --- a/docs/en/operations/settings/settings-profiles.md +++ b/docs/en/operations/settings/settings-profiles.md @@ -39,7 +39,7 @@ Example: 8
- + 1000000000 100000000000 @@ -67,6 +67,8 @@ Example: 50 100 + 4 + 1 diff --git a/docs/ru/operations/settings/query-complexity.md b/docs/ru/operations/settings/query-complexity.md index de9bb969085..5f3f4b74f51 100644 --- a/docs/ru/operations/settings/query-complexity.md +++ b/docs/ru/operations/settings/query-complexity.md @@ -314,3 +314,40 @@ FORMAT Null; При вставке данных, ClickHouse вычисляет количество партиций во вставленном блоке. Если число партиций больше, чем `max_partitions_per_insert_block`, ClickHouse генерирует исключение со следующим текстом: > «Too many partitions for single INSERT block (more than» + toString(max_parts) + «). The limit is controlled by ‘max_partitions_per_insert_block’ setting. Large number of partitions is a common misconception. It will lead to severe negative performance impact, including slow server startup, slow INSERT queries and slow SELECT queries. Recommended total number of partitions for a table is under 1000..10000. Please note, that partitioning is not intended to speed up SELECT queries (ORDER BY key is sufficient to make range queries fast). Partitions are intended for data manipulation (DROP PARTITION, etc).» + +## max_sessions_for_user {#max-sessions-per-user} + +Максимальное количество одновременных сессий на одного аутентифицированного пользователя. + +Пример: + +``` xml + + + 1 + + + 2 + + + 0 + + + + + + single_session_profile + + + + two_sessions_profile + + + + unlimited_sessions_profile + + +``` + +Значение по умолчанию: 0 (неограниченное количество сессий). diff --git a/docs/ru/operations/settings/settings-profiles.md b/docs/ru/operations/settings/settings-profiles.md index ba2cb9a601f..0d094c637ac 100644 --- a/docs/ru/operations/settings/settings-profiles.md +++ b/docs/ru/operations/settings/settings-profiles.md @@ -39,7 +39,7 @@ SET profile = 'web' 8 - + 1000000000 100000000000 @@ -67,6 +67,7 @@ SET profile = 'web' 50 100 + 4 1 diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 9e9d8644539..e277c49e39d 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -328,9 +328,6 @@ void ContextAccess::setRolesInfo(const std::shared_ptr & enabled_row_policies = access_control->getEnabledRowPolicies(*params.user_id, roles_info->enabled_roles); - enabled_quota = access_control->getEnabledQuota( - *params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key); - enabled_settings = access_control->getEnabledSettings( *params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles); @@ -416,19 +413,32 @@ RowPolicyFilterPtr ContextAccess::getRowPolicyFilter(const String & database, co std::shared_ptr ContextAccess::getQuota() const { std::lock_guard lock{mutex}; - if (enabled_quota) - return enabled_quota; - static const auto unlimited_quota = EnabledQuota::getUnlimitedQuota(); - return unlimited_quota; + + if (!enabled_quota) + { + if (roles_info) + { + enabled_quota = access_control->getEnabledQuota(*params.user_id, + user_name, + roles_info->enabled_roles, + params.address, + params.forwarded_address, + params.quota_key); + } + else + { + static const auto unlimited_quota = EnabledQuota::getUnlimitedQuota(); + return unlimited_quota; + } + } + + return enabled_quota; } std::optional ContextAccess::getQuotaUsage() const { - std::lock_guard lock{mutex}; - if (enabled_quota) - return enabled_quota->getUsage(); - return {}; + return getQuota()->getUsage(); } diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 12f584cab83..74c6bbe0353 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -6,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -20,6 +22,39 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } +namespace +{ +struct SettingSourceRestrictions +{ + constexpr SettingSourceRestrictions() { allowed_sources.set(); } + + constexpr SettingSourceRestrictions(std::initializer_list allowed_sources_) + { + for (auto allowed_source : allowed_sources_) + setSourceAllowed(allowed_source, true); + } + + constexpr bool isSourceAllowed(SettingSource source) { return allowed_sources[source]; } + constexpr void setSourceAllowed(SettingSource source, bool allowed) { allowed_sources[source] = allowed; } + + std::bitset allowed_sources; +}; + +const std::unordered_map SETTINGS_SOURCE_RESTRICTIONS = { + {"max_sessions_for_user", {SettingSource::PROFILE}}, +}; + +SettingSourceRestrictions getSettingSourceRestrictions(std::string_view name) +{ + auto settingConstraintIter = SETTINGS_SOURCE_RESTRICTIONS.find(name); + if (settingConstraintIter != SETTINGS_SOURCE_RESTRICTIONS.end()) + return settingConstraintIter->second; + else + return SettingSourceRestrictions(); // allows everything +} + +} + SettingsConstraints::SettingsConstraints(const AccessControl & access_control_) : access_control(&access_control_) { } @@ -98,7 +133,7 @@ void SettingsConstraints::merge(const SettingsConstraints & other) } -void SettingsConstraints::check(const Settings & current_settings, const SettingsProfileElements & profile_elements) const +void SettingsConstraints::check(const Settings & current_settings, const SettingsProfileElements & profile_elements, SettingSource source) const { for (const auto & element : profile_elements) { @@ -108,19 +143,19 @@ void SettingsConstraints::check(const Settings & current_settings, const Setting if (element.value) { SettingChange value(element.setting_name, *element.value); - check(current_settings, value); + check(current_settings, value, source); } if (element.min_value) { SettingChange value(element.setting_name, *element.min_value); - check(current_settings, value); + check(current_settings, value, source); } if (element.max_value) { SettingChange value(element.setting_name, *element.max_value); - check(current_settings, value); + check(current_settings, value, source); } SettingConstraintWritability new_value = SettingConstraintWritability::WRITABLE; @@ -142,24 +177,24 @@ void SettingsConstraints::check(const Settings & current_settings, const Setting } } -void SettingsConstraints::check(const Settings & current_settings, const SettingChange & change) const +void SettingsConstraints::check(const Settings & current_settings, const SettingChange & change, SettingSource source) const { - checkImpl(current_settings, const_cast(change), THROW_ON_VIOLATION); + checkImpl(current_settings, const_cast(change), THROW_ON_VIOLATION, source); } -void SettingsConstraints::check(const Settings & current_settings, const SettingsChanges & changes) const +void SettingsConstraints::check(const Settings & current_settings, const SettingsChanges & changes, SettingSource source) const { for (const auto & change : changes) - check(current_settings, change); + check(current_settings, change, source); } -void SettingsConstraints::check(const Settings & current_settings, SettingsChanges & changes) const +void SettingsConstraints::check(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const { boost::range::remove_erase_if( changes, [&](SettingChange & change) -> bool { - return !checkImpl(current_settings, const_cast(change), THROW_ON_VIOLATION); + return !checkImpl(current_settings, const_cast(change), THROW_ON_VIOLATION, source); }); } @@ -174,13 +209,13 @@ void SettingsConstraints::check(const MergeTreeSettings & current_settings, cons check(current_settings, change); } -void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes) const +void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const { boost::range::remove_erase_if( changes, [&](SettingChange & change) -> bool { - return !checkImpl(current_settings, change, CLAMP_ON_VIOLATION); + return !checkImpl(current_settings, change, CLAMP_ON_VIOLATION, source); }); } @@ -215,7 +250,10 @@ bool getNewValueToCheck(const T & current_settings, SettingChange & change, Fiel return true; } -bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const +bool SettingsConstraints::checkImpl(const Settings & current_settings, + SettingChange & change, + ReactionOnViolation reaction, + SettingSource source) const { std::string_view setting_name = Settings::Traits::resolveName(change.name); @@ -247,7 +285,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION)) return false; - return getChecker(current_settings, setting_name).check(change, new_value, reaction); + return getChecker(current_settings, setting_name).check(change, new_value, reaction, source); } bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const @@ -255,10 +293,13 @@ bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings, Field new_value; if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION)) return false; - return getMergeTreeChecker(change.name).check(change, new_value, reaction); + return getMergeTreeChecker(change.name).check(change, new_value, reaction, SettingSource::QUERY); } -bool SettingsConstraints::Checker::check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const +bool SettingsConstraints::Checker::check(SettingChange & change, + const Field & new_value, + ReactionOnViolation reaction, + SettingSource source) const { if (!explain.empty()) { @@ -326,6 +367,14 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n change.value = max_value; } + if (!getSettingSourceRestrictions(setting_name).isSourceAllowed(source)) + { + if (reaction == THROW_ON_VIOLATION) + throw Exception(ErrorCodes::READONLY, "Setting {} is not allowed to be set by {}", setting_name, toString(source)); + else + return false; + } + return true; } diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 667d57a90ae..d09e60cc9d5 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace Poco::Util @@ -73,17 +74,18 @@ public: void merge(const SettingsConstraints & other); /// Checks whether `change` violates these constraints and throws an exception if so. - void check(const Settings & current_settings, const SettingsProfileElements & profile_elements) const; - void check(const Settings & current_settings, const SettingChange & change) const; - void check(const Settings & current_settings, const SettingsChanges & changes) const; - void check(const Settings & current_settings, SettingsChanges & changes) const; + void check(const Settings & current_settings, const SettingsProfileElements & profile_elements, SettingSource source) const; + void check(const Settings & current_settings, const SettingChange & change, SettingSource source) const; + void check(const Settings & current_settings, const SettingsChanges & changes, SettingSource source) const; + void check(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const; /// Checks whether `change` violates these constraints and throws an exception if so. (setting short name is expected inside `changes`) void check(const MergeTreeSettings & current_settings, const SettingChange & change) const; void check(const MergeTreeSettings & current_settings, const SettingsChanges & changes) const; /// Checks whether `change` violates these and clamps the `change` if so. - void clamp(const Settings & current_settings, SettingsChanges & changes) const; + void clamp(const Settings & current_settings, SettingsChanges & changes, SettingSource source) const; + friend bool operator ==(const SettingsConstraints & left, const SettingsConstraints & right); friend bool operator !=(const SettingsConstraints & left, const SettingsConstraints & right) { return !(left == right); } @@ -133,7 +135,10 @@ private: {} // Perform checking - bool check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const; + bool check(SettingChange & change, + const Field & new_value, + ReactionOnViolation reaction, + SettingSource source) const; }; struct StringHash @@ -145,7 +150,11 @@ private: } }; - bool checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const; + bool checkImpl(const Settings & current_settings, + SettingChange & change, + ReactionOnViolation reaction, + SettingSource source) const; + bool checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const; Checker getChecker(const Settings & current_settings, std::string_view setting_name) const; diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 1723f85dc16..00e0ebd8b91 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -124,6 +124,9 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p if (e.code() == ErrorCodes::DEADLOCK_AVOIDED) continue; + /// Client can successfully connect to the server and + /// get ErrorCodes::USER_SESSION_LIMIT_EXCEEDED for suggestion connection. + /// We should not use std::cerr here, because this method works concurrently with the main thread. /// WriteBufferFromFileDescriptor will write directly to the file descriptor, avoiding data race on std::cerr. diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ae8d5f8796d..393486f805c 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -582,6 +582,7 @@ M(697, CANNOT_RESTORE_TO_NONENCRYPTED_DISK) \ M(698, INVALID_REDIS_STORAGE_TYPE) \ M(699, INVALID_REDIS_TABLE_STRUCTURE) \ + M(700, USER_SESSION_LIMIT_EXCEEDED) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/SettingSource.h b/src/Common/SettingSource.h new file mode 100644 index 00000000000..38e1bcae295 --- /dev/null +++ b/src/Common/SettingSource.h @@ -0,0 +1,43 @@ +#pragma once + +#include + +namespace DB +{ + enum SettingSource + { + /// Query or session change: + /// SET = + /// SELECT ... SETTINGS [ = = = = ] + /// ALTER ROLE ... SETTINGS [ = = ] + /// ALTER USER ... SETTINGS [ = checkSettingsConstraints(*settings_from_query); + getContext()->checkSettingsConstraints(*settings_from_query, SettingSource::ROLE); } if (!query.cluster.empty()) diff --git a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp index 0727b6f2182..8a79bab0b0d 100644 --- a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp @@ -54,7 +54,7 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() settings_from_query = SettingsProfileElements{*query.settings, access_control}; if (!query.attach) - getContext()->checkSettingsConstraints(*settings_from_query); + getContext()->checkSettingsConstraints(*settings_from_query, SettingSource::PROFILE); } if (!query.cluster.empty()) diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 165937560cc..475ee270506 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -133,7 +133,7 @@ BlockIO InterpreterCreateUserQuery::execute() settings_from_query = SettingsProfileElements{*query.settings, access_control}; if (!query.attach) - getContext()->checkSettingsConstraints(*settings_from_query); + getContext()->checkSettingsConstraints(*settings_from_query, SettingSource::USER); } if (!query.cluster.empty()) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d1b88988e5e..b10f85a2197 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -45,6 +45,7 @@ #include #include #include +#include #include #include #include @@ -158,6 +159,7 @@ namespace CurrentMetrics extern const Metric IOWriterThreadsActive; } + namespace DB { @@ -276,6 +278,7 @@ struct ContextSharedPart : boost::noncopyable mutable QueryCachePtr query_cache; /// Cache of query results. mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. ProcessList process_list; /// Executing queries at the moment. + SessionTracker session_tracker; GlobalOvercommitTracker global_overcommit_tracker; MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) @@ -739,6 +742,9 @@ std::unique_lock Context::getLock() const ProcessList & Context::getProcessList() { return shared->process_list; } const ProcessList & Context::getProcessList() const { return shared->process_list; } OvercommitTracker * Context::getGlobalOvercommitTracker() const { return &shared->global_overcommit_tracker; } + +SessionTracker & Context::getSessionTracker() { return shared->session_tracker; } + MergeList & Context::getMergeList() { return shared->merge_list; } const MergeList & Context::getMergeList() const { return shared->merge_list; } MovesList & Context::getMovesList() { return shared->moves_list; } @@ -1094,7 +1100,7 @@ void Context::setUser(const UUID & user_id_, bool set_current_profiles_, bool se std::optional params; { auto lock = getLock(); - params.emplace(ContextAccessParams{user_id_, /* full_access= */ false, /* use_default_roles = */ true, {}, settings, current_database, client_info}); + params.emplace(ContextAccessParams{user_id_, /* full_access= */ false, /* use_default_roles = */ true, {}, settings, current_database, client_info }); } /// `temp_access` is used here only to extract information about the user, not to actually check access. /// NOTE: AccessControl::getContextAccess() may require some IO work, so Context::getLock() must be unlocked while we're doing this. @@ -1157,13 +1163,6 @@ std::optional Context::getUserID() const } -void Context::setQuotaKey(String quota_key_) -{ - auto lock = getLock(); - client_info.quota_key = std::move(quota_key_); -} - - void Context::setCurrentRoles(const std::vector & current_roles_) { auto lock = getLock(); @@ -1303,7 +1302,7 @@ void Context::setCurrentProfiles(const SettingsProfilesInfo & profiles_info, boo { auto lock = getLock(); if (check_constraints) - checkSettingsConstraints(profiles_info.settings); + checkSettingsConstraints(profiles_info.settings, SettingSource::PROFILE); applySettingsChanges(profiles_info.settings); settings_constraints_and_current_profiles = profiles_info.getConstraintsAndProfileIDs(settings_constraints_and_current_profiles); } @@ -1857,29 +1856,29 @@ void Context::applySettingsChanges(const SettingsChanges & changes) } -void Context::checkSettingsConstraints(const SettingsProfileElements & profile_elements) const +void Context::checkSettingsConstraints(const SettingsProfileElements & profile_elements, SettingSource source) const { - getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, profile_elements); + getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, profile_elements, source); } -void Context::checkSettingsConstraints(const SettingChange & change) const +void Context::checkSettingsConstraints(const SettingChange & change, SettingSource source) const { - getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, change); + getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, change, source); } -void Context::checkSettingsConstraints(const SettingsChanges & changes) const +void Context::checkSettingsConstraints(const SettingsChanges & changes, SettingSource source) const { - getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes); + getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes, source); } -void Context::checkSettingsConstraints(SettingsChanges & changes) const +void Context::checkSettingsConstraints(SettingsChanges & changes, SettingSource source) const { - getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes); + getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes, source); } -void Context::clampToSettingsConstraints(SettingsChanges & changes) const +void Context::clampToSettingsConstraints(SettingsChanges & changes, SettingSource source) const { - getSettingsConstraintsAndCurrentProfiles()->constraints.clamp(settings, changes); + getSettingsConstraintsAndCurrentProfiles()->constraints.clamp(settings, changes, source); } void Context::checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 0d567816ec9..676eb8412e5 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -202,6 +203,8 @@ using MergeTreeMetadataCachePtr = std::shared_ptr; class PreparedSetsCache; using PreparedSetsCachePtr = std::shared_ptr; +class SessionTracker; + /// An empty interface for an arbitrary object that may be attached by a shared pointer /// to query context, when using ClickHouse as a library. struct IHostContext @@ -539,8 +542,6 @@ public: String getUserName() const; - void setQuotaKey(String quota_key_); - void setCurrentRoles(const std::vector & current_roles_); void setCurrentRolesDefault(); boost::container::flat_set getCurrentRoles() const; @@ -735,11 +736,11 @@ public: void applySettingsChanges(const SettingsChanges & changes); /// Checks the constraints. - void checkSettingsConstraints(const SettingsProfileElements & profile_elements) const; - void checkSettingsConstraints(const SettingChange & change) const; - void checkSettingsConstraints(const SettingsChanges & changes) const; - void checkSettingsConstraints(SettingsChanges & changes) const; - void clampToSettingsConstraints(SettingsChanges & changes) const; + void checkSettingsConstraints(const SettingsProfileElements & profile_elements, SettingSource source) const; + void checkSettingsConstraints(const SettingChange & change, SettingSource source) const; + void checkSettingsConstraints(const SettingsChanges & changes, SettingSource source) const; + void checkSettingsConstraints(SettingsChanges & changes, SettingSource source) const; + void clampToSettingsConstraints(SettingsChanges & changes, SettingSource source) const; void checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const; /// Reset settings to default value @@ -861,6 +862,8 @@ public: OvercommitTracker * getGlobalOvercommitTracker() const; + SessionTracker & getSessionTracker(); + MergeList & getMergeList(); const MergeList & getMergeList() const; diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index e9118b747e5..2c0baa0d4b3 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -15,7 +15,7 @@ namespace DB BlockIO InterpreterSetQuery::execute() { const auto & ast = query_ptr->as(); - getContext()->checkSettingsConstraints(ast.changes); + getContext()->checkSettingsConstraints(ast.changes, SettingSource::QUERY); auto session_context = getContext()->getSessionContext(); session_context->applySettingsChanges(ast.changes); session_context->addQueryParameters(ast.query_parameters); @@ -28,7 +28,7 @@ void InterpreterSetQuery::executeForCurrentContext(bool ignore_setting_constrain { const auto & ast = query_ptr->as(); if (!ignore_setting_constraints) - getContext()->checkSettingsConstraints(ast.changes); + getContext()->checkSettingsConstraints(ast.changes, SettingSource::QUERY); getContext()->applySettingsChanges(ast.changes); getContext()->resetSettingsToDefaultValue(ast.default_settings); } diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index cadf619700c..de2a779b740 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -3,11 +3,13 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -200,7 +202,6 @@ private: LOG_TEST(log, "Schedule closing session with session_id: {}, user_id: {}", session.key.second, session.key.first); - } void cleanThread() @@ -336,6 +337,9 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So if (session_context) throw Exception(ErrorCodes::LOGICAL_ERROR, "If there is a session context it must be created after authentication"); + if (session_tracker_handle) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session tracker handle was created before authentication finish"); + auto address = address_; if ((address == Poco::Net::SocketAddress{}) && (prepared_client_info->interface == ClientInfo::Interface::LOCAL)) address = Poco::Net::SocketAddress{"127.0.0.1", 0}; @@ -490,6 +494,8 @@ ContextMutablePtr Session::makeSessionContext() throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created before any query context"); if (!user_id) throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created after authentication"); + if (session_tracker_handle) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session tracker handle was created before making session"); LOG_DEBUG(log, "{} Creating session context with user_id: {}", toString(auth_id), toString(*user_id)); @@ -503,13 +509,17 @@ ContextMutablePtr Session::makeSessionContext() prepared_client_info.reset(); /// Set user information for the new context: current profiles, roles, access rights. - if (user_id) - new_session_context->setUser(*user_id); + new_session_context->setUser(*user_id); /// Session context is ready. session_context = new_session_context; user = session_context->getUser(); + session_tracker_handle = session_context->getSessionTracker().trackSession( + *user_id, + {}, + session_context->getSettingsRef().max_sessions_for_user); + return session_context; } @@ -521,6 +531,8 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created before any query context"); if (!user_id) throw Exception(ErrorCodes::LOGICAL_ERROR, "Session context must be created after authentication"); + if (session_tracker_handle) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session tracker handle was created before making session"); LOG_DEBUG(log, "{} Creating named session context with name: {}, user_id: {}", toString(auth_id), session_name_, toString(*user_id)); @@ -541,9 +553,23 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: new_session_context->setClientInfo(*prepared_client_info); prepared_client_info.reset(); + auto access = new_session_context->getAccess(); + UInt64 max_sessions_for_user = 0; /// Set user information for the new context: current profiles, roles, access rights. - if (user_id && !new_session_context->getAccess()->tryGetUser()) + if (!access->tryGetUser()) + { new_session_context->setUser(*user_id); + max_sessions_for_user = new_session_context->getSettingsRef().max_sessions_for_user; + } + else + { + // Always get setting from profile + // profile can be changed by ALTER PROFILE during single session + auto settings = access->getDefaultSettings(); + const Field * max_session_for_user_field = settings.tryGet("max_sessions_for_user"); + if (max_session_for_user_field) + max_sessions_for_user = max_session_for_user_field->safeGet(); + } /// Session context is ready. session_context = std::move(new_session_context); @@ -551,6 +577,11 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: named_session_created = new_named_session_created; user = session_context->getUser(); + session_tracker_handle = session_context->getSessionTracker().trackSession( + *user_id, + { session_name_ }, + max_sessions_for_user); + return session_context; } diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 36f811ccd24..51c0e3c71fa 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -113,6 +114,8 @@ private: std::shared_ptr named_session; bool named_session_created = false; + SessionTracker::SessionTrackerHandle session_tracker_handle; + Poco::Logger * log = nullptr; }; diff --git a/src/Interpreters/SessionTracker.cpp b/src/Interpreters/SessionTracker.cpp new file mode 100644 index 00000000000..4636766e288 --- /dev/null +++ b/src/Interpreters/SessionTracker.cpp @@ -0,0 +1,62 @@ +#include "SessionTracker.h" + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int USER_SESSION_LIMIT_EXCEEDED; +} + +SessionTracker::Session::Session(SessionTracker & tracker_, + const UUID& user_id_, + SessionInfos::const_iterator session_info_iter_) noexcept + : tracker(tracker_), user_id(user_id_), session_info_iter(session_info_iter_) +{ +} + +SessionTracker::Session::~Session() +{ + tracker.stopTracking(user_id, session_info_iter); +} + +SessionTracker::SessionTrackerHandle +SessionTracker::trackSession(const UUID & user_id, + const SessionInfo & session_info, + size_t max_sessions_for_user) +{ + std::lock_guard lock(mutex); + + auto sessions_for_user_iter = sessions_for_user.find(user_id); + if (sessions_for_user_iter == sessions_for_user.end()) + sessions_for_user_iter = sessions_for_user.emplace(user_id, SessionInfos()).first; + + SessionInfos & session_infos = sessions_for_user_iter->second; + if (max_sessions_for_user && session_infos.size() >= max_sessions_for_user) + { + throw Exception(ErrorCodes::USER_SESSION_LIMIT_EXCEEDED, + "User {} has overflown session count {}", + toString(user_id), + max_sessions_for_user); + } + + session_infos.emplace_front(session_info); + + return std::make_unique(*this, user_id, session_infos.begin()); +} + +void SessionTracker::stopTracking(const UUID& user_id, SessionInfos::const_iterator session_info_iter) +{ + std::lock_guard lock(mutex); + + auto sessions_for_user_iter = sessions_for_user.find(user_id); + chassert(sessions_for_user_iter != sessions_for_user.end()); + + sessions_for_user_iter->second.erase(session_info_iter); + if (sessions_for_user_iter->second.empty()) + sessions_for_user.erase(sessions_for_user_iter); +} + +} diff --git a/src/Interpreters/SessionTracker.h b/src/Interpreters/SessionTracker.h new file mode 100644 index 00000000000..0827213aeed --- /dev/null +++ b/src/Interpreters/SessionTracker.h @@ -0,0 +1,60 @@ +#pragma once + +#include "ClientInfo.h" + +#include +#include +#include +#include + +namespace DB +{ + +struct SessionInfo +{ + const String session_id; +}; + +using SessionInfos = std::list; + +using SessionsForUser = std::unordered_map; + +class SessionTracker; + +class SessionTracker +{ +public: + class Session : boost::noncopyable + { + public: + explicit Session(SessionTracker & tracker_, + const UUID & user_id_, + SessionInfos::const_iterator session_info_iter_) noexcept; + + ~Session(); + + private: + friend class SessionTracker; + + SessionTracker & tracker; + const UUID user_id; + const SessionInfos::const_iterator session_info_iter; + }; + + using SessionTrackerHandle = std::unique_ptr; + + SessionTrackerHandle trackSession(const UUID & user_id, + const SessionInfo & session_info, + size_t max_sessions_for_user); + +private: + /// disallow manual messing with session tracking + friend class Session; + + std::mutex mutex; + SessionsForUser sessions_for_user TSA_GUARDED_BY(mutex); + + void stopTracking(const UUID& user_id, SessionInfos::const_iterator session_info_iter); +}; + +} diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 67d30012b0e..3370a8c009b 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -833,7 +833,7 @@ namespace { settings_changes.push_back({key, value}); } - query_context->checkSettingsConstraints(settings_changes); + query_context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); query_context->applySettingsChanges(settings_changes); query_context->setCurrentQueryId(query_info.query_id()); @@ -1118,7 +1118,7 @@ namespace SettingsChanges settings_changes; for (const auto & [key, value] : external_table.settings()) settings_changes.push_back({key, value}); - external_table_context->checkSettingsConstraints(settings_changes); + external_table_context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); external_table_context->applySettingsChanges(settings_changes); } auto in = external_table_context->getInputFormat( diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index a0bfcd49dfd..ff5690a3b07 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -764,7 +764,7 @@ void HTTPHandler::processQuery( context->setDefaultFormat(default_format); /// For external data we also want settings - context->checkSettingsConstraints(settings_changes); + context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); context->applySettingsChanges(settings_changes); /// Set the query id supplied by the user, if any, and also update the OpenTelemetry fields. diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a747f06f1ce..5f3a7614eee 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -184,14 +184,17 @@ void TCPHandler::runImpl() try { receiveHello(); + + /// In interserver mode queries are executed without a session context. + if (!is_interserver_mode) + session->makeSessionContext(); + sendHello(); if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) receiveAddendum(); - if (!is_interserver_mode) /// In interserver mode queries are executed without a session context. + if (!is_interserver_mode) { - session->makeSessionContext(); - /// If session created, then settings in session context has been updated. /// So it's better to update the connection settings for flexibility. extractConnectionSettingsFromContext(session->sessionContext()); @@ -1181,7 +1184,6 @@ std::unique_ptr TCPHandler::makeSession() res->setClientName(client_name); res->setClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); res->setConnectionClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version); - res->setQuotaClientKey(quota_key); res->setClientInterface(interface); return res; @@ -1274,11 +1276,10 @@ void TCPHandler::receiveHello() void TCPHandler::receiveAddendum() { if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_QUOTA_KEY) - { readStringBinary(quota_key, *in); - if (!is_interserver_mode) - session->setQuotaClientKey(quota_key); - } + + if (!is_interserver_mode) + session->setQuotaClientKey(quota_key); } @@ -1591,12 +1592,12 @@ void TCPHandler::receiveQuery() if (query_kind == ClientInfo::QueryKind::INITIAL_QUERY) { /// Throw an exception if the passed settings violate the constraints. - query_context->checkSettingsConstraints(settings_changes); + query_context->checkSettingsConstraints(settings_changes, SettingSource::QUERY); } else { /// Quietly clamp to the constraints if it's not an initial query. - query_context->clampToSettingsConstraints(settings_changes); + query_context->clampToSettingsConstraints(settings_changes, SettingSource::QUERY); } query_context->applySettingsChanges(settings_changes); diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 6e1604f4eb5..cb2b98937ca 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -81,5 +81,15 @@ "test_system_flush_logs/test.py::test_log_buffer_size_rows_flush_threshold", "test_system_flush_logs/test.py::test_log_max_size", "test_crash_log/test.py::test_pkill_query_log", - "test_crash_log/test.py::test_pkill" + "test_crash_log/test.py::test_pkill", + + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_postgres", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_mysql", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_http", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_http_named_session", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_grpc", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp_and_others", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_end_session" ] diff --git a/tests/integration/test_profile_max_sessions_for_user/__init__.py b/tests/integration/test_profile_max_sessions_for_user/__init__.py new file mode 100755 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/dhparam.pem b/tests/integration/test_profile_max_sessions_for_user/configs/dhparam.pem new file mode 100755 index 00000000000..2e6cee0798d --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/dhparam.pem @@ -0,0 +1,8 @@ +-----BEGIN DH PARAMETERS----- +MIIBCAKCAQEAua92DDli13gJ+//ZXyGaggjIuidqB0crXfhUlsrBk9BV1hH3i7fR +XGP9rUdk2ubnB3k2ejBStL5oBrkHm9SzUFSQHqfDjLZjKoUpOEmuDc4cHvX1XTR5 +Pr1vf5cd0yEncJWG5W4zyUB8k++SUdL2qaeslSs+f491HBLDYn/h8zCgRbBvxhxb +9qeho1xcbnWeqkN6Kc9bgGozA16P9NLuuLttNnOblkH+lMBf42BSne/TWt3AlGZf +slKmmZcySUhF8aKfJnLKbkBCFqOtFRh8zBA9a7g+BT/lSANATCDPaAk1YVih2EKb +dpc3briTDbRsiqg2JKMI7+VdULY9bh3EawIBAg== +-----END DH PARAMETERS----- diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/log.xml b/tests/integration/test_profile_max_sessions_for_user/configs/log.xml new file mode 100644 index 00000000000..22f95a8bd5d --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/log.xml @@ -0,0 +1,9 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/ports.xml b/tests/integration/test_profile_max_sessions_for_user/configs/ports.xml new file mode 100644 index 00000000000..3123c4a3d9c --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/ports.xml @@ -0,0 +1,9 @@ + + 5433 + 9001 + 9100 + + + false + + diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/server.crt b/tests/integration/test_profile_max_sessions_for_user/configs/server.crt new file mode 100755 index 00000000000..070d37f3b77 --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/server.crt @@ -0,0 +1,18 @@ +-----BEGIN CERTIFICATE----- +MIIC+zCCAeOgAwIBAgIJANhP897Se2gmMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV +BAMMCWxvY2FsaG9zdDAeFw0yMDA0MTgyMTE2NDBaFw0yMTA0MTgyMTE2NDBaMBQx +EjAQBgNVBAMMCWxvY2FsaG9zdDCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoC +ggEBAM92kcojQoMsjZ9YGhPMY6h/fDUsZeSKHLxgqE6wbmfU1oZKCPWqnvl+4n0J +pnT5h1ETxxYZLepimKq0DEVPUTmCl0xmcKbtUNiaTUKYKsdita6b2vZCX9wUPN9p +2Kjnm41l+aZNqIEBhIgHNWg9qowi20y0EIXR79jQLwwaInHAaJLZxVsqY2zjQ/D7 +1Zh82MXud7iqxBQiEfw9Cz35UFA239R8QTlPkVQfsN1gfLxnLk24QUX3o+hbUI1g +nlSpyYDHYQlOmwz8doDs6THHAZNJ4bPE9xHNFpw6dGZdbtH+IKQ/qRZIiOaiNuzJ +IOHl6XQDRDkW2LMTiCQ6fjC7Pz8CAwEAAaNQME4wHQYDVR0OBBYEFFvhaA/Eguyf +BXkMj8BkNLBqMnz2MB8GA1UdIwQYMBaAFFvhaA/EguyfBXkMj8BkNLBqMnz2MAwG +A1UdEwQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBACeU/oL48eVAKH7NQntHhRaJ +ZGeQzKIjrSBjFo8BGXD1nJZhUeFsylLrhCkC8/5/3grE3BNVX9bxcGjO81C9Mn4U +t0z13d6ovJjCZSQArtLwgeJGlpH7gNdD3DyT8DQmrqYVnmnB7UmBu45XH1LWGQZr +FAOhGRVs6s6mNj8QlLMgdmsOeOQnsGCMdoss8zV9vO2dc4A5SDSSL2mqGGY4Yjtt +X+XlEhXXnksGyx8NGVOZX4wcj8WeCAj/lihQ7Zh6XYwZH9i+E46ompUwoziZnNPu +2RH63tLNCxkOY2HF5VMlbMmzer3FkhlM6TAZZRPcvSphKPwXK4A33yqc6wnWvpc= +-----END CERTIFICATE----- diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/server.key b/tests/integration/test_profile_max_sessions_for_user/configs/server.key new file mode 100755 index 00000000000..b3dee82dcda --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/server.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQDPdpHKI0KDLI2f +WBoTzGOof3w1LGXkihy8YKhOsG5n1NaGSgj1qp75fuJ9CaZ0+YdRE8cWGS3qYpiq +tAxFT1E5gpdMZnCm7VDYmk1CmCrHYrWum9r2Ql/cFDzfadio55uNZfmmTaiBAYSI +BzVoPaqMIttMtBCF0e/Y0C8MGiJxwGiS2cVbKmNs40Pw+9WYfNjF7ne4qsQUIhH8 +PQs9+VBQNt/UfEE5T5FUH7DdYHy8Zy5NuEFF96PoW1CNYJ5UqcmAx2EJTpsM/HaA +7OkxxwGTSeGzxPcRzRacOnRmXW7R/iCkP6kWSIjmojbsySDh5el0A0Q5FtizE4gk +On4wuz8/AgMBAAECggEAJ54J2yL+mZQRe2NUn4FBarTloDXZQ1pIgISov1Ybz0Iq +sTxEF728XAKp95y3J9Fa0NXJB+RJC2BGrRpy2W17IlNY1yMc0hOxg5t7s4LhcG/e +J/jlSG+GZL2MnlFVKXQJFWhq0yIzUmdayqstvLlB7z7cx/n+yb88YRfoVBRNjZEL +Tdrsw+087igDjrIxZJ3eMN5Wi434n9s4yAoRQC1bP5wcWx0gD4MzdmL8ip6suiRc +LRuBAhV/Op812xlxUhrF5dInUM9OLlGTXpUzexAS8Cyy7S4bfkW2BaCxTF7I7TFw +Whx28CKn/G49tIuU0m6AlxWbXpLVePTFyMb7RJz5cQKBgQD7VQd2u3HM6eE3PcXD +p6ObdLTUk8OAJ5BMmADFc71W0Epyo26/e8KXKGYGxE2W3fr13y+9b0fl5fxZPuhS +MgvXEO7rItAVsLcp0IzaqY0WUee2b4XWPAU0XuPqvjYMpx8H5OEHqFK6lhZysAqM +X7Ot3/Hux9X0MC4v5a/HNbDUOQKBgQDTUPaP3ADRrmpmE2sWuzWEnCSEz5f0tCLO +wTqhV/UraWUNlAbgK5NB790IjH/gotBSqqNPLJwJh0LUfClKM4LiaHsEag0OArOF +GhPMK1Ohps8c2RRsiG8+hxX2HEHeAVbkouEDPDiHdIW/92pBViDoETXL6qxDKbm9 +LkOcVeDfNwKBgQChh1xsqrvQ/t+IKWNZA/zahH9TwEP9sW/ESkz0mhYuHWA7nV4o +ItpFW+l2n+Nd+vy32OFN1p9W2iD9GrklWpTRfEiRRqaFyjVt4mMkhaPvnGRXlAVo +Utrldbb1v5ntN9txr2ARE9VXpe53dzzQSxGnxi4vUK/paK3GitAWMCOdwQKBgQCi +hmGsUXQb0P6qVYMGr6PAw2re7t8baLRguoMCdqjs45nCMLh9D2apzvb8TTtJJU/+ +VJlYGqJEPdDrpjcHh8jBo8QBqCM0RGWYGG9jl2syKB6hPGCV/PU6bSE58Y/DVNpk +7NUM7PM5UyhPddY2PC0A78Ole29UFLJzSzLa+b4DTwKBgH9Wh2k4YPnPcRrX89UL +eSwWa1CGq6HWX8Kd5qyz256aeHWuG5nv15+rBt+D7nwajUsqeVkAXz5H/dHuG1xz +jb7RW+pEjx0GVAmIbkM9vOLqEUfHHHPuk4AXCGGZ5sarPiKg4BHKBBsY1dpoO5UH +0j71fRA6zurHnTXDaCLWlUpZ +-----END PRIVATE KEY----- diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/ssl_conf.xml b/tests/integration/test_profile_max_sessions_for_user/configs/ssl_conf.xml new file mode 100644 index 00000000000..778d327c460 --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/ssl_conf.xml @@ -0,0 +1,17 @@ + + + + + + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + + /etc/clickhouse-server/config.d/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + diff --git a/tests/integration/test_profile_max_sessions_for_user/configs/users.xml b/tests/integration/test_profile_max_sessions_for_user/configs/users.xml new file mode 100644 index 00000000000..3bed673b2ca --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/configs/users.xml @@ -0,0 +1,16 @@ + + + + 2 + 0 + + + + + + + + 123 + + + diff --git a/tests/integration/test_profile_max_sessions_for_user/protos/clickhouse_grpc.proto b/tests/integration/test_profile_max_sessions_for_user/protos/clickhouse_grpc.proto new file mode 120000 index 00000000000..25d15f11e3b --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/protos/clickhouse_grpc.proto @@ -0,0 +1 @@ +../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py new file mode 100755 index 00000000000..9e6a10e7e15 --- /dev/null +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -0,0 +1,222 @@ +import os + +import grpc +import pymysql.connections +import psycopg2 as py_psql +import pytest +import sys +import threading + +from helpers.cluster import ClickHouseCluster, run_and_check + +MAX_SESSIONS_FOR_USER = 2 +POSTGRES_SERVER_PORT = 5433 +MYSQL_SERVER_PORT = 9001 +GRPC_PORT = 9100 + +TEST_USER = "test_user" +TEST_PASSWORD = "123" + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_ENCODING = "utf-8" + +# Use grpcio-tools to generate *pb2.py files from *.proto. +proto_dir = os.path.join(SCRIPT_DIR, "./protos") +gen_dir = os.path.join(SCRIPT_DIR, "./_gen") +os.makedirs(gen_dir, exist_ok=True) +run_and_check( + "python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \ + {proto_dir}/clickhouse_grpc.proto".format( + proto_dir=proto_dir, gen_dir=gen_dir + ), + shell=True, +) + +sys.path.append(gen_dir) + +import clickhouse_grpc_pb2 +import clickhouse_grpc_pb2_grpc + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "node", + main_configs=[ + "configs/ports.xml", + "configs/log.xml", + "configs/ssl_conf.xml", + "configs/dhparam.pem", + "configs/server.crt", + "configs/server.key", + ], + user_configs=["configs/users.xml"], + env_variables={"UBSAN_OPTIONS": "print_stacktrace=1"}, +) + + +def get_query(name, id): + return f"SElECT '{name}', {id}, sleep(1)" + + +def grpc_get_url(): + return f"{instance.ip_address}:{GRPC_PORT}" + + +def grpc_create_insecure_channel(): + channel = grpc.insecure_channel(grpc_get_url()) + grpc.channel_ready_future(channel).result(timeout=2) + return channel + + +def grpc_query(query_text, channel, session_id_): + query_info = clickhouse_grpc_pb2.QueryInfo( + query=query_text, + session_id=session_id_, + user_name=TEST_USER, + password=TEST_PASSWORD, + ) + + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) + result = stub.ExecuteQuery(query_info) + if result and result.HasField("exception"): + raise Exception(result.exception.display_text) + return result.output.decode(DEFAULT_ENCODING) + + +def threaded_run_test(sessions): + thread_list = [] + for i in range(len(sessions)): + thread = ThreadWithException(target=sessions[i], args=(i,)) + thread_list.append(thread) + thread.start() + + for thread in thread_list: + thread.join() + + exception_count = 0 + for i in range(len(sessions)): + if thread_list[i].run_exception != None: + exception_count += 1 + + assert exception_count == 1 + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +class ThreadWithException(threading.Thread): + run_exception = None + + def run(self): + try: + super().run() + except: + self.run_exception = sys.exc_info() + + def join(self): + super().join() + + +def postgres_session(id): + ch = py_psql.connect( + host=instance.ip_address, + port=POSTGRES_SERVER_PORT, + user=TEST_USER, + password=TEST_PASSWORD, + database="default", + ) + cur = ch.cursor() + cur.execute(get_query("postgres_session", id)) + cur.fetchall() + + +def mysql_session(id): + client = pymysql.connections.Connection( + host=instance.ip_address, + user=TEST_USER, + password=TEST_PASSWORD, + database="default", + port=MYSQL_SERVER_PORT, + ) + cursor = client.cursor(pymysql.cursors.DictCursor) + cursor.execute(get_query("mysql_session", id)) + cursor.fetchall() + + +def tcp_session(id): + instance.query(get_query("tcp_session", id), user=TEST_USER, password=TEST_PASSWORD) + + +def http_session(id): + instance.http_query( + get_query("http_session", id), user=TEST_USER, password=TEST_PASSWORD + ) + + +def http_named_session(id): + instance.http_query( + get_query("http_named_session", id), + user=TEST_USER, + password=TEST_PASSWORD, + params={"session_id": id}, + ) + + +def grpc_session(id): + grpc_query( + get_query("grpc_session", id), grpc_create_insecure_channel(), f"session_{id}" + ) + + +def test_profile_max_sessions_for_user_tcp(started_cluster): + threaded_run_test([tcp_session] * 3) + + +def test_profile_max_sessions_for_user_postgres(started_cluster): + threaded_run_test([postgres_session] * 3) + + +def test_profile_max_sessions_for_user_mysql(started_cluster): + threaded_run_test([mysql_session] * 3) + + +def test_profile_max_sessions_for_user_http(started_cluster): + threaded_run_test([http_session] * 3) + + +def test_profile_max_sessions_for_user_http_named_session(started_cluster): + threaded_run_test([http_named_session] * 3) + + +def test_profile_max_sessions_for_user_grpc(started_cluster): + threaded_run_test([grpc_session] * 3) + + +def test_profile_max_sessions_for_user_tcp_and_others(started_cluster): + threaded_run_test([tcp_session, grpc_session, grpc_session]) + threaded_run_test([tcp_session, http_session, http_session]) + threaded_run_test([tcp_session, mysql_session, mysql_session]) + threaded_run_test([tcp_session, postgres_session, postgres_session]) + threaded_run_test([tcp_session, http_session, postgres_session]) + threaded_run_test([tcp_session, postgres_session, http_session]) + + +def test_profile_max_sessions_for_user_end_session(started_cluster): + for conection_func in [ + tcp_session, + http_session, + grpc_session, + mysql_session, + postgres_session, + ]: + threaded_run_test([conection_func] * MAX_SESSIONS_FOR_USER) + threaded_run_test([conection_func] * MAX_SESSIONS_FOR_USER) + + +def test_profile_max_sessions_for_user_end_session(started_cluster): + instance.query_and_get_error("SET max_sessions_for_user = 10") diff --git a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference new file mode 100644 index 00000000000..f80f8738ff8 --- /dev/null +++ b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.reference @@ -0,0 +1,12 @@ +test_alter_profile case: max_session_count 1 alter_sessions_count 1 +test_alter_profile case: max_session_count 2 alter_sessions_count 1 +USER_SESSION_LIMIT_EXCEEDED +test_alter_profile case: max_session_count 1 alter_sessions_count 2 +test_alter_profile case: max_session_count 2 alter_sessions_count 2 +READONLY +READONLY +READONLY +READONLY +READONLY +READONLY +READONLY diff --git a/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh new file mode 100755 index 00000000000..546c54a4de9 --- /dev/null +++ b/tests/queries/0_stateless/02832_alter_max_sessions_for_user.sh @@ -0,0 +1,64 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SESSION_ID_PREFIX="02832_alter_max_sessions_session_$$" +PROFILE="02832_alter_max_sessions_profile_$$" +USER="02832_alter_max_sessions_user_$$" +USER2="02832_alter_max_sessions_user_two_$$" +ROLE="02832_alter_max_sessions_role_$$" + +${CLICKHOUSE_CLIENT} -q $"DROP USER IF EXISTS '${USER}'" +${CLICKHOUSE_CLIENT} -q $"DROP PROFILE IF EXISTS ${PROFILE}" +${CLICKHOUSE_CLIENT} -q $"CREATE SETTINGS PROFILE ${PROFILE}" +${CLICKHOUSE_CLIENT} -q $"CREATE USER '${USER}' SETTINGS PROFILE '${PROFILE}'" + +function test_alter_profile() +{ + local max_session_count="$1" + local alter_sessions_count="$2" + echo $"test_alter_profile case: max_session_count ${max_session_count} alter_sessions_count ${alter_sessions_count}" + + ${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${max_session_count}" + + # Create sesssions with $max_session_count resriction + for ((i = 1 ; i <= ${max_session_count} ; i++)); do + local session_id="${SESSION_ID_PREFIX}_${i}" + # Skip output from this query + ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${USER}&session_id=${session_id}&session_check=0" --data-binary "SELECT 1" > /dev/null + done + + # Update resriction to $alter_sessions_count + ${CLICKHOUSE_CLIENT} -q $"ALTER SETTINGS PROFILE ${PROFILE} SETTINGS max_sessions_for_user = ${alter_sessions_count}" + + # Simultaneous sessions should use max settings from profile ($alter_sessions_count) + for ((i = 1 ; i <= ${max_session_count} ; i++)); do + local session_id="${SESSION_ID_PREFIX}_${i}" + # ignore select 1, we need only errors + ${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${USER}&session_id=${session_id}&session_check=1" --data-binary "select sleep(0.3)" | grep -o -m 1 'USER_SESSION_LIMIT_EXCEEDED' & + done + + wait +} + +test_alter_profile 1 1 +test_alter_profile 2 1 +test_alter_profile 1 2 +test_alter_profile 2 2 + +${CLICKHOUSE_CLIENT} -q "SELECT 1 SETTINGS max_sessions_for_user = 1" 2>&1 | grep -m 1 -o 'READONLY' | head -1 +${CLICKHOUSE_CLIENT} -q $"SET max_sessions_for_user = 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1 +${CLICKHOUSE_CLIENT} --max_sessions_for_user=1 -q $"SELECT 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1 +# max_sessions_for_user is profile setting +${CLICKHOUSE_CLIENT} -q $"CREATE USER ${USER2} SETTINGS max_sessions_for_user = 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1 +${CLICKHOUSE_CLIENT} -q $"ALTER USER ${USER} SETTINGS max_sessions_for_user = 1" 2>&1 | grep -o -m 1 'READONLY' | head -1 +${CLICKHOUSE_CLIENT} -q $"CREATE ROLE ${ROLE} SETTINGS max_sessions_for_user = 1" 2>&1 | grep -o -m 1 'READONLY' | head -1 +${CLICKHOUSE_CLIENT} -q $"CREATE ROLE ${ROLE}" +${CLICKHOUSE_CLIENT} -q $"ALTER ROLE ${ROLE} SETTINGS max_sessions_for_user = 1 " 2>&1 | grep -o -m 1 'READONLY' | head -1 + +${CLICKHOUSE_CLIENT} -q $"DROP USER IF EXISTS '${USER}'" +${CLICKHOUSE_CLIENT} -q $"DROP USER IF EXISTS '${USER2}'" +${CLICKHOUSE_CLIENT} -q $"DROP PROFILE IF EXISTS ${PROFILE}" +${CLICKHOUSE_CLIENT} -q $"DROP ROLE IF EXISTS ${ROLE}" From 6d971bc3a8985e8ddbaf243a01d1a68b872d5322 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 30 Jul 2023 22:35:19 +0000 Subject: [PATCH 1344/2047] Specific check for NaN in window functions with RANGE --- src/Interpreters/WindowDescription.cpp | 58 ++++++++++++------- .../02833_window_func_range_offset.reference | 9 +++ .../02833_window_func_range_offset.sql | 5 +- 3 files changed, 49 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/WindowDescription.cpp b/src/Interpreters/WindowDescription.cpp index 8a7a5024d69..702a042e74e 100644 --- a/src/Interpreters/WindowDescription.cpp +++ b/src/Interpreters/WindowDescription.cpp @@ -91,30 +91,46 @@ void WindowFrame::toString(WriteBuffer & buf) const void WindowFrame::checkValid() const { // Check the validity of offsets. - if (begin_type == BoundaryType::Offset - && !((begin_offset.getType() == Field::Types::UInt64 - || begin_offset.getType() == Field::Types::Int64) - && begin_offset.get() >= 0 - && begin_offset.get() < INT_MAX)) + if (type == WindowFrame::FrameType::ROWS + || type == WindowFrame::FrameType::GROUPS) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", - type, - applyVisitor(FieldVisitorToString(), begin_offset), - begin_offset.getType()); - } + if (begin_type == BoundaryType::Offset + && !((begin_offset.getType() == Field::Types::UInt64 + || begin_offset.getType() == Field::Types::Int64) + && begin_offset.get() >= 0 + && begin_offset.get() < INT_MAX)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", + type, + applyVisitor(FieldVisitorToString(), begin_offset), + begin_offset.getType()); + } - if (end_type == BoundaryType::Offset - && !((end_offset.getType() == Field::Types::UInt64 - || end_offset.getType() == Field::Types::Int64) - && end_offset.get() >= 0 - && end_offset.get() < INT_MAX)) + if (end_type == BoundaryType::Offset + && !((end_offset.getType() == Field::Types::UInt64 + || end_offset.getType() == Field::Types::Int64) + && end_offset.get() >= 0 + && end_offset.get() < INT_MAX)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", + type, + applyVisitor(FieldVisitorToString(), end_offset), + end_offset.getType()); + } + } + else if (type == WindowFrame::FrameType::RANGE) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", - type, - applyVisitor(FieldVisitorToString(), end_offset), - end_offset.getType()); + if (begin_type == BoundaryType::Offset && begin_offset.getType() == Field::Types::Float64 && isNaN(begin_offset.get())) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Frame start offset for '{}' frame cannot be NaN for Floats", type); + } + + if (end_type == BoundaryType::Offset && end_offset.getType() == Field::Types::Float64 && isNaN(end_offset.get())) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Frame end offset for '{}' frame cannot be NaN for Floats", type); + } } // Check relative positioning of offsets. diff --git a/tests/queries/0_stateless/02833_window_func_range_offset.reference b/tests/queries/0_stateless/02833_window_func_range_offset.reference index e69de29bb2d..cf254aa2024 100644 --- a/tests/queries/0_stateless/02833_window_func_range_offset.reference +++ b/tests/queries/0_stateless/02833_window_func_range_offset.reference @@ -0,0 +1,9 @@ +-- { echoOn } +-- invalid start offset with RANGE +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN 0.0 PRECEDING AND UNBOUNDED FOLLOWING); +1 +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN nan PRECEDING AND UNBOUNDED FOLLOWING); -- { serverError BAD_ARGUMENTS } +-- invalid end offset with RANGE +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.0 FOLLOWING); +1 +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND nan FOLLOWING); -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/02833_window_func_range_offset.sql b/tests/queries/0_stateless/02833_window_func_range_offset.sql index f1d26c5cbaf..1c75543b3f1 100644 --- a/tests/queries/0_stateless/02833_window_func_range_offset.sql +++ b/tests/queries/0_stateless/02833_window_func_range_offset.sql @@ -1,6 +1,7 @@ +-- { echoOn } -- invalid start offset with RANGE -SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN 0.0 PRECEDING AND UNBOUNDED FOLLOWING); -- { serverError BAD_ARGUMENTS } +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN 0.0 PRECEDING AND UNBOUNDED FOLLOWING); SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN nan PRECEDING AND UNBOUNDED FOLLOWING); -- { serverError BAD_ARGUMENTS } -- invalid end offset with RANGE -SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.0 FOLLOWING); -- { serverError BAD_ARGUMENTS } +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.0 FOLLOWING); SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND nan FOLLOWING); -- { serverError BAD_ARGUMENTS } From 3d7257cc7e2de2457ec7a2391bb2751c415ed125 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 29 Jul 2023 22:04:21 +0000 Subject: [PATCH 1345/2047] Allow Floats as boundaries for RANGE is nonsense --- src/Interpreters/WindowDescription.cpp | 58 +++++++------------ .../0_stateless/01591_window_functions.sql | 8 +-- .../02833_window_func_range_offset.reference | 9 --- .../02833_window_func_range_offset.sql | 5 +- 4 files changed, 27 insertions(+), 53 deletions(-) diff --git a/src/Interpreters/WindowDescription.cpp b/src/Interpreters/WindowDescription.cpp index 702a042e74e..8a7a5024d69 100644 --- a/src/Interpreters/WindowDescription.cpp +++ b/src/Interpreters/WindowDescription.cpp @@ -91,46 +91,30 @@ void WindowFrame::toString(WriteBuffer & buf) const void WindowFrame::checkValid() const { // Check the validity of offsets. - if (type == WindowFrame::FrameType::ROWS - || type == WindowFrame::FrameType::GROUPS) + if (begin_type == BoundaryType::Offset + && !((begin_offset.getType() == Field::Types::UInt64 + || begin_offset.getType() == Field::Types::Int64) + && begin_offset.get() >= 0 + && begin_offset.get() < INT_MAX)) { - if (begin_type == BoundaryType::Offset - && !((begin_offset.getType() == Field::Types::UInt64 - || begin_offset.getType() == Field::Types::Int64) - && begin_offset.get() >= 0 - && begin_offset.get() < INT_MAX)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", - type, - applyVisitor(FieldVisitorToString(), begin_offset), - begin_offset.getType()); - } - - if (end_type == BoundaryType::Offset - && !((end_offset.getType() == Field::Types::UInt64 - || end_offset.getType() == Field::Types::Int64) - && end_offset.get() >= 0 - && end_offset.get() < INT_MAX)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", - type, - applyVisitor(FieldVisitorToString(), end_offset), - end_offset.getType()); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", + type, + applyVisitor(FieldVisitorToString(), begin_offset), + begin_offset.getType()); } - else if (type == WindowFrame::FrameType::RANGE) - { - if (begin_type == BoundaryType::Offset && begin_offset.getType() == Field::Types::Float64 && isNaN(begin_offset.get())) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Frame start offset for '{}' frame cannot be NaN for Floats", type); - } - if (end_type == BoundaryType::Offset && end_offset.getType() == Field::Types::Float64 && isNaN(end_offset.get())) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Frame end offset for '{}' frame cannot be NaN for Floats", type); - } + if (end_type == BoundaryType::Offset + && !((end_offset.getType() == Field::Types::UInt64 + || end_offset.getType() == Field::Types::Int64) + && end_offset.get() >= 0 + && end_offset.get() < INT_MAX)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", + type, + applyVisitor(FieldVisitorToString(), end_offset), + end_offset.getType()); } // Check relative positioning of offsets. diff --git a/tests/queries/0_stateless/01591_window_functions.sql b/tests/queries/0_stateless/01591_window_functions.sql index 3c9c1f9cea7..07e323b3c40 100644 --- a/tests/queries/0_stateless/01591_window_functions.sql +++ b/tests/queries/0_stateless/01591_window_functions.sql @@ -474,10 +474,10 @@ select count() over () from numbers(4) where number < 2; -- floating point RANGE frame select - count(*) over (order by toFloat32(number) range 5. preceding), - count(*) over (order by toFloat64(number) range 5. preceding), - count(*) over (order by toFloat32(number) range between current row and 5. following), - count(*) over (order by toFloat64(number) range between current row and 5. following) + count(*) over (order by toFloat32(number) range 5 preceding), + count(*) over (order by toFloat64(number) range 5 preceding), + count(*) over (order by toFloat32(number) range between current row and 5 following), + count(*) over (order by toFloat64(number) range between current row and 5 following) from numbers(7) ; diff --git a/tests/queries/0_stateless/02833_window_func_range_offset.reference b/tests/queries/0_stateless/02833_window_func_range_offset.reference index cf254aa2024..e69de29bb2d 100644 --- a/tests/queries/0_stateless/02833_window_func_range_offset.reference +++ b/tests/queries/0_stateless/02833_window_func_range_offset.reference @@ -1,9 +0,0 @@ --- { echoOn } --- invalid start offset with RANGE -SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN 0.0 PRECEDING AND UNBOUNDED FOLLOWING); -1 -SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN nan PRECEDING AND UNBOUNDED FOLLOWING); -- { serverError BAD_ARGUMENTS } --- invalid end offset with RANGE -SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.0 FOLLOWING); -1 -SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND nan FOLLOWING); -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/02833_window_func_range_offset.sql b/tests/queries/0_stateless/02833_window_func_range_offset.sql index 1c75543b3f1..f1d26c5cbaf 100644 --- a/tests/queries/0_stateless/02833_window_func_range_offset.sql +++ b/tests/queries/0_stateless/02833_window_func_range_offset.sql @@ -1,7 +1,6 @@ --- { echoOn } -- invalid start offset with RANGE -SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN 0.0 PRECEDING AND UNBOUNDED FOLLOWING); +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN 0.0 PRECEDING AND UNBOUNDED FOLLOWING); -- { serverError BAD_ARGUMENTS } SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN nan PRECEDING AND UNBOUNDED FOLLOWING); -- { serverError BAD_ARGUMENTS } -- invalid end offset with RANGE -SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.0 FOLLOWING); +SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND 0.0 FOLLOWING); -- { serverError BAD_ARGUMENTS } SELECT count() OVER (ORDER BY 3.4028234663852886e38 RANGE BETWEEN UNBOUNDED PRECEDING AND nan FOLLOWING); -- { serverError BAD_ARGUMENTS } From afb76101b797a144bc2e8d6cff5109c9036fc46a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 Jul 2023 05:23:28 +0200 Subject: [PATCH 1346/2047] Remove obsolete part of a check name --- .github/workflows/master.yml | 2 +- .github/workflows/pull_request.yml | 2 +- tests/ci/ci_config.py | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index e5b797beebd..ae1862e327f 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -3643,7 +3643,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (release-clang) + CHECK_NAME=Unit tests (release) REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index dd834959578..d97b9975c3c 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -4541,7 +4541,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/unit_tests_asan REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (release-clang) + CHECK_NAME=Unit tests (release) REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index bea654ca76f..517e40fd2d6 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -346,7 +346,7 @@ CI_CONFIG = { "Compatibility check (aarch64)": { "required_build": "package_aarch64", }, - "Unit tests (release-clang)": { + "Unit tests (release)": { "required_build": "binary_release", }, "Unit tests (asan)": { @@ -509,7 +509,7 @@ REQUIRED_CHECKS = [ "Style Check", "Unit tests (asan)", "Unit tests (msan)", - "Unit tests (release-clang)", + "Unit tests (release)", "Unit tests (tsan)", "Unit tests (ubsan)", ] From fdfe64f1b0330522180811b83e8db8436839c829 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 31 Jul 2023 12:07:50 +0800 Subject: [PATCH 1347/2047] change as request --- .../functions/string-functions.md | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index b1ffb32e7cd..fff37257917 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -706,7 +706,15 @@ endsWithUTF8(str, suffix) **Example** ``` sql -SELECT endsWithUTF8('中国', '国'); +SELECT endsWithUTF8('中国', '\xbd'), endsWith('中国', '\xbd') +``` + +Result: + +```result +┌─endsWithUTF8('中国', '½')─┬─endsWith('中国', '½')─┐ +│ 0 │ 1 │ +└──────────────────────────┴──────────────────────┘ ``` ## startsWith @@ -733,9 +741,16 @@ Returns whether string `str` starts with `prefix`, the difference between `start **Example** ``` sql -SELECT startsWithUTF8('中国', '中'); +SELECT startsWithUTF8('中国', '\xe4'), startsWith('中国', '\xe4') ``` +Result: + +```result +┌─startsWithUTF8('中国', '⥩─┬─startsWith('中国', '⥩─┐ +│ 0 │ 1 │ +└────────────────────────────┴────────────────────────┘ +``` ## trim From efad90d0f2731142677287495f4d8f47a4c32b51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 Jul 2023 07:26:36 +0200 Subject: [PATCH 1348/2047] Maybe fix TLS tests --- tests/integration/test_ssl_cert_authentication/test.py | 6 +++--- tests/integration/test_tlsv1_3/test.py | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index ff2de7491e1..fe6eb52e50e 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -208,7 +208,7 @@ def test_https_wrong_cert(): with pytest.raises(Exception) as err: execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning(f"Failed attempt with wrong cert, err: {err_str}") continue @@ -314,7 +314,7 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: peter, err: {err_str}" @@ -334,7 +334,7 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: jane, err: {err_str}" diff --git a/tests/integration/test_tlsv1_3/test.py b/tests/integration/test_tlsv1_3/test.py index f5c2be51ed7..094804bf963 100644 --- a/tests/integration/test_tlsv1_3/test.py +++ b/tests/integration/test_tlsv1_3/test.py @@ -96,7 +96,7 @@ def test_https_wrong_cert(): with pytest.raises(Exception) as err: execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning(f"Failed attempt with wrong cert, err: {err_str}") continue @@ -202,7 +202,7 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: peter, err: {err_str}" @@ -222,7 +222,7 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and "Broken pipe" in err_str: + if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: jane, err: {err_str}" From 010f3f1db1fc8adad0a74f424e153c28f7072e16 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 31 Jul 2023 05:39:49 +0000 Subject: [PATCH 1349/2047] Automatic style fix --- .../integration/test_ssl_cert_authentication/test.py | 12 +++++++++--- tests/integration/test_tlsv1_3/test.py | 12 +++++++++--- 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index fe6eb52e50e..d1ae39ca378 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -208,7 +208,9 @@ def test_https_wrong_cert(): with pytest.raises(Exception) as err: execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning(f"Failed attempt with wrong cert, err: {err_str}") continue @@ -314,7 +316,9 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: peter, err: {err_str}" @@ -334,7 +338,9 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: jane, err: {err_str}" diff --git a/tests/integration/test_tlsv1_3/test.py b/tests/integration/test_tlsv1_3/test.py index 094804bf963..87c03c56f91 100644 --- a/tests/integration/test_tlsv1_3/test.py +++ b/tests/integration/test_tlsv1_3/test.py @@ -96,7 +96,9 @@ def test_https_wrong_cert(): with pytest.raises(Exception) as err: execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning(f"Failed attempt with wrong cert, err: {err_str}") continue @@ -202,7 +204,9 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: peter, err: {err_str}" @@ -222,7 +226,9 @@ def test_https_non_ssl_auth(): cert_name="wrong", ) err_str = str(err.value) - if count < MAX_RETRY and (("Broken pipe" in err_str) or ("EOF occurred" in err_str)): + if count < MAX_RETRY and ( + ("Broken pipe" in err_str) or ("EOF occurred" in err_str) + ): count = count + 1 logging.warning( f"Failed attempt with wrong cert, user: jane, err: {err_str}" From 36b5ccfa3ad355af55ee2a6fa1b82bc631af1289 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 31 Jul 2023 10:03:44 +0200 Subject: [PATCH 1350/2047] update tests --- .../queries/0_stateless/02813_float_parsing.reference | 1 + tests/queries/0_stateless/02813_float_parsing.sql | 11 +++++++++++ 2 files changed, 12 insertions(+) diff --git a/tests/queries/0_stateless/02813_float_parsing.reference b/tests/queries/0_stateless/02813_float_parsing.reference index ed4797f7a37..c83331e0138 100644 --- a/tests/queries/0_stateless/02813_float_parsing.reference +++ b/tests/queries/0_stateless/02813_float_parsing.reference @@ -1 +1,2 @@ +1.7090999999999998 15008753.000000002 6.000000000000001e-9 6.000000000000002e-9 1.7091 15008752 5.9999996e-9 5.9999996e-9 1.7091 15008753 6e-9 6.000000000000001e-9 1.7091 15008753 6e-9 6e-9 diff --git a/tests/queries/0_stateless/02813_float_parsing.sql b/tests/queries/0_stateless/02813_float_parsing.sql index ab74f1e64e4..ba57b87f191 100644 --- a/tests/queries/0_stateless/02813_float_parsing.sql +++ b/tests/queries/0_stateless/02813_float_parsing.sql @@ -1,3 +1,14 @@ +SELECT + toFloat64('1.7091'), + toFloat64('1.5008753E7'), + toFloat64('6e-09'), + toFloat64('6.000000000000001e-9'), + toFloat32('1.7091'), + toFloat32('1.5008753E7'), + toFloat32('6e-09'), + toFloat32('6.000000000000001e-9') +SETTINGS precise_float_parsing = 0; + SELECT toFloat64('1.7091'), toFloat64('1.5008753E7'), From 987dd6602ab1ccf7e3af017f59d275c241406138 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Mon, 31 Jul 2023 11:08:51 +0300 Subject: [PATCH 1351/2047] Update docs/en/operations/settings/settings.md Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 886177b253b..345257ac6ed 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4573,7 +4573,7 @@ Default: 0 Switches [Float32/Float64](../../sql-reference/data-types/float.md) parsing algorithms: * If the value is `1`, then precise method is used. It is slower than fast method, but it always returns a number that is the closest machine representable number to the input. -* Otherwise, fast method is used (default). It usually returns the same value as precise, but in rare cases result may differ by one or two least significant bits. +* Otherwise, fast method is used (default). It usually returns the same value as precise, but in rare cases result may differ by one or two least significant digits. Possible values: `0`, `1`. From cb5bebf4e4582a1c581128d5087393ce37e3848a Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Mon, 31 Jul 2023 11:09:01 +0300 Subject: [PATCH 1352/2047] Update docs/ru/operations/settings/settings.md Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index bb84c4225f7..87a7aa3b0d4 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -4217,7 +4217,7 @@ SELECT *, timezone() FROM test_tz WHERE d = '2000-01-01 00:00:00' SETTINGS sessi Позволяет выбрать алгоритм, используемый при парсинге [Float32/Float64](../../sql-reference/data-types/float.md): * Если установлено значение `1`, то используется точный метод. Он более медленный, но всегда возвращает число, наиболее близкое к входному значению. -* В противном случае используется быстрый метод (поведение по умолчанию). Обычно результат его работы совпадает с результатом, полученным точным методом, однако в редких случаях он может отличаться на 1 или 2 наименее значимых бит. +* В противном случае используется быстрый метод (поведение по умолчанию). Обычно результат его работы совпадает с результатом, полученным точным методом, однако в редких случаях он может отличаться на 1 или 2 наименее значимых цифры. Возможные значения: `0`, `1`. From 64f11d4853ebadc50e198612071d00aedcf6f79c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 31 Jul 2023 08:30:55 +0000 Subject: [PATCH 1353/2047] Update reference file --- .../queries/0_stateless/01591_window_functions.reference | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index 8939ea1111d..ce9c6f4589e 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -1193,10 +1193,10 @@ select count() over () from numbers(4) where number < 2; 2 -- floating point RANGE frame select - count(*) over (order by toFloat32(number) range 5. preceding), - count(*) over (order by toFloat64(number) range 5. preceding), - count(*) over (order by toFloat32(number) range between current row and 5. following), - count(*) over (order by toFloat64(number) range between current row and 5. following) + count(*) over (order by toFloat32(number) range 5 preceding), + count(*) over (order by toFloat64(number) range 5 preceding), + count(*) over (order by toFloat32(number) range between current row and 5 following), + count(*) over (order by toFloat64(number) range between current row and 5 following) from numbers(7) ; 1 1 6 6 From 2e0d82765c2dbb4e021c9f174eaba8637f74401f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 Jul 2023 11:43:04 +0300 Subject: [PATCH 1354/2047] Move "reconfig" to experimental in the changelog See https://github.com/ClickHouse/ClickHouse/issues/52798 Automatic continuous fuzzing has found an issue. --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f401b346726..bf26708ebb0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,7 +23,6 @@ * Added `Overlay` database engine to combine multiple databases into one. Added `Filesystem` database engine to represent a directory in the filesystem as a set of implicitly available tables with auto-detected formats and structures. A new `S3` database engine allows to read-only interact with s3 storage by representing a prefix as a set of tables. A new `HDFS` database engine allows to interact with HDFS storage in the same way. [#48821](https://github.com/ClickHouse/ClickHouse/pull/48821) ([alekseygolub](https://github.com/alekseygolub)). * Add support for external disks in Keeper for storing snapshots and logs. [#50098](https://github.com/ClickHouse/ClickHouse/pull/50098) ([Antonio Andelic](https://github.com/antonio2368)). * Add support for multi-directory selection (`{}`) globs. [#50559](https://github.com/ClickHouse/ClickHouse/pull/50559) ([Andrey Zvonov](https://github.com/zvonand)). -* Support ZooKeeper `reconfig` command for ClickHouse Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). * Kafka connector can fetch Avro schema from schema registry with basic authentication using url-encoded credentials. [#49664](https://github.com/ClickHouse/ClickHouse/pull/49664) ([Ilya Golshtein](https://github.com/ilejn)). * Add function `arrayJaccardIndex` which computes the Jaccard similarity between two arrays. [#50076](https://github.com/ClickHouse/ClickHouse/pull/50076) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). * Add a column `is_obsolete` to `system.settings` and similar tables. Closes [#50819](https://github.com/ClickHouse/ClickHouse/issues/50819). [#50826](https://github.com/ClickHouse/ClickHouse/pull/50826) ([flynn](https://github.com/ucasfl)). @@ -124,6 +123,7 @@ * (experimental MaterializedMySQL) Now double quoted comments are supported in MaterializedMySQL. [#52355](https://github.com/ClickHouse/ClickHouse/pull/52355) ([Val Doroshchuk](https://github.com/valbok)). * Upgrade Intel QPL from v1.1.0 to v1.2.0 2. Upgrade Intel accel-config from v3.5 to v4.0 3. Fixed issue that Device IOTLB miss has big perf. impact for IAA accelerators. [#52180](https://github.com/ClickHouse/ClickHouse/pull/52180) ([jasperzhu](https://github.com/jinjunzh)). * The `session_timezone` setting (new in version 23.6) is demoted to experimental. [#52445](https://github.com/ClickHouse/ClickHouse/pull/52445) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Support ZooKeeper `reconfig` command for ClickHouse Keeper with incremental reconfiguration which can be enabled via `keeper_server.enable_reconfiguration` setting. Support adding servers, removing servers, and changing server priorities. [#49450](https://github.com/ClickHouse/ClickHouse/pull/49450) ([Mike Kot](https://github.com/myrrc)). It is suspected that this feature is incomplete. #### Build/Testing/Packaging Improvement * Add experimental ClickHouse builds for Linux RISC-V 64 to CI. [#31398](https://github.com/ClickHouse/ClickHouse/pull/31398) ([Alexey Milovidov](https://github.com/alexey-milovidov)). From df456021651b7d9c4b4458a15060a24bbfee9d73 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 08:50:09 +0000 Subject: [PATCH 1355/2047] Refactoring --- ...ArchiveReader.cpp => LibArchiveReader.cpp} | 63 ++++-- ...{TarArchiveReader.h => LibArchiveReader.h} | 19 +- src/IO/Archives/SevenZipArchiveReader.cpp | 188 ------------------ src/IO/Archives/SevenZipArchiveReader.h | 63 ------ src/IO/Archives/createArchiveReader.cpp | 3 +- src/Storages/StorageFile.cpp | 16 +- src/TableFunctions/ITableFunctionFileLike.cpp | 21 +- src/TableFunctions/TableFunctionFile.cpp | 10 +- .../02661_select_from_table_in_archive.sh | 17 +- 9 files changed, 100 insertions(+), 300 deletions(-) rename src/IO/Archives/{TarArchiveReader.cpp => LibArchiveReader.cpp} (65%) rename src/IO/Archives/{TarArchiveReader.h => LibArchiveReader.h} (72%) delete mode 100644 src/IO/Archives/SevenZipArchiveReader.cpp delete mode 100644 src/IO/Archives/SevenZipArchiveReader.h diff --git a/src/IO/Archives/TarArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp similarity index 65% rename from src/IO/Archives/TarArchiveReader.cpp rename to src/IO/Archives/LibArchiveReader.cpp index 5d88e0e5cd4..54a228858ee 100644 --- a/src/IO/Archives/TarArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -1,9 +1,10 @@ -#include +#include +#include #include -#include #include + namespace DB { @@ -14,8 +15,12 @@ namespace ErrorCodes extern const int CANNOT_UNPACK_ARCHIVE; extern const int LOGICAL_ERROR; extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int NOT_IMPLEMENTED; } -class TarArchiveReader::Handle + + +template +class LibArchiveReader::Handle { public: explicit Handle(const String & path_to_archive_) : path_to_archive(path_to_archive_) @@ -25,10 +30,11 @@ public: archive_read_support_format_all(archive); if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) { - throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open tar archive {}", quoteString(path_to_archive)); + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open {} archive: {}", ArchiveInfo::name, quoteString(path_to_archive)); } entry = archive_entry_new(); } + ~Handle() { archive_read_close(archive); @@ -52,10 +58,11 @@ private: const String path_to_archive; }; -class TarArchiveReader::ReadBufferFromTarArchive : public ReadBufferFromFileBase +template +class LibArchiveReader::ReadBufferFromLibArchive : public ReadBufferFromFileBase { public: - explicit ReadBufferFromTarArchive(const String & path_to_archive_, const String & filename_) + explicit ReadBufferFromLibArchive(const String & path_to_archive_, const String & filename_) : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0) , handle(path_to_archive_) , path_to_archive(path_to_archive_) @@ -121,24 +128,29 @@ private: const String filename; }; -TarArchiveReader::TarArchiveReader(const String & path_to_archive_) : path_to_archive(path_to_archive_) +template +LibArchiveReader::LibArchiveReader(const String & path_to_archive_) : path_to_archive(path_to_archive_) { } -TarArchiveReader::TarArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_) +template +LibArchiveReader::LibArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_) : path_to_archive(path_to_archive_), archive_read_function(archive_read_function_) { } -TarArchiveReader::~TarArchiveReader() = default; +template +LibArchiveReader::~LibArchiveReader() = default; -bool TarArchiveReader::fileExists(const String & filename) +template +bool LibArchiveReader::fileExists(const String & filename) { Handle handle(path_to_archive); return handle.locateFile(filename); } -TarArchiveReader::FileInfo TarArchiveReader::getFileInfo(const String & filename) +template +LibArchiveReader::FileInfo LibArchiveReader::getFileInfo(const String & filename) { Handle handle(path_to_archive); @@ -151,35 +163,44 @@ TarArchiveReader::FileInfo TarArchiveReader::getFileInfo(const String & filename return info; } -std::unique_ptr TarArchiveReader::firstFile() +template +std::unique_ptr::FileEnumerator> LibArchiveReader::firstFile() { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for tar archives"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for {} archives", ArchiveInfo::name); } -std::unique_ptr TarArchiveReader::readFile(const String & filename) +template +std::unique_ptr LibArchiveReader::readFile(const String & filename) { Handle handle(path_to_archive); handle.locateFile(filename); - return std::make_unique(path_to_archive, filename); + return std::make_unique(path_to_archive, filename); } -std::unique_ptr TarArchiveReader::readFile(std::unique_ptr /*enumerator*/) +template +std::unique_ptr LibArchiveReader::readFile(std::unique_ptr /*enumerator*/) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for tar archives"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for {} archives", ArchiveInfo::name); } -std::unique_ptr TarArchiveReader::nextFile(std::unique_ptr /*read_buffer*/) +template +std::unique_ptr::FileEnumerator> +LibArchiveReader::nextFile(std::unique_ptr /*read_buffer*/) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for tar archives"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for {} archives", ArchiveInfo::name); } -void TarArchiveReader::setPassword(const String & /*password_*/) +template +void LibArchiveReader::setPassword(const String & /*password_*/) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to tar archive"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to {} archive", ArchiveInfo::name); } +template class LibArchiveReader; +template class LibArchiveReader; + #endif } diff --git a/src/IO/Archives/TarArchiveReader.h b/src/IO/Archives/LibArchiveReader.h similarity index 72% rename from src/IO/Archives/TarArchiveReader.h rename to src/IO/Archives/LibArchiveReader.h index 8968a2b53dc..754df21c3ff 100644 --- a/src/IO/Archives/TarArchiveReader.h +++ b/src/IO/Archives/LibArchiveReader.h @@ -4,6 +4,7 @@ #include +#include namespace DB { @@ -14,18 +15,19 @@ class ReadBuffer; class ReadBufferFromFileBase; class SeekableReadBuffer; -/// Implementation of IArchiveReader for reading tar archives. -class TarArchiveReader : public IArchiveReader +/// Implementation of IArchiveReader for reading archives using libarchive. +template +class LibArchiveReader : public IArchiveReader { public: /// Constructs an archive's reader that will read from a file in the local filesystem. - explicit TarArchiveReader(const String & path_to_archive_); + explicit LibArchiveReader(const String & path_to_archive_); /// Constructs an archive's reader that will read by making a read buffer by using /// a specified function. - TarArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_); + LibArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_); - ~TarArchiveReader() override; + ~LibArchiveReader() override; /// Returns true if there is a specified file in the archive. bool fileExists(const String & filename) override; @@ -49,13 +51,18 @@ public: void setPassword(const String & password_) override; private: - class ReadBufferFromTarArchive; + class ReadBufferFromLibArchive; class Handle; const String path_to_archive; const ReadArchiveFunction archive_read_function; }; +struct TarArchiveInfo { static constexpr std::string_view name = "tar"; }; +using TarArchiveReader = LibArchiveReader; +struct SevenZipArchiveInfo { static constexpr std::string_view name = "7z"; }; +using SevenZipArchiveReader = LibArchiveReader; + #endif } diff --git a/src/IO/Archives/SevenZipArchiveReader.cpp b/src/IO/Archives/SevenZipArchiveReader.cpp deleted file mode 100644 index 2daef777e94..00000000000 --- a/src/IO/Archives/SevenZipArchiveReader.cpp +++ /dev/null @@ -1,188 +0,0 @@ -#include -#include -#include - -#include - - -namespace DB -{ - -#if USE_LIBARCHIVE - -namespace ErrorCodes -{ - extern const int CANNOT_UNPACK_ARCHIVE; - extern const int LOGICAL_ERROR; - extern const int SEEK_POSITION_OUT_OF_BOUND; - extern const int NOT_IMPLEMENTED; -} -class SevenZipArchiveReader::Handle -{ -public: - explicit Handle(const String & path_to_archive_) : path_to_archive(path_to_archive_) - { - archive = archive_read_new(); - archive_read_support_filter_all(archive); - archive_read_support_format_all(archive); - if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) - { - throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open 7z archive {}", quoteString(path_to_archive)); - } - entry = archive_entry_new(); - } - ~Handle() - { - archive_read_close(archive); - archive_read_free(archive); - } - - bool locateFile(const String & filename) - { - while (archive_read_next_header(archive, &entry) == ARCHIVE_OK) - { - if (archive_entry_pathname(entry) == filename) - return true; - } - return false; - } - - struct archive * archive; - struct archive_entry * entry; - -private: - const String path_to_archive; -}; - -class SevenZipArchiveReader::ReadBufferFromSevenZipArchive : public ReadBufferFromFileBase -{ -public: - explicit ReadBufferFromSevenZipArchive(const String & path_to_archive_, const String & filename_) - : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0) - , handle(path_to_archive_) - , path_to_archive(path_to_archive_) - , filename(filename_) - { - handle.locateFile(filename_); - } - - off_t seek(off_t off, int whence) override - { - off_t current_pos = getPosition(); - off_t new_pos; - if (whence == SEEK_SET) - new_pos = off; - else if (whence == SEEK_CUR) - new_pos = off + current_pos; - else - throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Only SEEK_SET and SEEK_CUR seek modes allowed."); - - if (new_pos == current_pos) - return current_pos; /// The position is the same. - - if (new_pos < 0) - throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); - - off_t working_buffer_start_pos = current_pos - offset(); - off_t working_buffer_end_pos = current_pos + available(); - - if ((working_buffer_start_pos <= new_pos) && (new_pos <= working_buffer_end_pos)) - { - /// The new position is still inside the buffer. - position() += new_pos - current_pos; - return new_pos; - } - - /// Check that the new position is now beyond the end of the file. - if (new_pos > archive_entry_size(handle.entry)) - throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); - - ignore(new_pos - current_pos); - return new_pos; - } - - off_t getPosition() override { return archive_entry_size(handle.entry) - available(); } - - String getFileName() const override { return filename; } - - -private: - bool nextImpl() override - { - auto bytes_read = archive_read_data(handle.archive, internal_buffer.begin(), static_cast(internal_buffer.size())); - - if (!bytes_read) - return false; - - working_buffer = internal_buffer; - working_buffer.resize(bytes_read); - return true; - } - Handle handle; - const String path_to_archive; - const String filename; -}; - -SevenZipArchiveReader::SevenZipArchiveReader(const String & path_to_archive_) : path_to_archive(path_to_archive_) -{ -} - -SevenZipArchiveReader::SevenZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_) - : path_to_archive(path_to_archive_), archive_read_function(archive_read_function_) -{ -} - -SevenZipArchiveReader::~SevenZipArchiveReader() = default; - -bool SevenZipArchiveReader::fileExists(const String & filename) -{ - Handle handle(path_to_archive); - return handle.locateFile(filename); -} - -SevenZipArchiveReader::FileInfo SevenZipArchiveReader::getFileInfo(const String & filename) -{ - Handle handle(path_to_archive); - - handle.locateFile(filename); - FileInfo info; - info.uncompressed_size = archive_entry_size(handle.entry); - info.compressed_size = archive_entry_size(handle.entry); - info.is_encrypted = false; - - return info; -} - -std::unique_ptr SevenZipArchiveReader::firstFile() -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for 7z archives"); -} - -std::unique_ptr SevenZipArchiveReader::readFile(const String & filename) -{ - Handle handle(path_to_archive); - handle.locateFile(filename); - - return std::make_unique(path_to_archive, filename); -} - -std::unique_ptr SevenZipArchiveReader::readFile(std::unique_ptr /*enumerator*/) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for 7z archives"); -} - -std::unique_ptr -SevenZipArchiveReader::nextFile(std::unique_ptr /*read_buffer*/) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for 7z archives"); -} - - -void SevenZipArchiveReader::setPassword(const String & /*password_*/) -{ - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to 7z archive"); -} - -#endif - -} diff --git a/src/IO/Archives/SevenZipArchiveReader.h b/src/IO/Archives/SevenZipArchiveReader.h deleted file mode 100644 index de7150f5a10..00000000000 --- a/src/IO/Archives/SevenZipArchiveReader.h +++ /dev/null @@ -1,63 +0,0 @@ -#pragma once - -#include "config.h" - -#include - -#include - -namespace DB -{ - -#if USE_LIBARCHIVE - -class ReadBuffer; -class ReadBufferFromFileBase; -class SeekableReadBuffer; - -/// Implementation of IArchiveReader for reading SevenZip archives. -class SevenZipArchiveReader : public IArchiveReader -{ -public: - /// Constructs an archive's reader that will read from a file in the local filesystem. - explicit SevenZipArchiveReader(const String & path_to_archive_); - - /// Constructs an archive's reader that will read by making a read buffer by using - /// a specified function. - SevenZipArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_); - - ~SevenZipArchiveReader() override; - - /// Returns true if there is a specified file in the archive. - bool fileExists(const String & filename) override; - - /// Returns the information about a file stored in the archive. - FileInfo getFileInfo(const String & filename) override; - - /// Starts enumerating files in the archive. - std::unique_ptr firstFile() override; - - /// Starts reading a file from the archive. The function returns a read buffer, - /// you can read that buffer to extract uncompressed data from the archive. - /// Several read buffers can be used at the same time in parallel. - std::unique_ptr readFile(const String & filename) override; - - /// It's possible to convert a file enumerator to a read buffer and vice versa. - std::unique_ptr readFile(std::unique_ptr enumerator) override; - std::unique_ptr nextFile(std::unique_ptr read_buffer) override; - - /// Sets password used to decrypt the contents of the files in the archive. - void setPassword(const String & password_) override; - -private: - class ReadBufferFromSevenZipArchive; - class Handle; - - const String path_to_archive; - String password; - const ReadArchiveFunction archive_read_function; -}; - -#endif - -} diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index b84f69ceb41..d33af227975 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -1,7 +1,6 @@ #include #include -#include -#include +#include #include diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 30905f20aeb..54ab2555b4e 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -530,15 +530,14 @@ ColumnsDescription StorageFile::getTableStructureFromFile( } else { - read_buffer_iterator = [&, it = paths_to_archive.begin()](ColumnsDescription &) mutable -> std::unique_ptr + read_buffer_iterator = [&, path_it = paths.begin(), archive_it = paths_to_archive.begin()](ColumnsDescription &) mutable -> std::unique_ptr { - if (it == paths_to_archive.end()) + if (archive_it == paths_to_archive.end()) return nullptr; - const auto & path = *it; - auto file_stat = getFileStat(path, false, -1, "File"); + auto file_stat = getFileStat(*archive_it, false, -1, "File"); - return createReadBuffer(path, file_stat, false, -1, compression_method, context, *it); + return createReadBuffer(*path_it, file_stat, false, -1, compression_method, context, *archive_it); }; } @@ -851,7 +850,12 @@ public: if (!read_buf) { - auto file_stat = getFileStat(current_path, storage->use_table_fd, storage->table_fd, storage->getName()); + struct stat file_stat; + if (files_info->paths_to_archive.empty()) + file_stat = getFileStat(current_path, storage->use_table_fd, storage->table_fd, storage->getName()); + else + file_stat = getFileStat(current_archive_path, storage->use_table_fd, storage->table_fd, storage->getName()); + if (context->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) continue; diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index a8b81822eed..baab7f5143e 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -25,15 +25,30 @@ namespace ErrorCodes void ITableFunctionFileLike::parseFirstArguments(const ASTPtr & arg, const ContextPtr &) { String path = checkAndGetLiteralArgument(arg, "source"); - size_t pos = path.find(" :: "); + size_t pos = path.find("::"); if (pos == String::npos) { filename = path; } else { - path_to_archive = path.substr(0, pos); - filename = path.substr(pos + 4, path.size() - pos - 3); + std::string_view path_to_archive_view = std::string_view{path}.substr(0, pos); + while (path_to_archive_view.back() == ' ') + path_to_archive_view.remove_suffix(1); + + if (path_to_archive_view.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to archive is empty"); + + path_to_archive = path_to_archive_view; + + std::string_view filename_view = std::string_view{path}.substr(pos + 2); + while (filename_view.front() == ' ') + filename_view.remove_prefix(1); + + if (filename_view.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filename is empty"); + + filename = filename_view; } } diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 1992f06c398..1ba40e25bc0 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -91,8 +91,14 @@ ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context if (fd >= 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Schema inference is not supported for table function '{}' with file descriptor", getName()); size_t total_bytes_to_read = 0; - Strings paths = StorageFile::getPathsList(filename, context->getUserFilesPath(), context, total_bytes_to_read); - return StorageFile::getTableStructureFromFile(format, paths, compression_method, std::nullopt, context); + + Strings paths; + Strings paths_to_archives; + if (path_to_archive.empty()) + paths = StorageFile::getPathsList(filename, context->getUserFilesPath(), context, total_bytes_to_read); + else + paths_to_archives = StorageFile::getPathsList(path_to_archive, context->getUserFilesPath(), context, total_bytes_to_read); + return StorageFile::getTableStructureFromFile(format, paths, compression_method, std::nullopt, context, paths_to_archives); } diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh index f72bba719a5..a0e165989ae 100755 --- a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh +++ b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh @@ -1,17 +1,16 @@ #!/usr/bin/env bash -# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -echo -e "1,2\n3,4" > 02661_data.csv -zip archive1.zip 02661_data.csv > /dev/null -zip archive2.zip 02661_data.csv > /dev/null +echo -e "1,2\n3,4" >${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv +zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null +zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null -$CLICKHOUSE_LOCAL --query "SELECT * FROM file('archive1.zip :: 02661_data.csv')" -$CLICKHOUSE_LOCAL --query "SELECT c1 FROM file('archive{1..2}.zip :: 02661_data.csv')" +$CLICKHOUSE_LOCAL --query "SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv')" +$CLICKHOUSE_LOCAL --query "SELECT c1 FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1..2}.zip :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv')" -rm 02661_data.csv -rm archive1.zip -rm archive2.zip +rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv +rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip +rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip From a473dc4b51a62f8b44137a9b84c8dc09f6b97542 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 09:24:06 +0000 Subject: [PATCH 1356/2047] Wait for response --- src/Coordination/KeeperServer.cpp | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index a4c3d91e1c9..88b9f1cedb4 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -794,8 +794,14 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction & action) std::lock_guard _{server_write_mutex}; if (const auto * add = std::get_if(&action)) - return raft_instance->get_srv_config(add->id) != nullptr - || raft_instance->add_srv(static_cast(*add))->get_accepted(); + { + if (raft_instance->get_srv_config(add->id) != nullptr) + return true; + + auto resp = raft_instance->add_srv(static_cast(*add)); + resp->get(); + return resp->get_accepted(); + } else if (const auto * remove = std::get_if(&action)) { if (remove->id == raft_instance->get_leader()) @@ -807,8 +813,12 @@ bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction & action) return false; } - return raft_instance->get_srv_config(remove->id) == nullptr - || raft_instance->remove_srv(remove->id)->get_accepted(); + if (raft_instance->get_srv_config(remove->id) == nullptr) + return true; + + auto resp = raft_instance->remove_srv(remove->id); + resp->get(); + return resp->get_accepted(); } else if (const auto * update = std::get_if(&action)) { From d136f18c3084b4af14c0bbf9ff8ef5028edf6ecf Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Jul 2023 12:17:30 +0200 Subject: [PATCH 1357/2047] Fix --- src/Interpreters/Cache/FileCacheKey.cpp | 6 ++++++ .../0_stateless/02808_filesystem_cache_drop_query.reference | 1 + .../0_stateless/02808_filesystem_cache_drop_query.sh | 4 ++++ 3 files changed, 11 insertions(+) diff --git a/src/Interpreters/Cache/FileCacheKey.cpp b/src/Interpreters/Cache/FileCacheKey.cpp index 772fcd600bf..360a9df7ad8 100644 --- a/src/Interpreters/Cache/FileCacheKey.cpp +++ b/src/Interpreters/Cache/FileCacheKey.cpp @@ -7,6 +7,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} FileCacheKey::FileCacheKey(const std::string & path) : key(sipHash128(path.data(), path.size())) @@ -30,6 +34,8 @@ FileCacheKey FileCacheKey::random() FileCacheKey FileCacheKey::fromKeyString(const std::string & key_str) { + if (key_str.size() != 32) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid key: {}", key_str); return FileCacheKey(unhexUInt(key_str.data())); } diff --git a/tests/queries/0_stateless/02808_filesystem_cache_drop_query.reference b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.reference index d80fc78e03d..d2c8c83d997 100644 --- a/tests/queries/0_stateless/02808_filesystem_cache_drop_query.reference +++ b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.reference @@ -1,3 +1,4 @@ +OK 1 0 1 diff --git a/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh index 9d987d0ebf2..f5670f1261e 100755 --- a/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh +++ b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh @@ -22,6 +22,10 @@ query_id=$RANDOM $CLICKHOUSE_CLIENT --query_id "$query_id" --query "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1" +$CLICKHOUSE_CLIENT -nm --query """ +SYSTEM DROP FILESYSTEM CACHE '$disk_name' KEY kek; +""" 2>&1 | grep -q "Invalid key: kek" && echo "OK" || echo "FAIL" + ${CLICKHOUSE_CLIENT} -q " system flush logs" key=$($CLICKHOUSE_CLIENT -nm --query """ From 7c49105cd56395bde38f0d7aff862c65c78c6989 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 Jul 2023 12:27:35 +0200 Subject: [PATCH 1358/2047] Allow OOM in Stress and Upgrade checks --- docker/test/stress/run.sh | 6 ++++++ docker/test/upgrade/run.sh | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 9217fcfddd9..a2264b8f3e6 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -233,4 +233,10 @@ rowNumberInAllBlocks() LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv || echo "failure\tCannot parse test_results.tsv" > /test_output/check_status.tsv [ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv +# But OOMs in stress test are allowed +if rg 'OOM in dmesg|Signal 9' /test_output/check_status.tsv +then + sed -i 's/failure/success/' /test_output/check_status.tsv +fi + collect_core_dumps diff --git a/docker/test/upgrade/run.sh b/docker/test/upgrade/run.sh index 13c352d5d41..d6cd6987e83 100644 --- a/docker/test/upgrade/run.sh +++ b/docker/test/upgrade/run.sh @@ -231,4 +231,10 @@ rowNumberInAllBlocks() LIMIT 1" < /test_output/test_results.tsv > /test_output/check_status.tsv || echo "failure\tCannot parse test_results.tsv" > /test_output/check_status.tsv [ -s /test_output/check_status.tsv ] || echo -e "success\tNo errors found" > /test_output/check_status.tsv +# But OOMs in stress test are allowed +if rg 'OOM in dmesg|Signal 9' /test_output/check_status.tsv +then + sed -i 's/failure/success/' /test_output/check_status.tsv +fi + collect_core_dumps From af60b4bb313bc33771562b3dd830395548952041 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 31 Jul 2023 10:29:15 +0000 Subject: [PATCH 1359/2047] fixed style check --- src/Functions/FunctionBinaryArithmetic.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index b4806d24a8e..0dcd24cc266 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1201,7 +1201,7 @@ class FunctionBinaryArithmetic : public IFunction } /// Unpacking non-const arrays and checking sizes of them. - if (*typeid_cast(arguments[0].column.get())->getOffsets().data() != + if (*typeid_cast(arguments[0].column.get())->getOffsets().data() != *typeid_cast(arguments[1].column.get())->getOffsets().data()) { throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, @@ -1217,7 +1217,7 @@ class FunctionBinaryArithmetic : public IFunction a = typeid_cast(arguments[1].column.get())->getData().getPtr(); t = typeid_cast(arguments[1].type.get())->getNestedType(); new_arguments[1] = {a, t, arguments[1].name}; - + auto res = executeImpl(new_arguments, t, input_rows_count); return ColumnArray::create(res, typeid_cast(arguments[0].column.get())->getOffsetsPtr()); } From 6f9cdab8a4e603cec1d6c0194a5222e5e1bf1465 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Jul 2023 12:41:53 +0200 Subject: [PATCH 1360/2047] Fix --- tests/integration/test_temporary_data_in_cache/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_temporary_data_in_cache/test.py b/tests/integration/test_temporary_data_in_cache/test.py index de806ddafaf..e6142c7eef1 100644 --- a/tests/integration/test_temporary_data_in_cache/test.py +++ b/tests/integration/test_temporary_data_in_cache/test.py @@ -41,7 +41,7 @@ def test_cache_evicted_by_temporary_data(start_cluster): # Codec is NONE to make cache size predictable q( - "CREATE TABLE t1 (x UInt64 CODEC(NONE)) ENGINE = MergeTree ORDER BY x SETTINGS storage_policy = 'tiny_local_cache'" + "CREATE TABLE t1 (x UInt64 CODEC(NONE), y UInt64 CODEC(NONE)) ENGINE = MergeTree ORDER BY x SETTINGS storage_policy = 'tiny_local_cache'" ) q("INSERT INTO t1 SELECT number, number FROM numbers(1024 * 1024)") From f457a51ac02ffa6b4ec00c416be2e5d9958f12e0 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 31 Jul 2023 13:50:36 +0200 Subject: [PATCH 1361/2047] Fix default value --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9823f8e9c95..fbe805cdfa5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -622,7 +622,7 @@ class IColumn; M(Bool, engine_file_allow_create_multiple_files, false, "Enables or disables creating a new file on each insert in file engine tables if format has suffix.", 0) \ M(Bool, engine_file_skip_empty_files, false, "Allows to skip empty files in file table engine", 0) \ M(Bool, engine_url_skip_empty_files, false, "Allows to skip empty files in url table engine", 0) \ - M(Bool, enable_url_encoding, false, " Allows to enable/disable decoding/encoding path in uri in URL table engine", 0) \ + M(Bool, enable_url_encoding, true, " Allows to enable/disable decoding/encoding path in uri in URL table engine", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(Bool, database_replicated_enforce_synchronous_settings, false, "Enforces synchronous waiting for some queries (see also database_atomic_wait_for_drop_and_detach_synchronously, mutation_sync, alter_sync). Not recommended to enable these settings.", 0) \ From 9fb86f134b8e72efc214e9089a523e8bd866ff18 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 12:04:27 +0000 Subject: [PATCH 1362/2047] Fix tests --- src/Client/ClientBase.cpp | 1 + src/Storages/StorageFile.cpp | 43 ++++++++++++++++--- src/Storages/StorageFile.h | 4 +- src/TableFunctions/ITableFunctionFileLike.cpp | 26 +---------- src/TableFunctions/ITableFunctionFileLike.h | 2 +- src/TableFunctions/TableFunctionFile.cpp | 4 ++ ...661_select_from_table_in_archive.reference | 7 ++- .../02661_select_from_table_in_archive.sh | 25 ++++++++--- 8 files changed, 72 insertions(+), 40 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 06dabf96c28..407156698e5 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1436,6 +1436,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des ConstraintsDescription{}, String{}, {}, + String{}, }; StoragePtr storage = std::make_shared(in_file, global_context->getUserFilesPath(), args); storage->startup(); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 54ab2555b4e..d59c1b4a1b4 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -380,11 +380,11 @@ std::unique_ptr createReadBuffer( int table_fd, const String & compression_method, ContextPtr context, - const String & path_to_archive = "auto") + const String & path_to_archive = "") { CompressionMethod method; - if (path_to_archive != "auto") + if (!path_to_archive.empty()) { auto reader = createArchiveReader(path_to_archive); std::unique_ptr in = reader->readFile(current_path); @@ -591,7 +591,7 @@ StorageFile::StorageFile(int table_fd_, CommonArguments args) StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args) : StorageFile(args) { - if (args.path_to_archive != "auto") + if (!args.path_to_archive.empty()) { paths_to_archive = getPathsList(args.path_to_archive, user_files_path, args.getContext(), total_bytes_to_read); paths = {table_path_}; @@ -600,6 +600,7 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us { paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read); } + is_db_table = false; is_path_with_globs = paths.size() > 1; if (!paths.empty()) @@ -822,8 +823,11 @@ public: if (!storage->use_table_fd) { size_t current_file = 0, current_archive = 0; - if (files_info->files.size() == 1 && !files_info->paths_to_archive.empty()) + if (!files_info->paths_to_archive.empty()) { + if (files_info->files.size() != 1) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Can only read a single file from archive"); + current_archive = files_info->next_archive_to_read.fetch_add(1); if (current_archive >= files_info->paths_to_archive.size()) return {}; @@ -1478,6 +1482,7 @@ void registerStorageFile(StorageFactory & factory) factory_args.constraints, factory_args.comment, {}, + {}, }; ASTs & engine_args_ast = factory_args.engine_args; @@ -1548,7 +1553,7 @@ void registerStorageFile(StorageFactory & factory) else if (type == Field::Types::UInt64) source_fd = static_cast(literal->value.get()); else if (type == Field::Types::String) - source_path = literal->value.get(); + StorageFile::parseFileSource(literal->value.get(), source_path, storage_args.path_to_archive); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second argument must be path or file descriptor"); } @@ -1620,4 +1625,32 @@ void StorageFile::addColumnsToCache( schema_cache.addMany(cache_keys, columns); } +void StorageFile::parseFileSource(String source, String & filename, String & path_to_archive) +{ + size_t pos = source.find("::"); + if (pos == String::npos) + { + filename = std::move(source); + return; + } + + std::string_view path_to_archive_view = std::string_view{source}.substr(0, pos); + while (path_to_archive_view.back() == ' ') + path_to_archive_view.remove_suffix(1); + + if (path_to_archive_view.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to archive is empty"); + + path_to_archive = path_to_archive_view; + + std::string_view filename_view = std::string_view{source}.substr(pos + 2); + while (filename_view.front() == ' ') + filename_view.remove_prefix(1); + + if (filename_view.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filename is empty"); + + filename = filename_view; +} + } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 10c234811ff..96dcfb4cb54 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -23,7 +23,7 @@ public: const ConstraintsDescription & constraints; const String & comment; const std::string rename_after_processing; - std::string path_to_archive = "auto"; + std::string path_to_archive; }; /// From file descriptor @@ -93,6 +93,8 @@ public: static SchemaCache & getSchemaCache(const ContextPtr & context); + static void parseFileSource(String source, String & filename, String & path_to_archive); + protected: friend class StorageFileSource; friend class StorageFileSink; diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index baab7f5143e..2ddc0b6dba1 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -25,31 +25,7 @@ namespace ErrorCodes void ITableFunctionFileLike::parseFirstArguments(const ASTPtr & arg, const ContextPtr &) { String path = checkAndGetLiteralArgument(arg, "source"); - size_t pos = path.find("::"); - if (pos == String::npos) - { - filename = path; - } - else - { - std::string_view path_to_archive_view = std::string_view{path}.substr(0, pos); - while (path_to_archive_view.back() == ' ') - path_to_archive_view.remove_suffix(1); - - if (path_to_archive_view.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path to archive is empty"); - - path_to_archive = path_to_archive_view; - - std::string_view filename_view = std::string_view{path}.substr(pos + 2); - while (filename_view.front() == ' ') - filename_view.remove_prefix(1); - - if (filename_view.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filename is empty"); - - filename = filename_view; - } + StorageFile::parseFileSource(std::move(path), filename, path_to_archive); } String ITableFunctionFileLike::getFormatFromFirstArgument() diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index 81a38d53d8c..7f212cb60e9 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -42,7 +42,7 @@ protected: virtual String getFormatFromFirstArgument(); String filename; - String path_to_archive = "auto"; + String path_to_archive; String format = "auto"; String structure = "auto"; String compression_method = "auto"; diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 1ba40e25bc0..3319d163410 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -39,6 +39,8 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr fd = STDOUT_FILENO; else if (filename == "stderr") fd = STDERR_FILENO; + else + StorageFile::parseFileSource(std::move(filename), filename, path_to_archive); } else if (type == Field::Types::Int64 || type == Field::Types::UInt64) { @@ -78,6 +80,7 @@ StoragePtr TableFunctionFile::getStorage(const String & source, global_context->getSettingsRef().rename_files_after_processing, path_to_archive, }; + if (fd >= 0) return std::make_shared(fd, args); @@ -98,6 +101,7 @@ ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context paths = StorageFile::getPathsList(filename, context->getUserFilesPath(), context, total_bytes_to_read); else paths_to_archives = StorageFile::getPathsList(path_to_archive, context->getUserFilesPath(), context, total_bytes_to_read); + return StorageFile::getTableStructureFromFile(format, paths, compression_method, std::nullopt, context, paths_to_archives); } diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.reference b/tests/queries/0_stateless/02661_select_from_table_in_archive.reference index 9869a226367..a13e8c56aa4 100644 --- a/tests/queries/0_stateless/02661_select_from_table_in_archive.reference +++ b/tests/queries/0_stateless/02661_select_from_table_in_archive.reference @@ -1,7 +1,12 @@ 1 2 3 4 +1 2 +3 4 +1 +3 +1 +3 1 3 1 3 - diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh index a0e165989ae..9366b7bd654 100755 --- a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh +++ b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh @@ -4,13 +4,24 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -echo -e "1,2\n3,4" >${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv -zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null -zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS 02661_archive_table" -$CLICKHOUSE_LOCAL --query "SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv')" -$CLICKHOUSE_LOCAL --query "SELECT c1 FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1..2}.zip :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv')" +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +echo -e "1,2\n3,4" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv +zip ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null +zip ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null + +function read_archive_file() { + $CLICKHOUSE_LOCAL --query "SELECT $1 FROM file('${user_files_path}/$2')" + $CLICKHOUSE_CLIENT --query "CREATE TABLE 02661_archive_table Engine=File('CSV', '${user_files_path}/$2')" + $CLICKHOUSE_CLIENT --query "SELECT $1 FROM 02661_archive_table" + $CLICKHOUSE_CLIENT --query "DROP TABLE 02661_archive_table" +} + +read_archive_file "*" "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" +read_archive_file "c1" "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1..2}.zip :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv -rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip -rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip +rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip +rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip From b683477b2034aaa3a5af7e21795d4a928cd1a719 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Jul 2023 14:14:15 +0200 Subject: [PATCH 1363/2047] Better dynamic disk configuration --- src/Disks/getDiskConfigurationFromAST.cpp | 10 ++++---- src/Disks/getDiskConfigurationFromAST.h | 8 +++---- src/Disks/getOrCreateDiskFromAST.cpp | 24 ++++++++----------- src/Parsers/ParserSetQuery.cpp | 4 ++-- ...2808_custom_disk_with_user_defined_name.sh | 4 ++-- 5 files changed, 22 insertions(+), 28 deletions(-) diff --git a/src/Disks/getDiskConfigurationFromAST.cpp b/src/Disks/getDiskConfigurationFromAST.cpp index 4b1323b4db8..76a257d3b52 100644 --- a/src/Disks/getDiskConfigurationFromAST.cpp +++ b/src/Disks/getDiskConfigurationFromAST.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes message.empty() ? "" : ": " + message); } -Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::string & root_name, const ASTs & disk_args, ContextPtr context) +Poco::AutoPtr getDiskConfigurationFromASTImpl(const ASTs & disk_args, ContextPtr context) { if (disk_args.empty()) throwBadConfiguration("expected non-empty list of arguments"); @@ -39,8 +39,6 @@ Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::st Poco::AutoPtr xml_document(new Poco::XML::Document()); Poco::AutoPtr root(xml_document->createElement("disk")); xml_document->appendChild(root); - Poco::AutoPtr disk_configuration(xml_document->createElement(root_name)); - root->appendChild(disk_configuration); for (const auto & arg : disk_args) { @@ -62,7 +60,7 @@ Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::st const std::string & key = key_identifier->name(); Poco::AutoPtr key_element(xml_document->createElement(key)); - disk_configuration->appendChild(key_element); + root->appendChild(key_element); if (!function_args[1]->as() && !function_args[1]->as()) throwBadConfiguration("expected values to be literals or identifiers"); @@ -75,9 +73,9 @@ Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::st return xml_document; } -DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context) +DiskConfigurationPtr getDiskConfigurationFromAST(const ASTs & disk_args, ContextPtr context) { - auto xml_document = getDiskConfigurationFromASTImpl(root_name, disk_args, context); + auto xml_document = getDiskConfigurationFromASTImpl(disk_args, context); Poco::AutoPtr conf(new Poco::Util::XMLConfiguration()); conf->load(xml_document); return conf; diff --git a/src/Disks/getDiskConfigurationFromAST.h b/src/Disks/getDiskConfigurationFromAST.h index 5697955e914..f23fb37b9dc 100644 --- a/src/Disks/getDiskConfigurationFromAST.h +++ b/src/Disks/getDiskConfigurationFromAST.h @@ -14,19 +14,19 @@ using DiskConfigurationPtr = Poco::AutoPtr; /** * Transform a list of pairs ( key1=value1, key2=value2, ... ), where keys and values are ASTLiteral or ASTIdentifier * into - * + * * value1 * value2 * ... - * + * * * Used in case disk configuration is passed via AST when creating * a disk object on-the-fly without any configuration file. */ -DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context); +DiskConfigurationPtr getDiskConfigurationFromAST(const ASTs & disk_args, ContextPtr context); /// The same as above function, but return XML::Document for easier modification of result configuration. -[[ maybe_unused ]] Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::string & root_name, const ASTs & disk_args, ContextPtr context); +[[ maybe_unused ]] Poco::AutoPtr getDiskConfigurationFromASTImpl(const ASTs & disk_args, ContextPtr context); /* * A reverse function. diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index a9a0e972bd1..da318303f62 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -26,8 +26,16 @@ namespace { std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context) { + const auto * function_args_expr = assert_cast(function.arguments.get()); + const auto & function_args = function_args_expr->children; + auto config = getDiskConfigurationFromAST(function_args, context); + std::string disk_name; - if (function.name == "disk") + if (config->has("name")) + { + disk_name = config->getString("name"); + } + else { /// We need a unique name for a created custom disk, but it needs to be the same /// after table is reattached or server is restarted, so take a hash of the disk @@ -36,21 +44,9 @@ namespace disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); } - else - { - static constexpr std::string_view custom_disk_prefix = "disk_"; - - if (function.name.size() <= custom_disk_prefix.size() || !function.name.starts_with(custom_disk_prefix)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid disk name: {}", function.name); - - disk_name = function.name.substr(custom_disk_prefix.size()); - } auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { - const auto * function_args_expr = assert_cast(function.arguments.get()); - const auto & function_args = function_args_expr->children; - auto config = getDiskConfigurationFromAST(disk_name, function_args, context); - auto disk = DiskFactory::instance().create(disk_name, *config, disk_name, context, disks_map); + auto disk = DiskFactory::instance().create(disk_name, *config, "", context, disks_map); /// Mark that disk can be used without storage policy. disk->markDiskAsCustom(); return disk; diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 727d037112f..4df74c2dd82 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -215,7 +215,7 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p else if (ParserKeyword("FALSE").ignore(pos, expected)) value = std::make_shared(Field(static_cast(0))); /// for SETTINGS disk=disk(type='s3', path='', ...) - else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name.starts_with("disk")) + else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { tryGetIdentifierNameInto(name, change.name); change.value = createFieldFromAST(function_ast); @@ -280,7 +280,7 @@ bool ParserSetQuery::parseNameValuePairWithParameterOrDefault( node = std::make_shared(Field(static_cast(1))); else if (ParserKeyword("FALSE").ignore(pos, expected)) node = std::make_shared(Field(static_cast(0))); - else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name.starts_with("disk")) + else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { change.name = name; change.value = createFieldFromAST(function_ast); diff --git a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh index a43cd6deb9e..333bc1bc25d 100755 --- a/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh +++ b/tests/queries/0_stateless/02808_custom_disk_with_user_defined_name.sh @@ -12,7 +12,7 @@ $CLICKHOUSE_CLIENT -nm --query """ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS disk = disk_s3_disk(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); +SETTINGS disk = disk(name = 's3_disk', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); """ 2>&1 | grep -q "Disk with name \`s3_disk\` already exist" && echo 'OK' || echo 'FAIL' disk_name="${CLICKHOUSE_TEST_UNIQUE_NAME}" @@ -25,7 +25,7 @@ $CLICKHOUSE_CLIENT -nm --query """ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS disk = disk_$disk_name(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); +SETTINGS disk = disk(name = '$disk_name', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); """ $CLICKHOUSE_CLIENT -nm --query """ From 985b2a010af9ba5de9e72fc4623ea05e7d91c4b2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 Jul 2023 15:17:47 +0300 Subject: [PATCH 1364/2047] Add a tool to upload `-ftime-trace` to ClickHouse (#52776) * Add a tool to upload `-ftime-trace` to ClickHouse * Add a tool to upload `-ftime-trace` to ClickHouse * Add a tool to upload `-ftime-trace` to ClickHouse --- .../prepare-time-trace/prepare-time-trace.sh | 51 +++++++++++++++++++ 1 file changed, 51 insertions(+) create mode 100755 utils/prepare-time-trace/prepare-time-trace.sh diff --git a/utils/prepare-time-trace/prepare-time-trace.sh b/utils/prepare-time-trace/prepare-time-trace.sh new file mode 100755 index 00000000000..300a32b0fd1 --- /dev/null +++ b/utils/prepare-time-trace/prepare-time-trace.sh @@ -0,0 +1,51 @@ +#!/bin/bash + +# This scripts transforms the output of clang's -ftime-trace JSON files into a format to upload to ClickHouse + +# Example: +# mkdir time_trace +# utils/prepare-time-trace/prepare-time-trace.sh build time_trace + +# See also https://docs.google.com/document/d/1CvAClvFfyA5R-PhYUmn5OOQtYMH4h6I0nSsKchNAySU/preview + +< \"${OUTPUT_DIR}/\$\$\" +" + +# Now you can upload it as follows: + +#cat "$OUTPUT_DIR"/* | clickhouse-client --progress --query "INSERT INTO build_time_trace (extra_column_names, file, library, time, pid, tid, ph, ts, dur, cat, name, detail, count, avgMs, args_name) FORMAT JSONCompactEachRow" From fb8502ba7671f75cb420ab5937cced224f5be7c6 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 31 Jul 2023 12:20:27 +0000 Subject: [PATCH 1365/2047] do not test upper bounds for throttlers --- tests/integration/test_throttling/test.py | 45 ++++++++++++----------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 62640394a85..526c34ac916 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -117,7 +117,8 @@ def assert_took(took, should_took): # we need to decrease the lower limit because the server limits could # be enforced by throttling some server background IO instead of query IO # and we have no control over it - assert took >= should_took[0] * 0.85 and took < should_took[1] + # Note that throttler does not apply any restrictions on upper bound, so we can only tell how much time required "at least", not "at most" + assert took >= should_took * 0.85 @pytest.mark.parametrize( @@ -132,7 +133,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_local_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds @@ -142,7 +143,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7, 14), + 7, id="user_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -152,7 +153,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_local_throttling", ), # @@ -164,7 +165,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_remote_to_local_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds @@ -184,7 +185,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_to_local_throttling", ), # @@ -196,7 +197,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_remote_to_remote_throttling", ), # No throttling for S3-to-S3, uses native copy @@ -206,7 +207,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (0, 3), + 0, id="user_remote_to_remote_throttling", ), # No throttling for S3-to-S3, uses native copy @@ -216,7 +217,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (0, 3), + 0, id="server_remote_to_remote_throttling", ), # @@ -233,7 +234,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_local_to_remote_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds, but for S3Client it is 2x more @@ -243,7 +244,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7 * 3, 7 * 4 - 1), + 7 * 3, id="user_local_to_remote_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds, but for S3Client it is 2x more @@ -253,7 +254,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3 * 3, 3 * 5), + 3 * 3, id="server_local_to_remote_throttling", ), ], @@ -306,7 +307,7 @@ def test_backup_throttling_override(): "user", "max_local_read_bandwidth", "1M", - (7, 14), + 7, id="user_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -315,7 +316,7 @@ def test_backup_throttling_override(): "server", "max_local_read_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_local_throttling", ), # @@ -328,7 +329,7 @@ def test_backup_throttling_override(): "user", "max_remote_read_network_bandwidth", "1M", - (7, 14), + 7, id="user_remote_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -337,7 +338,7 @@ def test_backup_throttling_override(): "server", "max_remote_read_network_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_throttling", ), ], @@ -368,7 +369,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): "user", "max_local_write_bandwidth", "1M", - (7, 14), + 7, id="local_user_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -377,29 +378,29 @@ def test_read_throttling(policy, mode, setting, value, should_took): "server", "max_local_write_bandwidth_for_server", "2M", - (3, 7), + 3, id="local_server_throttling", ), # # Remote # pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), - # writeing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + # writing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "s3", "user", "max_remote_write_network_bandwidth", "1M", - (7, 14), + 7, id="user_remote_throttling", ), - # writeing 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + # writing 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds pytest.param( "s3", "server", "max_remote_write_network_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_throttling", ), ], From efba3a21139adf8004d975b2d276edfa39782b19 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 31 Jul 2023 14:32:09 +0200 Subject: [PATCH 1366/2047] Add more logging and touch test for materialize mysql --- src/Common/mysqlxx/Pool.cpp | 3 +++ src/Common/mysqlxx/mysqlxx/Pool.h | 2 +- src/Databases/MySQL/DatabaseMaterializedMySQL.cpp | 2 ++ src/Databases/MySQL/DatabaseMaterializedMySQL.h | 1 + src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 6 +++++- .../integration/test_materialized_mysql_database/test.py | 8 -------- 6 files changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index 64a69c48e1d..43df0aa6708 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -153,7 +153,10 @@ Pool::Entry Pool::get(uint64_t wait_timeout) for (auto & connection : connections) { if (connection->ref_count == 0) + { + logger.test("Found free connection in pool, returning it to the caller"); return Entry(connection, this); + } } logger.trace("(%s): Trying to allocate a new connection.", getDescription()); diff --git a/src/Common/mysqlxx/mysqlxx/Pool.h b/src/Common/mysqlxx/mysqlxx/Pool.h index c2190fba684..52d116e39ce 100644 --- a/src/Common/mysqlxx/mysqlxx/Pool.h +++ b/src/Common/mysqlxx/mysqlxx/Pool.h @@ -26,7 +26,7 @@ namespace mysqlxx * * void thread() * { - * mysqlxx::Pool::Entry connection = pool.Get(); + * mysqlxx::Pool::Entry connection = pool.Get(); * std::string s = connection->query("SELECT 'Hello, world!' AS world").use().fetch()["world"].getString(); * } * TODO: simplify with PoolBase. diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 653c2dc27b6..f7e669d9feb 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -65,6 +65,7 @@ void DatabaseMaterializedMySQL::setException(const std::exception_ptr & exceptio void DatabaseMaterializedMySQL::startupTables(ThreadPool & thread_pool, LoadingStrictnessLevel mode) { + LOG_TRACE(log, "Starting MaterializeMySQL tables"); DatabaseAtomic::startupTables(thread_pool, mode); if (mode < LoadingStrictnessLevel::FORCE_ATTACH) @@ -122,6 +123,7 @@ void DatabaseMaterializedMySQL::alterTable(ContextPtr context_, const StorageID void DatabaseMaterializedMySQL::drop(ContextPtr context_) { + LOG_TRACE(log, "Dropping MaterializeMySQL database"); /// Remove metadata info fs::path metadata(getMetadataPath() + "/.metadata"); diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index 3698abf5542..60a88ea0d67 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 673bd155f77..c9e10b27caa 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -1,3 +1,4 @@ +#include "Common/logger_useful.h" #include "config.h" #if USE_MYSQL @@ -499,7 +500,10 @@ bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & meta { throw; } - catch (const mysqlxx::ConnectionFailed &) {} + catch (const mysqlxx::ConnectionFailed & ex) + { + LOG_TRACE(log, "Connection to MySQL failed {}", ex.displayText()); + } catch (const mysqlxx::BadQuery & e) { // Lost connection to MySQL server during query diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 1fd09f733f0..5096e0a03b7 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -1,17 +1,11 @@ -import os -import os.path as p import time -import pwd -import re import pymysql.cursors import pytest from helpers.cluster import ( ClickHouseCluster, ClickHouseInstance, get_docker_compose_path, - run_and_check, ) -import docker import logging from . import materialized_with_ddl @@ -63,8 +57,6 @@ class MySQLConnection: user="root", password="clickhouse", ip_address=None, - docker_compose=None, - project_name=cluster.project_name, ): self.user = user self.port = port From 52d13bd348dd362fab57d4285eb488ddf0d4446d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 31 Jul 2023 14:40:29 +0200 Subject: [PATCH 1367/2047] may be fix test --- .../00002_log_and_exception_messages_formatting.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference index 11b660b54a3..8a38707f8da 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -1,7 +1,7 @@ runtime messages 0.001 runtime exceptions 0.05 messages shorter than 10 1 -messages shorter than 16 3 +messages shorter than 16 4 exceptions shorter than 30 3 noisy messages 0.3 noisy Trace messages 0.16 From 8f5d0c0cd4b150d32a1257d18ee77cd582ac43cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 Jul 2023 13:03:04 +0000 Subject: [PATCH 1368/2047] Revert "Merge pull request #52514 from azat/tests/fix-test_version_update_after_mutation" This reverts commit dd491eeae2e1ee030f4db5d3f0837067a2ee0ca8, reversing changes made to b225f9c34bf6064d246ede33f98e3c573fcda142. --- tests/integration/helpers/cluster.py | 9 --------- .../force_remove_data_recursively_on_drop.xml | 7 ------- .../test_version_update_after_mutation/test.py | 13 +++---------- 3 files changed, 3 insertions(+), 26 deletions(-) delete mode 100644 tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0448eb2437f..eff44de842a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3199,7 +3199,6 @@ class ClickHouseInstance: ): self.name = name self.base_cmd = cluster.base_cmd - self.base_dir = base_path self.docker_id = cluster.get_instance_docker_id(self.name) self.cluster = cluster self.hostname = hostname if hostname is not None else self.name @@ -4194,14 +4193,6 @@ class ClickHouseInstance: ["bash", "-c", f"sed -i 's/{replace}/{replacement}/g' {path_to_config}"] ) - def put_users_config(self, config_path): - """Put new config (useful if you cannot put it at the start)""" - - instance_config_dir = p.abspath(p.join(self.path, "configs")) - users_d_dir = p.abspath(p.join(instance_config_dir, "users.d")) - config_path = p.join(self.base_dir, config_path) - shutil.copy(config_path, users_d_dir) - def create_dir(self): """Create the instance directory and all the needed files there.""" diff --git a/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml b/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml deleted file mode 100644 index 7a00648b28e..00000000000 --- a/tests/integration/test_version_update_after_mutation/configs/force_remove_data_recursively_on_drop.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - - 1 - - - diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 416220c93c3..c80205d48c1 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -51,12 +51,6 @@ def start_cluster(): cluster.shutdown() -def restart_node(node): - # set force_remove_data_recursively_on_drop (cannot be done before, because the version is too old) - node.put_users_config("configs/force_remove_data_recursively_on_drop.xml") - node.restart_with_latest_version(signal=9, fix_metadata=True) - - def test_mutate_and_upgrade(start_cluster): for node in [node1, node2]: node.query("DROP TABLE IF EXISTS mt") @@ -73,9 +67,8 @@ def test_mutate_and_upgrade(start_cluster): node2.query("DETACH TABLE mt") # stop being leader node1.query("DETACH TABLE mt") # stop being leader - - restart_node(node1) - restart_node(node2) + node1.restart_with_latest_version(signal=9, fix_metadata=True) + node2.restart_with_latest_version(signal=9, fix_metadata=True) # After hard restart table can be in readonly mode exec_query_with_retry( @@ -131,7 +124,7 @@ def test_upgrade_while_mutation(start_cluster): # (We could be in process of creating some system table, which will leave empty directory on restart, # so when we start moving system tables from ordinary to atomic db, it will complain about some undeleted files) node3.query("SYSTEM FLUSH LOGS") - restart_node(node3) + node3.restart_with_latest_version(signal=9, fix_metadata=True) # checks for readonly exec_query_with_retry(node3, "OPTIMIZE TABLE mt1", sleep_time=5, retry_count=60) From 964e4311c3950aa516b6cf435afc75f3e3245fb7 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 31 Jul 2023 13:03:56 +0000 Subject: [PATCH 1369/2047] reviewed changes --- .../System/attachInformationSchemaTables.cpp | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index efa24daeffe..544ab75ac29 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -40,15 +40,9 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d StoragePtr view = createTableFromAST(ast_create, database.getDatabaseName(), database.getTableDataPath(ast_create), context, true).second; - database.createTable(context, ast_create.getTable(), view, ast); - if (database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA) - { - database.createTable(context, Poco::toUpper(ast_create.getTable()), view, ast); - } - else - { - database.createTable(context, Poco::toLower(ast_create.getTable()), view, ast); - } + database.createTable(context, Poco::toUpper(ast_create.getTable()), view, ast); + database.createTable(context, Poco::toLower(ast_create.getTable()), view, ast); + } catch (...) { From b98d54cdf3a2f931040067418797336c6fb218a4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 Jul 2023 13:05:53 +0000 Subject: [PATCH 1370/2047] Add flushing logs --- tests/integration/test_version_update_after_mutation/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index c80205d48c1..eac214ea99f 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -67,6 +67,8 @@ def test_mutate_and_upgrade(start_cluster): node2.query("DETACH TABLE mt") # stop being leader node1.query("DETACH TABLE mt") # stop being leader + node1.query("SYSTEM FLUSH LOGS") + node2.query("SYSTEM FLUSH LOGS") node1.restart_with_latest_version(signal=9, fix_metadata=True) node2.restart_with_latest_version(signal=9, fix_metadata=True) From 74c0574d519378feb36a657988c22d0b93c31638 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 31 Jul 2023 15:22:16 +0200 Subject: [PATCH 1371/2047] Update ReadBufferFromHDFS.cpp --- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 3476defd5b1..122ed335265 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -75,7 +75,10 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(file_info->mSize); } } From 2b5a559d7816fe0036413572d3fe8c83a15d9e39 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Jul 2023 14:28:13 +0200 Subject: [PATCH 1372/2047] Fix test --- tests/clickhouse-test | 10 ++++++++++ tests/queries/0_stateless/01304_direct_io_long.sh | 2 +- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c63e1e3ae52..e44509bdbbd 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -954,6 +954,16 @@ class TestCase: elif tags and ("no-s3-storage" in tags) and args.s3_storage: return FailureReason.S3_STORAGE + elif ( + tags + and ("no-s3-storage-with-slow-build" in tags) + and args.s3_storage + and ( + BuildFlags.THREAD in args.build_flags + or BuildFlags.DEBUG in args.build_flags + ) + ): + return FailureReason.S3_STORAGE elif tags: for build_flag in args.build_flags: diff --git a/tests/queries/0_stateless/01304_direct_io_long.sh b/tests/queries/0_stateless/01304_direct_io_long.sh index bd503f40396..97148dc268e 100755 --- a/tests/queries/0_stateless/01304_direct_io_long.sh +++ b/tests/queries/0_stateless/01304_direct_io_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-s3-storage-with-slow-build CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 6a1cf5c6830be1f6a06210f2ef89564c7c6f0e3d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 13:27:06 +0000 Subject: [PATCH 1373/2047] Extend tests --- contrib/libarchive-cmake/CMakeLists.txt | 2 + contrib/libarchive-cmake/config.h | 6 +-- docker/test/stateless/Dockerfile | 2 + src/IO/Archives/LibArchiveReader.cpp | 10 +++-- src/IO/Archives/createArchiveReader.cpp | 2 +- ...661_select_from_table_in_archive.reference | 44 +++++++++++++++++++ .../02661_select_from_table_in_archive.sh | 43 +++++++++++++++--- 7 files changed, 97 insertions(+), 12 deletions(-) diff --git a/contrib/libarchive-cmake/CMakeLists.txt b/contrib/libarchive-cmake/CMakeLists.txt index 400bded56dc..3aca1c7eeab 100644 --- a/contrib/libarchive-cmake/CMakeLists.txt +++ b/contrib/libarchive-cmake/CMakeLists.txt @@ -146,4 +146,6 @@ target_compile_definitions(_libarchive PUBLIC target_compile_options(_libarchive PRIVATE "-Wno-reserved-macro-identifier") +target_link_libraries(_libarchive PRIVATE ch_contrib::zlib ch_contrib::xz) + add_library(ch_contrib::libarchive ALIAS _libarchive) \ No newline at end of file diff --git a/contrib/libarchive-cmake/config.h b/contrib/libarchive-cmake/config.h index 9861e88359d..c2fecf657bc 100644 --- a/contrib/libarchive-cmake/config.h +++ b/contrib/libarchive-cmake/config.h @@ -843,7 +843,7 @@ typedef uint64_t uintmax_t; /* #undef HAVE_LZMADEC_H */ /* Define to 1 if you have the header file. */ -/* #undef HAVE_LZMA_H */ +#define HAVE_LZMA_H 1 /* Define to 1 if you have a working `lzma_stream_encoder_mt' function. */ /* #undef HAVE_LZMA_STREAM_ENCODER_MT */ @@ -909,7 +909,7 @@ typedef uint64_t uintmax_t; /* #undef HAVE_NETTLE_SHA_H */ /* Define to 1 if you have the `nl_langinfo' function. */ -#define HAVE_NL_LANGINFO 1 +/* #undef HAVE_NL_LANGINFO */ /* Define to 1 if you have the `openat' function. */ #define HAVE_OPENAT 1 @@ -1249,7 +1249,7 @@ typedef uint64_t uintmax_t; #define HAVE_WORKING_FS_IOC_GETFLAGS 1 /* Define to 1 if you have the header file. */ -/* #undef HAVE_ZLIB_H */ +#define HAVE_ZLIB_H 1 /* Define to 1 if you have the header file. */ /* #undef HAVE_ZSTD_H */ diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index e1e84c427ba..c91d3c65bf6 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -41,6 +41,8 @@ RUN apt-get update -y \ zstd \ file \ pv \ + zip \ + p7zip-full \ && apt-get clean RUN pip3 install numpy scipy pandas Jinja2 diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 54a228858ee..f1c2858dba9 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -116,6 +116,9 @@ private: { auto bytes_read = archive_read_data(handle.archive, internal_buffer.begin(), static_cast(internal_buffer.size())); + if (bytes_read < 0) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to read file {} from {}: {}", filename, path_to_archive, archive_error_string(handle.archive)); + if (!bytes_read) return false; @@ -123,6 +126,7 @@ private: working_buffer.resize(bytes_read); return true; } + Handle handle; const String path_to_archive; const String filename; @@ -166,7 +170,7 @@ LibArchiveReader::FileInfo LibArchiveReader::getFileIn template std::unique_ptr::FileEnumerator> LibArchiveReader::firstFile() { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for {} archives", ArchiveInfo::name); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implemented for {} archives", ArchiveInfo::name); } template @@ -181,14 +185,14 @@ std::unique_ptr LibArchiveReader::readFile( template std::unique_ptr LibArchiveReader::readFile(std::unique_ptr /*enumerator*/) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for {} archives", ArchiveInfo::name); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implemented for {} archives", ArchiveInfo::name); } template std::unique_ptr::FileEnumerator> LibArchiveReader::nextFile(std::unique_ptr /*read_buffer*/) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implementaed for {} archives", ArchiveInfo::name); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implemented for {} archives", ArchiveInfo::name); } diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index d33af227975..7be0a57f2fb 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -32,7 +32,7 @@ std::shared_ptr createArchiveReader( throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "minizip library is disabled"); #endif } - else if (path_to_archive.ends_with(".tar")) + else if (path_to_archive.ends_with(".tar") || path_to_archive.ends_with("tar.gz")) { #if USE_LIBARCHIVE return std::make_shared(path_to_archive, archive_read_function); diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.reference b/tests/queries/0_stateless/02661_select_from_table_in_archive.reference index a13e8c56aa4..6f7ca964a7e 100644 --- a/tests/queries/0_stateless/02661_select_from_table_in_archive.reference +++ b/tests/queries/0_stateless/02661_select_from_table_in_archive.reference @@ -1,7 +1,51 @@ +Reading 02661_select_from_table_in_archive_test_archive1.zip :: 02661_select_from_table_in_archive_test_data.csv 1 2 3 4 1 2 3 4 +Reading 02661_select_from_table_in_archive_test_archive{1..2}.zip :: 02661_select_from_table_in_archive_test_data.csv +1 +3 +1 +3 +1 +3 +1 +3 +Reading 02661_select_from_table_in_archive_test_archive1.tar.gz :: 02661_select_from_table_in_archive_test_data.csv +1 2 +3 4 +1 2 +3 4 +Reading 02661_select_from_table_in_archive_test_archive{1..2}.tar.gz :: 02661_select_from_table_in_archive_test_data.csv +1 +3 +1 +3 +1 +3 +1 +3 +Reading 02661_select_from_table_in_archive_test_archive1.tar :: 02661_select_from_table_in_archive_test_data.csv +1 2 +3 4 +1 2 +3 4 +Reading 02661_select_from_table_in_archive_test_archive{1..2}.tar :: 02661_select_from_table_in_archive_test_data.csv +1 +3 +1 +3 +1 +3 +1 +3 +Reading 02661_select_from_table_in_archive_test_archive1.7z :: 02661_select_from_table_in_archive_test_data.csv +1 2 +3 4 +1 2 +3 4 +Reading 02661_select_from_table_in_archive_test_archive{1..2}.7z :: 02661_select_from_table_in_archive_test_data.csv 1 3 1 diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh index 9366b7bd654..af3366ab924 100755 --- a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh +++ b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh @@ -9,19 +9,52 @@ $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS 02661_archive_table" user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') echo -e "1,2\n3,4" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv -zip ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null -zip ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null function read_archive_file() { + echo "Reading $2" $CLICKHOUSE_LOCAL --query "SELECT $1 FROM file('${user_files_path}/$2')" $CLICKHOUSE_CLIENT --query "CREATE TABLE 02661_archive_table Engine=File('CSV', '${user_files_path}/$2')" $CLICKHOUSE_CLIENT --query "SELECT $1 FROM 02661_archive_table" $CLICKHOUSE_CLIENT --query "DROP TABLE 02661_archive_table" } -read_archive_file "*" "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" -read_archive_file "c1" "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1..2}.zip :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" +function run_archive_test() { + read_archive_file "*" "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" + read_archive_file "c1" "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1..2}.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" + + +} + +zip ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null +zip ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null + +run_archive_test "zip" -rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip + +tar -cvzf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar.gz ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null +tar -cvzf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar.gz ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null + +run_archive_test "tar.gz" + +rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar.gz +rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar.gz + +tar -cvf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null +tar -cvf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null + +run_archive_test "tar" + +rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar +rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar + +7z a ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.7z ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null +7z a ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.7z ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null + +run_archive_test "7z" + +rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.7z +rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.7z + +rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv From e869adf64573f8e2732c9aa65192ea1b745b766e Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 31 Jul 2023 06:48:50 -0700 Subject: [PATCH 1374/2047] Improve function naming --- src/Analyzer/IQueryTreeNode.cpp | 2 +- src/Client/QueryFuzzer.cpp | 2 +- src/Common/SipHash.h | 2 +- src/Parsers/IAST.cpp | 2 +- .../Formats/Impl/ConstantExpressionTemplate.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp | 2 +- 7 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 7e5d778f1c1..babc76f29d6 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -229,7 +229,7 @@ IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const } } - return getSipHash128AsLoHi(hash_state); + return getSipHash128AsPair(hash_state); } QueryTreeNodePtr IQueryTreeNode::clone() const diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index f20c869e119..86cedf65345 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -521,7 +521,7 @@ void QueryFuzzer::fuzzCreateQuery(ASTCreateQuery & create) if (create.storage) create.storage->updateTreeHash(sip_hash); - const auto hash = getSipHash128AsLoHi(sip_hash); + const auto hash = getSipHash128AsPair(sip_hash); /// Save only tables with unique definition. if (created_tables_hashes.insert(hash).second) diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 22c2c61abdc..43d620adde0 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -249,7 +249,7 @@ inline std::array getSipHash128AsArray(SipHash & sip_hash) return arr; } -inline CityHash_v1_0_2::uint128 getSipHash128AsLoHi(SipHash & sip_hash) +inline CityHash_v1_0_2::uint128 getSipHash128AsPair(SipHash & sip_hash) { CityHash_v1_0_2::uint128 result; sip_hash.get128(result.low64, result.high64); diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index b3cb5a65098..a494a528cd2 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -118,7 +118,7 @@ IAST::Hash IAST::getTreeHash() const { SipHash hash_state; updateTreeHash(hash_state); - return getSipHash128AsLoHi(hash_state); + return getSipHash128AsPair(hash_state); } diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 92bb1a15693..fe6fb42d0a0 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -402,7 +402,7 @@ size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTP /// Allows distinguish expression in the last column in Values format hash_state.update(salt); - const auto res128 = getSipHash128AsLoHi(hash_state); + const auto res128 = getSipHash128AsPair(hash_state); size_t res = 0; boost::hash_combine(res, res128.low64); boost::hash_combine(res, res128.high64); diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index d09cae7870f..55484a2c8cb 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -323,7 +323,7 @@ MergeTreeDataPartChecksums::Checksum::uint128 MergeTreeDataPartChecksums::getTot hash_of_all_files.update(checksum.file_hash); } - return getSipHash128AsLoHi(hash_of_all_files); + return getSipHash128AsPair(hash_of_all_files); } void MinimalisticDataPartChecksums::serialize(WriteBuffer & to) const @@ -413,9 +413,9 @@ void MinimalisticDataPartChecksums::computeTotalChecksums(const MergeTreeDataPar } } - hash_of_all_files = getSipHash128AsLoHi(hash_of_all_files_state); - hash_of_uncompressed_files = getSipHash128AsLoHi(hash_of_uncompressed_files_state); - uncompressed_hash_of_compressed_files = getSipHash128AsLoHi(uncompressed_hash_of_compressed_files_state); + hash_of_all_files = getSipHash128AsPair(hash_of_all_files_state); + hash_of_uncompressed_files = getSipHash128AsPair(hash_of_uncompressed_files_state); + uncompressed_hash_of_compressed_files = getSipHash128AsPair(uncompressed_hash_of_compressed_files_state); } String MinimalisticDataPartChecksums::getSerializedString(const MergeTreeDataPartChecksums & full_checksums, bool minimalistic) diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index a8d38a1bff8..179871c9ad9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -167,7 +167,7 @@ IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() column.column->updateHashFast(hash); checksum.uncompressed_size = block.bytes(); - checksum.uncompressed_hash = getSipHash128AsLoHi(hash); + checksum.uncompressed_hash = getSipHash128AsPair(hash); return checksum; } From e02805d9f777cddbc3d933c781aa6585efe3a6b3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 31 Jul 2023 16:55:51 +0300 Subject: [PATCH 1375/2047] Update test.py --- tests/integration/test_crash_log/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_crash_log/test.py b/tests/integration/test_crash_log/test.py index 1b7e7f38242..a5b82039a84 100644 --- a/tests/integration/test_crash_log/test.py +++ b/tests/integration/test_crash_log/test.py @@ -30,7 +30,7 @@ def send_signal(started_node, signal): def wait_for_clickhouse_stop(started_node): result = None - for attempt in range(60): + for attempt in range(120): time.sleep(1) pid = started_node.get_process_pid("clickhouse") if pid is None: From 76331a9b490daea5cf87f8089dc0acfeef85582d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 14:20:03 +0000 Subject: [PATCH 1376/2047] Update dockerfile --- docker/test/fasttest/Dockerfile | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index da4baa8c687..5bd8cd9b332 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -19,6 +19,8 @@ RUN apt-get update \ unixodbc \ pv \ zstd \ + zip \ + p7zip-full \ --yes --no-install-recommends RUN pip3 install numpy scipy pandas Jinja2 From 2088f846ad0e3fcedc37e8123cbc8c6d185958d4 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 31 Jul 2023 16:26:18 +0200 Subject: [PATCH 1377/2047] update as per reviews --- src/Functions/FunctionsConversion.h | 14 ++++++++++++-- src/IO/ReadHelpers.h | 10 ++++++++-- src/IO/readFloatText.cpp | 12 ++++-------- src/IO/readFloatText.h | 16 ++-------------- 4 files changed, 26 insertions(+), 26 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 01ff47af18c..f3b93e5cf7a 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1028,7 +1028,12 @@ template void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { if constexpr (std::is_floating_point_v) - readFloatText(x, rb); + { + if constexpr (precise_float_parsing) + readFloatTextPrecise(x, rb); + else + readFloatTextFast(x, rb); + } else parseType(x, rb, time_zone); } @@ -1093,7 +1098,12 @@ template bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { if constexpr (std::is_floating_point_v) - return tryReadFloatText(x, rb); + { + if constexpr (precise_float_parsing) + return tryReadFloatTextPrecise(x, rb); + else + return tryReadFloatTextFast(x, rb); + } else /*if constexpr (is_integer_v)*/ return tryParseTypeImpl(x, rb, time_zone); } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index f4d31c2d3d9..20fe8bd5998 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -526,8 +526,14 @@ void tryReadIntTextUnsafe(T & x, ReadBuffer & buf) /// Look at readFloatText.h -template void readFloatText(T & x, ReadBuffer & in); -template bool tryReadFloatText(T & x, ReadBuffer & in); +template void readFloatText(T & x, ReadBuffer & in); +template bool tryReadFloatText(T & x, ReadBuffer & in); + +template void readFloatTextPrecise(T & x, ReadBuffer & in); +template bool tryReadFloatTextPrecise(T & x, ReadBuffer & in); +template void readFloatTextFast(T & x, ReadBuffer & in); +template bool tryReadFloatTextFast(T & x, ReadBuffer & in); + /// simple: all until '\n' or '\t' diff --git a/src/IO/readFloatText.cpp b/src/IO/readFloatText.cpp index 60505e0fb22..d1143f7c62c 100644 --- a/src/IO/readFloatText.cpp +++ b/src/IO/readFloatText.cpp @@ -62,13 +62,9 @@ template void readFloatTextSimple(Float64 &, ReadBuffer &); template bool tryReadFloatTextSimple(Float32 &, ReadBuffer &); template bool tryReadFloatTextSimple(Float64 &, ReadBuffer &); -template void readFloatText(Float32 &, ReadBuffer &); -template void readFloatText(Float64 &, ReadBuffer &); -template void readFloatText(Float32 &, ReadBuffer &); -template void readFloatText(Float64 &, ReadBuffer &); -template bool tryReadFloatText(Float32 &, ReadBuffer &); -template bool tryReadFloatText(Float64 &, ReadBuffer &); -template bool tryReadFloatText(Float32 &, ReadBuffer &); -template bool tryReadFloatText(Float64 &, ReadBuffer &); +template void readFloatText(Float32 &, ReadBuffer &); +template void readFloatText(Float64 &, ReadBuffer &); +template bool tryReadFloatText(Float32 &, ReadBuffer &); +template bool tryReadFloatText(Float64 &, ReadBuffer &); } diff --git a/src/IO/readFloatText.h b/src/IO/readFloatText.h index de80342687f..da4719b8dcb 100644 --- a/src/IO/readFloatText.h +++ b/src/IO/readFloatText.h @@ -590,19 +590,7 @@ template bool tryReadFloatTextSimple(T & x, ReadBuffer & in) { retu /// Implementation that is selected as default. -template void readFloatText(T & x, ReadBuffer & in) -{ - if constexpr (precise_float_parsing) - readFloatTextPrecise(x, in); - else - readFloatTextFast(x, in); -} -template bool tryReadFloatText(T & x, ReadBuffer & in) -{ - if constexpr (precise_float_parsing) - return tryReadFloatTextPrecise(x, in); - else - return tryReadFloatTextFast(x, in); -} +template void readFloatText(T & x, ReadBuffer & in) { readFloatTextFast(x, in); } +template bool tryReadFloatText(T & x, ReadBuffer & in) { return tryReadFloatTextFast(x, in); } } From e50ffa118134e534c275d1a744e2668a6e05eb26 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 31 Jul 2023 16:44:46 +0200 Subject: [PATCH 1378/2047] fix style --- src/IO/ReadHelpers.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 20fe8bd5998..ba6b6f453af 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -535,7 +535,6 @@ template void readFloatTextFast(T & x, ReadBuffer & in); template bool tryReadFloatTextFast(T & x, ReadBuffer & in); - /// simple: all until '\n' or '\t' void readString(String & s, ReadBuffer & buf); From 86a251931bf5b026589beca33a6c0d001ff0cc51 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 14:53:40 +0000 Subject: [PATCH 1379/2047] Fix style --- src/IO/Archives/LibArchiveReader.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index f1c2858dba9..87ff734eab7 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int SEEK_POSITION_OUT_OF_BOUND; extern const int NOT_IMPLEMENTED; + extern const int CANNOT_READ_ALL_DATA; } From 8b0f52f4114099b8a3fc0e37caa8b3f9a99cc054 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 31 Jul 2023 15:04:30 +0000 Subject: [PATCH 1380/2047] Don't report LOGICAL_ERROR if a file got truncated during read --- src/IO/ParallelReadBuffer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index 92af1ed0b04..8d73f221748 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -8,7 +8,7 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int UNEXPECTED_END_OF_FILE; extern const int CANNOT_SEEK_THROUGH_FILE; extern const int SEEK_POSITION_OUT_OF_BOUND; @@ -260,7 +260,7 @@ void ParallelReadBuffer::readerThreadFunction(ReadWorkerPtr read_worker) if (!on_progress(r) && r < read_worker->segment.size()) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::UNEXPECTED_END_OF_FILE, "Failed to read all the data from the reader at offset {}, got {}/{} bytes", read_worker->start_offset, r, read_worker->segment.size()); } From c4a00b8c68ce39c4ee325ab8b29ea86e58dea8af Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 Jul 2023 15:39:07 +0000 Subject: [PATCH 1381/2047] Throw S3Exception whenever possible. --- src/Backups/BackupIO_S3.cpp | 6 +++--- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/IO/S3/copyS3File.cpp | 2 +- src/IO/S3/getObjectInfo.cpp | 2 +- src/Storages/DataLakes/S3MetadataReader.cpp | 4 ++-- src/Storages/StorageS3.cpp | 4 ++-- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index d487ec6e80e..40a043be552 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -88,7 +88,7 @@ namespace request.SetMaxKeys(1); auto outcome = client.ListObjects(request); if (!outcome.IsSuccess()) - throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); return outcome.GetResult().GetContents(); } @@ -271,7 +271,7 @@ void BackupWriterS3::removeFile(const String & file_name) request.SetKey(fs::path(s3_uri.key) / file_name); auto outcome = client->DeleteObject(request); if (!outcome.IsSuccess() && !isNotFoundError(outcome.GetError().GetErrorType())) - throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } void BackupWriterS3::removeFiles(const Strings & file_names) @@ -329,7 +329,7 @@ void BackupWriterS3::removeFilesBatch(const Strings & file_names) auto outcome = client->DeleteObjects(request); if (!outcome.IsSuccess() && !isNotFoundError(outcome.GetError().GetErrorType())) - throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index e46ca3d0828..4da7b3e892f 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -135,7 +135,7 @@ private: return result; } - throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", + throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", quoteString(request.GetBucket()), quoteString(request.GetPrefix()), backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); } diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 2de2ccd0f9f..dda107840cb 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -783,7 +783,7 @@ namespace if (!outcome.IsSuccess()) { abortMultipartUpload(); - throw Exception::createDeprecated(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } return outcome.GetResult().GetCopyPartResult().GetETag(); diff --git a/src/IO/S3/getObjectInfo.cpp b/src/IO/S3/getObjectInfo.cpp index c652f16ab20..88f79f8d8d5 100644 --- a/src/IO/S3/getObjectInfo.cpp +++ b/src/IO/S3/getObjectInfo.cpp @@ -85,7 +85,7 @@ ObjectInfo getObjectInfo( } else if (throw_on_error) { - throw DB::Exception(ErrorCodes::S3_ERROR, + throw S3Exception(error.GetErrorType(), "Failed to get object info: {}. HTTP response code: {}", error.GetMessage(), static_cast(error.GetResponseCode())); } diff --git a/src/Storages/DataLakes/S3MetadataReader.cpp b/src/Storages/DataLakes/S3MetadataReader.cpp index f62c440bc2f..ac472c190e4 100644 --- a/src/Storages/DataLakes/S3MetadataReader.cpp +++ b/src/Storages/DataLakes/S3MetadataReader.cpp @@ -57,8 +57,8 @@ std::vector S3DataLakeMetadataReadHelper::listFiles( { outcome = client->ListObjectsV2(request); if (!outcome.IsSuccess()) - throw Exception( - ErrorCodes::S3_ERROR, + throw S3Exception( + outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", quoteString(bucket), quoteString(base_configuration.url.key), diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index b52150250b8..ebce3a7aeca 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -245,7 +245,7 @@ private: if (!outcome.IsSuccess()) { - throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", + throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", quoteString(request.GetBucket()), quoteString(request.GetPrefix()), backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); } @@ -1195,7 +1195,7 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, if (!response.IsSuccess()) { const auto & err = response.GetError(); - throw Exception(ErrorCodes::S3_ERROR, "{}: {}", std::to_string(static_cast(err.GetErrorType())), err.GetMessage()); + throw S3Exception(err.GetMessage(), err.GetErrorType()); } for (const auto & error : response.GetResult().GetErrors()) From cbedc5553c526968671267d4ec6136181d1b96ec Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 31 Jul 2023 18:15:59 +0200 Subject: [PATCH 1382/2047] Update 00002_log_and_exception_messages_formatting.reference --- .../00002_log_and_exception_messages_formatting.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference index 8a38707f8da..11b660b54a3 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -1,7 +1,7 @@ runtime messages 0.001 runtime exceptions 0.05 messages shorter than 10 1 -messages shorter than 16 4 +messages shorter than 16 3 exceptions shorter than 30 3 noisy messages 0.3 noisy Trace messages 0.16 From f9e0a202a7eb55b17b427f4ad9439ca80ce21821 Mon Sep 17 00:00:00 2001 From: Andy Fiddaman Date: Mon, 31 Jul 2023 17:45:54 +0100 Subject: [PATCH 1383/2047] clickhouse-keeper: fix implementation of server with poll() --- src/Server/KeeperTCPHandler.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index e3edc281e83..e44ab4a4bd7 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -175,10 +175,12 @@ struct SocketInterruptablePollWrapper } while (rc < 0 && errno == POCO_EINTR); - if (rc >= 1 && poll_buf[0].revents & POLLIN) - socket_ready = true; - if (rc >= 2 && poll_buf[1].revents & POLLIN) - fd_ready = true; + if (rc >= 1) { + if (poll_buf[0].revents & POLLIN) + socket_ready = true; + if (poll_buf[1].revents & POLLIN) + fd_ready = true; + } #endif } From 8447a93931d59b3b3a2c1fc1fef0c14ae43d62b4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 31 Jul 2023 14:22:30 +0000 Subject: [PATCH 1384/2047] fix sorting of sparse columns with large limit --- src/Columns/ColumnSparse.cpp | 3 ++- .../Transforms/PartialSortingTransform.cpp | 2 +- .../02834_sparse_columns_sort_with_limit.reference | 1 + .../02834_sparse_columns_sort_with_limit.sql | 12 ++++++++++++ 4 files changed, 16 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02834_sparse_columns_sort_with_limit.reference create mode 100644 tests/queries/0_stateless/02834_sparse_columns_sort_with_limit.sql diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 0cad2c2d36b..d5edae7e7ea 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -1,3 +1,4 @@ +#include "Common/typeid_cast.h" #include #include #include @@ -439,7 +440,7 @@ void ColumnSparse::compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const { - if (row_indexes) + if (row_indexes || !typeid_cast(&rhs)) { /// TODO: implement without conversion to full column. auto this_full = convertToFullColumnIfSparse(); diff --git a/src/Processors/Transforms/PartialSortingTransform.cpp b/src/Processors/Transforms/PartialSortingTransform.cpp index 33c2d870b76..3fc9a4e71db 100644 --- a/src/Processors/Transforms/PartialSortingTransform.cpp +++ b/src/Processors/Transforms/PartialSortingTransform.cpp @@ -159,7 +159,7 @@ void PartialSortingTransform::transform(Chunk & chunk) { MutableColumnPtr sort_description_threshold_column_updated = raw_block_columns[i]->cloneEmpty(); sort_description_threshold_column_updated->insertFrom(*raw_block_columns[i], min_row_to_compare); - sort_description_threshold_columns_updated[i] = std::move(sort_description_threshold_column_updated); + sort_description_threshold_columns_updated[i] = sort_description_threshold_column_updated->convertToFullColumnIfSparse(); } sort_description_threshold_columns = std::move(sort_description_threshold_columns_updated); diff --git a/tests/queries/0_stateless/02834_sparse_columns_sort_with_limit.reference b/tests/queries/0_stateless/02834_sparse_columns_sort_with_limit.reference new file mode 100644 index 00000000000..e2ed8f4daf2 --- /dev/null +++ b/tests/queries/0_stateless/02834_sparse_columns_sort_with_limit.reference @@ -0,0 +1 @@ +65536 diff --git a/tests/queries/0_stateless/02834_sparse_columns_sort_with_limit.sql b/tests/queries/0_stateless/02834_sparse_columns_sort_with_limit.sql new file mode 100644 index 00000000000..32bd9694bd0 --- /dev/null +++ b/tests/queries/0_stateless/02834_sparse_columns_sort_with_limit.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS t_sparse_sort_limit; + +CREATE TABLE t_sparse_sort_limit (date Date, i UInt64, v Int16) +ENGINE = MergeTree ORDER BY (date, i) +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9; + +INSERT INTO t_sparse_sort_limit SELECT '2020-10-10', number % 10, number FROM numbers(100000); +INSERT INTO t_sparse_sort_limit SELECT '2020-10-11', number % 10, number FROM numbers(100000); + +SELECT count() FROM (SELECT toStartOfMonth(date) AS d FROM t_sparse_sort_limit ORDER BY -i LIMIT 65536); + +DROP TABLE IF EXISTS t_sparse_sort_limit; From fe6202c1cc505076165729ae14bc3841c2ee04e7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 17:49:34 +0000 Subject: [PATCH 1385/2047] Disable for fasttest --- docker/test/fasttest/Dockerfile | 2 -- tests/queries/0_stateless/02661_select_from_table_in_archive.sh | 1 + 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 5bd8cd9b332..da4baa8c687 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -19,8 +19,6 @@ RUN apt-get update \ unixodbc \ pv \ zstd \ - zip \ - p7zip-full \ --yes --no-install-recommends RUN pip3 install numpy scipy pandas Jinja2 diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh index af3366ab924..14443f62770 100755 --- a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh +++ b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From f13343b7ee49838fafa3ba71dd37d19741c98785 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 31 Jul 2023 17:50:24 +0000 Subject: [PATCH 1386/2047] ASD --- src/Columns/ColumnSparse.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index d5edae7e7ea..4f76a9be4b9 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -1,4 +1,3 @@ -#include "Common/typeid_cast.h" #include #include #include From 731b780b84876a7fbb9d7c9a9cd10fe197ea65e0 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Mon, 31 Jul 2023 21:06:13 +0300 Subject: [PATCH 1387/2047] Add integration test --- .../__init__.py | 0 .../configs/storage_config.xml | 32 ++++++++ .../test.py | 80 +++++++++++++++++++ 3 files changed, 112 insertions(+) create mode 100644 tests/integration/test_backup_restore_storage_policy/__init__.py create mode 100644 tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml create mode 100644 tests/integration/test_backup_restore_storage_policy/test.py diff --git a/tests/integration/test_backup_restore_storage_policy/__init__.py b/tests/integration/test_backup_restore_storage_policy/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml b/tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml new file mode 100644 index 00000000000..960e71294bb --- /dev/null +++ b/tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml @@ -0,0 +1,32 @@ + + + + + local + /var/lib/backups/ + + + local + /var/lib/disks/one/ + + + local + /var/lib/disks/two/ + + + + + + + one + two + + + + + + + bak + /var/lib/backups/ + + diff --git a/tests/integration/test_backup_restore_storage_policy/test.py b/tests/integration/test_backup_restore_storage_policy/test.py new file mode 100644 index 00000000000..78ce4d70180 --- /dev/null +++ b/tests/integration/test_backup_restore_storage_policy/test.py @@ -0,0 +1,80 @@ +import pytest +from helpers.cluster import ClickHouseCluster + + +backup_id_counter = 0 + +cluster = ClickHouseCluster(__file__) + +instance = cluster.add_instance( + "instance", + main_configs=["configs/storage_config.xml"], +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def cleanup_after_test(): + try: + yield + finally: + instance.query("DROP DATABASE IF EXISTS test") + + +def new_backup_name(): + global backup_id_counter + backup_id_counter += 1 + return f"Disk('bak', '{backup_id_counter}/')" + + +def create_and_fill_table(n=100): + instance.query("CREATE DATABASE test") + instance.query( + "CREATE TABLE test.table(x UInt32, y String) ENGINE=MergeTree ORDER BY y PARTITION BY x%10" + ) + instance.query( + f"INSERT INTO test.table SELECT number, toString(number) FROM numbers({n})" + ) + + +@pytest.mark.parametrize("policy", ["disks_in_order", "", None]) +def test_restore_table(policy): + backup_name = new_backup_name() + n = 20 + sum_n = int((n * (n - 1)) / 2) + expected = f"{n}\t{sum_n}\n" + + create_and_fill_table(n) + + assert instance.query("SELECT count(), sum(x) FROM test.table") == expected + + instance.query(f"BACKUP TABLE test.table TO {backup_name}") + + instance.query("DROP TABLE test.table SYNC") + + assert instance.query("EXISTS test.table") == "0\n" + + restore_query = f"RESTORE TABLE test.table FROM {backup_name}" + if policy is None: + policy = "default" + else: + restore_query += f" SETTINGS storage_policy = '{policy}'" + if policy == "": + policy = "default" + + instance.query(restore_query) + + assert instance.query("SELECT count(), sum(x) FROM test.table") == expected + + assert ( + instance.query("SELECT storage_policy FROM system.tables WHERE name='table'") + == f"{policy}\n" + ) From 870a506a0bb477cd17b65c06b12a73ef4320958c Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Jul 2023 20:07:23 +0200 Subject: [PATCH 1388/2047] Some fixes --- src/Core/Settings.h | 2 +- src/Storages/S3Queue/S3QueueHolder.cpp | 135 ++++++++++-------------- src/Storages/S3Queue/S3QueueHolder.h | 15 ++- src/Storages/S3Queue/S3QueueSource.cpp | 58 ++++------ src/Storages/S3Queue/S3QueueSource.h | 7 -- src/Storages/S3Queue/StorageS3Queue.cpp | 68 ++++++------ src/Storages/S3Queue/StorageS3Queue.h | 5 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3.h | 15 ++- 9 files changed, 131 insertions(+), 176 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e6c756ec4ae..923576b5f24 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -104,7 +104,7 @@ class IColumn; M(UInt64, s3_retry_attempts, 10, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(UInt64, s3_request_timeout_ms, 3000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ - M(String, s3queue_default_zookeeper_path, "/", "Default zookeeper path prefix for S3Queue engine", 0) \ + M(String, s3queue_default_zookeeper_path, "/s3queue/", "Default zookeeper path prefix for S3Queue engine", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 7997fd2e051..66f540ba24e 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -1,3 +1,4 @@ +#include "IO/VarInt.h" #include "config.h" #if USE_AWS_S3 @@ -23,33 +24,34 @@ namespace ErrorCodes void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) { - assertString("collection:\n", in); - files = {}; - while (!in.eof()) - { - String file_name; - UInt64 timestamp; - UInt64 retries_count; - in >> file_name >> "\n"; - in >> timestamp >> "\n"; - in >> retries_count >> "\n"; - TrackedCollectionItem item = {.file_path=file_name, .timestamp=timestamp, .retries_count=retries_count}; + if (in.eof()) + return; + + size_t files_num; + in >> files_num >> "\n"; + while (files_num--) + { + TrackedCollectionItem item; + in >> item.file_path >> "\n"; + in >> item.timestamp >> "\n"; + in >> item.retries_count >> "\n"; + in >> item.last_exception >> "\n"; files.push_back(item); } } void S3QueueHolder::S3QueueCollection::write(WriteBuffer & out) const { - out << "collection:\n"; + out << files.size() << "\n"; for (const auto & processed_file : files) { out << processed_file.file_path << "\n"; out << processed_file.timestamp << "\n"; out << processed_file.retries_count << "\n"; + out << processed_file.last_exception << "\n"; } - /// todo(kssenii): use a more flexible format? } String S3QueueHolder::S3QueueCollection::toString() const @@ -79,16 +81,14 @@ void S3QueueHolder::S3QueueProcessedCollection::parse(const String & collection_ { ReadBufferFromString buf(collection_str); read(buf); - // Remove old items - if (max_age > 0) + if (max_age > 0) // Remove old items { UInt64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); UInt64 max_seconds_diff = max_age; - auto new_end = std::remove_if( - files.begin(), files.end(), - [×tamp, &max_seconds_diff](TrackedCollectionItem processed_file) + std::erase_if( + files, + [×tamp, &max_seconds_diff](const TrackedCollectionItem & processed_file) { return (timestamp - processed_file.timestamp) > max_seconds_diff; }); - files.erase(new_end, files.end()); } } @@ -100,12 +100,10 @@ void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) TrackedCollectionItem processed_file = {.file_path=file_name, .timestamp=timestamp}; files.push_back(processed_file); - // Check set size - if (files.size() > max_size) + while (files.size() > max_size) { - files.erase(files.begin(), files.begin() + (files.size() - max_size)); + files.pop_front(); } - /// todo(kssenii): use deque here } @@ -120,7 +118,7 @@ void S3QueueHolder::S3QueueFailedCollection::parse(const String & collection_str } -bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) +bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name, const String & exception_message) { auto failed_it = std::find_if(files.begin(), files.end(), [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); @@ -133,7 +131,7 @@ bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) } else { - TrackedCollectionItem failed_file = {.file_path=file_name, .retries_count=max_retries_count}; + TrackedCollectionItem failed_file = { .file_path=file_name, .retries_count=max_retries_count, .last_exception = exception_message }; files.push_back(failed_file); } return true; @@ -163,6 +161,7 @@ S3QueueHolder::S3QueueHolder( , max_set_size(max_set_size_) , max_set_age_sec(max_set_age_sec_) , max_loading_retries(max_loading_retries_) + , zk_client(getContext()->getZooKeeper()) , zookeeper_path(zookeeper_path_) , zookeeper_failed_path(fs::path(zookeeper_path_) / "failed") , zookeeper_processing_path(fs::path(zookeeper_path_) / "processing") @@ -171,31 +170,20 @@ S3QueueHolder::S3QueueHolder( , mode(mode_) , log(&Poco::Logger::get("S3QueueHolder")) { - current_zookeeper = getContext()->getZooKeeper(); - if (!current_zookeeper) - throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); } -zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const -{ - /// todo(kssenii): current_zookeeper is not updated at all apart from in constructor, remove the lock? - std::lock_guard lock(current_zookeeper_mutex); - return current_zookeeper; -} - void S3QueueHolder::setFileProcessed(const String & file_path) { - auto zookeeper = getZooKeeper(); auto lock = acquireLock(); if (mode == S3QueueMode::UNORDERED) { - String processed_files = zookeeper->get(zookeeper_processed_path); + String processed_files = zk_client->get(zookeeper_processed_path); auto processed = S3QueueProcessedCollection(max_set_size, max_set_age_sec); processed.parse(processed_files); processed.add(file_path); - zookeeper->set(zookeeper_processed_path, processed.toString()); + zk_client->set(zookeeper_processed_path, processed.toString()); } else if (mode == S3QueueMode::ORDERED) { @@ -204,24 +192,22 @@ void S3QueueHolder::setFileProcessed(const String & file_path) // This check can be useful, when multiple table engines consume in ordered mode. if (max_file.compare(file_path) <= 0) { - zookeeper->set(zookeeper_processed_path, file_path); + zk_client->set(zookeeper_processed_path, file_path); } } removeProcessingFile(file_path); } -bool S3QueueHolder::markFailedAndCheckRetry(const String & file_path) +bool S3QueueHolder::setFileFailed(const String & file_path, const String & exception_message) { - auto zookeeper = getZooKeeper(); auto lock = acquireLock(); - String failed_files = zookeeper->get(zookeeper_failed_path); auto failed_collection = S3QueueFailedCollection(max_loading_retries); - failed_collection.parse(failed_files); - bool retry_later = failed_collection.add(file_path); + failed_collection.parse(zk_client->get(zookeeper_failed_path)); + bool retry_later = failed_collection.add(file_path, exception_message); - zookeeper->set(zookeeper_failed_path, failed_collection.toString()); + zk_client->set(zookeeper_failed_path, failed_collection.toString()); removeProcessingFile(file_path); return retry_later; @@ -229,8 +215,7 @@ bool S3QueueHolder::markFailedAndCheckRetry(const String & file_path) S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() { - auto zookeeper = getZooKeeper(); - String failed_files = zookeeper->get(zookeeper_failed_path); + String failed_files = zk_client->get(zookeeper_failed_path); auto failed_collection = S3QueueFailedCollection(max_loading_retries); failed_collection.parse(failed_files); @@ -240,85 +225,79 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() String S3QueueHolder::getMaxProcessedFile() { - auto zookeeper = getZooKeeper(); - String processed = zookeeper->get(zookeeper_processed_path); + String processed = zk_client->get(zookeeper_processed_path); return processed; } S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() { - auto zookeeper = getZooKeeper(); - String processing = zookeeper->get(zookeeper_processing_path); - return parseCollection(processing); + String processing_files; + if (!zk_client->tryGet(zookeeper_processing_path, processing_files)) + return {}; + return parseCollection(processing_files); } void S3QueueHolder::setFilesProcessing(Strings & file_paths) { - auto zookeeper = getZooKeeper(); std::unordered_set processing_files(file_paths.begin(), file_paths.end()); processing_files.merge(getProcessingFiles()); + String processing_files_str = toString(Strings(processing_files.begin(), processing_files.end())); - Strings processing_file_paths(processing_files.begin(), processing_files.end()); - zookeeper->set(fs::path(zookeeper_processing_path), toString(processing_file_paths)); + if (zk_client->exists(zookeeper_processing_path)) + zk_client->set(fs::path(zookeeper_processing_path), processing_files_str); + else + zk_client->create(fs::path(zookeeper_processing_path), processing_files_str, zkutil::CreateMode::Ephemeral); } S3QueueHolder::S3FilesCollection S3QueueHolder::getUnorderedProcessedFiles() { - auto zookeeper = getZooKeeper(); - - String processed = zookeeper->get(zookeeper_processed_path); + String processed = zk_client->get(zookeeper_processed_path); auto collection = S3QueueProcessedCollection(max_set_size, max_set_age_sec); collection.parse(processed); - return collection.getFileNames(); } -S3QueueHolder::S3FilesCollection S3QueueHolder::getExcludedFiles() +S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessedAndFailedFiles() { - auto zookeeper = getZooKeeper(); - - S3FilesCollection exclude_files = getFailedFiles(); + S3FilesCollection processed_and_failed_files = getFailedFiles(); if (mode == S3QueueMode::UNORDERED) { - S3FilesCollection processed_files = getUnorderedProcessedFiles(); - exclude_files.merge(processed_files); + processed_and_failed_files.merge(getUnorderedProcessedFiles()); } else { String processed = getMaxProcessedFile(); - exclude_files.insert(processed); + processed_and_failed_files.insert(processed); } S3FilesCollection processing_files = getProcessingFiles(); - exclude_files.merge(processing_files); + processed_and_failed_files.merge(processing_files); - return exclude_files; + return processed_and_failed_files; } void S3QueueHolder::removeProcessingFile(const String & file_path) { - auto zookeeper = getZooKeeper(); String node_data; - String processing = zookeeper->get(zookeeper_processing_path); + String processing = zk_client->get(zookeeper_processing_path); S3FilesCollection processing_files = parseCollection(processing); processing_files.erase(file_path); Strings file_paths(processing_files.begin(), processing_files.end()); - zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); + zk_client->set(fs::path(zookeeper_processing_path), toString(file_paths)); } std::shared_ptr S3QueueHolder::acquireLock() { - auto zookeeper = getZooKeeper(); UInt32 retry_count = 200; UInt32 sleep_ms = 100; UInt32 retries = 0; while (true) { - Coordination::Error code = zookeeper->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); + Coordination::Error code = zk_client->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { retries++; @@ -334,14 +313,14 @@ std::shared_ptr S3QueueHolder::acquireLock() } else { - return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zookeeper); + return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zk_client); } } } -S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) +S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(const String & collection_str) { - ReadBufferFromString rb(files); + ReadBufferFromString rb(collection_str); Strings deserialized; try { @@ -353,9 +332,7 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) deserialized = {}; } - std::unordered_set processed(deserialized.begin(), deserialized.end()); - - return processed; + return std::unordered_set(deserialized.begin(), deserialized.end()); } } diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index 83edd8a2e74..de7f1f56f9a 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -17,10 +17,11 @@ public: String file_path; UInt64 timestamp = 0; UInt64 retries_count = 0; + String last_exception; }; using S3FilesCollection = std::unordered_set; - using TrackedFiles = std::vector; + using TrackedFiles = std::deque; S3QueueHolder( const String & zookeeper_path_, @@ -31,9 +32,9 @@ public: UInt64 & max_loading_retries_); void setFileProcessed(const String & file_path); - bool markFailedAndCheckRetry(const String & file_path); + bool setFileFailed(const String & file_path, const String & exception_message); void setFilesProcessing(Strings & file_paths); - S3FilesCollection getExcludedFiles(); + S3FilesCollection getProcessedAndFailedFiles(); String getMaxProcessedFile(); std::shared_ptr acquireLock(); @@ -73,7 +74,7 @@ public: S3QueueFailedCollection(const UInt64 & max_retries_count_); void parse(const String & collection_str) override; - bool add(const String & file_name); + bool add(const String & file_name, const String & exception_message); S3FilesCollection getFileNames(); @@ -87,7 +88,7 @@ private: const UInt64 max_set_age_sec; const UInt64 max_loading_retries; - zkutil::ZooKeeperPtr current_zookeeper; + zkutil::ZooKeeperPtr zk_client; mutable std::mutex current_zookeeper_mutex; mutable std::mutex mutex; const String zookeeper_path; @@ -99,14 +100,12 @@ private: const UUID table_uuid; Poco::Logger * log; - zkutil::ZooKeeperPtr getZooKeeper() const; - S3FilesCollection getFailedFiles(); S3FilesCollection getProcessingFiles(); S3FilesCollection getUnorderedProcessedFiles(); void removeProcessingFile(const String & file_path); - S3FilesCollection parseCollection(String & files); + S3FilesCollection parseCollection(const String & collection_str); }; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 31eda8db31f..4785a131f60 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -21,7 +21,6 @@ # include # include -# include # include # include # include @@ -74,21 +73,17 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( const Block & virtual_header, ContextPtr context, UInt64 & max_poll_size_, - StorageS3QueueSource::KeysWithInfo * read_keys_, const S3Settings::RequestSettings & request_settings_) : max_poll_size(max_poll_size_) - , bucket(globbed_uri_.bucket) , glob_iterator(std::make_unique( - client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) + client_, globbed_uri_, query, virtual_header, context, nullptr, request_settings_)) { /// todo(kssenii): remove this loop, it should not be here while (true) { KeyWithInfo val = glob_iterator->next(); if (val.key.empty()) - { break; - } keys_buf.push_back(val); } } @@ -98,16 +93,17 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( { for (const KeyWithInfo & val : keys_buf) { - auto full_path = bucket + '/' + val.key; + auto full_path = val.key; if (exclude_keys.find(full_path) != exclude_keys.end()) { - LOG_TRACE(log, "Found in exclude keys {}", val.key); + LOG_TEST(log, "File {} will be skipped, because it was found in exclude files list " + "(either already processed or failed to be processed)", val.key); continue; } + if ((engine_mode == S3QueueMode::ORDERED) && (full_path.compare(max_file) <= 0)) - { continue; - } + if ((processing_keys.size() < max_poll_size) || (engine_mode == S3QueueMode::ORDERED)) { processing_keys.push_back(val); @@ -124,6 +120,7 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( processing_keys.begin(), processing_keys.end(), [](const KeyWithInfo & lhs, const KeyWithInfo & rhs) { return lhs.key.compare(rhs.key) < 0; }); + if (processing_keys.size() > max_poll_size) { processing_keys.erase(processing_keys.begin() + max_poll_size, processing_keys.end()); @@ -132,11 +129,9 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( Strings keys; for (const auto & key_info : processing_keys) - { - keys.push_back(bucket + '/' + key_info.key); - } - processing_keys.push_back(KeyWithInfo()); + keys.push_back(key_info.key); + processing_keys.push_back(KeyWithInfo()); processing_iterator = processing_keys.begin(); return keys; } @@ -153,12 +148,6 @@ StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next( return KeyWithInfo(); } -size_t StorageS3QueueSource::QueueGlobIterator::getTotalSize() const -{ - return glob_iterator->getTotalSize(); -} - - Block StorageS3QueueSource::getHeader(Block sample_block, const std::vector & requested_virtual_columns) { for (const auto & virtual_column : requested_virtual_columns) @@ -232,6 +221,7 @@ String StorageS3QueueSource::getName() const Chunk StorageS3QueueSource::generate() { + auto file_progress = getContext()->getFileProgressCallback(); while (true) { if (isCancelled() || !reader) @@ -243,22 +233,12 @@ Chunk StorageS3QueueSource::generate() Chunk chunk; bool success_in_pulling = false; - String file_path; try { if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); - - file_path = reader.getPath(); - size_t total_size = file_iterator->getTotalSize(); - if (num_rows && total_size) - { - size_t chunk_size = reader.getFormat()->getApproxBytesReadForChunk(); - if (!chunk_size) - chunk_size = chunk.bytes(); - updateRowsProgressApprox(*this, num_rows, chunk_size, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); - } + auto file_path = reader.getPath(); for (const auto & virtual_column : requested_virtual_columns) { @@ -279,14 +259,13 @@ Chunk StorageS3QueueSource::generate() catch (const Exception & e) { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); - const auto & failed_file_path = reader.getPath(); - queue_holder->markFailedAndCheckRetry(failed_file_path); + queue_holder->setFileFailed(reader.getFile(), e.message()); success_in_pulling = false; } if (success_in_pulling) { - applyActionAfterProcessing(file_path); - queue_holder->setFileProcessed(file_path); + applyActionAfterProcessing(reader.getFile()); + queue_holder->setFileProcessed(reader.getFile()); return chunk; } @@ -296,6 +275,7 @@ Chunk StorageS3QueueSource::generate() if (!reader) break; + /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. internal_source->create_reader_pool.wait(); @@ -320,12 +300,10 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) void StorageS3QueueSource::deleteProcessedObject(const String & file_path) { - LOG_WARNING(log, "Delete processed file {} from bucket {}", file_path, bucket); - S3::DeleteObjectRequest request; - /// todo(kssenii): looks incorrect - String delete_key = file_path.substr(bucket.length() + 1); + LOG_INFO(log, "Delete processed file {} from bucket {}", file_path, bucket); - request.WithKey(delete_key).WithBucket(bucket); + S3::DeleteObjectRequest request; + request.WithKey(file_path).WithBucket(bucket); auto outcome = client->DeleteObject(request); if (!outcome.IsSuccess()) { diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index a55dd2cbe40..e2e472b5007 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -46,18 +46,15 @@ public: const Block & virtual_header, ContextPtr context, UInt64 & max_poll_size_, - KeysWithInfo * read_keys_ = nullptr, const S3Settings::RequestSettings & request_settings_ = {}); KeyWithInfo next() override; - size_t getTotalSize() const override; Strings filterProcessingFiles(const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file = ""); private: UInt64 max_poll_size; - const String bucket; KeysWithInfo keys_buf; KeysWithInfo processing_keys; mutable std::mutex mutex; @@ -116,10 +113,6 @@ private: std::future reader_future; - UInt64 total_rows_approx_max = 0; - size_t total_rows_count_times = 0; - UInt64 total_rows_approx_accumulated = 0; - mutable std::mutex mutex; std::shared_ptr internal_source; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 71a93884896..d60c9adb4c8 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -24,7 +24,6 @@ # include # include -# include # include # include # include @@ -107,44 +106,43 @@ StorageS3Queue::StorageS3Queue( , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { if (!withGlobs()) - { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); - } - String setting_zookeeper_path = s3queue_settings->keeper_path; - LOG_INFO(log, "Settings zookeeper_path={}", setting_zookeeper_path); + std::string setting_zookeeper_path = s3queue_settings->keeper_path; + std::string zk_path_prefix; if (setting_zookeeper_path.empty()) { - auto table_id = getStorageID(); - auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); + auto database = DatabaseCatalog::instance().getDatabase(table_id_.database_name); bool is_in_replicated_database = database->getEngineName() == "Replicated"; auto default_path = getContext()->getSettingsRef().s3queue_default_zookeeper_path.value; if (!default_path.empty()) { - zookeeper_path - = zkutil::extractZooKeeperPath(fs::path(default_path) / toString(table_id.uuid), /* check_starts_with_slash */ true, log); + zk_path_prefix = default_path; } else if (is_in_replicated_database) { - LOG_INFO(log, "S3Queue engine keeper_path not specified. Use replicated database zookeeper path"); - String base_zookeeper_path = assert_cast(database.get())->getZooKeeperPath(); - zookeeper_path = zkutil::extractZooKeeperPath( - fs::path(base_zookeeper_path) / "s3queue" / toString(table_id.uuid), /* check_starts_with_slash */ true, log); + LOG_INFO(log, "S3Queue engine zookeeper path is not specified. " + "Using replicated database zookeeper path"); + + zk_path_prefix = fs::path(assert_cast(database.get())->getZooKeeperPath()) / "s3queue"; } else { - throw Exception( - ErrorCodes::NO_ZOOKEEPER, - "S3Queue keeper_path engine setting not specified, s3queue_default_zookeeper_path_prefix not specified and table not in " - "replicated database."); + throw Exception(ErrorCodes::NO_ZOOKEEPER, + "S3Queue keeper_path engine setting not specified, " + "s3queue_default_zookeeper_path_prefix not specified"); } } else { - zookeeper_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path, /* check_starts_with_slash */ true, log); + zk_path_prefix = s3queue_settings->keeper_path.value; } - LOG_INFO(log, "Set zookeeper_path={}", zookeeper_path); + + zookeeper_path = zkutil::extractZooKeeperPath( + fs::path(zk_path_prefix) / toString(table_id_.uuid), /* check_starts_with_slash */ true, log); + + LOG_INFO(log, "Using zookeeper path: {}", zookeeper_path); FormatFactory::instance().checkFormatName(format_name); context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); @@ -550,8 +548,8 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ { String metadata_str = S3QueueTableMetadata(s3_configuration, *s3queue_settings).toString(); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "collection:\n", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "collection:\n", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); @@ -612,11 +610,8 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const std::shared_ptr -StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys) +StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query) { - /// Iterate through disclosed globs and make a source for each file - std::lock_guard lock{sync_mutex}; - auto it = std::make_shared( *s3_configuration.client, s3_configuration.url, @@ -624,26 +619,35 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW virtual_block, local_context, s3queue_settings->s3queue_polling_size.value, - read_keys, s3_configuration.request_settings); - auto zookeeper_lock = queue_holder->acquireLock(); - S3QueueHolder::S3FilesCollection exclude = queue_holder->getExcludedFiles(); + auto lock = queue_holder->acquireLock(); + S3QueueHolder::S3FilesCollection files_to_skip = queue_holder->getProcessedAndFailedFiles(); - Strings processing_files; + Strings files_to_process; if (mode == S3QueueMode::UNORDERED) { - processing_files = it->filterProcessingFiles(mode, exclude); + files_to_process = it->filterProcessingFiles(mode, files_to_skip); } else { String max_processed_file = queue_holder->getMaxProcessedFile(); - processing_files = it->filterProcessingFiles(mode, exclude, max_processed_file); + files_to_process = it->filterProcessingFiles(mode, files_to_skip, max_processed_file); } - queue_holder->setFilesProcessing(processing_files); + + LOG_TEST(log, "Found files to process: {}", fmt::join(files_to_process, ", ")); + + queue_holder->setFilesProcessing(files_to_process); return it; } +void StorageS3Queue::drop() +{ + auto zk_client = getZooKeeper(); + if (zk_client->exists(zookeeper_path)) + zk_client->removeRecursive(zookeeper_path); +} + void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { factory.registerStorage( diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index f9500371489..6af38058dd8 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -109,9 +109,9 @@ private: std::atomic shutdown_called{false}; Poco::Logger * log; - void startup() override; void shutdown() override; + void drop() override; struct TaskContext { @@ -126,7 +126,6 @@ private: zkutil::ZooKeeperPtr current_zookeeper; mutable std::mutex current_zookeeper_mutex; - mutable std::mutex sync_mutex; void setZooKeeper(); zkutil::ZooKeeperPtr tryGetZooKeeper() const; @@ -140,7 +139,7 @@ private: using KeysWithInfo = StorageS3QueueSource::KeysWithInfo; std::shared_ptr - createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys = nullptr); + createFileIterator(ContextPtr local_context, ASTPtr query); void streamToViews(); Configuration updateConfigurationAndGetCopy(ContextPtr local_context); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index b52150250b8..7525edfc718 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -596,7 +596,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); - return ReaderHolder{fs::path(bucket) / key_with_info.key, std::move(read_buf), std::move(input_format), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{key_with_info.key, bucket, std::move(read_buf), std::move(input_format), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3Source::createReaderAsync() diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index dd2a750f2b2..e08c01d0c91 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -159,12 +159,14 @@ private: { public: ReaderHolder( - String path_, + String key_, + String bucket_, std::unique_ptr read_buf_, std::shared_ptr input_format_, std::unique_ptr pipeline_, std::unique_ptr reader_) - : path(std::move(path_)) + : key(std::move(key_)) + , bucket(std::move(bucket_)) , read_buf(std::move(read_buf_)) , input_format(std::move(input_format_)) , pipeline(std::move(pipeline_)) @@ -189,19 +191,22 @@ private: pipeline = std::move(other.pipeline); input_format = std::move(other.input_format); read_buf = std::move(other.read_buf); - path = std::move(other.path); + key = std::move(other.key); + bucket = std::move(other.bucket); return *this; } explicit operator bool() const { return reader != nullptr; } PullingPipelineExecutor * operator->() { return reader.get(); } const PullingPipelineExecutor * operator->() const { return reader.get(); } - const String & getPath() const { return path; } + String getPath() const { return fs::path(bucket) / key; } + const String & getFile() const { return key; } const IInputFormat * getInputFormat() const { return input_format.get(); } private: - String path; + String key; + String bucket; std::unique_ptr read_buf; std::shared_ptr input_format; std::unique_ptr pipeline; From 75f6a50079dda28dc88b038b618175bfd8178ff3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Jul 2023 20:08:31 +0200 Subject: [PATCH 1389/2047] Add todo --- src/Storages/S3Queue/S3QueueHolder.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 66f540ba24e..24bb31517a5 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -22,6 +22,8 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; } +/// TODO: update zk session if expired + void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) { files = {}; From 525da383163bebd72e3a1a90fc0f7ea73c2bd868 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 31 Jul 2023 18:58:11 +0000 Subject: [PATCH 1390/2047] increase min protocol version for sparse serialization --- src/Core/ProtocolDefines.h | 22 +++++++++++----------- src/Formats/NativeReader.cpp | 2 +- src/Formats/NativeWriter.cpp | 2 +- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index dbe13d9502a..1f7f1481aba 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -44,17 +44,6 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS 54451 -#define DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION 54454 - -/// Version of ClickHouse TCP protocol. -/// -/// Should be incremented manually on protocol changes. -/// -/// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, -/// later is just a number for server version (one number instead of commit SHA) -/// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54464 - #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 #define DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS_IN_INSERT 54456 @@ -77,3 +66,14 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_TOTAL_BYTES_IN_PROGRESS 54463 #define DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES 54464 + +#define DBMS_MIN_REVISION_WITH_SPARSE_SERIALIZATION 54465 + +/// Version of ClickHouse TCP protocol. +/// +/// Should be incremented manually on protocol changes. +/// +/// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, +/// later is just a number for server version (one number instead of commit SHA) +/// for simplicity (sometimes it may be more convenient in some use cases). +#define DBMS_TCP_PROTOCOL_VERSION 54465 diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index 4c25460eb63..bfe4aab6e3a 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -173,7 +173,7 @@ Block NativeReader::read() setVersionToAggregateFunctions(column.type, true, server_revision); SerializationPtr serialization; - if (server_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) + if (server_revision >= DBMS_MIN_REVISION_WITH_SPARSE_SERIALIZATION) { auto info = column.type->createSerializationInfo({}); diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index 8100a3868e6..e50ff224257 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -132,7 +132,7 @@ size_t NativeWriter::write(const Block & block) /// Serialization. Dynamic, if client supports it. SerializationPtr serialization; - if (client_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) + if (client_revision >= DBMS_MIN_REVISION_WITH_SPARSE_SERIALIZATION) { auto info = column.type->getSerializationInfo(*column.column); serialization = column.type->getSerialization(*info); From 2ed74507e77a308912cd9341b4e4c5d3ac5cd6fd Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Mon, 31 Jul 2023 22:04:54 +0300 Subject: [PATCH 1391/2047] Blacken test_backup_restore_storage_policy/test.py --- tests/integration/test_backup_restore_storage_policy/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_storage_policy/test.py b/tests/integration/test_backup_restore_storage_policy/test.py index 78ce4d70180..cc1412c03c3 100644 --- a/tests/integration/test_backup_restore_storage_policy/test.py +++ b/tests/integration/test_backup_restore_storage_policy/test.py @@ -51,7 +51,7 @@ def test_restore_table(policy): n = 20 sum_n = int((n * (n - 1)) / 2) expected = f"{n}\t{sum_n}\n" - + create_and_fill_table(n) assert instance.query("SELECT count(), sum(x) FROM test.table") == expected From f2fd1da89163b037495f2b10a6af936d1874be3b Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 31 Jul 2023 19:13:27 +0000 Subject: [PATCH 1392/2047] Fix a build error --- src/Functions/FunctionsHashing.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 929cb2f77f4..5363220bd01 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1074,7 +1074,7 @@ private: { // IPv6 addresses are parsed into four 32-bit components in big-endian ordering on both platforms, so no change is necessary. // Reference: `parseIPv6orIPv4` in src/Common/formatIPv6.h. - if constexpr (std::endian::native == std::endian::big && std::is_same_v) + if constexpr (std::endian::native == std::endian::big && std::is_same_v, IPv6>) return; transformEndianness(value); From 453463ba22e3083dd9e848b3a6089cb0be34ea98 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 Jul 2023 19:47:23 +0000 Subject: [PATCH 1393/2047] Cleaup on every iteration. --- src/Storages/MergeTree/localBackup.cpp | 31 +++++++++++++++++++------- 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/localBackup.cpp b/src/Storages/MergeTree/localBackup.cpp index 6faacf3c066..cae6993e3e8 100644 --- a/src/Storages/MergeTree/localBackup.cpp +++ b/src/Storages/MergeTree/localBackup.cpp @@ -125,9 +125,6 @@ void localBackup( size_t try_no = 0; const size_t max_tries = 10; - CleanupOnFail cleanup(disk_transaction ? std::function([]{}) : - [disk, destination_path]() { disk->removeRecursive(destination_path); }); - /** Files in the directory can be permanently added and deleted. * If some file is deleted during an attempt to make a backup, then try again, * because it's important to take into account any new files that might appear. @@ -136,10 +133,30 @@ void localBackup( { try { - if (copy_instead_of_hardlinks && !disk_transaction) - disk->copyDirectoryContent(source_path, disk, destination_path); - else + if (disk_transaction) + { localBackupImpl(disk, disk_transaction.get(), source_path, destination_path, make_source_readonly, 0, max_level, copy_instead_of_hardlinks, files_to_copy_intead_of_hardlinks); + } + else if (copy_instead_of_hardlinks) + { + CleanupOnFail cleanup([disk, destination_path]() { disk->removeRecursive(destination_path); }); + disk->copyDirectoryContent(source_path, disk, destination_path); + cleanup.success(); + } + else + { + std::function cleaner; + if (disk->supportZeroCopyReplication()) + /// Note: this code will create garbage on s3. We should always remove `copy_instead_of_hardlinks` files. + /// The third agrument should be a list of excpetions, but (looks like) it is ignored for keep_all_shared_data = true. + cleaner = [disk, destination_path]() { disk->removeSharedRecursive(destination_path, /*keep_all_shared_data*/ true, {}); }; + else + cleaner = [disk, destination_path]() { disk->removeRecursive(destination_path); }; + + CleanupOnFail cleanup(std::move(cleaner)); + localBackupImpl(disk, disk_transaction.get(), source_path, destination_path, make_source_readonly, 0, max_level, false, files_to_copy_intead_of_hardlinks); + cleanup.success(); + } } catch (const DB::ErrnoException & e) { @@ -166,8 +183,6 @@ void localBackup( break; } - - cleanup.success(); } } From 5570333c7c13773342c4190a6b829d00a7a3f9a3 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 31 Jul 2023 15:00:32 -0700 Subject: [PATCH 1394/2047] Add support for arbitrary endianness transformation --- src/Common/TransformEndianness.hpp | 36 +++++++++---------- .../Serializations/SerializationNumber.cpp | 2 +- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index fe43861f66f..05f7778a12e 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -9,19 +9,19 @@ namespace DB { -template +template requires std::is_integral_v inline void transformEndianness(T & value) { - if constexpr (endian != std::endian::native) + if constexpr (ToEndian != FromEndian) value = std::byteswap(value); } -template +template requires is_big_int_v inline void transformEndianness(T & x) { - if constexpr (std::endian::native != endian) + if constexpr (ToEndian != FromEndian) { auto & items = x.items; std::transform(std::begin(items), std::end(items), std::begin(items), [](auto & item) { return std::byteswap(item); }); @@ -29,49 +29,49 @@ inline void transformEndianness(T & x) } } -template +template requires is_decimal inline void transformEndianness(T & x) { - transformEndianness(x.value); + transformEndianness(x.value); } -template +template requires std::is_floating_point_v inline void transformEndianness(T & value) { - if constexpr (std::endian::native != endian) + if constexpr (ToEndian != FromEndian) { auto * start = reinterpret_cast(&value); std::reverse(start, start + sizeof(T)); } } -template +template requires std::is_scoped_enum_v inline void transformEndianness(T & x) { using UnderlyingType = std::underlying_type_t; - transformEndianness(reinterpret_cast(x)); + transformEndianness(reinterpret_cast(x)); } -template +template inline void transformEndianness(std::pair & pair) { - transformEndianness(pair.first); - transformEndianness(pair.second); + transformEndianness(pair.first); + transformEndianness(pair.second); } -template +template inline void transformEndianness(StrongTypedef & x) { - transformEndianness(x.toUnderType()); + transformEndianness(x.toUnderType()); } -template +template inline void transformEndianness(CityHash_v1_0_2::uint128 & x) { - transformEndianness(x.low64); - transformEndianness(x.high64); + transformEndianness(x.low64); + transformEndianness(x.high64); } } diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index ca2616f4582..6b9fd9c3982 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -169,7 +169,7 @@ void SerializationNumber::deserializeBinaryBulk(IColumn & column, ReadBuffer x.resize(initial_size + size / sizeof(typename ColumnVector::ValueType)); if constexpr (std::endian::native == std::endian::big && sizeof(T) >= 2) - std::ranges::for_each(x | std::views::drop(initial_size), [](auto & i) { transformEndianness(i); }); + std::ranges::for_each(x | std::views::drop(initial_size), [](auto & i) { transformEndianness(i); }); } template class SerializationNumber; From e5240f900b2f0c05060ce538e0d07fe233f53883 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 1 Aug 2023 00:09:13 +0200 Subject: [PATCH 1395/2047] make regexp analyzer recognize named capturing groups --- src/Common/OptimizedRegularExpression.cpp | 24 +++++++++++++++++++ src/Common/tests/gtest_optimize_re.cpp | 4 ++++ .../02751_match_constant_needle.reference | 1 + .../02751_match_constant_needle.sql | 1 + 4 files changed, 30 insertions(+) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 05e6aefbb5e..bcef49ce00c 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -45,6 +45,25 @@ size_t shortest_literal_length(const Literals & literals) return shortest; } +const char * skipNameCapturingGroup(const char * pos, size_t offset, const char * end) +{ + const char special = *(pos + offset) == '<' ? '>' : '\''; + offset ++; + while (pos + offset < end) + { + const char cur = *(pos + offset); + if (cur == special) + { + return pos + offset; + } + if (('0' <= cur && cur <= '9') || ('a' <= cur && cur <= 'z') || ('A' <= cur && cur <= 'Z')) + offset ++; + else + return pos; + } + return pos; +} + const char * analyzeImpl( std::string_view regexp, const char * pos, @@ -247,10 +266,15 @@ const char * analyzeImpl( break; } } + /// (?regex) means non-capturing parentheses group if (pos + 2 < end && pos[1] == '?' && pos[2] == ':') { pos += 2; } + if (pos + 3 < end && pos[1] == '?' && (pos[2] == '<' || pos[2] == '\'' || (pos[2] == 'P' && pos[3] == '<'))) + { + pos = skipNameCapturingGroup(pos, pos[2] == 'P' ? 3: 2, end); + } Literal group_required_substr; bool group_is_trival = true; Literals group_alters; diff --git a/src/Common/tests/gtest_optimize_re.cpp b/src/Common/tests/gtest_optimize_re.cpp index 3710666d336..a9fcb918b24 100644 --- a/src/Common/tests/gtest_optimize_re.cpp +++ b/src/Common/tests/gtest_optimize_re.cpp @@ -47,4 +47,8 @@ TEST(OptimizeRE, analyze) test_f("abc|(:?xx|yy|zz|x?)def", "", {"abc", "def"}); test_f("abc|(:?xx|yy|zz|x?){1,2}def", "", {"abc", "def"}); test_f(R"(\\A(?:(?:[-0-9_a-z]+(?:\\.[-0-9_a-z]+)*)/k8s1)\\z)", "/k8s1"); + test_f("[a-zA-Z]+(?P\\d+)", ""); + test_f("[a-zA-Z]+(?\\d+)", ""); + test_f("[a-zA-Z]+(?'num'\\d+)", ""); + test_f("[a-zA-Z]+(?x\\d+)", "x"); } diff --git a/tests/queries/0_stateless/02751_match_constant_needle.reference b/tests/queries/0_stateless/02751_match_constant_needle.reference index d00491fd7e5..6ed281c757a 100644 --- a/tests/queries/0_stateless/02751_match_constant_needle.reference +++ b/tests/queries/0_stateless/02751_match_constant_needle.reference @@ -1 +1,2 @@ 1 +1 diff --git a/tests/queries/0_stateless/02751_match_constant_needle.sql b/tests/queries/0_stateless/02751_match_constant_needle.sql index 71bdcc7cb0a..9980c3760f3 100644 --- a/tests/queries/0_stateless/02751_match_constant_needle.sql +++ b/tests/queries/0_stateless/02751_match_constant_needle.sql @@ -1 +1,2 @@ select match('default/k8s1', '\\A(?:(?:[-0-9_a-z]+(?:\\.[-0-9_a-z]+)*)/k8s1)\\z'); +select match('abc123', '[a-zA-Z]+(?P\\d+)'); From 4894a357f5a39943db1a95a65b3da507e82df83e Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Mon, 31 Jul 2023 17:49:59 -0600 Subject: [PATCH 1396/2047] Update datetime.md Correct URLs for settings-formats.md to date_time_input_format and date_time_output_format --- docs/en/sql-reference/data-types/datetime.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index fe279edb709..c99c8791542 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -140,8 +140,8 @@ Time shifts for multiple days. Some pacific islands changed their timezone offse - [Type conversion functions](../../sql-reference/functions/type-conversion-functions.md) - [Functions for working with dates and times](../../sql-reference/functions/date-time-functions.md) - [Functions for working with arrays](../../sql-reference/functions/array-functions.md) -- [The `date_time_input_format` setting](../../operations/settings/settings.md#settings-date_time_input_format) -- [The `date_time_output_format` setting](../../operations/settings/settings.md#settings-date_time_output_format) +- [The `date_time_input_format` setting](../../operations/settings/settings-formats.md#settings-date_time_input_format) +- [The `date_time_output_format` setting](../../operations/settings/settings-formats.md#settings-date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) - [The `session_timezone` setting](../../operations/settings/settings.md#session_timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-datetime) From 1fcbf5f96d1fb13c118c497fa0a2732629e99c75 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 07:33:14 +0200 Subject: [PATCH 1397/2047] Update tests --- tests/queries/0_stateless/02724_database_s3.sh | 4 ++-- tests/queries/0_stateless/02725_database_hdfs.sh | 10 +++++----- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index bb8f1f5f7ee..ca6264ac681 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -46,12 +46,12 @@ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = S3; USE test3; SELECT * FROM \"http://localhost:11111/test/a.myext\" -""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK" +""" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ USE test3; SELECT * FROM \"abacaba\" -""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK" +""" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK" # Cleanup ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index 89ff7421a6f..aed5a6a0a39 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -43,12 +43,12 @@ DROP DATABASE IF EXISTS test4; CREATE DATABASE test4 ENGINE = HDFS; USE test4; SELECT * FROM \"abacaba/file.tsv\" -""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK1" +""" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK1" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK2" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK3" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK4" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK5" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK2" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK3" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK4" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK5" # Cleanup From a0ff04e0e7cc7bd911d80a01aee34846ec26bb3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 08:03:09 +0200 Subject: [PATCH 1398/2047] Add a test --- .../02834_client_yaml_configs.reference | 3 +++ .../0_stateless/02834_client_yaml_configs.sh | 21 +++++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/02834_client_yaml_configs.reference create mode 100755 tests/queries/0_stateless/02834_client_yaml_configs.sh diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.reference b/tests/queries/0_stateless/02834_client_yaml_configs.reference new file mode 100644 index 00000000000..302360f2570 --- /dev/null +++ b/tests/queries/0_stateless/02834_client_yaml_configs.reference @@ -0,0 +1,3 @@ +31337 +31337 +31337 diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh new file mode 100755 index 00000000000..f17186328b4 --- /dev/null +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +pushd "${CLICKHOUSE_TMP}" > /dev/null + +echo "max_block_size: 31337" > clickhouse-client.yaml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.yaml + +echo "max_block_size: 31337" > clickhouse-client.yml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.yml + +echo "31337" > clickhouse-client.xml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.xml + +popd > /dev/null From 0374b3de3e80edeac7e0b8580d32f313e2734d30 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 1 Aug 2023 09:04:55 +0200 Subject: [PATCH 1399/2047] Update src/Server/KeeperTCPHandler.cpp Co-authored-by: Azat Khuzhin --- src/Server/KeeperTCPHandler.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index e44ab4a4bd7..58d227a5ae5 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -175,7 +175,8 @@ struct SocketInterruptablePollWrapper } while (rc < 0 && errno == POCO_EINTR); - if (rc >= 1) { + if (rc >= 1) + { if (poll_buf[0].revents & POLLIN) socket_ready = true; if (poll_buf[1].revents & POLLIN) From 10662b64256dc85469b7f03dc73e8c889b2ae07c Mon Sep 17 00:00:00 2001 From: daviddhc20120601 Date: Tue, 1 Aug 2023 15:35:25 +0800 Subject: [PATCH 1400/2047] Update annindexes.md explain more about l2distance and cosine distance Update annindexes.md explain more about l2distance and cosine distance --- .../table-engines/mergetree-family/annindexes.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 4a4ebb47bdc..5944048f6c3 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -193,6 +193,19 @@ index creation, `L2Distance` is used as default. Parameter `NumTrees` is the num specified: 100). Higher values of `NumTree` mean more accurate search results but slower index creation / query times (approximately linearly) as well as larger index sizes. +`L2Distance` is also called Euclidean distance, the Euclidean distance between two points in Euclidean space is the length of a line segment between the two points. +For example: If we have point P(p1,p2), Q(q1,q2), their distance will be d(p,q) +![L2Distance](https://en.wikipedia.org/wiki/Euclidean_distance#/media/File:Euclidean_distance_2d.svg) + +`cosineDistance` also called cosine similarity is a measure of similarity between two non-zero vectors defined in an inner product space. Cosine similarity is the cosine of the angle between the vectors; that is, it is the dot product of the vectors divided by the product of their lengths. +![cosineDistance](https://www.tyrrell4innovation.ca/wp-content/uploads/2021/06/rsz_jenny_du_miword.png) + +The Euclidean distance corresponds to the L2-norm of a difference between vectors. The cosine similarity is proportional to the dot product of two vectors and inversely proportional to the product of their magnitudes. +![compare](https://www.researchgate.net/publication/320914786/figure/fig2/AS:558221849841664@1510101868614/The-difference-between-Euclidean-distance-and-cosine-similarity.png) +In one sentence: cosine similarity care only about the angle between them, but do not care about the "distance" we normally think. +![L2 distance](https://www.baeldung.com/wp-content/uploads/sites/4/2020/06/4-1.png) +![cosineDistance](https://www.baeldung.com/wp-content/uploads/sites/4/2020/06/5.png) + :::note Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 From 628786e7eb6a55d5f5b2881a0fc34c3edcf7d0e8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 10:52:06 +0300 Subject: [PATCH 1401/2047] Update 02834_client_yaml_configs.sh --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index f17186328b4..5a6d26808a3 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 71e58699b5f3f8e61a8b748e2172bdf9ad00b4ac Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 1 Aug 2023 07:52:50 +0000 Subject: [PATCH 1402/2047] Fix build and test --- contrib/libarchive-cmake/CMakeLists.txt | 19 ++++++++++++++++++- contrib/libarchive-cmake/config.h | 10 +++++----- ...661_select_from_table_in_archive.reference | 12 ++++-------- .../02661_select_from_table_in_archive.sh | 4 +--- 4 files changed, 28 insertions(+), 17 deletions(-) diff --git a/contrib/libarchive-cmake/CMakeLists.txt b/contrib/libarchive-cmake/CMakeLists.txt index 3aca1c7eeab..4be9b206368 100644 --- a/contrib/libarchive-cmake/CMakeLists.txt +++ b/contrib/libarchive-cmake/CMakeLists.txt @@ -146,6 +146,23 @@ target_compile_definitions(_libarchive PUBLIC target_compile_options(_libarchive PRIVATE "-Wno-reserved-macro-identifier") -target_link_libraries(_libarchive PRIVATE ch_contrib::zlib ch_contrib::xz) +if (TARGET ch_contrib::xz) + target_compile_definitions(_libarchive PUBLIC HAVE_LZMA_H=1) + target_link_libraries(_libarchive PRIVATE ch_contrib::xz) +endif() + +if (TARGET ch_contrib::zlib) + target_compile_definitions(_libarchive PUBLIC HAVE_ZLIB_H=1) + target_link_libraries(_libarchive PRIVATE ch_contrib::zlib) +endif() + +if (OS_LINUX) + target_compile_definitions( + _libarchive PUBLIC + MAJOR_IN_SYSMACROS=1 + HAVE_LINUX_FS_H=1 + HAVE_STRUCT_STAT_ST_MTIM_TV_NSEC=1 + ) +endif() add_library(ch_contrib::libarchive ALIAS _libarchive) \ No newline at end of file diff --git a/contrib/libarchive-cmake/config.h b/contrib/libarchive-cmake/config.h index c2fecf657bc..b5033587c85 100644 --- a/contrib/libarchive-cmake/config.h +++ b/contrib/libarchive-cmake/config.h @@ -782,7 +782,7 @@ typedef uint64_t uintmax_t; /* #undef HAVE_LINUX_FIEMAP_H */ /* Define to 1 if you have the header file. */ -#define HAVE_LINUX_FS_H 1 +/* #undef HAVE_LINUX_FS_H */ /* Define to 1 if you have the header file. */ /* #undef HAVE_LINUX_MAGIC_H */ @@ -843,7 +843,7 @@ typedef uint64_t uintmax_t; /* #undef HAVE_LZMADEC_H */ /* Define to 1 if you have the header file. */ -#define HAVE_LZMA_H 1 +/* #undef HAVE_LZMA_H */ /* Define to 1 if you have a working `lzma_stream_encoder_mt' function. */ /* #undef HAVE_LZMA_STREAM_ENCODER_MT */ @@ -1057,7 +1057,7 @@ typedef uint64_t uintmax_t; /* #undef HAVE_STRUCT_STAT_ST_MTIME_USEC */ /* Define to 1 if `st_mtim.tv_nsec' is a member of `struct stat'. */ -#define HAVE_STRUCT_STAT_ST_MTIM_TV_NSEC 1 +/* #undef HAVE_STRUCT_STAT_ST_MTIM_TV_NSEC */ /* Define to 1 if `st_umtime' is a member of `struct stat'. */ /* #undef HAVE_STRUCT_STAT_ST_UMTIME */ @@ -1249,7 +1249,7 @@ typedef uint64_t uintmax_t; #define HAVE_WORKING_FS_IOC_GETFLAGS 1 /* Define to 1 if you have the header file. */ -#define HAVE_ZLIB_H 1 +/* #undef HAVE_ZLIB_H */ /* Define to 1 if you have the header file. */ /* #undef HAVE_ZSTD_H */ @@ -1291,7 +1291,7 @@ typedef uint64_t uintmax_t; /* Define to 1 if `major', `minor', and `makedev' are declared in . */ -#define MAJOR_IN_SYSMACROS 1 +/* #undef MAJOR_IN_SYSMACROS */ /* Define to 1 if your C compiler doesn't accept -c and -o together. */ /* #undef NO_MINUS_C_MINUS_O */ diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.reference b/tests/queries/0_stateless/02661_select_from_table_in_archive.reference index 6f7ca964a7e..67f1d859da5 100644 --- a/tests/queries/0_stateless/02661_select_from_table_in_archive.reference +++ b/tests/queries/0_stateless/02661_select_from_table_in_archive.reference @@ -1,9 +1,8 @@ -Reading 02661_select_from_table_in_archive_test_archive1.zip :: 02661_select_from_table_in_archive_test_data.csv +Running for zip files 1 2 3 4 1 2 3 4 -Reading 02661_select_from_table_in_archive_test_archive{1..2}.zip :: 02661_select_from_table_in_archive_test_data.csv 1 3 1 @@ -12,12 +11,11 @@ Reading 02661_select_from_table_in_archive_test_archive{1..2}.zip :: 02661_selec 3 1 3 -Reading 02661_select_from_table_in_archive_test_archive1.tar.gz :: 02661_select_from_table_in_archive_test_data.csv +Running for tar.gz files 1 2 3 4 1 2 3 4 -Reading 02661_select_from_table_in_archive_test_archive{1..2}.tar.gz :: 02661_select_from_table_in_archive_test_data.csv 1 3 1 @@ -26,12 +24,11 @@ Reading 02661_select_from_table_in_archive_test_archive{1..2}.tar.gz :: 02661_se 3 1 3 -Reading 02661_select_from_table_in_archive_test_archive1.tar :: 02661_select_from_table_in_archive_test_data.csv +Running for tar files 1 2 3 4 1 2 3 4 -Reading 02661_select_from_table_in_archive_test_archive{1..2}.tar :: 02661_select_from_table_in_archive_test_data.csv 1 3 1 @@ -40,12 +37,11 @@ Reading 02661_select_from_table_in_archive_test_archive{1..2}.tar :: 02661_selec 3 1 3 -Reading 02661_select_from_table_in_archive_test_archive1.7z :: 02661_select_from_table_in_archive_test_data.csv +Running for 7z files 1 2 3 4 1 2 3 4 -Reading 02661_select_from_table_in_archive_test_archive{1..2}.7z :: 02661_select_from_table_in_archive_test_data.csv 1 3 1 diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh index 14443f62770..b38f80e5d3f 100755 --- a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh +++ b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh @@ -12,7 +12,6 @@ user_files_path=$(clickhouse-client --query "select _path,_file from file('nonex echo -e "1,2\n3,4" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv function read_archive_file() { - echo "Reading $2" $CLICKHOUSE_LOCAL --query "SELECT $1 FROM file('${user_files_path}/$2')" $CLICKHOUSE_CLIENT --query "CREATE TABLE 02661_archive_table Engine=File('CSV', '${user_files_path}/$2')" $CLICKHOUSE_CLIENT --query "SELECT $1 FROM 02661_archive_table" @@ -20,10 +19,9 @@ function read_archive_file() { } function run_archive_test() { + echo "Running for $1 files" read_archive_file "*" "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" read_archive_file "c1" "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1..2}.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" - - } zip ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null From 6e39065bd0054aa1d9f12352f9a962b3f1f9c1b3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 10:14:49 +0200 Subject: [PATCH 1403/2047] Better tests --- tests/queries/0_stateless/02724_database_s3.sh | 4 ++-- tests/queries/0_stateless/02725_database_hdfs.sh | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index ca6264ac681..ae3b048938e 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -46,12 +46,12 @@ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = S3; USE test3; SELECT * FROM \"http://localhost:11111/test/a.myext\" -""" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK" +""" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ USE test3; SELECT * FROM \"abacaba\" -""" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK" +""" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" # Cleanup ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index aed5a6a0a39..6b69051de3b 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -45,10 +45,10 @@ USE test4; SELECT * FROM \"abacaba/file.tsv\" """ 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK1" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK2" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK3" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK4" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK5" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" # Cleanup From 8b37abfa598b80bf9a1281d519f09d953e18523e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 11:16:00 +0300 Subject: [PATCH 1404/2047] Update 02834_client_yaml_configs.sh --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index 5a6d26808a3..dbb40d33e0a 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -pushd "${CLICKHOUSE_TMP}" > /dev/null +pushd "${CLICKHOUSE_TMP}" || exit > /dev/null echo "max_block_size: 31337" > clickhouse-client.yaml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" @@ -19,4 +19,4 @@ echo "31337" > clickho ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.xml -popd > /dev/null +popd || exit > /dev/null From 513d3fd3c1c1484cc908987f135f153f59d36405 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Aug 2023 09:15:35 +0000 Subject: [PATCH 1405/2047] fix more places --- tests/integration/test_throttling/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 526c34ac916..91b472513c7 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -175,7 +175,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7, 14), + 7, id="user_remote_to_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -291,7 +291,7 @@ def test_backup_throttling_override(): }, ) # reading 1e6*8 bytes with 500Ki default bandwith should take (8-0.5)/0.5=15 seconds - assert_took(took, (15, 20)) + assert_took(took, 15) @pytest.mark.parametrize( From 16851040df34f6f5508e292b16663a938e681392 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 1 Aug 2023 11:21:46 +0200 Subject: [PATCH 1406/2047] Fix annoying test --- src/Interpreters/Cache/FileCacheKey.cpp | 2 +- tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileCacheKey.cpp b/src/Interpreters/Cache/FileCacheKey.cpp index 360a9df7ad8..75a8ac2934e 100644 --- a/src/Interpreters/Cache/FileCacheKey.cpp +++ b/src/Interpreters/Cache/FileCacheKey.cpp @@ -35,7 +35,7 @@ FileCacheKey FileCacheKey::random() FileCacheKey FileCacheKey::fromKeyString(const std::string & key_str) { if (key_str.size() != 32) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid key: {}", key_str); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid cache key hex: {}", key_str); return FileCacheKey(unhexUInt(key_str.data())); } diff --git a/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh index f5670f1261e..4335bc34a28 100755 --- a/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh +++ b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh @@ -24,7 +24,7 @@ $CLICKHOUSE_CLIENT --query_id "$query_id" --query "SELECT * FROM test FORMAT Nul $CLICKHOUSE_CLIENT -nm --query """ SYSTEM DROP FILESYSTEM CACHE '$disk_name' KEY kek; -""" 2>&1 | grep -q "Invalid key: kek" && echo "OK" || echo "FAIL" +""" 2>&1 | grep -q "Invalid cache key hex: kek" && echo "OK" || echo "FAIL" ${CLICKHOUSE_CLIENT} -q " system flush logs" From 946f2f0b3919bc4022195896fa4f843a6b08317a Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 1 Aug 2023 12:24:08 +0300 Subject: [PATCH 1407/2047] Simplify the integration test - dispose of the excessive details that cluttered it --- .../test.py | 26 +++---------------- 1 file changed, 4 insertions(+), 22 deletions(-) diff --git a/tests/integration/test_backup_restore_storage_policy/test.py b/tests/integration/test_backup_restore_storage_policy/test.py index cc1412c03c3..9c462e6060e 100644 --- a/tests/integration/test_backup_restore_storage_policy/test.py +++ b/tests/integration/test_backup_restore_storage_policy/test.py @@ -35,33 +35,18 @@ def new_backup_name(): return f"Disk('bak', '{backup_id_counter}/')" -def create_and_fill_table(n=100): +def create_and_fill_table(): instance.query("CREATE DATABASE test") - instance.query( - "CREATE TABLE test.table(x UInt32, y String) ENGINE=MergeTree ORDER BY y PARTITION BY x%10" - ) - instance.query( - f"INSERT INTO test.table SELECT number, toString(number) FROM numbers({n})" - ) + instance.query("CREATE TABLE test.table(x UInt32) ENGINE=MergeTree ORDER BY x") + instance.query(f"INSERT INTO test.table SELECT number FROM numbers(10)") @pytest.mark.parametrize("policy", ["disks_in_order", "", None]) def test_restore_table(policy): backup_name = new_backup_name() - n = 20 - sum_n = int((n * (n - 1)) / 2) - expected = f"{n}\t{sum_n}\n" - - create_and_fill_table(n) - - assert instance.query("SELECT count(), sum(x) FROM test.table") == expected - + create_and_fill_table() instance.query(f"BACKUP TABLE test.table TO {backup_name}") - instance.query("DROP TABLE test.table SYNC") - - assert instance.query("EXISTS test.table") == "0\n" - restore_query = f"RESTORE TABLE test.table FROM {backup_name}" if policy is None: policy = "default" @@ -69,11 +54,8 @@ def test_restore_table(policy): restore_query += f" SETTINGS storage_policy = '{policy}'" if policy == "": policy = "default" - instance.query(restore_query) - assert instance.query("SELECT count(), sum(x) FROM test.table") == expected - assert ( instance.query("SELECT storage_policy FROM system.tables WHERE name='table'") == f"{policy}\n" From 0037ab759d8c7b997c9e6a26732f53a6b4fe80fa Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 1 Aug 2023 11:26:55 +0200 Subject: [PATCH 1408/2047] Fix build on mac --- contrib/libarchive-cmake/CMakeLists.txt | 4 ++++ contrib/libarchive-cmake/config.h | 10 +++++----- .../0_stateless/02661_select_from_table_in_archive.sh | 10 +++++----- 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/contrib/libarchive-cmake/CMakeLists.txt b/contrib/libarchive-cmake/CMakeLists.txt index 4be9b206368..fb64266185e 100644 --- a/contrib/libarchive-cmake/CMakeLists.txt +++ b/contrib/libarchive-cmake/CMakeLists.txt @@ -162,6 +162,10 @@ if (OS_LINUX) MAJOR_IN_SYSMACROS=1 HAVE_LINUX_FS_H=1 HAVE_STRUCT_STAT_ST_MTIM_TV_NSEC=1 + HAVE_LINUX_TYPES_H=1 + HAVE_SYS_STATFS_H=1 + HAVE_FUTIMESAT=1 + HAVE_ICONV=1 ) endif() diff --git a/contrib/libarchive-cmake/config.h b/contrib/libarchive-cmake/config.h index b5033587c85..0b0cab47a52 100644 --- a/contrib/libarchive-cmake/config.h +++ b/contrib/libarchive-cmake/config.h @@ -628,7 +628,7 @@ typedef uint64_t uintmax_t; #define HAVE_FUTIMES 1 /* Define to 1 if you have the `futimesat' function. */ -#define HAVE_FUTIMESAT 1 +/* #undef HAVE_FUTIMESAT */ /* Define to 1 if you have the `getea' function. */ /* #undef HAVE_GETEA */ @@ -667,7 +667,7 @@ typedef uint64_t uintmax_t; #define HAVE_GRP_H 1 /* Define to 1 if you have the `iconv' function. */ -#define HAVE_ICONV 1 +/* #undef HAVE_ICONV */ /* Define to 1 if you have the header file. */ #define HAVE_ICONV_H 1 @@ -788,7 +788,7 @@ typedef uint64_t uintmax_t; /* #undef HAVE_LINUX_MAGIC_H */ /* Define to 1 if you have the header file. */ -#define HAVE_LINUX_TYPES_H 1 +/* #undef HAVE_LINUX_TYPES_H */ /* Define to 1 if you have the `listea' function. */ /* #undef HAVE_LISTEA */ @@ -1081,7 +1081,7 @@ typedef uint64_t uintmax_t; /* #undef HAVE_SYS_ACL_H */ /* Define to 1 if you have the header file. */ -#define HAVE_SYS_CDEFS_H 1 +/* #undef HAVE_SYS_CDEFS_H */ /* Define to 1 if you have the header file, and it defines `DIR'. */ @@ -1122,7 +1122,7 @@ typedef uint64_t uintmax_t; #define HAVE_SYS_SELECT_H 1 /* Define to 1 if you have the header file. */ -#define HAVE_SYS_STATFS_H 1 +/* #undef HAVE_SYS_STATFS_H */ /* Define to 1 if you have the header file. */ #define HAVE_SYS_STATVFS_H 1 diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh index b38f80e5d3f..6e6342e3d54 100755 --- a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh +++ b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh @@ -32,23 +32,23 @@ run_archive_test "zip" rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip -tar -cvzf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar.gz ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null -tar -cvzf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar.gz ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null +tar -cvzf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar.gz ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv &> /dev/null +tar -cvzf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar.gz ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv &> /dev/null run_archive_test "tar.gz" rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar.gz rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar.gz -tar -cvf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null -tar -cvf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null +tar -cvf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv &> /dev/null +tar -cvf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv &> /dev/null run_archive_test "tar" rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar -7z a ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.7z ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null +7z a ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.7z ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null 7z a ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.7z ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null run_archive_test "7z" From 9ab875f5ad115002148addc0e3757610b93965d0 Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 1 Aug 2023 11:38:15 +0200 Subject: [PATCH 1409/2047] add explicit nullptr for now --- src/Functions/FunctionsConversion.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index e95699a0cec..d4dd843a66e 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1371,8 +1371,8 @@ struct ConvertThroughParsing precise_float_parsing = query_context->getSettingsRef().precise_float_parsing; } - std::function parseFunction; - std::function tryParseFunction; + std::function parseFunction = nullptr; + std::function tryParseFunction = nullptr; if constexpr (!(parsing_mode == ConvertFromStringParsingMode::BestEffort || parsing_mode == ConvertFromStringParsingMode::BestEffortUS || From c8a8b057cd7ab31d8c9ce66cf49b4bd5fee2be04 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 1 Aug 2023 11:43:30 +0200 Subject: [PATCH 1410/2047] Update 01603_read_with_backoff_bug.sql --- tests/queries/0_stateless/01603_read_with_backoff_bug.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql index 35e20b057e7..569a92f3048 100644 --- a/tests/queries/0_stateless/01603_read_with_backoff_bug.sql +++ b/tests/queries/0_stateless/01603_read_with_backoff_bug.sql @@ -1,6 +1,8 @@ -- Tags: no-tsan -- Tag no-tsan: Too long for TSan +set enable_filesystem_cache=0; +set enable_filesystem_cache_on_write_operations=0; drop table if exists t; create table t (x UInt64, s String) engine = MergeTree order by x; @@ -11,7 +13,6 @@ FROM numbers_mt((8129 * 1024) * 3) settings max_insert_threads=8; -- optimize table t final; -set enable_filesystem_cache=0; select count(), sum(length(s)) from t settings max_threads = 3, read_backoff_min_latency_ms = 1, read_backoff_max_throughput = 1000000000, read_backoff_min_interval_between_events_ms = 1, read_backoff_min_events = 1, read_backoff_min_concurrency = 1; select count(), sum(length(s)) from t settings max_threads = 3, read_backoff_min_latency_ms = 1, read_backoff_max_throughput = 1000000000, read_backoff_min_interval_between_events_ms = 1, read_backoff_min_events = 1, read_backoff_min_concurrency = 1; select count(), sum(length(s)) from t settings max_threads = 3, read_backoff_min_latency_ms = 1, read_backoff_max_throughput = 1000000000, read_backoff_min_interval_between_events_ms = 1, read_backoff_min_events = 1, read_backoff_min_concurrency = 1; From 135790c0d6caed1075399cab69a5036c71860cd8 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Aug 2023 09:44:08 +0000 Subject: [PATCH 1411/2047] more fixes --- tests/integration/test_throttling/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 91b472513c7..a27bb472ea8 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -300,7 +300,7 @@ def test_backup_throttling_override(): # # Local # - pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + pytest.param("default", None, None, None, 0, id="no_local_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "default", @@ -322,7 +322,7 @@ def test_backup_throttling_override(): # # Remote # - pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), + pytest.param("s3", None, None, None, 0, id="no_remote_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "s3", @@ -362,7 +362,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): # # Local # - pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + pytest.param("default", None, None, None, 0, id="no_local_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "default", @@ -384,7 +384,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): # # Remote # - pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), + pytest.param("s3", None, None, None, 0, id="no_remote_throttling"), # writing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "s3", From 5880a0a3721d8d3d3f39ce7f1196eac4be433bf8 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 1 Aug 2023 11:50:31 +0200 Subject: [PATCH 1412/2047] Update ReadBufferFromHDFS.cpp --- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 122ed335265..150c40a8151 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -88,7 +88,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory Date: Tue, 1 Aug 2023 11:50:56 +0200 Subject: [PATCH 1413/2047] Update src/Storages/MergeTree/localBackup.cpp Co-authored-by: Sergei Trifonov --- src/Storages/MergeTree/localBackup.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/localBackup.cpp b/src/Storages/MergeTree/localBackup.cpp index cae6993e3e8..4adde07da97 100644 --- a/src/Storages/MergeTree/localBackup.cpp +++ b/src/Storages/MergeTree/localBackup.cpp @@ -148,7 +148,7 @@ void localBackup( std::function cleaner; if (disk->supportZeroCopyReplication()) /// Note: this code will create garbage on s3. We should always remove `copy_instead_of_hardlinks` files. - /// The third agrument should be a list of excpetions, but (looks like) it is ignored for keep_all_shared_data = true. + /// The third argument should be a list of exceptions, but (looks like) it is ignored for keep_all_shared_data = true. cleaner = [disk, destination_path]() { disk->removeSharedRecursive(destination_path, /*keep_all_shared_data*/ true, {}); }; else cleaner = [disk, destination_path]() { disk->removeRecursive(destination_path); }; From ccf9fa96419c38513b4bd96d0332c8613a3abf17 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 1 Aug 2023 11:55:08 +0200 Subject: [PATCH 1414/2047] Update ReadBufferFromHDFS.cpp --- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 150c40a8151..9455afe7833 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -80,6 +80,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(file_info->mSize); + hdfsFreeFileInfo(file_info, 1); } } From be458fd44b7eca63080fc821e9bf88c968a4eb28 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Aug 2023 12:06:25 +0200 Subject: [PATCH 1415/2047] Fix style check --- src/Storages/S3Queue/S3QueueHolder.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 24bb31517a5..860484da671 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -18,7 +18,6 @@ namespace DB namespace ErrorCodes { - extern const int NO_ZOOKEEPER; extern const int TIMEOUT_EXCEEDED; } From 08f5ebf3e86528b900341fe5c09cc222a3584230 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Aug 2023 12:19:15 +0200 Subject: [PATCH 1416/2047] Fix test --- src/Storages/S3Queue/StorageS3Queue.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index d60c9adb4c8..baacb3397c7 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -108,14 +108,15 @@ StorageS3Queue::StorageS3Queue( if (!withGlobs()) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); - std::string setting_zookeeper_path = s3queue_settings->keeper_path; - std::string zk_path_prefix; + String setting_zookeeper_path = s3queue_settings->keeper_path; if (setting_zookeeper_path.empty()) { auto database = DatabaseCatalog::instance().getDatabase(table_id_.database_name); bool is_in_replicated_database = database->getEngineName() == "Replicated"; auto default_path = getContext()->getSettingsRef().s3queue_default_zookeeper_path.value; + String zk_path_prefix; + if (!default_path.empty()) { zk_path_prefix = default_path; @@ -133,15 +134,16 @@ StorageS3Queue::StorageS3Queue( "S3Queue keeper_path engine setting not specified, " "s3queue_default_zookeeper_path_prefix not specified"); } + + zookeeper_path = zkutil::extractZooKeeperPath( + fs::path(zk_path_prefix) / toString(table_id_.uuid), /* check_starts_with_slash */ true, log); } else { - zk_path_prefix = s3queue_settings->keeper_path.value; + /// We do not add table uuid here on purpose. + zookeeper_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path.value, /* check_starts_with_slash */ true, log); } - zookeeper_path = zkutil::extractZooKeeperPath( - fs::path(zk_path_prefix) / toString(table_id_.uuid), /* check_starts_with_slash */ true, log); - LOG_INFO(log, "Using zookeeper path: {}", zookeeper_path); FormatFactory::instance().checkFormatName(format_name); From fe869207a1b6930826723cbaae6f0b391f1a04c0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Aug 2023 10:28:23 +0000 Subject: [PATCH 1417/2047] Disable a couple of long tests for debug build. --- .../00840_long_concurrent_select_and_drop_deadlock.sh | 2 +- .../0_stateless/02151_hash_table_sizes_stats_distributed.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh b/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh index 6714d8b35ca..cbe37de6651 100755 --- a/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh +++ b/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: deadlock, no-parallel +# Tags: deadlock, no-parallel, no-debug # NOTE: database = $CLICKHOUSE_DATABASE is unwanted diff --git a/tests/queries/0_stateless/02151_hash_table_sizes_stats_distributed.sh b/tests/queries/0_stateless/02151_hash_table_sizes_stats_distributed.sh index 703b2c4357c..b23be4283b2 100755 --- a/tests/queries/0_stateless/02151_hash_table_sizes_stats_distributed.sh +++ b/tests/queries/0_stateless/02151_hash_table_sizes_stats_distributed.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, distributed, no-tsan +# Tags: long, distributed, no-tsan, no-debug # These tests don't use `current_database = currentDatabase()` condition, because database name isn't propagated during remote queries. From 505b4d01121d3d6cfd48d281a68fe00ca9ecadf2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 1 Aug 2023 13:39:23 +0300 Subject: [PATCH 1418/2047] Simplify the integration test - use backup to File instead of Disk --- .../configs/storage_config.xml | 7 +------ .../integration/test_backup_restore_storage_policy/test.py | 2 +- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml b/tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml index 960e71294bb..d27a919aa7a 100644 --- a/tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml +++ b/tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml @@ -1,10 +1,6 @@ - - local - /var/lib/backups/ - local /var/lib/disks/one/ @@ -26,7 +22,6 @@ - bak - /var/lib/backups/ + /backups diff --git a/tests/integration/test_backup_restore_storage_policy/test.py b/tests/integration/test_backup_restore_storage_policy/test.py index 9c462e6060e..50e71a11ae9 100644 --- a/tests/integration/test_backup_restore_storage_policy/test.py +++ b/tests/integration/test_backup_restore_storage_policy/test.py @@ -32,7 +32,7 @@ def cleanup_after_test(): def new_backup_name(): global backup_id_counter backup_id_counter += 1 - return f"Disk('bak', '{backup_id_counter}/')" + return f"File('/backups/{backup_id_counter}/')" def create_and_fill_table(): From ce4f273be1f6cf7168347a5d1507580a06ee8255 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 1 Aug 2023 12:39:36 +0200 Subject: [PATCH 1419/2047] Updated list of retryable errors and updated both tests to use same azure_query function --- .../test.py | 23 +++---------------- .../test_storage_azure_blob_storage/test.py | 2 ++ 2 files changed, 5 insertions(+), 20 deletions(-) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 86b70f8db70..dcca85fca7a 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -7,6 +7,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.utility import generate_values, replace_config, SafeThread from azure.storage.blob import BlobServiceClient +from test_storage_azure_blob_storage.test import azure_query SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -38,28 +39,10 @@ def cluster(): cluster.shutdown() -# Note: use this for selects and inserts and create table queries. +# Note: use azure_query for selects and inserts and create table queries. # For inserts there is no guarantee that retries will not result in duplicates. -# But it is better to retry anyway because 'Connection was closed by the server' error +# But it is better to retry anyway because connection related errors # happens in fact only for inserts because reads already have build-in retries in code. -def azure_query(node, query, try_num=3, settings={}): - for i in range(try_num): - try: - return node.query(query, settings=settings) - except Exception as ex: - retriable_errors = [ - "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response" - ] - retry = False - for error in retriable_errors: - if error in str(ex): - retry = True - logging.info(f"Try num: {i}. Having retriable error: {ex}") - break - if not retry or i == try_num - 1: - raise Exception(ex) - continue - def create_table(node, table_name, **additional_settings): settings = { diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 21f57a67495..6e103bdf96d 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -43,6 +43,8 @@ def azure_query(node, query, try_num=10, settings={}): retriable_errors = [ "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", + "DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read", ] retry = False for error in retriable_errors: From 4561378c2085eca3bc4ff3dc8e3b512ee6407398 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 1 Aug 2023 10:57:46 +0000 Subject: [PATCH 1420/2047] Automatic style fix --- tests/integration/test_merge_tree_azure_blob_storage/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index dcca85fca7a..121b9b294f0 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -44,6 +44,7 @@ def cluster(): # But it is better to retry anyway because connection related errors # happens in fact only for inserts because reads already have build-in retries in code. + def create_table(node, table_name, **additional_settings): settings = { "storage_policy": "blob_storage_policy", From 9caef8b4a56e94d684f18318af0928c437ae2b83 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Aug 2023 11:18:53 +0000 Subject: [PATCH 1421/2047] Try to fix a rare fail in 00612_http_max_query_size --- tests/queries/0_stateless/00612_http_max_query_size.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00612_http_max_query_size.sh b/tests/queries/0_stateless/00612_http_max_query_size.sh index cfcae330b85..6289470c21e 100755 --- a/tests/queries/0_stateless/00612_http_max_query_size.sh +++ b/tests/queries/0_stateless/00612_http_max_query_size.sh @@ -36,7 +36,7 @@ def gen_data(q): pattern = ''' or toString(number) = '{}'\n''' - for i in range(1, 4 * 1024): + for i in range(0, 1024 * 2): yield pattern.format(str(i).zfill(1024 - len(pattern) + 2)).encode() s = requests.Session() From a14a6b56b3b99ea7aa96078f51d93e6e250723fb Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Aug 2023 14:31:05 +0200 Subject: [PATCH 1422/2047] Better --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 351 ++++++++++++++++++ ...S3QueueHolder.h => S3QueueFilesMetadata.h} | 53 +-- src/Storages/S3Queue/S3QueueHolder.cpp | 341 ----------------- src/Storages/S3Queue/S3QueueSource.cpp | 8 +- src/Storages/S3Queue/S3QueueSource.h | 6 +- src/Storages/S3Queue/StorageS3Queue.cpp | 175 ++++----- src/Storages/S3Queue/StorageS3Queue.h | 48 ++- 7 files changed, 489 insertions(+), 493 deletions(-) create mode 100644 src/Storages/S3Queue/S3QueueFilesMetadata.cpp rename src/Storages/S3Queue/{S3QueueHolder.h => S3QueueFilesMetadata.h} (73%) delete mode 100644 src/Storages/S3Queue/S3QueueHolder.cpp diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp new file mode 100644 index 00000000000..514baeb7b07 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -0,0 +1,351 @@ +#include "IO/VarInt.h" +#include "config.h" + +#if USE_AWS_S3 +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TIMEOUT_EXCEEDED; +} + +namespace +{ + UInt64 getCurrentTime() + { + return std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + } +} + +void S3QueueFilesMetadata::S3QueueCollection::read(ReadBuffer & in) +{ + files = {}; + if (in.eof()) + return; + + size_t files_num; + in >> files_num >> "\n"; + while (files_num--) + { + TrackedCollectionItem item; + in >> item.file_path >> "\n"; + in >> item.timestamp >> "\n"; + in >> item.retries_count >> "\n"; + in >> item.last_exception >> "\n"; + files.push_back(item); + } +} + +void S3QueueFilesMetadata::S3QueueCollection::write(WriteBuffer & out) const +{ + out << files.size() << "\n"; + for (const auto & processed_file : files) + { + out << processed_file.file_path << "\n"; + out << processed_file.timestamp << "\n"; + out << processed_file.retries_count << "\n"; + out << processed_file.last_exception << "\n"; + } +} + +String S3QueueFilesMetadata::S3QueueCollection::toString() const +{ + WriteBufferFromOwnString out; + write(out); + return out.str(); +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::S3QueueCollection::getFileNames() +{ + S3FilesCollection keys = {}; + for (const auto & pair : files) + keys.insert(pair.file_path); + return keys; +} + + +S3QueueFilesMetadata::S3QueueProcessedCollection::S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) + : max_size(max_size_), max_age(max_age_) +{ +} + +void S3QueueFilesMetadata::S3QueueProcessedCollection::parse(const String & collection_str) +{ + ReadBufferFromString buf(collection_str); + read(buf); + if (max_age > 0) // Remove old items + { + std::erase_if( + files, + [timestamp = getCurrentTime(), this](const TrackedCollectionItem & processed_file) + { return (timestamp - processed_file.timestamp) > max_age; }); + } +} + + +void S3QueueFilesMetadata::S3QueueProcessedCollection::add(const String & file_name) +{ + TrackedCollectionItem processed_file = { .file_path=file_name, .timestamp = getCurrentTime() }; + files.push_back(processed_file); + + /// TODO: it is strange that in parse() we take into account only max_age, but here only max_size. + while (files.size() > max_size) + { + files.pop_front(); + } +} + + +S3QueueFilesMetadata::S3QueueFailedCollection::S3QueueFailedCollection(const UInt64 & max_retries_count_) + : max_retries_count(max_retries_count_) +{ +} + +void S3QueueFilesMetadata::S3QueueFailedCollection::parse(const String & collection_str) +{ + ReadBufferFromString buf(collection_str); + read(buf); +} + + +bool S3QueueFilesMetadata::S3QueueFailedCollection::add(const String & file_name, const String & exception_message) +{ + auto failed_it = std::find_if( + files.begin(), files.end(), + [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); + + if (failed_it == files.end()) + { + files.emplace_back(file_name, 0, max_retries_count, exception_message); + } + else if (failed_it->retries_count == 0 || --failed_it->retries_count == 0) + { + return false; + } + return true; +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::S3QueueFailedCollection::getFileNames() +{ + S3FilesCollection failed_keys; + for (const auto & pair : files) + { + if (pair.retries_count == 0) + failed_keys.insert(pair.file_path); + } + return failed_keys; +} + +void S3QueueFilesMetadata::S3QueueProcessingCollection::parse(const String & collection_str) +{ + ReadBufferFromString rb(collection_str); + Strings result; + readQuoted(result, rb); + files = S3FilesCollection(result.begin(), result.end()); +} + +void S3QueueFilesMetadata::S3QueueProcessingCollection::add(const Strings & file_names) +{ + files.insert(file_names.begin(), file_names.end()); +} + +void S3QueueFilesMetadata::S3QueueProcessingCollection::remove(const String & file_name) +{ + files.erase(file_name); +} + +String S3QueueFilesMetadata::S3QueueProcessingCollection::toString() const +{ + return DB::toString(Strings(files.begin(), files.end())); +} + + +S3QueueFilesMetadata::S3QueueFilesMetadata( + const StorageS3Queue * storage_, + const S3QueueSettings & settings_) + : storage(storage_) + , mode(settings_.mode) + , max_set_size(settings_.s3queue_tracked_files_limit.value) + , max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value) + , max_loading_retries(settings_.s3queue_loading_retries.value) + , zookeeper_processing_path(fs::path(storage->getZooKeeperPath()) / "processing") + , zookeeper_processed_path(fs::path(storage->getZooKeeperPath()) / "processed") + , zookeeper_failed_path(fs::path(storage->getZooKeeperPath()) / "failed") + , zookeeper_lock_path(fs::path(storage->getZooKeeperPath()) / "lock") + , log(&Poco::Logger::get("S3QueueFilesMetadata")) +{ +} + +void S3QueueFilesMetadata::setFileProcessed(const String & file_path) +{ + auto zookeeper = storage->getZooKeeper(); + auto lock = acquireLock(zookeeper); + + switch (mode) + { + case S3QueueMode::UNORDERED: + { + S3QueueProcessedCollection processed_files(max_set_size, max_set_age_sec); + processed_files.parse(zookeeper->get(zookeeper_processed_path)); + processed_files.add(file_path); + zookeeper->set(zookeeper_processed_path, processed_files.toString()); + break; + } + case S3QueueMode::ORDERED: + { + // Check that we set in ZooKeeper node only maximum processed file path. + // This check can be useful, when multiple table engines consume in ordered mode. + String max_file = getMaxProcessedFile(); + if (max_file.compare(file_path) <= 0) + zookeeper->set(zookeeper_processed_path, file_path); + break; + } + } + removeProcessingFile(file_path); +} + + +bool S3QueueFilesMetadata::setFileFailed(const String & file_path, const String & exception_message) +{ + auto zookeeper = storage->getZooKeeper(); + auto lock = acquireLock(zookeeper); + + S3QueueFailedCollection failed_collection(max_loading_retries); + failed_collection.parse(zookeeper->get(zookeeper_failed_path)); + const bool can_be_retried = failed_collection.add(file_path, exception_message); + zookeeper->set(zookeeper_failed_path, failed_collection.toString()); + removeProcessingFile(file_path); + return can_be_retried; +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getFailedFiles() +{ + auto zookeeper = storage->getZooKeeper(); + String failed_files = zookeeper->get(zookeeper_failed_path); + + S3QueueFailedCollection failed_collection(max_loading_retries); + failed_collection.parse(failed_files); + return failed_collection.getFileNames(); +} + +String S3QueueFilesMetadata::getMaxProcessedFile() +{ + auto zookeeper = storage->getZooKeeper(); + return zookeeper->get(zookeeper_processed_path); +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getProcessingFiles() +{ + auto zookeeper = storage->getZooKeeper(); + String processing_files; + if (!zookeeper->tryGet(zookeeper_processing_path, processing_files)) + return {}; + + S3QueueProcessingCollection processing_collection; + if (!processing_files.empty()) + processing_collection.parse(processing_files); + return processing_collection.getFileNames(); +} + +void S3QueueFilesMetadata::setFilesProcessing(const Strings & file_paths) +{ + auto zookeeper = storage->getZooKeeper(); + String processing_files; + zookeeper->tryGet(zookeeper_processing_path, processing_files); + + S3QueueProcessingCollection processing_collection; + if (!processing_files.empty()) + processing_collection.parse(processing_files); + processing_collection.add(file_paths); + + if (zookeeper->exists(zookeeper_processing_path)) + zookeeper->set(zookeeper_processing_path, processing_collection.toString()); + else + zookeeper->create(zookeeper_processing_path, processing_collection.toString(), zkutil::CreateMode::Ephemeral); +} + +void S3QueueFilesMetadata::removeProcessingFile(const String & file_path) +{ + auto zookeeper = storage->getZooKeeper(); + String processing_files; + zookeeper->tryGet(zookeeper_processing_path, processing_files); + + S3QueueProcessingCollection processing_collection; + processing_collection.parse(processing_files); + processing_collection.remove(file_path); + zookeeper->set(zookeeper_processing_path, processing_collection.toString()); +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getUnorderedProcessedFiles() +{ + auto zookeeper = storage->getZooKeeper(); + S3QueueProcessedCollection processed_collection(max_set_size, max_set_age_sec); + processed_collection.parse(zookeeper->get(zookeeper_processed_path)); + return processed_collection.getFileNames(); +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getProcessedFailedAndProcessingFiles() +{ + S3FilesCollection processed_and_failed_files = getFailedFiles(); + switch (mode) + { + case S3QueueMode::UNORDERED: + { + processed_and_failed_files.merge(getUnorderedProcessedFiles()); + break; + } + case S3QueueMode::ORDERED: + { + processed_and_failed_files.insert(getMaxProcessedFile()); + break; + } + } + processed_and_failed_files.merge(getProcessingFiles()); + return processed_and_failed_files; +} + +std::shared_ptr S3QueueFilesMetadata::acquireLock(zkutil::ZooKeeperPtr zookeeper) +{ + UInt32 retry_count = 200; + UInt32 sleep_ms = 100; + UInt32 retries = 0; + + while (true) + { + Coordination::Error code = zookeeper->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + { + retries++; + if (retries > retry_count) + { + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Can't acquire zookeeper lock"); + } + sleepForMilliseconds(sleep_ms); + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception(code, zookeeper_lock_path); + } + else + { + return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zookeeper); + } + } +} + +} + +#endif diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h similarity index 73% rename from src/Storages/S3Queue/S3QueueHolder.h rename to src/Storages/S3Queue/S3QueueFilesMetadata.h index de7f1f56f9a..c436de946ff 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -9,7 +9,10 @@ namespace DB { -class S3QueueHolder : public WithContext +class StorageS3Queue; +struct S3QueueSettings; + +class S3QueueFilesMetadata { public: struct TrackedCollectionItem @@ -23,27 +26,21 @@ public: using S3FilesCollection = std::unordered_set; using TrackedFiles = std::deque; - S3QueueHolder( - const String & zookeeper_path_, - const S3QueueMode & mode_, - ContextPtr context_, - UInt64 & max_set_size_, - UInt64 & max_set_age_sec_, - UInt64 & max_loading_retries_); + S3QueueFilesMetadata(const StorageS3Queue * storage_, const S3QueueSettings & settings_); + void setFilesProcessing(const Strings & file_paths); void setFileProcessed(const String & file_path); bool setFileFailed(const String & file_path, const String & exception_message); - void setFilesProcessing(Strings & file_paths); - S3FilesCollection getProcessedAndFailedFiles(); - String getMaxProcessedFile(); - std::shared_ptr acquireLock(); + S3FilesCollection getProcessedFailedAndProcessingFiles(); + String getMaxProcessedFile(); + std::shared_ptr acquireLock(zkutil::ZooKeeperPtr zookeeper); struct S3QueueCollection { public: virtual ~S3QueueCollection() = default; - String toString() const; + virtual String toString() const; S3FilesCollection getFileNames(); virtual void parse(const String & collection_str) = 0; @@ -82,30 +79,42 @@ public: UInt64 max_retries_count; }; + struct S3QueueProcessingCollection + { + public: + S3QueueProcessingCollection() = default; + + void parse(const String & collection_str); + void add(const Strings & file_names); + void remove(const String & file_name); + + String toString() const; + const S3FilesCollection & getFileNames() const { return files; } + + private: + S3FilesCollection files; + }; private: + const StorageS3Queue * storage; + const S3QueueMode mode; const UInt64 max_set_size; const UInt64 max_set_age_sec; const UInt64 max_loading_retries; - zkutil::ZooKeeperPtr zk_client; - mutable std::mutex current_zookeeper_mutex; - mutable std::mutex mutex; - const String zookeeper_path; - const String zookeeper_failed_path; const String zookeeper_processing_path; const String zookeeper_processed_path; + const String zookeeper_failed_path; const String zookeeper_lock_path; - const S3QueueMode mode; - const UUID table_uuid; + + mutable std::mutex mutex; Poco::Logger * log; S3FilesCollection getFailedFiles(); S3FilesCollection getProcessingFiles(); S3FilesCollection getUnorderedProcessedFiles(); - void removeProcessingFile(const String & file_path); - S3FilesCollection parseCollection(const String & collection_str); + void removeProcessingFile(const String & file_path); }; diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp deleted file mode 100644 index 860484da671..00000000000 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ /dev/null @@ -1,341 +0,0 @@ -#include "IO/VarInt.h" -#include "config.h" - -#if USE_AWS_S3 -# include -# include -# include -# include -# include -# include -# include -# include -# include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int TIMEOUT_EXCEEDED; -} - -/// TODO: update zk session if expired - -void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) -{ - files = {}; - - if (in.eof()) - return; - - size_t files_num; - in >> files_num >> "\n"; - while (files_num--) - { - TrackedCollectionItem item; - in >> item.file_path >> "\n"; - in >> item.timestamp >> "\n"; - in >> item.retries_count >> "\n"; - in >> item.last_exception >> "\n"; - files.push_back(item); - } -} - -void S3QueueHolder::S3QueueCollection::write(WriteBuffer & out) const -{ - out << files.size() << "\n"; - for (const auto & processed_file : files) - { - out << processed_file.file_path << "\n"; - out << processed_file.timestamp << "\n"; - out << processed_file.retries_count << "\n"; - out << processed_file.last_exception << "\n"; - } -} - -String S3QueueHolder::S3QueueCollection::toString() const -{ - WriteBufferFromOwnString out; - write(out); - return out.str(); -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueCollection::getFileNames() -{ - S3FilesCollection keys = {}; - for (const auto & pair : files) - { - keys.insert(pair.file_path); - } - return keys; -} - - -S3QueueHolder::S3QueueProcessedCollection::S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) - : max_size(max_size_), max_age(max_age_) -{ -} - -void S3QueueHolder::S3QueueProcessedCollection::parse(const String & collection_str) -{ - ReadBufferFromString buf(collection_str); - read(buf); - if (max_age > 0) // Remove old items - { - UInt64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - UInt64 max_seconds_diff = max_age; - std::erase_if( - files, - [×tamp, &max_seconds_diff](const TrackedCollectionItem & processed_file) - { return (timestamp - processed_file.timestamp) > max_seconds_diff; }); - } -} - - -void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) -{ - UInt64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - - TrackedCollectionItem processed_file = {.file_path=file_name, .timestamp=timestamp}; - files.push_back(processed_file); - - while (files.size() > max_size) - { - files.pop_front(); - } -} - - -S3QueueHolder::S3QueueFailedCollection::S3QueueFailedCollection(const UInt64 & max_retries_count_) : max_retries_count(max_retries_count_) -{ -} - -void S3QueueHolder::S3QueueFailedCollection::parse(const String & collection_str) -{ - ReadBufferFromString buf(collection_str); - read(buf); -} - - -bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name, const String & exception_message) -{ - auto failed_it - = std::find_if(files.begin(), files.end(), [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); - if (failed_it != files.end()) - { - if (failed_it->retries_count == 0 || --failed_it->retries_count == 0) - { - return false; - } - } - else - { - TrackedCollectionItem failed_file = { .file_path=file_name, .retries_count=max_retries_count, .last_exception = exception_message }; - files.push_back(failed_file); - } - return true; -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueFailedCollection::getFileNames() -{ - S3FilesCollection failed_keys; - for (const auto & pair : files) - { - if (pair.retries_count <= 0) - { - failed_keys.insert(pair.file_path); - } - } - return failed_keys; -} - -S3QueueHolder::S3QueueHolder( - const String & zookeeper_path_, - const S3QueueMode & mode_, - ContextPtr context_, - UInt64 & max_set_size_, - UInt64 & max_set_age_sec_, - UInt64 & max_loading_retries_) - : WithContext(context_) - , max_set_size(max_set_size_) - , max_set_age_sec(max_set_age_sec_) - , max_loading_retries(max_loading_retries_) - , zk_client(getContext()->getZooKeeper()) - , zookeeper_path(zookeeper_path_) - , zookeeper_failed_path(fs::path(zookeeper_path_) / "failed") - , zookeeper_processing_path(fs::path(zookeeper_path_) / "processing") - , zookeeper_processed_path(fs::path(zookeeper_path_) / "processed") - , zookeeper_lock_path(fs::path(zookeeper_path_) / "lock") - , mode(mode_) - , log(&Poco::Logger::get("S3QueueHolder")) -{ -} - - -void S3QueueHolder::setFileProcessed(const String & file_path) -{ - auto lock = acquireLock(); - - if (mode == S3QueueMode::UNORDERED) - { - String processed_files = zk_client->get(zookeeper_processed_path); - auto processed = S3QueueProcessedCollection(max_set_size, max_set_age_sec); - processed.parse(processed_files); - processed.add(file_path); - zk_client->set(zookeeper_processed_path, processed.toString()); - } - else if (mode == S3QueueMode::ORDERED) - { - String max_file = getMaxProcessedFile(); - // Check that we set in ZooKeeper node only maximum processed file path. - // This check can be useful, when multiple table engines consume in ordered mode. - if (max_file.compare(file_path) <= 0) - { - zk_client->set(zookeeper_processed_path, file_path); - } - } - removeProcessingFile(file_path); -} - - -bool S3QueueHolder::setFileFailed(const String & file_path, const String & exception_message) -{ - auto lock = acquireLock(); - - auto failed_collection = S3QueueFailedCollection(max_loading_retries); - failed_collection.parse(zk_client->get(zookeeper_failed_path)); - bool retry_later = failed_collection.add(file_path, exception_message); - - zk_client->set(zookeeper_failed_path, failed_collection.toString()); - removeProcessingFile(file_path); - - return retry_later; -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() -{ - String failed_files = zk_client->get(zookeeper_failed_path); - - auto failed_collection = S3QueueFailedCollection(max_loading_retries); - failed_collection.parse(failed_files); - - return failed_collection.getFileNames(); -} - -String S3QueueHolder::getMaxProcessedFile() -{ - String processed = zk_client->get(zookeeper_processed_path); - return processed; -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() -{ - String processing_files; - if (!zk_client->tryGet(zookeeper_processing_path, processing_files)) - return {}; - return parseCollection(processing_files); -} - -void S3QueueHolder::setFilesProcessing(Strings & file_paths) -{ - std::unordered_set processing_files(file_paths.begin(), file_paths.end()); - processing_files.merge(getProcessingFiles()); - String processing_files_str = toString(Strings(processing_files.begin(), processing_files.end())); - - if (zk_client->exists(zookeeper_processing_path)) - zk_client->set(fs::path(zookeeper_processing_path), processing_files_str); - else - zk_client->create(fs::path(zookeeper_processing_path), processing_files_str, zkutil::CreateMode::Ephemeral); -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getUnorderedProcessedFiles() -{ - String processed = zk_client->get(zookeeper_processed_path); - auto collection = S3QueueProcessedCollection(max_set_size, max_set_age_sec); - collection.parse(processed); - return collection.getFileNames(); -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessedAndFailedFiles() -{ - S3FilesCollection processed_and_failed_files = getFailedFiles(); - - if (mode == S3QueueMode::UNORDERED) - { - processed_and_failed_files.merge(getUnorderedProcessedFiles()); - } - else - { - String processed = getMaxProcessedFile(); - processed_and_failed_files.insert(processed); - } - - S3FilesCollection processing_files = getProcessingFiles(); - processed_and_failed_files.merge(processing_files); - - return processed_and_failed_files; -} - -void S3QueueHolder::removeProcessingFile(const String & file_path) -{ - String node_data; - String processing = zk_client->get(zookeeper_processing_path); - S3FilesCollection processing_files = parseCollection(processing); - - processing_files.erase(file_path); - - Strings file_paths(processing_files.begin(), processing_files.end()); - zk_client->set(fs::path(zookeeper_processing_path), toString(file_paths)); -} - -std::shared_ptr S3QueueHolder::acquireLock() -{ - UInt32 retry_count = 200; - UInt32 sleep_ms = 100; - - UInt32 retries = 0; - while (true) - { - Coordination::Error code = zk_client->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); - if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) - { - retries++; - if (retries > retry_count) - { - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Can't acquire zookeeper lock"); - } - sleepForMilliseconds(sleep_ms); - } - else if (code != Coordination::Error::ZOK) - { - throw Coordination::Exception(code, zookeeper_lock_path); - } - else - { - return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zk_client); - } - } -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(const String & collection_str) -{ - ReadBufferFromString rb(collection_str); - Strings deserialized; - try - { - readQuoted(deserialized, rb); - } - catch (const Exception & e) - { - LOG_WARNING(log, "Can't parse collection from ZooKeeper node: {}", e.displayText()); - deserialized = {}; - } - - return std::unordered_set(deserialized.begin(), deserialized.end()); -} - -} - -#endif diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 4785a131f60..57d2d6304b0 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -171,7 +171,7 @@ StorageS3QueueSource::StorageS3QueueSource( const String & bucket_, const String & version_id_, std::shared_ptr file_iterator_, - std::shared_ptr queue_holder_, + std::shared_ptr files_metadata_, const S3QueueAction & action_, const size_t download_thread_num_) : ISource(getHeader(sample_block_, requested_virtual_columns_)) @@ -183,7 +183,7 @@ StorageS3QueueSource::StorageS3QueueSource( , columns_desc(columns_) , request_settings(request_settings_) , client(client_) - , queue_holder(queue_holder_) + , files_metadata(files_metadata_) , requested_virtual_columns(requested_virtual_columns_) , file_iterator(file_iterator_) , action(action_) @@ -259,13 +259,13 @@ Chunk StorageS3QueueSource::generate() catch (const Exception & e) { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); - queue_holder->setFileFailed(reader.getFile(), e.message()); + files_metadata->setFileFailed(reader.getFile(), e.message()); success_in_pulling = false; } if (success_in_pulling) { applyActionAfterProcessing(reader.getFile()); - queue_holder->setFileProcessed(reader.getFile()); + files_metadata->setFileProcessed(reader.getFile()); return chunk; } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index e2e472b5007..a85fce46ad8 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -8,7 +8,7 @@ # include # include -# include +# include # include # include @@ -81,7 +81,7 @@ public: const String & bucket, const String & version_id, std::shared_ptr file_iterator_, - std::shared_ptr queue_holder_, + std::shared_ptr files_metadata_, const S3QueueAction & action_, size_t download_thread_num); @@ -101,7 +101,7 @@ private: S3Settings::RequestSettings request_settings; std::shared_ptr client; - std::shared_ptr queue_holder; + std::shared_ptr files_metadata; using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index baacb3397c7..87bff398172 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -93,23 +93,21 @@ StorageS3Queue::StorageS3Queue( : IStorage(table_id_) , WithContext(context_) , s3queue_settings(std::move(s3queue_settings_)) - , s3_configuration{configuration_} - , keys({s3_configuration.url.key}) - , mode(s3queue_settings->mode) , after_processing(s3queue_settings->after_processing) - , milliseconds_to_wait(s3queue_settings->s3queue_polling_min_timeout_ms) - , format_name(configuration_.format) - , compression_method(configuration_.compression_method) - , name(s3_configuration.url.storage_name) + , configuration{configuration_} + , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) , format_settings(format_settings_) , partition_by(partition_by_) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { - if (!withGlobs()) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); + if (configuration.url.key.ends_with('/')) + configuration.url.key += '*'; - String setting_zookeeper_path = s3queue_settings->keeper_path; - if (setting_zookeeper_path.empty()) + if (!withGlobs()) + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs"); + + String setting_zk_path = s3queue_settings->keeper_path; + if (setting_zk_path.empty()) { auto database = DatabaseCatalog::instance().getDatabase(table_id_.database_name); bool is_in_replicated_database = database->getEngineName() == "Replicated"; @@ -135,25 +133,25 @@ StorageS3Queue::StorageS3Queue( "s3queue_default_zookeeper_path_prefix not specified"); } - zookeeper_path = zkutil::extractZooKeeperPath( + zk_path = zkutil::extractZooKeeperPath( fs::path(zk_path_prefix) / toString(table_id_.uuid), /* check_starts_with_slash */ true, log); } else { /// We do not add table uuid here on purpose. - zookeeper_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path.value, /* check_starts_with_slash */ true, log); + zk_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path.value, /* check_starts_with_slash */ true, log); } - LOG_INFO(log, "Using zookeeper path: {}", zookeeper_path); + LOG_INFO(log, "Using zookeeper path: {}", zk_path); - FormatFactory::instance().checkFormatName(format_name); - context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); + FormatFactory::instance().checkFormatName(configuration.format); + context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri); StorageInMemoryMetadata storage_metadata; - s3_configuration.update(context_); + configuration.update(context_); if (columns_.empty()) { - auto columns = StorageS3::getTableStructureFromDataImpl(s3_configuration, format_settings, context_); + auto columns = StorageS3::getTableStructureFromDataImpl(configuration, format_settings, context_); storage_metadata.setColumns(columns); } else @@ -163,22 +161,15 @@ StorageS3Queue::StorageS3Queue( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - setZooKeeper(); auto metadata_snapshot = getInMemoryMetadataPtr(); const bool is_first_replica = createTableIfNotExists(metadata_snapshot); if (!is_first_replica) { - checkTableStructure(zookeeper_path, metadata_snapshot); + checkTableStructure(zk_path, metadata_snapshot); } - queue_holder = std::make_unique( - zookeeper_path, - mode, - getContext(), - s3queue_settings->s3queue_tracked_files_limit.value, - s3queue_settings->s3queue_tracked_file_ttl_sec.value, - s3queue_settings->s3queue_loading_retries.value); + files_metadata = std::make_shared(this, *s3queue_settings); auto default_virtuals = NamesAndTypesList{ {"_path", std::make_shared(std::make_shared())}, @@ -196,12 +187,12 @@ StorageS3Queue::StorageS3Queue( bool StorageS3Queue::supportsSubcolumns() const { - return FormatFactory::instance().checkIfFormatSupportsSubcolumns(format_name); + return FormatFactory::instance().checkIfFormatSupportsSubcolumns(configuration.format); } bool StorageS3Queue::supportsSubsetOfColumns() const { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); } Pipe StorageS3Queue::read( @@ -220,7 +211,7 @@ Pipe StorageS3Queue::read( if (mv_attached) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageS3Queue with attached materialized views"); - auto query_s3_configuration = updateConfigurationAndGetCopy(local_context); + auto query_configuration = updateConfigurationAndGetCopy(local_context); Pipes pipes; @@ -262,24 +253,23 @@ Pipe StorageS3Queue::read( } const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; - auto zookeeper = getZooKeeper(); return Pipe(std::make_shared( requested_virtual_columns, - format_name, + configuration.format, getName(), block_for_format, local_context, format_settings, columns_description, max_block_size, - query_s3_configuration.request_settings, - compression_method, - query_s3_configuration.client, - query_s3_configuration.url.bucket, - query_s3_configuration.url.version_id, + query_configuration.request_settings, + configuration.compression_method, + query_configuration.client, + query_configuration.url.bucket, + query_configuration.url.version_id, iterator_wrapper, - queue_holder, + files_metadata, after_processing, max_download_threads)); } @@ -387,7 +377,7 @@ void StorageS3Queue::threadFunc() break; } - milliseconds_to_wait = s3queue_settings->s3queue_polling_min_timeout_ms; + reschedule_processing_interval_ms = s3queue_settings->s3queue_polling_min_timeout_ms; } } } @@ -402,9 +392,9 @@ void StorageS3Queue::threadFunc() { LOG_TRACE(log, "Reschedule S3 Queue thread func."); /// Reschedule with backoff. - if (milliseconds_to_wait < s3queue_settings->s3queue_polling_max_timeout_ms) - milliseconds_to_wait += s3queue_settings->s3queue_polling_backoff_ms; - task->holder->scheduleAfter(milliseconds_to_wait); + if (reschedule_processing_interval_ms < s3queue_settings->s3queue_polling_max_timeout_ms) + reschedule_processing_interval_ms += s3queue_settings->s3queue_polling_backoff_ms; + task->holder->scheduleAfter(reschedule_processing_interval_ms); } } @@ -426,7 +416,7 @@ void StorageS3Queue::streamToViews() auto s3queue_context = Context::createCopy(getContext()); s3queue_context->makeQueryContext(); - auto query_s3_configuration = updateConfigurationAndGetCopy(s3queue_context); + auto query_configuration = updateConfigurationAndGetCopy(s3queue_context); // Create a stream for each consumer and join them in a union stream // Only insert into dependent views and expect that input blocks contain virtual columns @@ -473,23 +463,22 @@ void StorageS3Queue::streamToViews() Pipes pipes; - auto zookeeper = getZooKeeper(); auto pipe = Pipe(std::make_shared( requested_virtual_columns, - format_name, + configuration.format, getName(), block_for_format, s3queue_context, format_settings, columns_description, block_size, - query_s3_configuration.request_settings, - compression_method, - query_s3_configuration.client, - query_s3_configuration.url.bucket, - query_s3_configuration.url.version_id, + query_configuration.request_settings, + configuration.compression_method, + query_configuration.client, + query_configuration.url.bucket, + query_configuration.url.version_id, iterator_wrapper, - queue_holder, + files_metadata, after_processing, max_download_threads)); @@ -505,65 +494,56 @@ void StorageS3Queue::streamToViews() StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) { - s3_configuration.update(local_context); - return s3_configuration; -} - -void StorageS3Queue::setZooKeeper() -{ - std::lock_guard lock(current_zookeeper_mutex); - current_zookeeper = getContext()->getZooKeeper(); -} - -zkutil::ZooKeeperPtr StorageS3Queue::tryGetZooKeeper() const -{ - std::lock_guard lock(current_zookeeper_mutex); - return current_zookeeper; + configuration.update(local_context); + return configuration; } zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const { - auto res = tryGetZooKeeper(); - if (!res) - throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); - return res; + std::lock_guard lock{zk_mutex}; + if (!zk_client || zk_client->expired()) + { + zk_client = getContext()->getZooKeeper(); + zk_client->sync(zk_path); + } + return zk_client; } bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot) { auto zookeeper = getZooKeeper(); - zookeeper->createAncestors(zookeeper_path); + zookeeper->createAncestors(zk_path); for (size_t i = 0; i < zk_create_table_retries; ++i) { Coordination::Requests ops; bool is_first_replica = true; - if (zookeeper->exists(zookeeper_path + "/metadata")) + if (zookeeper->exists(zk_path + "/metadata")) { - if (!zookeeper->exists(zookeeper_path + "/processing")) - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); - LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zookeeper_path); + if (!zookeeper->exists(zk_path + "/processing")) + ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processing", "", zkutil::CreateMode::Ephemeral)); + LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zk_path); is_first_replica = false; } else { - String metadata_str = S3QueueTableMetadata(s3_configuration, *s3queue_settings).toString(); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); + String metadata_str = S3QueueTableMetadata(configuration, *s3queue_settings).toString(); + ops.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processed", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/failed", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processing", "", zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); + zk_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/metadata", metadata_str, zkutil::CreateMode::Persistent)); } Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZNODEEXISTS) { - LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zookeeper_path); + LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zk_path); continue; } else if (code != Coordination::Error::ZOK) @@ -577,7 +557,7 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ throw Exception( ErrorCodes::REPLICA_ALREADY_EXISTS, "Cannot create table, because it is created concurrently every time or because " - "of wrong zookeeper_path or because of logical error"); + "of wrong zk_path or because of logical error"); } @@ -588,7 +568,7 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const { auto zookeeper = getZooKeeper(); - S3QueueTableMetadata old_metadata(s3_configuration, *s3queue_settings); + S3QueueTableMetadata old_metadata(configuration, *s3queue_settings); Coordination::Stat metadata_stat; String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata", &metadata_stat); @@ -615,39 +595,40 @@ std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query) { auto it = std::make_shared( - *s3_configuration.client, - s3_configuration.url, + *configuration.client, + configuration.url, query, virtual_block, local_context, s3queue_settings->s3queue_polling_size.value, - s3_configuration.request_settings); + configuration.request_settings); - auto lock = queue_holder->acquireLock(); - S3QueueHolder::S3FilesCollection files_to_skip = queue_holder->getProcessedAndFailedFiles(); + auto zookeeper = getZooKeeper(); + auto lock = files_metadata->acquireLock(zookeeper); + S3QueueFilesMetadata::S3FilesCollection files_to_skip = files_metadata->getProcessedFailedAndProcessingFiles(); Strings files_to_process; - if (mode == S3QueueMode::UNORDERED) + if (s3queue_settings->mode == S3QueueMode::UNORDERED) { - files_to_process = it->filterProcessingFiles(mode, files_to_skip); + files_to_process = it->filterProcessingFiles(s3queue_settings->mode, files_to_skip); } else { - String max_processed_file = queue_holder->getMaxProcessedFile(); - files_to_process = it->filterProcessingFiles(mode, files_to_skip, max_processed_file); + String max_processed_file = files_metadata->getMaxProcessedFile(); + files_to_process = it->filterProcessingFiles(s3queue_settings->mode, files_to_skip, max_processed_file); } LOG_TEST(log, "Found files to process: {}", fmt::join(files_to_process, ", ")); - queue_holder->setFilesProcessing(files_to_process); + files_metadata->setFilesProcessing(files_to_process); return it; } void StorageS3Queue::drop() { - auto zk_client = getZooKeeper(); - if (zk_client->exists(zookeeper_path)) - zk_client->removeRecursive(zookeeper_path); + auto zookeeper = getZooKeeper(); + if (zookeeper->exists(zk_path)) + zookeeper->removeRecursive(zk_path); } void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 6af38058dd8..9737d5fcefa 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -11,7 +11,7 @@ # include # include -# include +# include # include # include # include @@ -41,6 +41,7 @@ class StorageS3Queue : public IStorage, WithContext { public: using Configuration = typename StorageS3::Configuration; + StorageS3Queue( std::unique_ptr s3queue_settings_, const Configuration & configuration_, @@ -79,35 +80,39 @@ public: bool supportsPartitionBy() const override; - const auto & getFormatName() const { return format_name; } + const auto & getFormatName() const { return configuration.format; } + + const String & getZooKeeperPath() const { return zk_path; } + + zkutil::ZooKeeperPtr getZooKeeper() const; private: - std::unique_ptr s3queue_settings; - std::shared_ptr queue_holder; - Configuration s3_configuration; - std::vector keys; + const std::unique_ptr s3queue_settings; + const S3QueueAction after_processing; + + std::shared_ptr files_metadata; + Configuration configuration; NamesAndTypesList virtual_columns; Block virtual_block; - S3QueueMode mode; - S3QueueAction after_processing; - uint64_t milliseconds_to_wait = 10000; - - String format_name; - String compression_method; - String name; + UInt64 reschedule_processing_interval_ms; std::optional format_settings; ASTPtr partition_by; + String zk_path; + mutable zkutil::ZooKeeperPtr zk_client; + mutable std::mutex zk_mutex; + + std::atomic mv_attached = false; + std::atomic shutdown_called{false}; + Poco::Logger * log; + bool supportsSubcolumns() const override; - bool withGlobs() const { return s3_configuration.url.key.find_first_of("*?{") != std::string::npos; } + bool withGlobs() const { return configuration.url.key.find_first_of("*?{") != std::string::npos; } void threadFunc(); size_t getTableDependentCount() const; - std::atomic mv_attached = false; bool hasDependencies(const StorageID & table_id); - std::atomic shutdown_called{false}; - Poco::Logger * log; void startup() override; void shutdown() override; @@ -122,19 +127,10 @@ private: std::shared_ptr task; bool supportsSubsetOfColumns() const override; - String zookeeper_path; - - zkutil::ZooKeeperPtr current_zookeeper; - mutable std::mutex current_zookeeper_mutex; - - void setZooKeeper(); - zkutil::ZooKeeperPtr tryGetZooKeeper() const; - zkutil::ZooKeeperPtr getZooKeeper() const; const UInt32 zk_create_table_retries = 1000; bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); - const String & getZooKeeperPath() const { return zookeeper_path; } using KeysWithInfo = StorageS3QueueSource::KeysWithInfo; From 6859c2c7fe49ca683879ac81b886a94d8869a8cd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 1 Aug 2023 10:09:48 +0000 Subject: [PATCH 1423/2047] Try to fix 02352_rwlock --- tests/queries/0_stateless/02352_rwlock.sh | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/02352_rwlock.sh b/tests/queries/0_stateless/02352_rwlock.sh index 7505a03a382..44d08b9b5c4 100755 --- a/tests/queries/0_stateless/02352_rwlock.sh +++ b/tests/queries/0_stateless/02352_rwlock.sh @@ -12,6 +12,11 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function wait_query_by_id_started() { + # As the query we are waiting for is running simultaneously, let's give it a little time to actually start. The + # queries are supposed to run for multiple seconds, so sleeping 0.5 seconds is not a big deal, especially when + # flushing the logs can take up to 3 to 5 seconds. Basically waiting a bit here we can increase the chance that we + # don't have spend precious time on flushing logs. + sleep 0.5 local query_id=$1 && shift # wait for query to be started while [ "$($CLICKHOUSE_CLIENT "$@" -q "select count() from system.processes where query_id = '$query_id'")" -ne 1 ]; do From 1cbd4ff2d65b3ecb5dedef80c2f77d77e2dd9b8d Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 1 Aug 2023 12:42:36 +0000 Subject: [PATCH 1424/2047] Fix possible assert in ~PushingAsyncPipelineExecutor in clickhouse-local --- src/Client/LocalConnection.cpp | 11 +++++++++++ .../0_stateless/02841_local_assert.reference | 1 + .../queries/0_stateless/02841_local_assert.sh | 18 ++++++++++++++++++ 3 files changed, 30 insertions(+) create mode 100644 tests/queries/0_stateless/02841_local_assert.reference create mode 100755 tests/queries/0_stateless/02841_local_assert.sh diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 6bb792ac51e..749ac7886c8 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -35,6 +35,9 @@ LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_, bool LocalConnection::~LocalConnection() { + /// Last query can be not finished or cancelled due to exception on client side. + if (state && !state->is_finished && !state->is_cancelled) + sendCancel(); state.reset(); } @@ -73,6 +76,10 @@ void LocalConnection::sendQuery( bool, std::function process_progress_callback) { + /// Last query can be not finished or cancelled due to exception on client side. + if (state && !state->is_finished && !state->is_cancelled) + sendCancel(); + /// Suggestion comes without client_info. if (client_info) query_context = session.makeQueryContext(*client_info); @@ -204,6 +211,10 @@ void LocalConnection::sendCancel() state->is_cancelled = true; if (state->executor) state->executor->cancel(); + if (state->pushing_executor) + state->pushing_executor->cancel(); + if (state->pushing_async_executor) + state->pushing_async_executor->cancel(); } bool LocalConnection::pullBlock(Block & block) diff --git a/tests/queries/0_stateless/02841_local_assert.reference b/tests/queries/0_stateless/02841_local_assert.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02841_local_assert.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02841_local_assert.sh b/tests/queries/0_stateless/02841_local_assert.sh new file mode 100755 index 00000000000..a167c09da1f --- /dev/null +++ b/tests/queries/0_stateless/02841_local_assert.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo "create table test (x UInt64) engine=Memory; +insert into test from infile 'data'; -- {clientError BAD_ARGUMENTS}" | $CLICKHOUSE_LOCAL -nm + +echo "create table test (x UInt64) engine=Memory; +insert into test from infile 'data';" | $CLICKHOUSE_LOCAL -nm --ignore-error + +echo "create table test (x UInt64) engine=Memory; +insert into test from infile 'data'; -- {clientError BAD_ARGUMENTS} +select 1" | $CLICKHOUSE_LOCAL -nm + From 9027ab968eaa3389c75d8717a031e28c86b213f6 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 1 Aug 2023 15:04:53 +0300 Subject: [PATCH 1425/2047] Extend the set of test cases in the integration test --- .../configs/storage_config.xml | 10 ++++- .../test.py | 44 ++++++++++++------- 2 files changed, 37 insertions(+), 17 deletions(-) diff --git a/tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml b/tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml index d27a919aa7a..de9f68cbe87 100644 --- a/tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml +++ b/tests/integration/test_backup_restore_storage_policy/configs/storage_config.xml @@ -11,14 +11,20 @@
- + one + + + + + + two - + diff --git a/tests/integration/test_backup_restore_storage_policy/test.py b/tests/integration/test_backup_restore_storage_policy/test.py index 50e71a11ae9..4e9d309a220 100644 --- a/tests/integration/test_backup_restore_storage_policy/test.py +++ b/tests/integration/test_backup_restore_storage_policy/test.py @@ -35,28 +35,42 @@ def new_backup_name(): return f"File('/backups/{backup_id_counter}/')" -def create_and_fill_table(): +def create_table_backup(backup_name, storage_policy=None): instance.query("CREATE DATABASE test") - instance.query("CREATE TABLE test.table(x UInt32) ENGINE=MergeTree ORDER BY x") + create_query = "CREATE TABLE test.table(x UInt32) ENGINE=MergeTree ORDER BY x" + if storage_policy is not None: + create_query += f" SETTINGS storage_policy = '{storage_policy}'" + instance.query(create_query) instance.query(f"INSERT INTO test.table SELECT number FROM numbers(10)") - - -@pytest.mark.parametrize("policy", ["disks_in_order", "", None]) -def test_restore_table(policy): - backup_name = new_backup_name() - create_and_fill_table() instance.query(f"BACKUP TABLE test.table TO {backup_name}") instance.query("DROP TABLE test.table SYNC") + + +def restore_table(backup_name, storage_policy=None): restore_query = f"RESTORE TABLE test.table FROM {backup_name}" - if policy is None: - policy = "default" - else: - restore_query += f" SETTINGS storage_policy = '{policy}'" - if policy == "": - policy = "default" + if storage_policy is not None: + restore_query += f" SETTINGS storage_policy = '{storage_policy}'" instance.query(restore_query) + +@pytest.mark.parametrize( + "origin_policy, restore_policy, expected_policy", + [ + (None, "", "default"), + (None, None, "default"), + (None, "policy1", "policy1"), + ("policy1", "policy1", "policy1"), + ("policy1", "policy2", "policy2"), + ("policy1", "", "default"), + ("policy1", None, "policy1"), + ], +) +def test_storage_policies(origin_policy, restore_policy, expected_policy): + backup_name = new_backup_name() + create_table_backup(backup_name, origin_policy) + restore_table(backup_name, restore_policy) + assert ( instance.query("SELECT storage_policy FROM system.tables WHERE name='table'") - == f"{policy}\n" + == f"{expected_policy}\n" ) From f9a9b02439a93ce6709175b6f2ffd6bbcd9c43fd Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 1 Aug 2023 12:45:56 +0000 Subject: [PATCH 1426/2047] Better comment --- src/Client/LocalConnection.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 749ac7886c8..5caa2a57c7e 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -35,7 +35,7 @@ LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_, bool LocalConnection::~LocalConnection() { - /// Last query can be not finished or cancelled due to exception on client side. + /// Last query may not have been finished or cancelled due to exception on client side. if (state && !state->is_finished && !state->is_cancelled) sendCancel(); state.reset(); @@ -76,7 +76,7 @@ void LocalConnection::sendQuery( bool, std::function process_progress_callback) { - /// Last query can be not finished or cancelled due to exception on client side. + /// Last query may not have been finished or cancelled due to exception on client side. if (state && !state->is_finished && !state->is_cancelled) sendCancel(); From dc07ede6848b8966760879f8ff3f5eeefdbd686c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 1 Aug 2023 14:47:41 +0200 Subject: [PATCH 1427/2047] Update 02808_filesystem_cache_drop_query.sh --- tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh index 4335bc34a28..b44f9e50513 100755 --- a/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh +++ b/tests/queries/0_stateless/02808_filesystem_cache_drop_query.sh @@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -nm --query """ DROP TABLE IF EXISTS test; CREATE TABLE test (a Int32, b String) ENGINE = MergeTree() ORDER BY tuple() -SETTINGS disk = disk_$disk_name(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); +SETTINGS disk = disk(name = '$disk_name', type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, disk = s3_disk); INSERT INTO test SELECT 1, 'test'; """ From 2227945529fb62261e204dfd5a5bc874f097d917 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 1 Aug 2023 13:34:19 +0000 Subject: [PATCH 1428/2047] fixed tests --- tests/integration/test_backup_restore_new/test.py | 1 + tests/integration/test_mysql_protocol/golang.reference | 1 + tests/integration/test_mysql_protocol/test.py | 8 ++++---- .../test_select_access_rights/test_from_system_tables.py | 2 +- 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 53f1599a0d6..92965d910f4 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1227,6 +1227,7 @@ def test_backup_all(exclude_system_log_tables): "asynchronous_insert_log", ] exclude_from_backup += ["system." + table_name for table_name in log_tables] + exclude_from_backup += ["information_schema.COLUMNS", "information_schema.SCHEMATA", "information_schema.TABLES", "information_schema.VIEWS"] backup_command = f"BACKUP ALL {'EXCEPT TABLES ' + ','.join(exclude_from_backup) if exclude_from_backup else ''} TO {backup_name}" diff --git a/tests/integration/test_mysql_protocol/golang.reference b/tests/integration/test_mysql_protocol/golang.reference index 4069b2a086a..7680c8c74dc 100644 --- a/tests/integration/test_mysql_protocol/golang.reference +++ b/tests/integration/test_mysql_protocol/golang.reference @@ -14,6 +14,7 @@ a UNSIGNED TINYINT Result: tables 1 tables 1 +tables 1 Columns: a b diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 5e03c4883b4..494af3c219d 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -679,7 +679,7 @@ def test_php_client(started_cluster, php_container): demux=True, ) assert code == 0 - assert stdout.decode() == "tables\ntables\n" + assert stdout.decode() == "tables\ntables\ntables\n" code, (stdout, stderr) = php_container.exec_run( "php -f test_ssl.php {host} {port} default 123".format( @@ -688,7 +688,7 @@ def test_php_client(started_cluster, php_container): demux=True, ) assert code == 0 - assert stdout.decode() == "tables\ntables\n" + assert stdout.decode() == "tables\ntables\ntables\n" code, (stdout, stderr) = php_container.exec_run( "php -f test.php {host} {port} user_with_double_sha1 abacaba".format( @@ -697,7 +697,7 @@ def test_php_client(started_cluster, php_container): demux=True, ) assert code == 0 - assert stdout.decode() == "tables\ntables\n" + assert stdout.decode() == "tables\ntables\ntables\n" code, (stdout, stderr) = php_container.exec_run( "php -f test_ssl.php {host} {port} user_with_double_sha1 abacaba".format( @@ -706,7 +706,7 @@ def test_php_client(started_cluster, php_container): demux=True, ) assert code == 0 - assert stdout.decode() == "tables\ntables\n" + assert stdout.decode() == "tables\ntables\ntables\n" def test_mysqljs_client(started_cluster, nodejs_container): diff --git a/tests/integration/test_select_access_rights/test_from_system_tables.py b/tests/integration/test_select_access_rights/test_from_system_tables.py index ccdd09c67ef..cd29183561a 100644 --- a/tests/integration/test_select_access_rights/test_from_system_tables.py +++ b/tests/integration/test_select_access_rights/test_from_system_tables.py @@ -194,7 +194,7 @@ def test_information_schema(): node.query( "SELECT count() FROM information_schema.TABLES WHERE table_name='TABLES'" ) - == "3\n" + == "2\n" ) assert ( node.query( From c24dc8e64eddf453f1a0524236e8497a7bdbcf62 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 1 Aug 2023 13:48:49 +0000 Subject: [PATCH 1429/2047] Multithreaded read of files from archives: --- src/IO/Archives/IArchiveReader.h | 4 + src/IO/Archives/LibArchiveReader.cpp | 262 ++++++++++++++++-------- src/IO/Archives/LibArchiveReader.h | 10 +- src/IO/Archives/ZipArchiveReader.cpp | 32 +++ src/IO/Archives/ZipArchiveReader.h | 4 + src/IO/Archives/createArchiveReader.cpp | 4 +- src/Storages/StorageFile.cpp | 103 +++++++--- 7 files changed, 298 insertions(+), 121 deletions(-) diff --git a/src/IO/Archives/IArchiveReader.h b/src/IO/Archives/IArchiveReader.h index 37629cd7eed..0b08a29099c 100644 --- a/src/IO/Archives/IArchiveReader.h +++ b/src/IO/Archives/IArchiveReader.h @@ -40,6 +40,8 @@ public: virtual bool nextFile() = 0; }; + virtual const std::string & getPath() const = 0; + /// Starts enumerating files in the archive. virtual std::unique_ptr firstFile() = 0; @@ -52,6 +54,8 @@ public: virtual std::unique_ptr readFile(std::unique_ptr enumerator) = 0; virtual std::unique_ptr nextFile(std::unique_ptr read_buffer) = 0; + virtual std::vector getAllFiles() = 0; + /// Sets password used to decrypt files in the archive. virtual void setPassword(const String & /* password */) {} diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 87ff734eab7..f3657d5908b 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes extern const int SEEK_POSITION_OUT_OF_BOUND; extern const int NOT_IMPLEMENTED; extern const int CANNOT_READ_ALL_DATA; + extern const int UNSUPPORTED_METHOD; } @@ -26,103 +27,191 @@ class LibArchiveReader::Handle public: explicit Handle(const String & path_to_archive_) : path_to_archive(path_to_archive_) { - archive = archive_read_new(); - archive_read_support_filter_all(archive); - archive_read_support_format_all(archive); - if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) - { - throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open {} archive: {}", ArchiveInfo::name, quoteString(path_to_archive)); - } - entry = archive_entry_new(); + current_archive = open(path_to_archive); + current_entry = archive_entry_new(); + } + + Handle(const Handle &) = delete; + Handle(Handle && other) noexcept + : current_archive(other.current_archive) + , current_entry(other.current_entry) + { + other.current_archive = nullptr; + other.current_entry = nullptr; } ~Handle() { - archive_read_close(archive); - archive_read_free(archive); + if (current_archive) + { + archive_read_close(current_archive); + archive_read_free(current_archive); + } } bool locateFile(const String & filename) { - while (archive_read_next_header(archive, &entry) == ARCHIVE_OK) + resetFileInfo(); + int err = ARCHIVE_OK; + while (true) { - if (archive_entry_pathname(entry) == filename) + err = archive_read_next_header(current_archive, ¤t_entry); + + if (err == ARCHIVE_RETRY) + continue; + + if (err != ARCHIVE_OK) + break; + + if (archive_entry_pathname(current_entry) == filename) return true; } + + checkError(err); return false; } - struct archive * archive; - struct archive_entry * entry; + bool nextFile() + { + resetFileInfo(); + int err = ARCHIVE_OK; + do + { + err = archive_read_next_header(current_archive, ¤t_entry); + } while (err == ARCHIVE_RETRY); + checkError(err); + return err == ARCHIVE_OK; + } + + static struct archive * open(const String & path_to_archive) + { + auto * archive = archive_read_new(); + archive_read_support_filter_all(archive); + archive_read_support_format_all(archive); + if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open {} archive: {}", ArchiveInfo::name, quoteString(path_to_archive)); + + return archive; + } + + std::vector getAllFiles() + { + auto * archive = open(path_to_archive); + auto * entry = archive_entry_new(); + + std::vector files; + int error = archive_read_next_header(archive, &entry); + while (error == ARCHIVE_OK || error == ARCHIVE_RETRY) + { + files.push_back(archive_entry_pathname(entry)); + error = archive_read_next_header(archive, &entry); + } + + archive_read_close(archive); + archive_read_free(archive); + + checkError(error); + return files; + } + + void checkError(int error) + { + if (error == ARCHIVE_FATAL) + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Failed to read archive while fetching all files: {}", archive_error_string(current_archive)); + } + + void resetFileInfo() + { + file_name.reset(); + file_info.reset(); + } + + const String & getFileName() const + { + if (!file_name) + file_name.emplace(archive_entry_pathname(current_entry)); + + return *file_name; + } + + const FileInfo & getFileInfo() const + { + if (!file_info) + { + file_info.emplace(); + file_info->uncompressed_size = archive_entry_size(current_entry); + file_info->compressed_size = archive_entry_size(current_entry); + file_info->is_encrypted = false; + } + + return *file_info; + } + + struct archive * current_archive; + struct archive_entry * current_entry; private: const String path_to_archive; + mutable std::optional file_name; + mutable std::optional file_info; +}; + +template +class LibArchiveReader::FileEnumeratorImpl : public FileEnumerator +{ +public: + explicit FileEnumeratorImpl(Handle handle_) : handle(std::move(handle_)) {} + + const String & getFileName() const override { return handle.getFileName(); } + const FileInfo & getFileInfo() const override { return handle.getFileInfo(); } + bool nextFile() override { return handle.nextFile(); } + + /// Releases owned handle to pass it to a read buffer. + Handle releaseHandle() && { return std::move(handle); } +private: + Handle handle; }; template class LibArchiveReader::ReadBufferFromLibArchive : public ReadBufferFromFileBase { public: - explicit ReadBufferFromLibArchive(const String & path_to_archive_, const String & filename_) + explicit ReadBufferFromLibArchive(Handle handle_, std::string path_to_archive_) : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0) - , handle(path_to_archive_) - , path_to_archive(path_to_archive_) - , filename(filename_) + , handle(std::move(handle_)) + , path_to_archive(std::move(path_to_archive_)) + {} + + off_t seek(off_t /* off */, int /* whence */) override { - handle.locateFile(filename_); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Seek is not supported when reading from archive"); } - off_t seek(off_t off, int whence) override + off_t getPosition() override { - off_t current_pos = getPosition(); - off_t new_pos; - if (whence == SEEK_SET) - new_pos = off; - else if (whence == SEEK_CUR) - new_pos = off + current_pos; - else - throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Only SEEK_SET and SEEK_CUR seek modes allowed."); - - if (new_pos == current_pos) - return current_pos; /// The position is the same. - - if (new_pos < 0) - throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); - - off_t working_buffer_start_pos = current_pos - offset(); - off_t working_buffer_end_pos = current_pos + available(); - - if ((working_buffer_start_pos <= new_pos) && (new_pos <= working_buffer_end_pos)) - { - /// The new position is still inside the buffer. - position() += new_pos - current_pos; - return new_pos; - } - - /// Check that the new position is now beyond the end of the file. - if (new_pos > archive_entry_size(handle.entry)) - throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bound"); - - ignore(new_pos - current_pos); - return new_pos; + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); } - off_t getPosition() override { return archive_entry_size(handle.entry) - available(); } - - String getFileName() const override { return filename; } + String getFileName() const override { return handle.getFileName(); } + Handle releaseHandle() && + { + return std::move(handle); + } private: bool nextImpl() override { - auto bytes_read = archive_read_data(handle.archive, internal_buffer.begin(), static_cast(internal_buffer.size())); + auto bytes_read = archive_read_data(handle.current_archive, internal_buffer.begin(), static_cast(internal_buffer.size())); if (bytes_read < 0) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to read file {} from {}: {}", filename, path_to_archive, archive_error_string(handle.archive)); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to read file {} from {}: {}", handle.getFileName(), path_to_archive, archive_error_string(handle.current_archive)); if (!bytes_read) return false; + total_bytes_read += bytes; + working_buffer = internal_buffer; working_buffer.resize(bytes_read); return true; @@ -130,23 +219,22 @@ private: Handle handle; const String path_to_archive; - const String filename; + size_t total_bytes_read = 0; }; template LibArchiveReader::LibArchiveReader(const String & path_to_archive_) : path_to_archive(path_to_archive_) -{ -} - -template -LibArchiveReader::LibArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_) - : path_to_archive(path_to_archive_), archive_read_function(archive_read_function_) -{ -} +{} template LibArchiveReader::~LibArchiveReader() = default; +template +const std::string & LibArchiveReader::getPath() const +{ + return path_to_archive; +} + template bool LibArchiveReader::fileExists(const String & filename) { @@ -158,20 +246,18 @@ template LibArchiveReader::FileInfo LibArchiveReader::getFileInfo(const String & filename) { Handle handle(path_to_archive); - handle.locateFile(filename); - FileInfo info; - info.uncompressed_size = archive_entry_size(handle.entry); - info.compressed_size = archive_entry_size(handle.entry); - info.is_encrypted = false; - - return info; + return handle.getFileInfo(); } template std::unique_ptr::FileEnumerator> LibArchiveReader::firstFile() { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implemented for {} archives", ArchiveInfo::name); + Handle handle(path_to_archive); + if (!handle.nextFile()) + return nullptr; + + return std::make_unique(std::move(handle)); } template @@ -179,24 +265,38 @@ std::unique_ptr LibArchiveReader::readFile( { Handle handle(path_to_archive); handle.locateFile(filename); - - return std::make_unique(path_to_archive, filename); + return std::make_unique(std::move(handle), path_to_archive); } template -std::unique_ptr LibArchiveReader::readFile(std::unique_ptr /*enumerator*/) +std::unique_ptr LibArchiveReader::readFile(std::unique_ptr enumerator) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implemented for {} archives", ArchiveInfo::name); + if (!dynamic_cast(enumerator.get())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong enumerator passed to readFile()"); + auto enumerator_impl = std::unique_ptr(static_cast(enumerator.release())); + auto handle = std::move(*enumerator_impl).releaseHandle(); + return std::make_unique(std::move(handle), path_to_archive); +} + +template std::unique_ptr::FileEnumerator> +LibArchiveReader::nextFile(std::unique_ptr read_buffer) +{ + if (!dynamic_cast(read_buffer.get())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong ReadBuffer passed to nextFile()"); + auto read_buffer_from_libarchive = std::unique_ptr(static_cast(read_buffer.release())); + auto handle = std::move(*read_buffer_from_libarchive).releaseHandle(); + if (!handle.nextFile()) + return nullptr; + return std::make_unique(std::move(handle)); } template -std::unique_ptr::FileEnumerator> -LibArchiveReader::nextFile(std::unique_ptr /*read_buffer*/) +std::vector LibArchiveReader::getAllFiles() { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Iterating files not implemented for {} archives", ArchiveInfo::name); + Handle handle(path_to_archive); + return handle.getAllFiles(); } - template void LibArchiveReader::setPassword(const String & /*password_*/) { diff --git a/src/IO/Archives/LibArchiveReader.h b/src/IO/Archives/LibArchiveReader.h index 754df21c3ff..596010c7fbd 100644 --- a/src/IO/Archives/LibArchiveReader.h +++ b/src/IO/Archives/LibArchiveReader.h @@ -23,12 +23,10 @@ public: /// Constructs an archive's reader that will read from a file in the local filesystem. explicit LibArchiveReader(const String & path_to_archive_); - /// Constructs an archive's reader that will read by making a read buffer by using - /// a specified function. - LibArchiveReader(const String & path_to_archive_, const ReadArchiveFunction & archive_read_function_); - ~LibArchiveReader() override; + const std::string & getPath() const override; + /// Returns true if there is a specified file in the archive. bool fileExists(const String & filename) override; @@ -47,15 +45,17 @@ public: std::unique_ptr readFile(std::unique_ptr enumerator) override; std::unique_ptr nextFile(std::unique_ptr read_buffer) override; + std::vector getAllFiles() override; + /// Sets password used to decrypt the contents of the files in the archive. void setPassword(const String & password_) override; private: class ReadBufferFromLibArchive; class Handle; + class FileEnumeratorImpl; const String path_to_archive; - const ReadArchiveFunction archive_read_function; }; struct TarArchiveInfo { static constexpr std::string_view name = "tar"; }; diff --git a/src/IO/Archives/ZipArchiveReader.cpp b/src/IO/Archives/ZipArchiveReader.cpp index 206c2c45ee5..bcb99553eae 100644 --- a/src/IO/Archives/ZipArchiveReader.cpp +++ b/src/IO/Archives/ZipArchiveReader.cpp @@ -1,4 +1,5 @@ #include +#include "mz_compat.h" #if USE_MINIZIP #include @@ -131,6 +132,26 @@ public: return *file_info; } + std::vector getAllFiles() + { + std::vector files; + resetFileInfo(); + int err = unzGoToFirstFile(raw_handle); + if (err == UNZ_END_OF_LIST_OF_FILE) + return files; + + do + { + checkResult(err); + resetFileInfo(); + retrieveFileInfo(); + files.push_back(*file_name); + err = unzGoToNextFile(raw_handle); + } while (err != UNZ_END_OF_LIST_OF_FILE); + + return files; + } + void closeFile() { int err = unzCloseCurrentFile(raw_handle); @@ -459,6 +480,11 @@ ZipArchiveReader::~ZipArchiveReader() } } +const std::string & ZipArchiveReader::getPath() const +{ + return path_to_archive; +} + bool ZipArchiveReader::fileExists(const String & filename) { return acquireHandle().tryLocateFile(filename); @@ -506,6 +532,12 @@ std::unique_ptr ZipArchiveReader::nextFile(std return std::make_unique(std::move(handle)); } +std::vector ZipArchiveReader::getAllFiles() +{ + auto handle = acquireHandle(); + return handle.getAllFiles(); +} + void ZipArchiveReader::setPassword(const String & password_) { std::lock_guard lock{mutex}; diff --git a/src/IO/Archives/ZipArchiveReader.h b/src/IO/Archives/ZipArchiveReader.h index 9d0da28b080..164518b1a37 100644 --- a/src/IO/Archives/ZipArchiveReader.h +++ b/src/IO/Archives/ZipArchiveReader.h @@ -27,6 +27,8 @@ public: ~ZipArchiveReader() override; + const std::string & getPath() const override; + /// Returns true if there is a specified file in the archive. bool fileExists(const String & filename) override; @@ -45,6 +47,8 @@ public: std::unique_ptr readFile(std::unique_ptr enumerator) override; std::unique_ptr nextFile(std::unique_ptr read_buffer) override; + std::vector getAllFiles() override; + /// Sets password used to decrypt the contents of the files in the archive. void setPassword(const String & password_) override; diff --git a/src/IO/Archives/createArchiveReader.cpp b/src/IO/Archives/createArchiveReader.cpp index 7be0a57f2fb..37743da7107 100644 --- a/src/IO/Archives/createArchiveReader.cpp +++ b/src/IO/Archives/createArchiveReader.cpp @@ -35,7 +35,7 @@ std::shared_ptr createArchiveReader( else if (path_to_archive.ends_with(".tar") || path_to_archive.ends_with("tar.gz")) { #if USE_LIBARCHIVE - return std::make_shared(path_to_archive, archive_read_function); + return std::make_shared(path_to_archive); #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "libarchive library is disabled"); #endif @@ -43,7 +43,7 @@ std::shared_ptr createArchiveReader( else if (path_to_archive.ends_with(".7z")) { #if USE_LIBARCHIVE - return std::make_shared(path_to_archive, archive_read_function); + return std::make_shared(path_to_archive); #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "libarchive library is disabled"); #endif diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index d59c1b4a1b4..37998f37c3f 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -387,8 +387,7 @@ std::unique_ptr createReadBuffer( if (!path_to_archive.empty()) { auto reader = createArchiveReader(path_to_archive); - std::unique_ptr in = reader->readFile(current_path); - return in; + return reader->readFile(current_path); } if (use_table_fd) @@ -688,7 +687,9 @@ public: struct FilesInfo { std::vector files; - std::vector paths_to_archive; + + std::vector archives; + std::vector> files_in_archive; std::atomic next_file_to_read = 0; std::atomic next_archive_to_read = 0; @@ -822,17 +823,31 @@ public: { if (!storage->use_table_fd) { - size_t current_file = 0, current_archive = 0; - if (!files_info->paths_to_archive.empty()) + size_t current_file = 0, current_file_in_archive = 0; + if (!files_info->files_in_archive.empty()) { - if (files_info->files.size() != 1) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Can only read a single file from archive"); - - current_archive = files_info->next_archive_to_read.fetch_add(1); - if (current_archive >= files_info->paths_to_archive.size()) + current_file_in_archive = files_info->next_archive_to_read.fetch_add(1); + if (current_file_in_archive >= files_info->files_in_archive.size()) return {}; - current_path = files_info->files[current_file]; - current_archive_path = files_info->paths_to_archive[current_archive]; + + const auto & [archive_index, filename] = files_info->files_in_archive[current_file_in_archive]; + const auto & archive = files_info->archives[archive_index]; + current_path = filename; + + if (!archive_reader || archive_reader->getPath() != archive) + { + archive_reader = createArchiveReader(archive); + file_enumerator = archive_reader->firstFile(); + } + + if (file_enumerator == nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to find a file in archive {}", archive); + + while (file_enumerator->getFileName() != filename) + { + if (!file_enumerator->nextFile()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected file {} is missing from archive {}", filename, archive); + } } else { @@ -855,25 +870,23 @@ public: if (!read_buf) { struct stat file_stat; - if (files_info->paths_to_archive.empty()) + if (archive_reader == nullptr) + { file_stat = getFileStat(current_path, storage->use_table_fd, storage->table_fd, storage->getName()); - else - file_stat = getFileStat(current_archive_path, storage->use_table_fd, storage->table_fd, storage->getName()); - if (context->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) - continue; + if (context->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0) + continue; + } - if (files_info->paths_to_archive.empty()) + if (archive_reader == nullptr) + { read_buf = createReadBuffer(current_path, file_stat, storage->use_table_fd, storage->table_fd, storage->compression_method, context); + } else - read_buf = createReadBuffer( - current_path, - file_stat, - storage->use_table_fd, - storage->table_fd, - storage->compression_method, - context, - current_archive_path); + { + chassert(file_enumerator); + read_buf = archive_reader->readFile(std::move(file_enumerator)); + } } const Settings & settings = context->getSettingsRef(); @@ -933,7 +946,11 @@ public: reader.reset(); pipeline.reset(); input_format.reset(); - read_buf.reset(); + + if (archive_reader != nullptr) + file_enumerator = archive_reader->nextFile(std::move(read_buf)); + else + read_buf.reset(); } return {}; @@ -945,13 +962,15 @@ private: StorageSnapshotPtr storage_snapshot; FilesInfoPtr files_info; String current_path; - String current_archive_path; Block sample_block; std::unique_ptr read_buf; InputFormatPtr input_format; std::unique_ptr pipeline; std::unique_ptr reader; + std::shared_ptr archive_reader; + std::unique_ptr file_enumerator = nullptr; + ColumnsDescription columns_description; Block block_for_format; @@ -979,18 +998,34 @@ Pipe StorageFile::read( } else { - if (paths.size() == 1 && paths_to_archive.empty() && !fs::exists(paths[0])) + const auto & p = paths_to_archive.empty() ? paths : paths_to_archive; + if (p.size() == 1 && !fs::exists(p[0])) { if (context->getSettingsRef().engine_file_empty_if_not_exists) return Pipe(std::make_shared(storage_snapshot->getSampleBlockForColumns(column_names))); else - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File {} doesn't exist", paths[0]); + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File {} doesn't exist", p[0]); + } + } + + std::vector> files_in_archive; + + if (!paths_to_archive.empty()) + { + for (size_t i = 0; i < paths_to_archive.size(); ++i) + { + const auto & path_to_archive = paths_to_archive[i]; + auto archive_reader = createArchiveReader(path_to_archive); + auto files = archive_reader->getAllFiles(); + for (auto & file : files) + files_in_archive.push_back({i, std::move(file)}); } } auto files_info = std::make_shared(); files_info->files = paths; - files_info->paths_to_archive = paths_to_archive; + files_info->archives = paths_to_archive; + files_info->files_in_archive = std::move(files_in_archive); files_info->total_bytes_to_read = total_bytes_to_read; for (const auto & column : column_names) @@ -1004,8 +1039,10 @@ Pipe StorageFile::read( auto this_ptr = std::static_pointer_cast(shared_from_this()); size_t num_streams = max_num_streams; - if (max_num_streams > paths.size()) - num_streams = paths.size(); + + auto files_to_read = std::max(files_info->files_in_archive.size(), paths.size()); + if (max_num_streams > files_to_read) + num_streams = files_to_read; Pipes pipes; pipes.reserve(num_streams); From 8646eefc9cfeedc840c14bc7571c0c6490679a4e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 17:05:24 +0300 Subject: [PATCH 1430/2047] Update docker_test.py --- tests/ci/docker_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 8542de412d2..bdbee92d2d3 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -43,7 +43,7 @@ class TestDockerImageCheck(unittest.TestCase): di.DockerImage( "docker/test/sqltest", "clickhouse/sqltest", - True, + False, "clickhouse/test-base", # type: ignore ), di.DockerImage( From 09cb5cbe8704f158b743b8c355e87aaf512b88e0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Aug 2023 14:30:41 +0000 Subject: [PATCH 1431/2047] Try to continue clickhouse process in stress test after terminating gdb. --- tests/ci/stress.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index 6d17384c63f..60ddeb85d35 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -125,6 +125,8 @@ def prepare_for_hung_check(drop_databases): # However, it obstruct checking for hung queries. logging.info("Will terminate gdb (if any)") call_with_retry("kill -TERM $(pidof gdb)") + # Sometimes there is a message `Child process was stopped by signal 19` in logs after stopping gdb + call_with_retry("kill -CONT $(lsof -ti:9000)") # ThreadFuzzer significantly slows down server and causes false-positive hung check failures call_with_retry("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'") From 07790ab485012f844a7efafa1729ec901df79c0b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 17:40:55 +0300 Subject: [PATCH 1432/2047] Update tests/queries/0_stateless/02834_client_yaml_configs.sh MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index dbb40d33e0a..d1c5a40c04f 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -11,11 +11,11 @@ echo "max_block_size: 31337" > clickhouse-client.yaml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.yaml -echo "max_block_size: 31337" > clickhouse-client.yml +echo "max_block_size: 31338" > clickhouse-client.yml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.yml -echo "31337" > clickhouse-client.xml +echo "31339" > clickhouse-client.xml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.xml From 93d766ba74f22f7de2dbbc3f196be06af91614d3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 17:41:00 +0300 Subject: [PATCH 1433/2047] Update tests/queries/0_stateless/02834_client_yaml_configs.reference MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- tests/queries/0_stateless/02834_client_yaml_configs.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.reference b/tests/queries/0_stateless/02834_client_yaml_configs.reference index 302360f2570..b2eddb19e52 100644 --- a/tests/queries/0_stateless/02834_client_yaml_configs.reference +++ b/tests/queries/0_stateless/02834_client_yaml_configs.reference @@ -1,3 +1,3 @@ 31337 -31337 -31337 +31338 +31339 From 4e849a8abac2baec0156c9d306607422b31a09c7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 1 Aug 2023 16:44:13 +0200 Subject: [PATCH 1434/2047] fix ci for #52091 --- .../Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp index 7205ac299a9..774c07ef1d4 100644 --- a/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp +++ b/src/Analyzer/Passes/OptimizeDateOrDateTimeConverterWithPreimagePass.cpp @@ -48,7 +48,7 @@ public: return true; } - void visitImpl(QueryTreeNodePtr & node) const + void enterImpl(QueryTreeNodePtr & node) const { const static std::unordered_map swap_relations = { {"equals", "equals"}, From 262d4c6316f1b6ace0aa99a251486e74a78e2e88 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 1 Aug 2023 16:44:46 +0200 Subject: [PATCH 1435/2047] Add try/catch in destructor --- src/Client/LocalConnection.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 5caa2a57c7e..2b28d1ee9d9 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -37,7 +37,16 @@ LocalConnection::~LocalConnection() { /// Last query may not have been finished or cancelled due to exception on client side. if (state && !state->is_finished && !state->is_cancelled) - sendCancel(); + { + try + { + LocalConnection::sendCancel(); + } + catch (...) + { + /// Just ignore any exception. + } + } state.reset(); } From ad2489be8578c24eaa3b2dfb092dde648061e09c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 16:45:10 +0200 Subject: [PATCH 1436/2047] Debug tests --- tests/queries/0_stateless/02724_database_s3.sh | 4 ++-- tests/queries/0_stateless/02725_database_hdfs.sh | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index ae3b048938e..d7eb5b101d2 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -46,12 +46,12 @@ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = S3; USE test3; SELECT * FROM \"http://localhost:11111/test/a.myext\" -""" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" +""" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ USE test3; SELECT * FROM \"abacaba\" -""" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" +""" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" # Cleanup ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index 6b69051de3b..aa1fb80ddca 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -45,10 +45,10 @@ USE test4; SELECT * FROM \"abacaba/file.tsv\" """ 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK1" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' | grep -oF "FILE_DOESNT_EXIST" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" # Cleanup From 79d49a006c2f179ea1aa4044bc1228eb6b687358 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Aug 2023 17:06:23 +0200 Subject: [PATCH 1437/2047] Fix the PR body check for `Reverts #number` --- tests/ci/cancel_and_rerun_workflow_lambda/app.py | 4 +++- tests/ci/lambda_shared_package/lambda_shared/pr.py | 8 ++------ tests/ci/run_check.py | 2 +- 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index 250655ddeb2..8282fb7768a 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -323,7 +323,9 @@ def main(event): if action == "edited": print("PR is edited, check if the body is correct") - error, category = check_pr_description(pull_request["body"]) + error, _ = check_pr_description( + pull_request["body"], pull_request["base"]["repo"]["full_name"] + ) if error: print( f"The PR's body is wrong, is going to comment it. The error is: {error}" diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index ef47eacc082..82fc53b9356 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -101,7 +101,7 @@ LABELS = { CATEGORY_TO_LABEL = {c: lb for lb, categories in LABELS.items() for c in categories} -def check_pr_description(pr_body: str) -> Tuple[str, str]: +def check_pr_description(pr_body: str, repo_name: str) -> Tuple[str, str]: """The function checks the body to being properly formatted according to .github/PULL_REQUEST_TEMPLATE.md, if the first returned string is not empty, then there is an error.""" @@ -109,11 +109,7 @@ def check_pr_description(pr_body: str) -> Tuple[str, str]: lines = [re.sub(r"\s+", " ", line) for line in lines] # Check if body contains "Reverts ClickHouse/ClickHouse#36337" - if [ - True - for line in lines - if re.match(r"\AReverts {GITHUB_REPOSITORY}#[\d]+\Z", line) - ]: + if [True for line in lines if re.match(rf"\AReverts {repo_name}#[\d]+\Z", line)]: return "", LABELS["pr-not-for-changelog"][0] category = "" diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 330a1309016..4f022b6c0a5 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -108,7 +108,7 @@ def main(): gh = Github(get_best_robot_token(), per_page=100) commit = get_commit(gh, pr_info.sha) - description_error, category = check_pr_description(pr_info.body) + description_error, category = check_pr_description(pr_info.body, GITHUB_REPOSITORY) pr_labels_to_add = [] pr_labels_to_remove = [] if ( From b79cd49ccf11f0d3fc74c495ea8cae6809862ec6 Mon Sep 17 00:00:00 2001 From: Alexander Zaitsev Date: Tue, 1 Aug 2023 17:27:36 +0200 Subject: [PATCH 1438/2047] doc: add PGO article --- .../profile-guided-optimization.md | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 docs/en/operations/optimizing-performance/profile-guided-optimization.md diff --git a/docs/en/operations/optimizing-performance/profile-guided-optimization.md b/docs/en/operations/optimizing-performance/profile-guided-optimization.md new file mode 100644 index 00000000000..cda21e3c604 --- /dev/null +++ b/docs/en/operations/optimizing-performance/profile-guided-optimization.md @@ -0,0 +1,26 @@ +--- +slug: /en/operations/optimizing-performance/profile-guided-optimization +sidebar_position: 54 +sidebar_label: Profile Guided Optimization (PGO) +--- +import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md'; + +# Profile Guided Optimization + +Profile-Guided Optimization (PGO) is a compiler optimization technique where a program is optimized based on the runtime profile. + +According to the tests, PGO helps with achieving better performance for ClickHouse. According to the tests, we see improvements up to 15% in QPS on the ClickBench test suite. The more detailed results are available [here](https://pastebin.com/xbue3HMU). The performance benefits depend on your typical workload - you can get better or worse results. + +More information about PGO in ClickHouse you can read in the corresponding GitHub [issue](https://github.com/ClickHouse/ClickHouse/issues/44567). + +## How to build ClickHouse with PGO? + +There are two major kinds of PGO: [Instrumentation](https://clang.llvm.org/docs/UsersManual.html#using-sampling-profilers) and [Sampling](https://clang.llvm.org/docs/UsersManual.html#using-sampling-profilers) (also known as AutoFDO). In this guide is described the Instrumentation PGO with ClickHouse. + +1. Build ClickHouse in Instrumented mode. In Clang it can be done via passing `-fprofile-instr-generate` option to `CXXFLAGS`. +2. Run instrumented ClickHouse on a sample workload. Here you need to use your usual workload. One of the approaches could be using [ClickBench](https://github.com/ClickHouse/ClickBench) as a sample workload. ClickHouse in the instrumentation mode could work slowly so be ready for that and do not run instrumented ClickHouse in performance-critical environments. +3. Recompile ClickHouse once again with `-fprofile-instr-use` compiler flags and profiles that are collected from the previous step. + +A more detailed guide on how to apply PGO is in the Clang [documentation](https://clang.llvm.org/docs/UsersManual.html#profile-guided-optimization). + +If you are going to collect a sample workload directly from a production environment, we recommend trying to use Sampling PGO. From a330a7ac38cb733ee386d8bd8a5a77e763b1b874 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Aug 2023 15:47:35 +0200 Subject: [PATCH 1439/2047] PyYAML>=5.4 conflicts with cython 3 --- docker/test/integration/runner/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 8e95d94b6dc..064538a0448 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -95,6 +95,7 @@ RUN python3 -m pip install --no-cache-dir \ pytest-timeout \ pytest-xdist \ pytz \ + pyyaml==5.3.1 \ redis \ requests-kerberos \ tzlocal==2.1 \ From 6b29cccf3e8c663cac47a7db076033bf7435e8bb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Aug 2023 15:50:02 +0200 Subject: [PATCH 1440/2047] Rename docker_images.json to docker_images_for_tests.json --- tests/ci/docker_test.py | 2 +- .../tests/{docker_images.json => docker_images_for_tests.json} | 0 2 files changed, 1 insertion(+), 1 deletion(-) rename tests/ci/tests/{docker_images.json => docker_images_for_tests.json} (100%) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index c679ab984ee..40c51702868 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -18,7 +18,7 @@ import docker_server as ds class TestDockerImageCheck(unittest.TestCase): docker_images_path = os.path.join( - os.path.dirname(__file__), "tests/docker_images.json" + os.path.dirname(__file__), "tests/docker_images_for_tests.json" ) def test_get_changed_docker_images(self): diff --git a/tests/ci/tests/docker_images.json b/tests/ci/tests/docker_images_for_tests.json similarity index 100% rename from tests/ci/tests/docker_images.json rename to tests/ci/tests/docker_images_for_tests.json From eb93d34e8db6a6ea3cffff91ed1583387fcac106 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Aug 2023 15:39:56 +0000 Subject: [PATCH 1441/2047] Disable more tests for debug. --- tests/queries/0_stateless/00900_long_parquet_load.sh | 2 +- tests/queries/0_stateless/02352_rwlock.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00900_long_parquet_load.sh b/tests/queries/0_stateless/00900_long_parquet_load.sh index 8142c5b5810..60317e1671c 100755 --- a/tests/queries/0_stateless/00900_long_parquet_load.sh +++ b/tests/queries/0_stateless/00900_long_parquet_load.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest +# Tags: long, no-fasttest, no-debug # # Load all possible .parquet files found in submodules. diff --git a/tests/queries/0_stateless/02352_rwlock.sh b/tests/queries/0_stateless/02352_rwlock.sh index 7505a03a382..22453be5c0b 100755 --- a/tests/queries/0_stateless/02352_rwlock.sh +++ b/tests/queries/0_stateless/02352_rwlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-parallel, no-fasttest, no-debug # Tag no-parallel -- to avoid running it in parallel, this will avoid possible issues due to high pressure # Test that ensures that WRITE lock failure notifies READ. From 107759aec834bf97c95bee1bce3e2c7852c7ce1b Mon Sep 17 00:00:00 2001 From: Alexander Zaitsev Date: Tue, 1 Aug 2023 18:00:53 +0200 Subject: [PATCH 1442/2047] doc: add ClickBench and AutoFDO to the aspell ignore list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 013ec546119..fe47cd83a41 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -41,6 +41,7 @@ AsynchronousMetricsUpdateInterval AsynchronousReadWait Authenticator Authenticators +AutoFDO AutoML Autocompletion AvroConfluent @@ -146,6 +147,7 @@ ChannelID Cidr Ciphertext CityHash +ClickBench ClickCat ClickHouse ClickHouse's From c485348486298ab66fac9ceab57b09004b81ff96 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Tue, 1 Aug 2023 09:28:27 -0700 Subject: [PATCH 1443/2047] Specify that written_bytes is uncompressed --- docs/en/operations/system-tables/query_log.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index c6f565b8748..7143520835f 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -48,7 +48,7 @@ Columns: - `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Total number of rows read from all tables and table functions participated in query. It includes usual subqueries, subqueries for `IN` and `JOIN`. For distributed queries `read_rows` includes the total number of rows read at all replicas. Each replica sends it’s `read_rows` value, and the server-initiator of the query summarizes all received and local values. The cache volumes do not affect this value. - `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Total number of bytes read from all tables and table functions participated in query. It includes usual subqueries, subqueries for `IN` and `JOIN`. For distributed queries `read_bytes` includes the total number of rows read at all replicas. Each replica sends it’s `read_bytes` value, and the server-initiator of the query summarizes all received and local values. The cache volumes do not affect this value. - `written_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` queries, the number of written rows. For other queries, the column value is 0. -- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` queries, the number of written bytes. For other queries, the column value is 0. +- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — For `INSERT` queries, the number of written bytes (uncompressed). For other queries, the column value is 0. - `result_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Number of rows in a result of the `SELECT` query, or a number of rows in the `INSERT` query. - `result_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — RAM volume in bytes used to store a query result. - `memory_usage` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Memory consumption by the query. From 490554eb4f9a2ee339dde3b7a7dd393f3bf130ff Mon Sep 17 00:00:00 2001 From: zvonand Date: Tue, 1 Aug 2023 18:29:44 +0200 Subject: [PATCH 1444/2047] simplify --- src/Functions/FunctionsConversion.h | 69 +++++++++-------------------- 1 file changed, 22 insertions(+), 47 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index d4dd843a66e..79d17d8ac98 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1039,28 +1039,22 @@ inline void convertFromTime(DataTypeDateTime::FieldType & x, t /** Conversion of strings to numbers, dates, datetimes: through parsing. */ -template -void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +template +void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool precise_float_parsing) { if constexpr (std::is_floating_point_v) { - if constexpr (precise_float_parsing) + if (precise_float_parsing) readFloatTextPrecise(x, rb); else readFloatTextFast(x, rb); } else - parseType(x, rb, time_zone); -} - -template -void parseType(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) -{ - readText(x, rb); + readText(x, rb); } template <> -inline void parseType(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline void parseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) { DayNum tmp(0); readDateText(tmp, rb, *time_zone); @@ -1068,7 +1062,7 @@ inline void parseType(DataTypeDate::FieldType & x, ReadBuffer & rb } template <> -inline void parseType(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline void parseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) { ExtendedDayNum tmp(0); readDateText(tmp, rb, *time_zone); @@ -1078,7 +1072,7 @@ inline void parseType(DataTypeDate32::FieldType & x, ReadBuffer // NOTE: no need of extra overload of DateTime64, since readDateTimeText64 has different signature and that case is explicitly handled in the calling code. template <> -inline void parseType(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline void parseImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) { time_t time = 0; readDateTimeText(time, rb, *time_zone); @@ -1086,7 +1080,7 @@ inline void parseType(DataTypeDateTime::FieldType & x, ReadBuf } template <> -inline void parseType(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline void parseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) { UUID tmp; readUUIDText(tmp, rb); @@ -1094,7 +1088,7 @@ inline void parseType(DataTypeUUID::FieldType & x, ReadBuffer & rb } template <> -inline void parseType(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline void parseImpl(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) { IPv4 tmp; readIPv4Text(tmp, rb); @@ -1102,35 +1096,29 @@ inline void parseType(DataTypeIPv4::FieldType & x, ReadBuffer & rb } template <> -inline void parseType(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline void parseImpl(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) { IPv6 tmp; readIPv6Text(tmp, rb); x = tmp; } -template -bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +template +bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool precise_float_parsing) { if constexpr (std::is_floating_point_v) { - if constexpr (precise_float_parsing) + if (precise_float_parsing) return tryReadFloatTextPrecise(x, rb); else return tryReadFloatTextFast(x, rb); } else /*if constexpr (is_integer_v)*/ - return tryParseTypeImpl(x, rb, time_zone); -} - -template -inline bool tryParseTypeImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) -{ - return tryReadIntText(x, rb); + return tryReadIntText(x, rb); } template <> -inline bool tryParseTypeImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline bool tryParseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) { DayNum tmp(0); if (!tryReadDateText(tmp, rb, *time_zone)) @@ -1140,7 +1128,7 @@ inline bool tryParseTypeImpl(DataTypeDate::FieldType & x, ReadBuff } template <> -inline bool tryParseTypeImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline bool tryParseImpl(DataTypeDate32::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) { ExtendedDayNum tmp(0); if (!tryReadDateText(tmp, rb, *time_zone)) @@ -1150,7 +1138,7 @@ inline bool tryParseTypeImpl(DataTypeDate32::FieldType & x, Read } template <> -inline bool tryParseTypeImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) +inline bool tryParseImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone, bool) { time_t tmp = 0; if (!tryReadDateTimeText(tmp, rb, *time_zone)) @@ -1160,7 +1148,7 @@ inline bool tryParseTypeImpl(DataTypeDateTime::FieldType & x, } template <> -inline bool tryParseTypeImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline bool tryParseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) { UUID tmp; if (!tryReadUUIDText(tmp, rb)) @@ -1171,7 +1159,7 @@ inline bool tryParseTypeImpl(DataTypeUUID::FieldType & x, ReadBuff } template <> -inline bool tryParseTypeImpl(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline bool tryParseImpl(DataTypeIPv4::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) { IPv4 tmp; if (!tryReadIPv4Text(tmp, rb)) @@ -1182,7 +1170,7 @@ inline bool tryParseTypeImpl(DataTypeIPv4::FieldType & x, ReadBuff } template <> -inline bool tryParseTypeImpl(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) +inline bool tryParseImpl(DataTypeIPv6::FieldType & x, ReadBuffer & rb, const DateLUTImpl *, bool) { IPv6 tmp; if (!tryReadIPv6Text(tmp, rb)) @@ -1371,19 +1359,6 @@ struct ConvertThroughParsing precise_float_parsing = query_context->getSettingsRef().precise_float_parsing; } - std::function parseFunction = nullptr; - std::function tryParseFunction = nullptr; - - if constexpr (!(parsing_mode == ConvertFromStringParsingMode::BestEffort || - parsing_mode == ConvertFromStringParsingMode::BestEffortUS || - to_datetime64 || IsDataTypeDecimal)) - { - if constexpr (exception_mode == ConvertFromStringExceptionMode::Throw) - parseFunction = precise_float_parsing ? parseImpl : parseImpl; - else - tryParseFunction = precise_float_parsing ? tryParseImpl : tryParseImpl; - } - for (size_t i = 0; i < size; ++i) { size_t next_offset = std::is_same_v ? (*offsets)[i] : (current_offset + fixed_string_size); @@ -1450,7 +1425,7 @@ struct ConvertThroughParsing } } - parseFunction(vec_to[i], read_buffer, local_time_zone); + parseImpl(vec_to[i], read_buffer, local_time_zone, precise_float_parsing); } while (false); } } @@ -1520,7 +1495,7 @@ struct ConvertThroughParsing } } - parsed = tryParseFunction(vec_to[i], read_buffer, local_time_zone); + parsed = tryParseImpl(vec_to[i], read_buffer, local_time_zone, precise_float_parsing); } while (false); } } From 5cc9600bbf718374ea8ca51d697d03328ac1e921 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Jul 2023 18:10:56 +0200 Subject: [PATCH 1445/2047] Update AllocationTrace.h --- src/Common/AllocationTrace.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/AllocationTrace.h b/src/Common/AllocationTrace.h index 3b0766f8f4d..cb4385f49e0 100644 --- a/src/Common/AllocationTrace.h +++ b/src/Common/AllocationTrace.h @@ -12,7 +12,7 @@ struct AllocationTrace ALWAYS_INLINE void onAlloc(void * ptr, size_t size) const { - if (likely(sample_probability < 0)) + if (likely(sample_probability <= 0)) return; onAllocImpl(ptr, size); @@ -20,7 +20,7 @@ struct AllocationTrace ALWAYS_INLINE void onFree(void * ptr, size_t size) const { - if (likely(sample_probability < 0)) + if (likely(sample_probability <= 0)) return; onFreeImpl(ptr, size); From 7aecec8bc727226bf9be98efa64fbc0f6858f707 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Tue, 1 Aug 2023 13:43:28 -0400 Subject: [PATCH 1446/2047] Update src/Common/OptimizedRegularExpression.cpp --- src/Common/OptimizedRegularExpression.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index bcef49ce00c..dc35ddc14a9 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -266,7 +266,7 @@ const char * analyzeImpl( break; } } - /// (?regex) means non-capturing parentheses group + /// (?:regex) means non-capturing parentheses group if (pos + 2 < end && pos[1] == '?' && pos[2] == ':') { pos += 2; From 17c4abce10c03bafc5316cde3f80a163fa56a944 Mon Sep 17 00:00:00 2001 From: Kenji Noguchi Date: Tue, 1 Aug 2023 10:44:43 -0700 Subject: [PATCH 1447/2047] CVE-2016-2183: disable 3DES --- base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h | 6 +++--- base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h | 2 +- base/poco/NetSSL_OpenSSL/src/Context.cpp | 2 +- .../operations/server-configuration-parameters/settings.md | 2 +- .../operations/server-configuration-parameters/settings.md | 2 +- .../operations/server-configuration-parameters/settings.md | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h index 65917ac9dd4..c19eecf5c73 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h @@ -146,7 +146,7 @@ namespace Net std::string cipherList; /// Specifies the supported ciphers in OpenSSL notation. - /// Defaults to "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH". + /// Defaults to "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH". std::string dhParamsFile; /// Specifies a file containing Diffie-Hellman parameters. @@ -172,7 +172,7 @@ namespace Net VerificationMode verificationMode = VERIFY_RELAXED, int verificationDepth = 9, bool loadDefaultCAs = false, - const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); + const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH"); /// Creates a Context. /// /// * usage specifies whether the context is used by a client or server. @@ -200,7 +200,7 @@ namespace Net VerificationMode verificationMode = VERIFY_RELAXED, int verificationDepth = 9, bool loadDefaultCAs = false, - const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); + const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH"); /// Creates a Context. /// /// * usage specifies whether the context is used by a client or server. diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h index 21a1ed685e5..e4037c87927 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h @@ -76,7 +76,7 @@ namespace Net /// none|relaxed|strict|once /// 1..9 /// true|false - /// ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH + /// ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH /// true|false /// /// KeyFileHandler diff --git a/base/poco/NetSSL_OpenSSL/src/Context.cpp b/base/poco/NetSSL_OpenSSL/src/Context.cpp index ca220c40a33..d0bab902b89 100644 --- a/base/poco/NetSSL_OpenSSL/src/Context.cpp +++ b/base/poco/NetSSL_OpenSSL/src/Context.cpp @@ -41,7 +41,7 @@ Context::Params::Params(): verificationMode(VERIFY_RELAXED), verificationDepth(9), loadDefaultCAs(false), - cipherList("ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH") + cipherList("ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH") { } diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 5187ccce789..a7637082496 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1640,7 +1640,7 @@ Keys for server/client settings: - verificationMode (default: relaxed) – The method for checking the node’s certificates. Details are in the description of the [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: `none`, `relaxed`, `strict`, `once`. - verificationDepth (default: 9) – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. - loadDefaultCAFile (default: true) – Wether built-in CA certificates for OpenSSL will be used. ClickHouse assumes that builtin CA certificates are in the file `/etc/ssl/cert.pem` (resp. the directory `/etc/ssl/certs`) or in file (resp. directory) specified by the environment variable `SSL_CERT_FILE` (resp. `SSL_CERT_DIR`). -- cipherList (default: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`) - Supported OpenSSL encryptions. +- cipherList (default: `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`) - Supported OpenSSL encryptions. - cacheSessions (default: false) – Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. Acceptable values: `true`, `false`. - sessionIdContext (default: `${application.name}`) – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. This parameter is always recommended since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: `${application.name}`. - sessionCacheSize (default: [1024\*20](https://github.com/ClickHouse/boringssl/blob/master/include/openssl/ssl.h#L1978)) – The maximum number of sessions that the server caches. A value of 0 means unlimited sessions. diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 81a696bcfc1..7b026244624 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1106,7 +1106,7 @@ ClickHouse использует потоки из глобального пул - verificationMode - Способ проверки сертификатов узла. Подробности находятся в описании класса [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h). Допустимые значения: `none`, `relaxed`, `strict`, `once`. - verificationDepth - Максимальная длина верификационной цепи. Верификация завершится ошибкой, если длина цепи сертификатов превысит установленное значение. - loadDefaultCAFile - Признак того, что будут использоваться встроенные CA-сертификаты для OpenSSL. Допустимые значения: `true`, `false`. \| -- cipherList - Поддерживаемые OpenSSL-шифры. Например, `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`. +- cipherList - Поддерживаемые OpenSSL-шифры. Например, `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`. - cacheSessions - Включение/выключение кеширования сессии. Использовать обязательно вместе с `sessionIdContext`. Допустимые значения: `true`, `false`. - sessionIdContext - Уникальный набор произвольных символов, которые сервер добавляет к каждому сгенерированному идентификатору. Длина строки не должна превышать `SSL_MAX_SSL_SESSION_ID_LENGTH`. Рекомендуется к использованию всегда, поскольку позволяет избежать проблем как в случае, если сервер кеширует сессию, так и если клиент затребовал кеширование. По умолчанию `${application.name}`. - sessionCacheSize - Максимальное количество сессий, которые кэширует сервер. По умолчанию - 1024\*20. 0 - неограниченное количество сессий. diff --git a/docs/zh/operations/server-configuration-parameters/settings.md b/docs/zh/operations/server-configuration-parameters/settings.md index f6106d8734e..8e2cb389f04 100644 --- a/docs/zh/operations/server-configuration-parameters/settings.md +++ b/docs/zh/operations/server-configuration-parameters/settings.md @@ -455,7 +455,7 @@ SSL客户端/服务器配置。 - verificationMode – The method for checking the node’s certificates. Details are in the description of the [A.背景](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) 同学们 可能的值: `none`, `relaxed`, `strict`, `once`. - verificationDepth – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. - loadDefaultCAFile – Indicates that built-in CA certificates for OpenSSL will be used. Acceptable values: `true`, `false`. \| -- cipherList – Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`. +- cipherList – Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`. - cacheSessions – Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. 可接受的值: `true`, `false`. - sessionIdContext – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. 始终建议使用此参数,因为如果服务器缓存会话,以及客户端请求缓存,它有助于避免出现问题。 默认值: `${application.name}`. - sessionCacheSize – The maximum number of sessions that the server caches. Default value: 1024\*20. 0 – Unlimited sessions. From aaba48f0df13aa9c314d604affc1ac491c4db5dd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Jul 2023 17:19:33 +0000 Subject: [PATCH 1448/2047] Fixing build and style. --- .../AggregateFunctionFlameGraph.cpp | 13 ++++++------- src/Common/MemoryTracker.cpp | 4 ++-- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionFlameGraph.cpp b/src/AggregateFunctions/AggregateFunctionFlameGraph.cpp index e25dfead466..f3d99046036 100644 --- a/src/AggregateFunctions/AggregateFunctionFlameGraph.cpp +++ b/src/AggregateFunctions/AggregateFunctionFlameGraph.cpp @@ -221,8 +221,7 @@ void dumpFlameGraph( std::unordered_map mapping; #if defined(__ELF__) && !defined(OS_FREEBSD) - auto symbol_index_ptr = DB::SymbolIndex::instance(); - const DB::SymbolIndex & symbol_index = *symbol_index_ptr; + const DB::SymbolIndex & symbol_index = DB::SymbolIndex::instance(); #endif for (const auto & trace : traces) @@ -537,10 +536,10 @@ public: void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override { - const auto * trace = typeid_cast(columns[0]); + const auto & trace = assert_cast(*columns[0]); - const auto & trace_offsets = trace->getOffsets(); - const auto & trace_values = typeid_cast(&trace->getData())->getData(); + const auto & trace_offsets = trace.getOffsets(); + const auto & trace_values = assert_cast(trace.getData()).getData(); UInt64 prev_offset = 0; if (row_num) prev_offset = trace_offsets[row_num - 1]; @@ -549,14 +548,14 @@ public: Int64 allocated = 1; if (argument_types.size() >= 2) { - const auto & sizes = typeid_cast(columns[1])->getData(); + const auto & sizes = assert_cast(*columns[1]).getData(); allocated = sizes[row_num]; } UInt64 ptr = 0; if (argument_types.size() >= 3) { - const auto & ptrs = typeid_cast(columns[2])->getData(); + const auto & ptrs = assert_cast(*columns[2]).getData(); ptr = ptrs[row_num]; } diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index cdc2f15385f..25d80d96956 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -1,7 +1,7 @@ #include "MemoryTracker.h" #include -#include +#include #include #include #include @@ -85,7 +85,7 @@ inline std::string_view toDescription(OvercommitResult result) bool shouldTrackAllocation(DB::Float64 probability, void * ptr) { - return sipHash64(uintptr_t(ptr)) < std::numeric_limits::max() * probability; + return intHash64(uintptr_t(ptr)) < std::numeric_limits::max() * probability; } } From 6db7eaf4d8bbb57ab4911748cfbb0dbcab499d0d Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 1 Aug 2023 19:14:11 +0000 Subject: [PATCH 1449/2047] fixed ast --- .../System/attachInformationSchemaTables.cpp | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 544ab75ac29..9a909110889 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -18,7 +18,7 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE); if (database.getEngineName() != "Memory") return; - bool is_uppercase = database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE; + // bool is_uppercase = database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE; String metadata_resource_name = view_name + ".sql"; auto attach_query = getResource(metadata_resource_name); @@ -34,14 +34,20 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d assert(view_name == ast_create.getTable()); ast_create.attach = false; ast_create.setDatabase(database.getDatabaseName()); - if (is_uppercase) - ast_create.setTable(Poco::toUpper(view_name)); + // if (is_uppercase) + // ast_create.setTable(Poco::toUpper(view_name)); StoragePtr view = createTableFromAST(ast_create, database.getDatabaseName(), database.getTableDataPath(ast_create), context, true).second; - - database.createTable(context, Poco::toUpper(ast_create.getTable()), view, ast); - database.createTable(context, Poco::toLower(ast_create.getTable()), view, ast); + database.createTable(context, ast_create.getTable(), view, ast); + ASTPtr ast_upper = ast_create.clone(); + auto & ast_create_upper = ast_upper->as(); + ast_create.setTable(Poco::toUpper(view_name)); + StoragePtr view_upper = createTableFromAST(ast_create_upper, database.getDatabaseName(), + database.getTableDataPath(ast_create_upper), context, true).second; + + database.createTable(context, ast_create_upper.getTable(), view_upper, ast_upper); + } catch (...) From d6a2c631da8131a18fea79d0501d30a79cd16ffe Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Aug 2023 21:16:22 +0200 Subject: [PATCH 1450/2047] fix assertion in mutations with transactions --- src/Storages/StorageMergeTree.cpp | 17 +++++++++++++---- .../01168_mutations_isolation.reference | 2 ++ .../0_stateless/01168_mutations_isolation.sh | 13 +++++++++++++ 3 files changed, 28 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 32e100edc4d..a279291aef1 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1154,16 +1154,25 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate( } TransactionID first_mutation_tid = mutations_begin_it->second.tid; - MergeTreeTransactionPtr txn = tryGetTransactionForMutation(mutations_begin_it->second, log); - assert(txn || first_mutation_tid.isPrehistoric()); + MergeTreeTransactionPtr txn; - if (txn) + if (!first_mutation_tid.isPrehistoric()) { + /// Mutate visible parts only /// NOTE Do not mutate visible parts in Outdated state, because it does not make sense: /// mutation will fail anyway due to serialization error. - if (!part->version.isVisible(*txn)) + + /// It's possible that both mutation and transaction are already finished, + /// because that part should not be mutated because it was not visible for that transaction. + if (!part->version.isVisible(first_mutation_tid.start_csn, first_mutation_tid)) continue; + + txn = tryGetTransactionForMutation(mutations_begin_it->second, log); + if (!txn) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find transaction {} that has started mutation {} " + "that is going to be applied to part {}", + first_mutation_tid, mutations_begin_it->second.file_name, part->name); } auto commands = std::make_shared(); diff --git a/tests/queries/0_stateless/01168_mutations_isolation.reference b/tests/queries/0_stateless/01168_mutations_isolation.reference index f9ebd1c5f83..44da63385ca 100644 --- a/tests/queries/0_stateless/01168_mutations_isolation.reference +++ b/tests/queries/0_stateless/01168_mutations_isolation.reference @@ -36,3 +36,5 @@ tx14 10 22 all_1_14_2_18 tx14 10 42 all_1_14_2_18 tx14 10 62 all_1_14_2_18 tx14 10 82 all_1_14_2_18 +11 2 all_2_2_0 +11 10 all_1_1_0_3 diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index 5d014e030f1..2b76e5742ac 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -94,3 +94,16 @@ tx 14 "begin transaction" tx 14 "select 10, n, _part from mt order by n" | accept_both_parts $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table mt" + +$CLICKHOUSE_CLIENT -q "create table mt (n int) engine=MergeTree order by tuple()" +$CLICKHOUSE_CLIENT --implicit_transaction=1 -q "insert into mt values (1)" + +tx 15 "begin transaction" +tx 16 "begin transaction" +tx 16 "insert into mt values (2)" +tx 15 "alter table mt update n = 10*n where 1" +tx 15 "commit" +tx 16 "commit" +$CLICKHOUSE_CLIENT --implicit_transaction=1 -q "select 11, n, _part from mt order by n" + +$CLICKHOUSE_CLIENT -q "drop table mt" From 0641dfd7e640bfd08d8533d41cb8ada524dd7b3a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 22:33:04 +0200 Subject: [PATCH 1451/2047] Add missing modification --- tests/ci/ci_config.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 517e40fd2d6..be8364f8874 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -402,6 +402,9 @@ CI_CONFIG = { "Sqllogic test (release)": { "required_build": "package_release", }, + "SQLTest": { + "required_build": "package_release", + }, }, } # type: dict From 625b6b2b1c8dc5c1375ab74ae2bb758ecb74d081 Mon Sep 17 00:00:00 2001 From: Hendrik M Date: Tue, 1 Aug 2023 22:33:42 +0200 Subject: [PATCH 1452/2047] use own parser only for table identifiers --- src/Parsers/ExpressionElementParsers.cpp | 50 ++++++++++++++++------- src/Parsers/ExpressionElementParsers.h | 13 ++++++ src/Parsers/ParserTablesInSelectQuery.cpp | 2 + 3 files changed, 51 insertions(+), 14 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 354c43d3348..52f87e2e048 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -188,20 +188,6 @@ bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ++pos; return true; } - else if (pos->type == TokenType::StringLiteral) - { - ReadBufferFromMemory buf(pos->begin, pos->size()); - String s; - - readQuotedStringWithSQLStyle(s, buf); - - if (s.empty()) /// Identifiers "empty string" are not allowed. - return false; - - node = std::make_shared(s); - ++pos; - return true; - } else if (pos->type == TokenType::BareWord) { node = std::make_shared(String(pos->begin, pos->end)); @@ -257,6 +243,42 @@ bool ParserIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } +bool ParserTableAsStringLiteralIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (pos->type != TokenType::StringLiteral) + return false; + + ReadBufferFromMemory in(pos->begin, pos->size()); + String s; + + try + { + readQuotedStringWithSQLStyle(s, in); + } + catch (const Exception &) + { + expected.add(pos, "string literal"); + return false; + } + + if (in.count() != pos->size()) + { + expected.add(pos, "string literal"); + return false; + } + + if (s.empty()) + { + expected.add(pos, "non-empty string literal"); + return false; + } + + node = std::make_shared(s); + ++pos; + return true; +} + + bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr id_list; diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index f33f2d99f71..2d06b26c6dc 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -34,6 +34,19 @@ protected: }; +/** An identifier for tables written as string literal, for example, 'mytable.avro' + */ +class ParserTableAsStringLiteralIdentifier : public IParserBase +{ +public: + explicit ParserTableAsStringLiteralIdentifier() {} + +protected: + const char * getName() const override { return "string literal table identifier"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + + /** An identifier, possibly containing a dot, for example, x_yz123 or `something special` or Hits.EventTime, * possibly with UUID clause like `db name`.`table name` UUID 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx' */ diff --git a/src/Parsers/ParserTablesInSelectQuery.cpp b/src/Parsers/ParserTablesInSelectQuery.cpp index 08b6f77fafa..b3ae6ca0bb9 100644 --- a/src/Parsers/ParserTablesInSelectQuery.cpp +++ b/src/Parsers/ParserTablesInSelectQuery.cpp @@ -24,6 +24,8 @@ bool ParserTableExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!ParserWithOptionalAlias(std::make_unique(), allow_alias_without_as_keyword).parse(pos, res->subquery, expected) && !ParserWithOptionalAlias(std::make_unique(false, true), allow_alias_without_as_keyword).parse(pos, res->table_function, expected) && !ParserWithOptionalAlias(std::make_unique(true, true), allow_alias_without_as_keyword) + .parse(pos, res->database_and_table_name, expected) + && !ParserWithOptionalAlias(std::make_unique(), allow_alias_without_as_keyword) .parse(pos, res->database_and_table_name, expected)) return false; From ddf55b5575f109f3049ab4c9cdf725df74b32656 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 23:03:47 +0200 Subject: [PATCH 1453/2047] Fix tests --- .../queries/0_stateless/02724_database_s3.reference | 4 ++-- tests/queries/0_stateless/02724_database_s3.sh | 4 ++-- .../0_stateless/02725_database_hdfs.reference | 12 ++++++------ tests/queries/0_stateless/02725_database_hdfs.sh | 12 ++++++------ 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02724_database_s3.reference b/tests/queries/0_stateless/02724_database_s3.reference index 425cca6a077..bf8ea70f1d1 100644 --- a/tests/queries/0_stateless/02724_database_s3.reference +++ b/tests/queries/0_stateless/02724_database_s3.reference @@ -17,5 +17,5 @@ test1 16 17 18 0 0 0 Test 2: check exceptions -OK -OK +BAD_ARGUMENTS +BAD_ARGUMENTS diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index d7eb5b101d2..15e93ff117f 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -46,12 +46,12 @@ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = S3; USE test3; SELECT * FROM \"http://localhost:11111/test/a.myext\" -""" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" +""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ USE test3; SELECT * FROM \"abacaba\" -""" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" +""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" # Cleanup ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ diff --git a/tests/queries/0_stateless/02725_database_hdfs.reference b/tests/queries/0_stateless/02725_database_hdfs.reference index ef8adae2bbc..9f63e757b27 100644 --- a/tests/queries/0_stateless/02725_database_hdfs.reference +++ b/tests/queries/0_stateless/02725_database_hdfs.reference @@ -4,9 +4,9 @@ test1 1 2 3 test2 Test 2: check exceptions -OK0 -OK1 -OK2 -OK3 -OK4 -OK5 +BAD_ARGUMENTS +CANNOT_EXTRACT_TABLE_STRUCTURE +BAD_ARGUMENTS +BAD_ARGUMENTS +CANNOT_EXTRACT_TABLE_STRUCTURE +BAD_ARGUMENTS diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index aa1fb80ddca..c0bd9ab5e57 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -36,19 +36,19 @@ echo "Test 2: check exceptions" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = HDFS('abacaba'); -""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK0" +""" | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test4; CREATE DATABASE test4 ENGINE = HDFS; USE test4; SELECT * FROM \"abacaba/file.tsv\" -""" 2>&1| grep -F "FILE_DOESNT_EXIST" > /dev/null && echo "OK1" +""" | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' #| grep -oF "FILE_DOESNT_EXIST" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" # Cleanup From a1c499d2acc2c46dccbacdeaed758e892de1db9f Mon Sep 17 00:00:00 2001 From: hendrik-m Date: Wed, 2 Aug 2023 00:01:26 +0200 Subject: [PATCH 1454/2047] change file permissions --- .../0_stateless/02816_clickhouse_local_table_name_expressions.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.sh diff --git a/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.sh b/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.sh old mode 100644 new mode 100755 From aec0667f16ad39126306ef7133a8af5722ed687e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 1 Aug 2023 23:39:52 +0000 Subject: [PATCH 1455/2047] better check of version for sparse serialization --- src/Core/ProtocolDefines.h | 2 ++ src/Formats/NativeReader.cpp | 2 +- src/Formats/NativeWriter.cpp | 16 +++++++++++++--- 3 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 1f7f1481aba..9897f314aa8 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -44,6 +44,8 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS 54451 +#define DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION 54454 + #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 #define DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS_IN_INSERT 54456 diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index bfe4aab6e3a..4c25460eb63 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -173,7 +173,7 @@ Block NativeReader::read() setVersionToAggregateFunctions(column.type, true, server_revision); SerializationPtr serialization; - if (server_revision >= DBMS_MIN_REVISION_WITH_SPARSE_SERIALIZATION) + if (server_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) { auto info = column.type->createSerializationInfo({}); diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index e50ff224257..70d5b7914a7 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -132,12 +132,22 @@ size_t NativeWriter::write(const Block & block) /// Serialization. Dynamic, if client supports it. SerializationPtr serialization; - if (client_revision >= DBMS_MIN_REVISION_WITH_SPARSE_SERIALIZATION) + if (client_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) { auto info = column.type->getSerializationInfo(*column.column); - serialization = column.type->getSerialization(*info); + bool has_custom = false; + + if (client_revision >= DBMS_MIN_REVISION_WITH_SPARSE_SERIALIZATION) + { + serialization = column.type->getSerialization(*info); + has_custom = info->hasCustomSerialization(); + } + else + { + serialization = column.type->getDefaultSerialization(); + column.column = recursiveRemoveSparse(column.column); + } - bool has_custom = info->hasCustomSerialization(); writeBinary(static_cast(has_custom), ostr); if (has_custom) info->serialializeKindBinary(ostr); From 7e37f1fabd259892b4fe6360015522213166c038 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 2 Aug 2023 00:32:20 +0000 Subject: [PATCH 1456/2047] fixed test flakiness --- tests/integration/parallel_skip.json | 3 +- .../test.py | 38 ++++++------------- 2 files changed, 12 insertions(+), 29 deletions(-) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 35052ad9a7f..c2b0e7eb61e 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -91,6 +91,5 @@ "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_http_named_session", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_grpc", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp_and_others", - "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp", - "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_end_session" + "test_profile_max_sessions_for_user/test.py:: test_profile_max_sessions_for_user_setting_in_query ] diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 9e6a10e7e15..72addd79ec5 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -8,6 +8,8 @@ import sys import threading from helpers.cluster import ClickHouseCluster, run_and_check +from helpers.test_tools import assert_logs_contain_with_retry + MAX_SESSIONS_FOR_USER = 2 POSTGRES_SERVER_PORT = 5433 @@ -54,7 +56,7 @@ instance = cluster.add_instance( def get_query(name, id): - return f"SElECT '{name}', {id}, sleep(1)" + return f"SElECT '{name}', {id}, number from system.numbers" def grpc_get_url(): @@ -83,22 +85,21 @@ def grpc_query(query_text, channel, session_id_): def threaded_run_test(sessions): + instance.rotate_logs() thread_list = [] for i in range(len(sessions)): thread = ThreadWithException(target=sessions[i], args=(i,)) thread_list.append(thread) thread.start() + if len(sessions) > MAX_SESSIONS_FOR_USER: + assert_logs_contain_with_retry(instance, "overflown session count") + + instance.query(f"KILL QUERY WHERE user='{TEST_USER}' SYNC") + for thread in thread_list: thread.join() - exception_count = 0 - for i in range(len(sessions)): - if thread_list[i].run_exception != None: - exception_count += 1 - - assert exception_count == 1 - @pytest.fixture(scope="module") def started_cluster(): @@ -110,16 +111,11 @@ def started_cluster(): class ThreadWithException(threading.Thread): - run_exception = None - def run(self): try: super().run() except: - self.run_exception = sys.exc_info() - - def join(self): - super().join() + pass def postgres_session(id): @@ -206,17 +202,5 @@ def test_profile_max_sessions_for_user_tcp_and_others(started_cluster): threaded_run_test([tcp_session, postgres_session, http_session]) -def test_profile_max_sessions_for_user_end_session(started_cluster): - for conection_func in [ - tcp_session, - http_session, - grpc_session, - mysql_session, - postgres_session, - ]: - threaded_run_test([conection_func] * MAX_SESSIONS_FOR_USER) - threaded_run_test([conection_func] * MAX_SESSIONS_FOR_USER) - - -def test_profile_max_sessions_for_user_end_session(started_cluster): +def test_profile_max_sessions_for_user_setting_in_query(started_cluster): instance.query_and_get_error("SET max_sessions_for_user = 10") From 288e3f75fb7e55e5c6b5dc2a36ec128c50eda194 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Wed, 2 Aug 2023 00:36:25 +0000 Subject: [PATCH 1457/2047] fix json --- tests/integration/parallel_skip.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index c2b0e7eb61e..dec51396c51 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -91,5 +91,5 @@ "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_http_named_session", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_grpc", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp_and_others", - "test_profile_max_sessions_for_user/test.py:: test_profile_max_sessions_for_user_setting_in_query + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_setting_in_query" ] From dc33564044c2c59680c4a046367178d3f08a2bf5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 2 Aug 2023 06:30:22 +0000 Subject: [PATCH 1458/2047] Disable reconfig test until proper fix --- .../test_keeper_reconfig_replace_leader_in_one_command/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py index e23d0674c12..3a8bf3fc9b1 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -42,7 +42,7 @@ def started_cluster(): def get_fake_zk(node): return ku.get_fake_zk(cluster, node) - +@pytest.mark.skip(reason="test is flaky because changes are not properly waited for") def test_reconfig_replace_leader_in_one_command(started_cluster): """ Remove leader from a cluster of 3 and add a new node to this cluster in a single command From c1752355b48e5da5a0154cd333f1ecbbee94f700 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 2 Aug 2023 06:40:32 +0000 Subject: [PATCH 1459/2047] Automatic style fix --- .../test_keeper_reconfig_replace_leader_in_one_command/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py index 3a8bf3fc9b1..1ec44d8a002 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -42,6 +42,7 @@ def started_cluster(): def get_fake_zk(node): return ku.get_fake_zk(cluster, node) + @pytest.mark.skip(reason="test is flaky because changes are not properly waited for") def test_reconfig_replace_leader_in_one_command(started_cluster): """ From 9423976b7a88e23bd98078760a23cb86bcb179a3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 2 Aug 2023 07:40:59 +0000 Subject: [PATCH 1460/2047] Add support for file globs --- src/IO/Archives/IArchiveReader.h | 4 ++ src/IO/Archives/LibArchiveReader.cpp | 32 ++++++++--- src/IO/Archives/LibArchiveReader.h | 2 + src/IO/Archives/ZipArchiveReader.cpp | 39 ++++++++++++-- src/IO/Archives/ZipArchiveReader.h | 2 + src/Storages/StorageFile.cpp | 81 +++++++++++++++++++++++----- 6 files changed, 139 insertions(+), 21 deletions(-) diff --git a/src/IO/Archives/IArchiveReader.h b/src/IO/Archives/IArchiveReader.h index 0b08a29099c..03e5392e970 100644 --- a/src/IO/Archives/IArchiveReader.h +++ b/src/IO/Archives/IArchiveReader.h @@ -45,16 +45,20 @@ public: /// Starts enumerating files in the archive. virtual std::unique_ptr firstFile() = 0; + using NameFilter = std::function; + /// Starts reading a file from the archive. The function returns a read buffer, /// you can read that buffer to extract uncompressed data from the archive. /// Several read buffers can be used at the same time in parallel. virtual std::unique_ptr readFile(const String & filename) = 0; + virtual std::unique_ptr readFile(NameFilter filter) = 0; /// It's possible to convert a file enumerator to a read buffer and vice versa. virtual std::unique_ptr readFile(std::unique_ptr enumerator) = 0; virtual std::unique_ptr nextFile(std::unique_ptr read_buffer) = 0; virtual std::vector getAllFiles() = 0; + virtual std::vector getAllFiles(NameFilter filter) = 0; /// Sets password used to decrypt files in the archive. virtual void setPassword(const String & /* password */) {} diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index f3657d5908b..0e0d035d98b 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -49,7 +49,12 @@ public: } } - bool locateFile(const String & filename) + bool locateFile(const std::string & filename) + { + return locateFile([&](const std::string & file) { return file == filename; }); + } + + bool locateFile(NameFilter filter) { resetFileInfo(); int err = ARCHIVE_OK; @@ -63,7 +68,7 @@ public: if (err != ARCHIVE_OK) break; - if (archive_entry_pathname(current_entry) == filename) + if (filter(archive_entry_pathname(current_entry))) return true; } @@ -95,7 +100,7 @@ public: return archive; } - std::vector getAllFiles() + std::vector getAllFiles(NameFilter filter) { auto * archive = open(path_to_archive); auto * entry = archive_entry_new(); @@ -104,7 +109,10 @@ public: int error = archive_read_next_header(archive, &entry); while (error == ARCHIVE_OK || error == ARCHIVE_RETRY) { - files.push_back(archive_entry_pathname(entry)); + std::string name = archive_entry_pathname(entry); + if (!filter || filter(name)) + files.push_back(std::move(name)); + error = archive_read_next_header(archive, &entry); } @@ -262,9 +270,15 @@ std::unique_ptr::FileEnumerator> LibArchi template std::unique_ptr LibArchiveReader::readFile(const String & filename) +{ + return readFile([&](const std::string & file) { return file == filename; }); +} + +template +std::unique_ptr LibArchiveReader::readFile(NameFilter filter) { Handle handle(path_to_archive); - handle.locateFile(filename); + handle.locateFile(filter); return std::make_unique(std::move(handle), path_to_archive); } @@ -292,9 +306,15 @@ LibArchiveReader::nextFile(std::unique_ptr read_buffer) template std::vector LibArchiveReader::getAllFiles() +{ + return getAllFiles({}); +} + +template +std::vector LibArchiveReader::getAllFiles(NameFilter filter) { Handle handle(path_to_archive); - return handle.getAllFiles(); + return handle.getAllFiles(filter); } template diff --git a/src/IO/Archives/LibArchiveReader.h b/src/IO/Archives/LibArchiveReader.h index 596010c7fbd..86127fa6953 100644 --- a/src/IO/Archives/LibArchiveReader.h +++ b/src/IO/Archives/LibArchiveReader.h @@ -40,12 +40,14 @@ public: /// you can read that buffer to extract uncompressed data from the archive. /// Several read buffers can be used at the same time in parallel. std::unique_ptr readFile(const String & filename) override; + std::unique_ptr readFile(NameFilter filter) override; /// It's possible to convert a file enumerator to a read buffer and vice versa. std::unique_ptr readFile(std::unique_ptr enumerator) override; std::unique_ptr nextFile(std::unique_ptr read_buffer) override; std::vector getAllFiles() override; + std::vector getAllFiles(NameFilter filter) override; /// Sets password used to decrypt the contents of the files in the archive. void setPassword(const String & password_) override; diff --git a/src/IO/Archives/ZipArchiveReader.cpp b/src/IO/Archives/ZipArchiveReader.cpp index bcb99553eae..84a8001e70e 100644 --- a/src/IO/Archives/ZipArchiveReader.cpp +++ b/src/IO/Archives/ZipArchiveReader.cpp @@ -86,6 +86,26 @@ public: file_name = file_name_; } + void locateFile(NameFilter filter) + { + int err = unzGoToFirstFile(raw_handle); + if (err == UNZ_END_OF_LIST_OF_FILE) + showError("No file was found satisfying the filter"); + + do + { + checkResult(err); + resetFileInfo(); + retrieveFileInfo(); + if (filter(getFileName())) + return; + + err = unzGoToNextFile(raw_handle); + } while (err != UNZ_END_OF_LIST_OF_FILE); + + showError("No file was found satisfying the filter"); + } + bool tryLocateFile(const String & file_name_) { resetFileInfo(); @@ -132,7 +152,7 @@ public: return *file_info; } - std::vector getAllFiles() + std::vector getAllFiles(NameFilter filter) { std::vector files; resetFileInfo(); @@ -145,7 +165,8 @@ public: checkResult(err); resetFileInfo(); retrieveFileInfo(); - files.push_back(*file_name); + if (!filter || filter(getFileName())) + files.push_back(*file_name); err = unzGoToNextFile(raw_handle); } while (err != UNZ_END_OF_LIST_OF_FILE); @@ -512,6 +533,13 @@ std::unique_ptr ZipArchiveReader::readFile(const String return std::make_unique(std::move(handle)); } +std::unique_ptr ZipArchiveReader::readFile(NameFilter filter) +{ + auto handle = acquireHandle(); + handle.locateFile(filter); + return std::make_unique(std::move(handle)); +} + std::unique_ptr ZipArchiveReader::readFile(std::unique_ptr enumerator) { if (!dynamic_cast(enumerator.get())) @@ -533,9 +561,14 @@ std::unique_ptr ZipArchiveReader::nextFile(std } std::vector ZipArchiveReader::getAllFiles() +{ + return getAllFiles({}); +} + +std::vector ZipArchiveReader::getAllFiles(NameFilter filter) { auto handle = acquireHandle(); - return handle.getAllFiles(); + return handle.getAllFiles(filter); } void ZipArchiveReader::setPassword(const String & password_) diff --git a/src/IO/Archives/ZipArchiveReader.h b/src/IO/Archives/ZipArchiveReader.h index 164518b1a37..0b5fa572860 100644 --- a/src/IO/Archives/ZipArchiveReader.h +++ b/src/IO/Archives/ZipArchiveReader.h @@ -42,12 +42,14 @@ public: /// you can read that buffer to extract uncompressed data from the archive. /// Several read buffers can be used at the same time in parallel. std::unique_ptr readFile(const String & filename) override; + std::unique_ptr readFile(NameFilter filter) override; /// It's possible to convert a file enumerator to a read buffer and vice versa. std::unique_ptr readFile(std::unique_ptr enumerator) override; std::unique_ptr nextFile(std::unique_ptr read_buffer) override; std::vector getAllFiles() override; + std::vector getAllFiles(NameFilter filter) override; /// Sets password used to decrypt the contents of the files in the archive. void setPassword(const String & password_) override; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 37998f37c3f..3d87793d06c 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -57,7 +57,6 @@ #include #include - namespace ProfileEvents { extern const Event CreatedReadBufferOrdinary; @@ -387,7 +386,23 @@ std::unique_ptr createReadBuffer( if (!path_to_archive.empty()) { auto reader = createArchiveReader(path_to_archive); - return reader->readFile(current_path); + + if (current_path.find_first_of("*?{") != std::string::npos) + { + auto matcher = std::make_shared(makeRegexpPatternFromGlobs(current_path)); + if (!matcher->ok()) + throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, + "Cannot compile regex from glob ({}): {}", current_path, matcher->error()); + + return reader->readFile([matcher = std::move(matcher)](const std::string & path) + { + return re2::RE2::FullMatch(path, *matcher); + }); + } + else + { + return reader->readFile(current_path); + } } if (use_table_fd) @@ -529,14 +544,30 @@ ColumnsDescription StorageFile::getTableStructureFromFile( } else { - read_buffer_iterator = [&, path_it = paths.begin(), archive_it = paths_to_archive.begin()](ColumnsDescription &) mutable -> std::unique_ptr + read_buffer_iterator = [&, path_it = paths.begin(), archive_it = paths_to_archive.begin(), first = true](ColumnsDescription &) mutable -> std::unique_ptr { - if (archive_it == paths_to_archive.end()) - return nullptr; + String path; + struct stat file_stat; + do + { + if (archive_it == paths_to_archive.end()) + { + if (first) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, because all files are empty. You must specify table structure manually", + format); + return nullptr; + } - auto file_stat = getFileStat(*archive_it, false, -1, "File"); + path = *archive_it++; + file_stat = getFileStat(path, false, -1, "File"); + } + while (context->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0); + + first = false; + return createReadBuffer(*path_it, file_stat, false, -1, compression_method, context, path); - return createReadBuffer(*path_it, file_stat, false, -1, compression_method, context, *archive_it); }; } @@ -1012,13 +1043,39 @@ Pipe StorageFile::read( if (!paths_to_archive.empty()) { + if (paths.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Multiple paths defined for reading from archive"); + + const auto & path = paths[0]; + + IArchiveReader::NameFilter filter; + if (path.find_first_of("*?{") != std::string::npos) + { + auto matcher = std::make_shared(makeRegexpPatternFromGlobs(path)); + if (!matcher->ok()) + throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, + "Cannot compile regex from glob ({}): {}", path, matcher->error()); + + filter = [matcher](const std::string & p) + { + return re2::RE2::FullMatch(p, *matcher); + }; + } + for (size_t i = 0; i < paths_to_archive.size(); ++i) { - const auto & path_to_archive = paths_to_archive[i]; - auto archive_reader = createArchiveReader(path_to_archive); - auto files = archive_reader->getAllFiles(); - for (auto & file : files) - files_in_archive.push_back({i, std::move(file)}); + if (filter) + { + const auto & path_to_archive = paths_to_archive[i]; + auto archive_reader = createArchiveReader(path_to_archive); + auto files = archive_reader->getAllFiles(filter); + for (auto & file : files) + files_in_archive.push_back({i, std::move(file)}); + } + else + { + files_in_archive.push_back({i, path}); + } } } From be0c5bf10a04b77a7d57d4e3241fc8cefb357401 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 2 Aug 2023 03:03:24 +0000 Subject: [PATCH 1461/2047] Use concepts to replace more std::enable_if_t fix --- src/Access/LDAPClient.cpp | 3 ++- src/Common/Exception.h | 6 +++--- src/Common/IntervalTree.h | 5 +++-- src/Common/NetException.h | 3 ++- src/Core/MultiEnum.h | 6 +++--- .../Serializations/SerializationNullable.cpp | 16 ++++++++-------- src/Functions/FunctionsHashing.h | 11 ++++------- src/Functions/TransformDateTime64.h | 8 +++++--- src/Parsers/Access/ParserCreateQuotaQuery.cpp | 3 ++- 9 files changed, 32 insertions(+), 29 deletions(-) diff --git a/src/Access/LDAPClient.cpp b/src/Access/LDAPClient.cpp index 9606656f732..ac2f1683f0c 100644 --- a/src/Access/LDAPClient.cpp +++ b/src/Access/LDAPClient.cpp @@ -18,7 +18,8 @@ namespace { -template >>> +template +requires std::is_fundamental_v> void updateHash(SipHash & hash, const T & value) { hash.update(value); diff --git a/src/Common/Exception.h b/src/Common/Exception.h index f80dfe7f0a2..0a94d39d846 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -71,9 +71,9 @@ public: } /// Message must be a compile-time constant - template>> - Exception(int code, T && message) - : Exception(message, code) + template + requires std::is_convertible_v + Exception(int code, T && message) : Exception(message, code) { capture_thread_frame_pointers = thread_frame_pointers; message_format_string = tryGetStaticFormatString(message); diff --git a/src/Common/IntervalTree.h b/src/Common/IntervalTree.h index 9a42aadf70e..1543275233a 100644 --- a/src/Common/IntervalTree.h +++ b/src/Common/IntervalTree.h @@ -3,8 +3,8 @@ #include #include -#include #include +#include namespace DB @@ -119,7 +119,8 @@ public: return true; } - template , bool> = true, typename... Args> + template + requires(!std::is_same_v) ALWAYS_INLINE bool emplace(Interval interval, Args &&... args) { assert(!tree_is_built); diff --git a/src/Common/NetException.h b/src/Common/NetException.h index 5f887f1f6ae..0ec3a7c423f 100644 --- a/src/Common/NetException.h +++ b/src/Common/NetException.h @@ -9,7 +9,8 @@ namespace DB class NetException : public Exception { public: - template>> + template + requires std::is_convertible_v NetException(int code, T && message) : Exception(std::forward(message), code) { message_format_string = tryGetStaticFormatString(message); diff --git a/src/Core/MultiEnum.h b/src/Core/MultiEnum.h index 32aae93c6d5..36b69b1b41b 100644 --- a/src/Core/MultiEnum.h +++ b/src/Core/MultiEnum.h @@ -12,9 +12,9 @@ struct MultiEnum MultiEnum() = default; - template ...>>> - constexpr explicit MultiEnum(EnumValues ... v) - : MultiEnum((toBitFlag(v) | ... | 0u)) + template + requires std::conjunction_v...> + constexpr explicit MultiEnum(EnumValues... v) : MultiEnum((toBitFlag(v) | ... | 0u)) {} template diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 6e4402740d9..774b86472be 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -189,10 +189,10 @@ void SerializationNullable::serializeBinary(const IColumn & column, size_t row_n /// Deserialize value into ColumnNullable. /// We need to insert both to nested column and to null byte map, or, in case of exception, to not insert at all. -template , ReturnType>* = nullptr> -static ReturnType safeDeserialize( - IColumn & column, const ISerialization &, - CheckForNull && check_for_null, DeserializeNested && deserialize_nested) +template +requires std::same_as +static ReturnType +safeDeserialize(IColumn & column, const ISerialization &, CheckForNull && check_for_null, DeserializeNested && deserialize_nested) { ColumnNullable & col = assert_cast(column); @@ -217,10 +217,10 @@ static ReturnType safeDeserialize( } /// Deserialize value into non-nullable column. In case of NULL, insert default value and return false. -template , ReturnType>* = nullptr> -static ReturnType safeDeserialize( - IColumn & column, const ISerialization &, - CheckForNull && check_for_null, DeserializeNested && deserialize_nested) +template +requires std::same_as +static ReturnType +safeDeserialize(IColumn & column, const ISerialization &, CheckForNull && check_for_null, DeserializeNested && deserialize_nested) { bool insert_default = check_for_null(); if (insert_default) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 6af683777c3..211862a673b 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -573,18 +573,15 @@ struct JavaHashImpl static_cast(x) ^ static_cast(static_cast(x) >> 32)); } - template - || std::is_same_v - || std::is_same_v, T>::type * = nullptr> + template + requires std::same_as || std::same_as || std::same_as static ReturnType apply(T x) { return x; } - template - && !std::is_same_v - && !std::is_same_v - && !std::is_same_v, T>::type * = nullptr> + template + requires(!std::same_as && !std::same_as && !std::same_as) static ReturnType apply(T x) { if (std::is_unsigned_v) diff --git a/src/Functions/TransformDateTime64.h b/src/Functions/TransformDateTime64.h index fcee2753066..8484846ddc5 100644 --- a/src/Functions/TransformDateTime64.h +++ b/src/Functions/TransformDateTime64.h @@ -88,8 +88,9 @@ public: } } - template >> - inline auto execute(const T & t, Args && ... args) const + template + requires (!std::same_as) + inline auto execute(const T & t, Args &&... args) const { return wrapped_transform.execute(t, std::forward(args)...); } @@ -128,7 +129,8 @@ public: } } - template >> + template + requires (!std::same_as) inline auto executeExtendedResult(const T & t, Args && ... args) const { return wrapped_transform.executeExtendedResult(t, std::forward(args)...); diff --git a/src/Parsers/Access/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp index 2330ba7771c..cd94fbb410f 100644 --- a/src/Parsers/Access/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -108,7 +108,8 @@ namespace }); } - template || std::is_same_v>> + template + requires std::same_as || std::same_as T fieldToNumber(const Field & f) { if (f.getType() == Field::Types::String) From 35941ddb927b32959e87831882acde544997df0d Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 2 Aug 2023 11:57:38 +0300 Subject: [PATCH 1462/2047] Refine RestorerFromBackup::applyCustomStoragePolicy infinitesimally --- src/Backups/RestorerFromBackup.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Backups/RestorerFromBackup.cpp b/src/Backups/RestorerFromBackup.cpp index 151813db574..8a0908c6b67 100644 --- a/src/Backups/RestorerFromBackup.cpp +++ b/src/Backups/RestorerFromBackup.cpp @@ -629,12 +629,10 @@ void RestorerFromBackup::checkDatabase(const String & database_name) void RestorerFromBackup::applyCustomStoragePolicy(ASTPtr query_ptr) { constexpr auto setting_name = "storage_policy"; - if (!query_ptr) - return; - auto storage = query_ptr->as().storage; - if (storage && storage->settings) + if (query_ptr && restore_settings.storage_policy.has_value()) { - if (restore_settings.storage_policy.has_value()) + ASTStorage * storage = query_ptr->as().storage; + if (storage && storage->settings) { if (restore_settings.storage_policy.value().empty()) /// it has been set to "" deliberately, so the source storage policy is erased From 5e544a5ae5e35bc6dce583564afb7c89bc0751f4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Aug 2023 12:00:57 +0200 Subject: [PATCH 1463/2047] empty commit From 0174244fe12666f3792586a641c46dab1f597ebf Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 2 Aug 2023 11:21:47 +0000 Subject: [PATCH 1464/2047] fixed tests --- src/Storages/System/attachInformationSchemaTables.cpp | 2 +- .../02206_information_schema_show_database.reference | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 9a909110889..3eb7d74b9d2 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -42,7 +42,7 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d database.createTable(context, ast_create.getTable(), view, ast); ASTPtr ast_upper = ast_create.clone(); auto & ast_create_upper = ast_upper->as(); - ast_create.setTable(Poco::toUpper(view_name)); + ast_create_upper.setTable(Poco::toUpper(view_name)); StoragePtr view_upper = createTableFromAST(ast_create_upper, database.getDatabaseName(), database.getTableDataPath(ast_create_upper), context, true).second; diff --git a/tests/queries/0_stateless/02206_information_schema_show_database.reference b/tests/queries/0_stateless/02206_information_schema_show_database.reference index f5fd221a689..0cf7913e28e 100644 --- a/tests/queries/0_stateless/02206_information_schema_show_database.reference +++ b/tests/queries/0_stateless/02206_information_schema_show_database.reference @@ -1,6 +1,6 @@ CREATE DATABASE INFORMATION_SCHEMA\nENGINE = Memory CREATE VIEW INFORMATION_SCHEMA.COLUMNS\n(\n `table_catalog` String,\n `table_schema` String,\n `table_name` String,\n `TABLE_SCHEMA` String,\n `TABLE_NAME` String,\n `column_name` String,\n `ordinal_position` UInt64,\n `column_default` String,\n `is_nullable` String,\n `data_type` String,\n `character_maximum_length` Nullable(UInt64),\n `character_octet_length` Nullable(UInt64),\n `numeric_precision` Nullable(UInt64),\n `numeric_precision_radix` Nullable(UInt64),\n `numeric_scale` Nullable(UInt64),\n `datetime_precision` Nullable(UInt64),\n `character_set_catalog` Nullable(String),\n `character_set_schema` Nullable(String),\n `character_set_name` Nullable(String),\n `collation_catalog` Nullable(String),\n `collation_schema` Nullable(String),\n `collation_name` Nullable(String),\n `domain_catalog` Nullable(String),\n `domain_schema` Nullable(String),\n `domain_name` Nullable(String),\n `column_comment` String,\n `column_type` String,\n `TABLE_CATALOG` String ALIAS table_catalog,\n `COLUMN_NAME` String ALIAS column_name,\n `ORDINAL_POSITION` UInt64 ALIAS ordinal_position,\n `COLUMN_DEFAULT` String ALIAS column_default,\n `IS_NULLABLE` String ALIAS is_nullable,\n `DATA_TYPE` String ALIAS data_type,\n `CHARACTER_MAXIMUM_LENGTH` Nullable(UInt64) ALIAS character_maximum_length,\n `CHARACTER_OCTET_LENGTH` Nullable(UInt64) ALIAS character_octet_length,\n `NUMERIC_PRECISION` Nullable(UInt64) ALIAS numeric_precision,\n `NUMERIC_PRECISION_RADIX` Nullable(UInt64) ALIAS numeric_precision_radix,\n `NUMERIC_SCALE` Nullable(UInt64) ALIAS numeric_scale,\n `DATETIME_PRECISION` Nullable(UInt64) ALIAS datetime_precision,\n `CHARACTER_SET_CATALOG` Nullable(String) ALIAS character_set_catalog,\n `CHARACTER_SET_SCHEMA` Nullable(String) ALIAS character_set_schema,\n `CHARACTER_SET_NAME` Nullable(String) ALIAS character_set_name,\n `COLLATION_CATALOG` Nullable(String) ALIAS collation_catalog,\n `COLLATION_SCHEMA` Nullable(String) ALIAS collation_schema,\n `COLLATION_NAME` Nullable(String) ALIAS collation_name,\n `DOMAIN_CATALOG` Nullable(String) ALIAS domain_catalog,\n `DOMAIN_SCHEMA` Nullable(String) ALIAS domain_schema,\n `DOMAIN_NAME` Nullable(String) ALIAS domain_name,\n `COLUMN_COMMENT` String ALIAS column_comment,\n `COLUMN_TYPE` String ALIAS column_type\n) AS\nSELECT\n database AS table_catalog,\n database AS table_schema,\n database AS TABLE_SCHEMA,\n table AS table_name,\n table AS TABLE_NAME,\n name AS column_name,\n position AS ordinal_position,\n default_expression AS column_default,\n type LIKE \'Nullable(%)\' AS is_nullable,\n type AS data_type,\n character_octet_length AS character_maximum_length,\n character_octet_length,\n numeric_precision,\n numeric_precision_radix,\n numeric_scale,\n datetime_precision,\n NULL AS character_set_catalog,\n NULL AS character_set_schema,\n NULL AS character_set_name,\n NULL AS collation_catalog,\n NULL AS collation_schema,\n NULL AS collation_name,\n NULL AS domain_catalog,\n NULL AS domain_schema,\n NULL AS domain_name,\n comment AS column_comment,\n type AS column_type\nFROM system.columns CREATE VIEW INFORMATION_SCHEMA.TABLES (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables -CREATE VIEW INFORMATION_SCHEMA.TABLES (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables -CREATE VIEW information_schema.tables (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables +CREATE VIEW INFORMATION_SCHEMA.tables (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables +CREATE VIEW information_schema.TABLES (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables CREATE VIEW information_schema.tables (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables From dab77b044be007f0c2854777d4057f2f07721dde Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 2 Aug 2023 13:23:18 +0200 Subject: [PATCH 1465/2047] Update attachInformationSchemaTables.cpp --- src/Storages/System/attachInformationSchemaTables.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 3eb7d74b9d2..07e9ab380d4 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -18,7 +18,6 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE); if (database.getEngineName() != "Memory") return; - // bool is_uppercase = database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE; String metadata_resource_name = view_name + ".sql"; auto attach_query = getResource(metadata_resource_name); @@ -34,8 +33,6 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d assert(view_name == ast_create.getTable()); ast_create.attach = false; ast_create.setDatabase(database.getDatabaseName()); - // if (is_uppercase) - // ast_create.setTable(Poco::toUpper(view_name)); StoragePtr view = createTableFromAST(ast_create, database.getDatabaseName(), database.getTableDataPath(ast_create), context, true).second; From ffdc174c9a9984042b947ddb5c20d2258554d34e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 2 Aug 2023 13:24:05 +0200 Subject: [PATCH 1466/2047] Update test.py --- tests/integration/test_backup_restore_new/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 92965d910f4..53f1599a0d6 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1227,7 +1227,6 @@ def test_backup_all(exclude_system_log_tables): "asynchronous_insert_log", ] exclude_from_backup += ["system." + table_name for table_name in log_tables] - exclude_from_backup += ["information_schema.COLUMNS", "information_schema.SCHEMATA", "information_schema.TABLES", "information_schema.VIEWS"] backup_command = f"BACKUP ALL {'EXCEPT TABLES ' + ','.join(exclude_from_backup) if exclude_from_backup else ''} TO {backup_name}" From 1335d06b8f49b3ec581cb24a2398c46d253647c7 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Wed, 2 Aug 2023 13:39:54 +0200 Subject: [PATCH 1467/2047] Do not replicate ALTER TABLE t FREEZE on Replicated engine (#52064) * fix * do not replicate alter freeze * fix --------- Co-authored-by: Alexander Tokmakov --- .../en/engines/database-engines/replicated.md | 2 +- src/Databases/DatabaseReplicated.cpp | 4 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 4 +- src/Interpreters/executeDDLQueryOnCluster.h | 2 +- src/Parsers/ASTAlterQuery.cpp | 49 +------------------ src/Parsers/ASTAlterQuery.h | 2 - 6 files changed, 7 insertions(+), 56 deletions(-) diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index 5672633c4a2..1f90f2acabf 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -35,7 +35,7 @@ The [system.clusters](../../operations/system-tables/clusters.md) system table c When creating a new replica of the database, this replica creates tables by itself. If the replica has been unavailable for a long time and has lagged behind the replication log — it checks its local metadata with the current metadata in ZooKeeper, moves the extra tables with data to a separate non-replicated database (so as not to accidentally delete anything superfluous), creates the missing tables, updates the table names if they have been renamed. The data is replicated at the `ReplicatedMergeTree` level, i.e. if the table is not replicated, the data will not be replicated (the database is responsible only for metadata). -[`ALTER TABLE ATTACH|FETCH|DROP|DROP DETACHED|DETACH PARTITION|PART`](../../sql-reference/statements/alter/partition.md) queries are allowed but not replicated. The database engine will only add/fetch/remove the partition/part to the current replica. However, if the table itself uses a Replicated table engine, then the data will be replicated after using `ATTACH`. +[`ALTER TABLE FREEZE|ATTACH|FETCH|DROP|DROP DETACHED|DETACH PARTITION|PART`](../../sql-reference/statements/alter/partition.md) queries are allowed but not replicated. The database engine will only add/fetch/remove the partition/part to the current replica. However, if the table itself uses a Replicated table engine, then the data will be replicated after using `ATTACH`. ## Usage Example {#usage-example} diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index ed56edd7503..b93ba731d4a 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -666,7 +666,7 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ { for (const auto & command : query_alter->command_list->children) { - if (!isSupportedAlterType(command->as().type)) + if (!isSupportedAlterTypeForOnClusterDDLQuery(command->as().type)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported type of ALTER query"); } } @@ -1474,7 +1474,7 @@ bool DatabaseReplicated::shouldReplicateQuery(const ContextPtr & query_context, /// Some ALTERs are not replicated on database level if (const auto * alter = query_ptr->as()) { - if (alter->isAttachAlter() || alter->isFetchAlter() || alter->isDropPartitionAlter() || is_keeper_map_table(query_ptr)) + if (alter->isAttachAlter() || alter->isFetchAlter() || alter->isDropPartitionAlter() || is_keeper_map_table(query_ptr) || alter->isFreezeAlter()) return false; if (has_many_shards() || !is_replicated_table(query_ptr)) diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 04c1d545207..b34108644fb 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -49,7 +49,7 @@ static ZooKeeperRetriesInfo getRetriesInfo() ); } -bool isSupportedAlterType(int type) +bool isSupportedAlterTypeForOnClusterDDLQuery(int type) { assert(type != ASTAlterCommand::NO_TYPE); static const std::unordered_set unsupported_alter_types{ @@ -90,7 +90,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, { for (const auto & command : query_alter->command_list->children) { - if (!isSupportedAlterType(command->as().type)) + if (!isSupportedAlterTypeForOnClusterDDLQuery(command->as().type)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported type of ALTER query"); } } diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index 40db13d7ef5..7daf9babf9f 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -21,7 +21,7 @@ class Cluster; using ClusterPtr = std::shared_ptr; /// Returns true if provided ALTER type can be executed ON CLUSTER -bool isSupportedAlterType(int type); +bool isSupportedAlterTypeForOnClusterDDLQuery(int type); struct DDLQueryOnClusterParams { diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 61e5903fad5..955320c318c 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -13,7 +13,7 @@ namespace ErrorCodes String ASTAlterCommand::getID(char delim) const { - return String("AlterCommand") + delim + typeToString(type); + return fmt::format("AlterCommand{}{}", delim, type); } ASTPtr ASTAlterCommand::clone() const @@ -80,53 +80,6 @@ ASTPtr ASTAlterCommand::clone() const return res; } -const char * ASTAlterCommand::typeToString(ASTAlterCommand::Type type) -{ - switch (type) - { - case ADD_COLUMN: return "ADD_COLUMN"; - case DROP_COLUMN: return "DROP_COLUMN"; - case MODIFY_COLUMN: return "MODIFY_COLUMN"; - case COMMENT_COLUMN: return "COMMENT_COLUMN"; - case RENAME_COLUMN: return "RENAME_COLUMN"; - case MATERIALIZE_COLUMN: return "MATERIALIZE_COLUMN"; - case MODIFY_ORDER_BY: return "MODIFY_ORDER_BY"; - case MODIFY_SAMPLE_BY: return "MODIFY_SAMPLE_BY"; - case MODIFY_TTL: return "MODIFY_TTL"; - case MATERIALIZE_TTL: return "MATERIALIZE_TTL"; - case MODIFY_SETTING: return "MODIFY_SETTING"; - case RESET_SETTING: return "RESET_SETTING"; - case MODIFY_QUERY: return "MODIFY_QUERY"; - case REMOVE_TTL: return "REMOVE_TTL"; - case REMOVE_SAMPLE_BY: return "REMOVE_SAMPLE_BY"; - case ADD_INDEX: return "ADD_INDEX"; - case DROP_INDEX: return "DROP_INDEX"; - case MATERIALIZE_INDEX: return "MATERIALIZE_INDEX"; - case ADD_CONSTRAINT: return "ADD_CONSTRAINT"; - case DROP_CONSTRAINT: return "DROP_CONSTRAINT"; - case ADD_PROJECTION: return "ADD_PROJECTION"; - case DROP_PROJECTION: return "DROP_PROJECTION"; - case MATERIALIZE_PROJECTION: return "MATERIALIZE_PROJECTION"; - case DROP_PARTITION: return "DROP_PARTITION"; - case DROP_DETACHED_PARTITION: return "DROP_DETACHED_PARTITION"; - case ATTACH_PARTITION: return "ATTACH_PARTITION"; - case MOVE_PARTITION: return "MOVE_PARTITION"; - case REPLACE_PARTITION: return "REPLACE_PARTITION"; - case FETCH_PARTITION: return "FETCH_PARTITION"; - case FREEZE_PARTITION: return "FREEZE_PARTITION"; - case FREEZE_ALL: return "FREEZE_ALL"; - case UNFREEZE_PARTITION: return "UNFREEZE_PARTITION"; - case UNFREEZE_ALL: return "UNFREEZE_ALL"; - case DELETE: return "DELETE"; - case UPDATE: return "UPDATE"; - case NO_TYPE: return "NO_TYPE"; - case LIVE_VIEW_REFRESH: return "LIVE_VIEW_REFRESH"; - case MODIFY_DATABASE_SETTING: return "MODIFY_DATABASE_SETTING"; - case MODIFY_COMMENT: return "MODIFY_COMMENT"; - } - UNREACHABLE(); -} - void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { if (type == ASTAlterCommand::ADD_COLUMN) diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 1400113fa9c..30cf0cac4ce 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -208,8 +208,6 @@ public: ASTPtr clone() const override; - static const char * typeToString(Type type); - protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; }; From e811d5c4e9d5c7eaa204410417f8e53e772950c9 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 2 Aug 2023 14:06:20 +0200 Subject: [PATCH 1468/2047] Update attachInformationSchemaTables.cpp --- src/Storages/System/attachInformationSchemaTables.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/System/attachInformationSchemaTables.cpp b/src/Storages/System/attachInformationSchemaTables.cpp index 07e9ab380d4..43a3840cdf0 100644 --- a/src/Storages/System/attachInformationSchemaTables.cpp +++ b/src/Storages/System/attachInformationSchemaTables.cpp @@ -42,9 +42,8 @@ static void createInformationSchemaView(ContextMutablePtr context, IDatabase & d ast_create_upper.setTable(Poco::toUpper(view_name)); StoragePtr view_upper = createTableFromAST(ast_create_upper, database.getDatabaseName(), database.getTableDataPath(ast_create_upper), context, true).second; - + database.createTable(context, ast_create_upper.getTable(), view_upper, ast_upper); - } catch (...) From 61977479bec5776a6d54148e3b9a773a83fa7e2e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Aug 2023 15:18:57 +0300 Subject: [PATCH 1469/2047] Update SystemLog.cpp (#52858) --- src/Interpreters/SystemLog.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index be0468aa876..12f3b9744cb 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -48,6 +48,11 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +namespace ActionLocks +{ + extern const StorageActionBlockType PartsMerge; +} + namespace { class StorageWithComment : public IAST @@ -560,6 +565,10 @@ void SystemLog::prepareTable() rename->elements.emplace_back(std::move(elem)); + ActionLock merges_lock; + if (DatabaseCatalog::instance().getDatabase(table_id.database_name)->getUUID() == UUIDHelpers::Nil) + merges_lock = table->getActionLock(ActionLocks::PartsMerge); + auto query_context = Context::createCopy(context); /// As this operation is performed automatically we don't want it to fail because of user dependencies on log tables query_context->setSetting("check_table_dependencies", Field{false}); From 6e49d3add69aa77057b6ef47081aed87392404e2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 2 Aug 2023 12:34:43 +0000 Subject: [PATCH 1470/2047] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 6ad0e76b4a0..ed66324da28 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -612,12 +612,14 @@ def test_filter_using_file(cluster): query = f"select count(*) from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_partition_tf_*.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') WHERE _file='test_partition_tf_3.csv'" assert azure_query(node, query) == "1\n" + def test_read_from_not_existing_container(cluster): node = cluster.instances["node"] query = f"select * from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont_not_exists', 'test_table.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto')" expected_err_msg = "container does not exist" assert expected_err_msg in node.query_and_get_error(query) + def test_function_signatures(cluster): node = cluster.instances["node"] connection_string = "DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1;" From 2cbe79b529e84166055d2af469fde9c28d1757c0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 2 Aug 2023 15:11:52 +0200 Subject: [PATCH 1471/2047] Fix memory consumption when `max_block_size` is huge in Aggregator + more memory checks (#51566) * impl * remove checks from without_key methods * maybe will improve smth * add test * Update 02797_aggregator_huge_mem_usage_bug.sql * Update 02797_aggregator_huge_mem_usage_bug.sql --------- Co-authored-by: Alexey Milovidov --- src/Common/MemoryTracker.cpp | 4 +- src/Interpreters/Aggregator.cpp | 16 ++-- src/Interpreters/Aggregator.h | 93 +++++++++---------- .../AggregatingInOrderTransform.cpp | 4 +- ...97_aggregator_huge_mem_usage_bug.reference | 0 .../02797_aggregator_huge_mem_usage_bug.sql | 12 +++ 6 files changed, 72 insertions(+), 57 deletions(-) create mode 100644 tests/queries/0_stateless/02797_aggregator_huge_mem_usage_bug.reference create mode 100644 tests/queries/0_stateless/02797_aggregator_huge_mem_usage_bug.sql diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 52cae0768dc..216f2e9f167 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -208,10 +208,10 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT * we allow exception about memory limit exceeded to be thrown only on next allocation. * So, we allow over-allocations. */ - Int64 will_be = size + amount.fetch_add(size, std::memory_order_relaxed); + Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); - if (metric_loaded != CurrentMetrics::end()) + if (metric_loaded != CurrentMetrics::end() && size) CurrentMetrics::add(metric_loaded, size); Int64 current_hard_limit = hard_limit.load(std::memory_order_relaxed); diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index c2914c938b5..91cd574708a 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -984,6 +984,8 @@ void Aggregator::executeOnBlockSmall( } executeImpl(result, row_begin, row_end, key_columns, aggregate_instructions); + + CurrentMemoryTracker::check(); } void Aggregator::mergeOnBlockSmall( @@ -1023,6 +1025,8 @@ void Aggregator::mergeOnBlockSmall( #undef M else throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant."); + + CurrentMemoryTracker::check(); } void Aggregator::executeImpl( @@ -1383,11 +1387,8 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl( } -void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl( - AggregatedDataVariants & data_variants, - size_t row_begin, - size_t row_end, - AggregateFunctionInstruction * aggregate_instructions) const +void NO_INLINE Aggregator::executeOnIntervalWithoutKey( + AggregatedDataVariants & data_variants, size_t row_begin, size_t row_end, AggregateFunctionInstruction * aggregate_instructions) const { /// `data_variants` will destroy the states of aggregate functions in the destructor data_variants.aggregator = this; @@ -1414,7 +1415,7 @@ void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl( } } -void NO_INLINE Aggregator::mergeOnIntervalWithoutKeyImpl( +void NO_INLINE Aggregator::mergeOnIntervalWithoutKey( AggregatedDataVariants & data_variants, size_t row_begin, size_t row_end, @@ -2921,6 +2922,7 @@ void NO_INLINE Aggregator::mergeBlockWithoutKeyStreamsImpl( AggregateColumnsConstData aggregate_columns = params.makeAggregateColumnsData(block); mergeWithoutKeyStreamsImpl(result, 0, block.rows(), aggregate_columns); } + void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl( AggregatedDataVariants & result, size_t row_begin, @@ -3139,6 +3141,8 @@ void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVari LOG_TRACE(log, "Merged partially aggregated single-level data."); } + + CurrentMemoryTracker::check(); } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 29096a38be6..4f2c86606c5 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1118,9 +1118,55 @@ public: AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block bool & no_more_keys) const; + /** This array serves two purposes. + * + * Function arguments are collected side by side, and they do not need to be collected from different places. Also the array is made zero-terminated. + * The inner loop (for the case without_key) is almost twice as compact; performance gain of about 30%. + */ + struct AggregateFunctionInstruction + { + const IAggregateFunction * that{}; + size_t state_offset{}; + const IColumn ** arguments{}; + const IAggregateFunction * batch_that{}; + const IColumn ** batch_arguments{}; + const UInt64 * offsets{}; + bool has_sparse_arguments = false; + }; + + /// Used for optimize_aggregation_in_order: + /// - No two-level aggregation + /// - No external aggregation + /// - No without_key support (it is implemented using executeOnIntervalWithoutKey()) + void executeOnBlockSmall( + AggregatedDataVariants & result, + size_t row_begin, + size_t row_end, + ColumnRawPtrs & key_columns, + AggregateFunctionInstruction * aggregate_instructions) const; + + void executeOnIntervalWithoutKey( + AggregatedDataVariants & data_variants, + size_t row_begin, + size_t row_end, + AggregateFunctionInstruction * aggregate_instructions) const; + /// Used for aggregate projection. bool mergeOnBlock(Block block, AggregatedDataVariants & result, bool & no_more_keys) const; + void mergeOnBlockSmall( + AggregatedDataVariants & result, + size_t row_begin, + size_t row_end, + const AggregateColumnsConstData & aggregate_columns_data, + const ColumnRawPtrs & key_columns) const; + + void mergeOnIntervalWithoutKey( + AggregatedDataVariants & data_variants, + size_t row_begin, + size_t row_end, + const AggregateColumnsConstData & aggregate_columns_data) const; + /** Convert the aggregation data structure into a block. * If overflow_row = true, then aggregates for rows that are not included in max_rows_to_group_by are put in the first block. * @@ -1178,22 +1224,6 @@ private: AggregateFunctionsPlainPtrs aggregate_functions; - /** This array serves two purposes. - * - * Function arguments are collected side by side, and they do not need to be collected from different places. Also the array is made zero-terminated. - * The inner loop (for the case without_key) is almost twice as compact; performance gain of about 30%. - */ - struct AggregateFunctionInstruction - { - const IAggregateFunction * that{}; - size_t state_offset{}; - const IColumn ** arguments{}; - const IAggregateFunction * batch_that{}; - const IColumn ** batch_arguments{}; - const UInt64 * offsets{}; - bool has_sparse_arguments = false; - }; - using AggregateFunctionInstructions = std::vector; using NestedColumnsHolder = std::vector>; @@ -1239,26 +1269,6 @@ private: */ void destroyAllAggregateStates(AggregatedDataVariants & result) const; - - /// Used for optimize_aggregation_in_order: - /// - No two-level aggregation - /// - No external aggregation - /// - No without_key support (it is implemented using executeOnIntervalWithoutKeyImpl()) - void executeOnBlockSmall( - AggregatedDataVariants & result, - size_t row_begin, - size_t row_end, - ColumnRawPtrs & key_columns, - AggregateFunctionInstruction * aggregate_instructions) const; - void mergeOnBlockSmall( - AggregatedDataVariants & result, - size_t row_begin, - size_t row_end, - const AggregateColumnsConstData & aggregate_columns_data, - const ColumnRawPtrs & key_columns) const; - - void mergeOnBlockImpl(Block block, AggregatedDataVariants & result, bool no_more_keys) const; - void executeImpl( AggregatedDataVariants & result, size_t row_begin, @@ -1300,17 +1310,6 @@ private: AggregateFunctionInstruction * aggregate_instructions, Arena * arena) const; - void executeOnIntervalWithoutKeyImpl( - AggregatedDataVariants & data_variants, - size_t row_begin, - size_t row_end, - AggregateFunctionInstruction * aggregate_instructions) const; - void mergeOnIntervalWithoutKeyImpl( - AggregatedDataVariants & data_variants, - size_t row_begin, - size_t row_end, - const AggregateColumnsConstData & aggregate_columns_data) const; - template void writeToTemporaryFileImpl( AggregatedDataVariants & data_variants, diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 3d5a910f054..4e9f7b7601a 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -159,14 +159,14 @@ void AggregatingInOrderTransform::consume(Chunk chunk) if (group_by_key) params->aggregator.mergeOnBlockSmall(variants, key_begin, key_end, aggregate_columns_data, key_columns_raw); else - params->aggregator.mergeOnIntervalWithoutKeyImpl(variants, key_begin, key_end, aggregate_columns_data); + params->aggregator.mergeOnIntervalWithoutKey(variants, key_begin, key_end, aggregate_columns_data); } else { if (group_by_key) params->aggregator.executeOnBlockSmall(variants, key_begin, key_end, key_columns_raw, aggregate_function_instructions.data()); else - params->aggregator.executeOnIntervalWithoutKeyImpl(variants, key_begin, key_end, aggregate_function_instructions.data()); + params->aggregator.executeOnIntervalWithoutKey(variants, key_begin, key_end, aggregate_function_instructions.data()); } } diff --git a/tests/queries/0_stateless/02797_aggregator_huge_mem_usage_bug.reference b/tests/queries/0_stateless/02797_aggregator_huge_mem_usage_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02797_aggregator_huge_mem_usage_bug.sql b/tests/queries/0_stateless/02797_aggregator_huge_mem_usage_bug.sql new file mode 100644 index 00000000000..e204d968382 --- /dev/null +++ b/tests/queries/0_stateless/02797_aggregator_huge_mem_usage_bug.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS v; + +create view v (s LowCardinality(String), n UInt8) as select 'test' as s, toUInt8(number) as n from numbers(10000000); + +-- this is what allows mem usage to go really high +set max_block_size=10000000000; + +set max_memory_usage = '1Gi'; + +select s, sum(n) from v group by s format Null; + +DROP TABLE v; From b4d692c95f93583b30d28c4c624979e4eb0763b3 Mon Sep 17 00:00:00 2001 From: Alexander Zaitsev Date: Wed, 2 Aug 2023 16:08:38 +0200 Subject: [PATCH 1472/2047] doc: try to fix PGO docs --- .../optimizing-performance/profile-guided-optimization.md | 1 + .../optimizing-performance/profile-guided-optimization.md | 1 + 2 files changed, 2 insertions(+) create mode 120000 docs/ru/operations/optimizing-performance/profile-guided-optimization.md create mode 120000 docs/zh/operations/optimizing-performance/profile-guided-optimization.md diff --git a/docs/ru/operations/optimizing-performance/profile-guided-optimization.md b/docs/ru/operations/optimizing-performance/profile-guided-optimization.md new file mode 120000 index 00000000000..31cb656bd99 --- /dev/null +++ b/docs/ru/operations/optimizing-performance/profile-guided-optimization.md @@ -0,0 +1 @@ +../../../en/operations/optimizing-performance/profile-guided-optimization.md \ No newline at end of file diff --git a/docs/zh/operations/optimizing-performance/profile-guided-optimization.md b/docs/zh/operations/optimizing-performance/profile-guided-optimization.md new file mode 120000 index 00000000000..31cb656bd99 --- /dev/null +++ b/docs/zh/operations/optimizing-performance/profile-guided-optimization.md @@ -0,0 +1 @@ +../../../en/operations/optimizing-performance/profile-guided-optimization.md \ No newline at end of file From 08aa90aa29760ca56f9520254f44f48cefaec13c Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 2 Aug 2023 17:26:59 +0300 Subject: [PATCH 1473/2047] Amend the documentation --- docs/en/operations/backup.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 62f931a76b4..3a5768a03af 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -84,6 +84,11 @@ The BACKUP and RESTORE statements take a list of DATABASE and TABLE names, a des - `password` for the file on disk - `base_backup`: the destination of the previous backup of this source. For example, `Disk('backups', '1.zip')` - `structure_only`: if enabled, allows to only backup or restore the CREATE statements without the data of tables + - `storage_policy`: storage policy for the table being restored. See [Using Multiple Block Devices for Data Storage](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). + +:::note storage_policy +`storage_policy` setting is only applicable to the `RESTORE TABLE` command with an engine from the `MergeTree` family. +::: ### Usage examples From d87162f43cc0efaceed08da0051ad651c0ad81d0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 2 Aug 2023 17:09:47 +0200 Subject: [PATCH 1474/2047] Fix --- tests/integration/test_storage_s3_queue/test.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 77e32e2922c..484ab6d7e95 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -761,7 +761,7 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): files_to_generate = 100 - poll_size = 10 + poll_size = 2 prefix = f"test_multiple_{mode}" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] @@ -785,7 +785,12 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): CREATE TABLE test.s3_queue_persistent ({table_format}) ENGINE = MergeTree() ORDER BY column1; + """ + ) + for inst in [instance, instance_2]: + inst.query( + f""" CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS SELECT * @@ -800,7 +805,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): def get_count(node, table_name): return int(run_query(node, f"SELECT count() FROM {table_name}")) - for _ in range(100): + for _ in range(150): if ( get_count(instance, "test.s3_queue_persistent") + get_count(instance_2, "test.s3_queue_persistent") @@ -816,11 +821,12 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): list(map(int, l.split())) for l in run_query(instance_2, get_query).splitlines() ] + assert len(res1) + len(res2) == files_to_generate + # Checking that all engines have made progress assert len(res1) > 0 assert len(res2) > 0 - assert len(res1) + len(res2) == files_to_generate assert {tuple(v) for v in res1 + res2} == set([tuple(i) for i in total_values]) # Checking that all files were processed only once From e369f0da7cdb4a2db51bb1f428830103011749f2 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 2 Aug 2023 17:31:35 +0200 Subject: [PATCH 1475/2047] Resolve expressions from WITH clause on usage --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 23 ++++++----------------- 1 file changed, 6 insertions(+), 17 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 34432d054e1..0b3d19f1861 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -6887,13 +6887,12 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier scope.scope_node->formatASTForErrorMessage()); } - std::erase_if(with_nodes, [](const QueryTreeNodePtr & node) - { - auto * subquery_node = node->as(); - auto * union_node = node->as(); - - return (subquery_node && subquery_node->isCTE()) || (union_node && union_node->isCTE()); - }); + /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions + * and CTE for other sections to use. + * + * Example: WITH 1 AS constant, (x -> x + 1) AS lambda, a AS (SELECT * FROM test_table); + */ + query_node_typed.getWith().getNodes().clear(); for (auto & window_node : query_node_typed.getWindow().getNodes()) { @@ -6952,9 +6951,6 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier scope.scope_node->formatASTForErrorMessage()); } - if (query_node_typed.hasWith()) - resolveExpressionNodeList(query_node_typed.getWithNode(), scope, true /*allow_lambda_expression*/, false /*allow_table_expression*/); - if (query_node_typed.getPrewhere()) resolveExpressionNode(query_node_typed.getPrewhere(), scope, false /*allow_lambda_expression*/, false /*allow_table_expression*/); @@ -7123,13 +7119,6 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier scope.scope_node->formatASTForErrorMessage()); } - /** WITH section can be safely removed, because WITH section only can provide aliases to query expressions - * and CTE for other sections to use. - * - * Example: WITH 1 AS constant, (x -> x + 1) AS lambda, a AS (SELECT * FROM test_table); - */ - query_node_typed.getWith().getNodes().clear(); - /** WINDOW section can be safely removed, because WINDOW section can only provide window definition to window functions. * * Example: SELECT count(*) OVER w FROM test_table WINDOW w AS (PARTITION BY id); From 4c5c3a338d9ce83b2788a24878507ce978f24e69 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 17:48:10 +0200 Subject: [PATCH 1476/2047] Maybe fix tests --- tests/queries/0_stateless/02724_database_s3.reference | 2 ++ tests/queries/0_stateless/02725_database_hdfs.sh | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02724_database_s3.reference b/tests/queries/0_stateless/02724_database_s3.reference index bf8ea70f1d1..437549a973a 100644 --- a/tests/queries/0_stateless/02724_database_s3.reference +++ b/tests/queries/0_stateless/02724_database_s3.reference @@ -19,3 +19,5 @@ test1 Test 2: check exceptions BAD_ARGUMENTS BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index c0bd9ab5e57..dab2d37605c 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -1,6 +1,8 @@ #!/usr/bin/env bash # Tags: no-fasttest, use-hdfs, no-parallel +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh From 3eeaf7af22ba5ca5b530fd85b9dc08995b2964c2 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 2 Aug 2023 16:08:53 +0000 Subject: [PATCH 1477/2047] Fix build error --- src/Functions/FunctionsHashing.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index a2083d0a88e..f70e91033ea 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1377,8 +1377,8 @@ public: if constexpr (std::is_same_v) /// backward-compatible { - if (std::endian::native == std::endian::big) - std::ranges::for_each(col_to->getData(), transformEndianness); + if constexpr (std::endian::native == std::endian::big) + std::ranges::for_each(col_to->getData(), transformEndianness); auto col_to_fixed_string = ColumnFixedString::create(sizeof(UInt128)); const auto & data = col_to->getData(); From 85ba27a2d4b5aa2fc80c90b7393ab327da8e79dc Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 2 Aug 2023 18:22:48 +0200 Subject: [PATCH 1478/2047] Add a test --- .../02841_with_clause_resolve.reference | 15 ++ .../0_stateless/02841_with_clause_resolve.sql | 141 ++++++++++++++++++ 2 files changed, 156 insertions(+) create mode 100644 tests/queries/0_stateless/02841_with_clause_resolve.reference create mode 100644 tests/queries/0_stateless/02841_with_clause_resolve.sql diff --git a/tests/queries/0_stateless/02841_with_clause_resolve.reference b/tests/queries/0_stateless/02841_with_clause_resolve.reference new file mode 100644 index 00000000000..e2dfc4d85a9 --- /dev/null +++ b/tests/queries/0_stateless/02841_with_clause_resolve.reference @@ -0,0 +1,15 @@ +2.5 +2.5 +2.5 +2.5 +2.5 +(9399,2858) +(7159,6972) +(7456,3524) +(12685,10191) +(12598,4979) +(9824,2699) +(5655,7793) +(14410,10296) +(16211,7662) +(9349,9053) diff --git a/tests/queries/0_stateless/02841_with_clause_resolve.sql b/tests/queries/0_stateless/02841_with_clause_resolve.sql new file mode 100644 index 00000000000..b416446461b --- /dev/null +++ b/tests/queries/0_stateless/02841_with_clause_resolve.sql @@ -0,0 +1,141 @@ +set allow_experimental_analyzer = 1; + +WITH + -- Input + 44100 AS sample_frequency + , number AS tick + , tick / sample_frequency AS time + + -- Delay + , (time, wave, delay_, decay, count) -> arraySum(n1 -> wave(time - delay_ * n1), range(count)) AS delay + + , delay(time, (time -> 0.5), 0.2, 0.5, 5) AS kick + +SELECT + + kick + +FROM system.numbers +LIMIT 5; + +WITH + -- Input + 44100 AS sample_frequency + , number AS tick + , tick / sample_frequency AS time + + -- Output control + , 1 AS master_volume + , level -> least(1.0, greatest(-1.0, level)) AS clamp + , level -> (clamp(level) * 0x7FFF * master_volume)::Int16 AS output + , x -> (x, x) AS mono + + -- Basic waves + , time -> sin(time * 2 * pi()) AS sine_wave + , time -> time::UInt64 % 2 * 2 - 1 AS square_wave + , time -> (time - floor(time)) * 2 - 1 AS sawtooth_wave + , time -> abs(sawtooth_wave(time)) * 2 - 1 AS triangle_wave + + -- Helpers + , (from, to, wave, time) -> from + ((wave(time) + 1) / 2) * (to - from) AS lfo + , (from, to, steps, time) -> from + floor((time - floor(time)) * steps) / steps * (to - from) AS step_lfo + , (from, to, steps, time) -> exp(step_lfo(log(from), log(to), steps, time)) AS exp_step_lfo + + -- Noise + , time -> cityHash64(time) / 0xFFFFFFFFFFFFFFFF AS uniform_noise + , time -> erf(uniform_noise(time)) AS white_noise + , time -> cityHash64(time) % 2 ? 1 : -1 AS bernoulli_noise + + -- Distortion + , (x, amount) -> clamp(x * amount) AS clipping + , (x, amount) -> clamp(x > 0 ? pow(x, amount) : -pow(-x, amount)) AS power_distortion + , (x, amount) -> round(x * exp2(amount)) / exp2(amount) AS bitcrush + , (time, sample_frequency) -> round(time * sample_frequency) / sample_frequency AS desample + , (time, wave, amount) -> (time - floor(time) < (1 - amount)) ? wave(time * (1 - amount)) : 0 AS thin + , (time, wave, amount) -> wave(floor(time) + pow(time - floor(time), amount)) AS skew + + -- Combining + , (a, b, weight) -> a * (1 - weight) + b * weight AS combine + + -- Envelopes + , (time, offset, attack, hold, release) -> + time < offset ? 0 + : (time < offset + attack ? ((time - offset) / attack) + : (time < offset + attack + hold ? 1 + : (time < offset + attack + hold + release ? (offset + attack + hold + release - time) / release + : 0))) AS envelope + + , (bpm, time, offset, attack, hold, release) -> + envelope( + time * (bpm / 60) - floor(time * (bpm / 60)), + offset, + attack, + hold, + release) AS running_envelope + + -- Sequencers + , (sequence, time) -> sequence[1 + time::UInt64 % length(sequence)] AS sequencer + + -- Delay + , (time, wave, delay, decay, count) -> arraySum(n -> wave(time - delay * n) * pow(decay, n), range(count)) AS delay + + + , delay(time, (time -> power_distortion(sine_wave(time * 80 + sine_wave(time * 2)), lfo(0.5, 1, sine_wave, time / 16)) + * running_envelope(60, time, 0, 0.0, 0.01, 0.1)), + 0.2, 0.5, 5) AS kick + +SELECT + + (output( + kick + + delay(time, (time -> + power_distortion( + sine_wave(time * 50 + 1 * sine_wave(time * 100 + 1/4)) + * running_envelope(60, time, 0, 0.01, 0.01, 0.1), + lfo(1, 0.75, triangle_wave, time / 8))), + 0.2, 0.5, 10) + * lfo(0.5, 1, triangle_wave, time / 7) + + + delay(time, (time -> + power_distortion( + sine_wave(time * sequencer([50, 100, 200, 400], time / 2) + 1 * sine_wave(time * sequencer([50, 100, 200], time / 4) + 1/4)) + * running_envelope(60, time, 0.5, 0.01, 0.01, 0.1), + lfo(1, 0.75, triangle_wave, time / 8))), + 0.2, 0.5, 10) + * lfo(0.5, 1, triangle_wave, 16 + time / 11) + + + delay(time, (time -> + white_noise(time) * running_envelope(60, time, 0.75, 0.01, 0.01, 0.1)), + 0.2, 0.5, 10) + * lfo(0.5, 1, triangle_wave, 24 + time / 13) + + + sine_wave(time * 100 + 1 * sine_wave(time * 10 + 1/4)) + * running_envelope(120, time, 0, 0.01, 0.01, 0.1) + ), + + output( + kick + + delay(time + 0.01, (time -> + power_distortion( + sine_wave(time * 50 + 1 * sine_wave(time * 100 + 1/4)) + * running_envelope(60, time, 0, 0.01, 0.01, 0.1), + lfo(1, 0.75, triangle_wave, time / 8))), + 0.2, 0.5, 10) + * lfo(0.5, 1, triangle_wave, time / 7) + + + delay(time - 0.01, (time -> + power_distortion( + sine_wave(time * sequencer([50, 100, 200, 400], time / 2) + 1 * sine_wave(time * sequencer([50, 100, 200], time / 4) + 1/4)) + * running_envelope(60, time, 0.5, 0.01, 0.01, 0.1), + lfo(1, 0.75, triangle_wave, time / 8))), + 0.2, 0.5, 10) + * lfo(0.5, 1, triangle_wave, 16 + time / 11) + + + delay(time + 0.005, (time -> + white_noise(time) * running_envelope(60, time, 0.75, 0.01, 0.01, 0.1)), + 0.2, 0.5, 10) + * lfo(0.5, 1, triangle_wave, 24 + time / 13) + )) + +FROM system.numbers +LIMIT 10; From 8cbadaa6d17deb911ac1653d7ddf8b260f9b2f5c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 2 Aug 2023 18:39:20 +0200 Subject: [PATCH 1479/2047] Update src/DataTypes/DataTypeArray.h MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/DataTypes/DataTypeArray.h | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index 82af498ab13..68b574b8ded 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -71,9 +71,4 @@ public: size_t getNumberOfDimensions() const; }; -template inline constexpr bool IsDataTypeArray() -{ - return false; -} - } From d3f5551bb4962d2f719445d589f34ae76a731951 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 2 Aug 2023 19:39:32 +0300 Subject: [PATCH 1480/2047] Clarify the documentation --- docs/en/operations/backup.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 3a5768a03af..498cabc9434 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -84,10 +84,10 @@ The BACKUP and RESTORE statements take a list of DATABASE and TABLE names, a des - `password` for the file on disk - `base_backup`: the destination of the previous backup of this source. For example, `Disk('backups', '1.zip')` - `structure_only`: if enabled, allows to only backup or restore the CREATE statements without the data of tables - - `storage_policy`: storage policy for the table being restored. See [Using Multiple Block Devices for Data Storage](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). + - `storage_policy`: storage policy for the tables being restored. See [Using Multiple Block Devices for Data Storage](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). :::note storage_policy -`storage_policy` setting is only applicable to the `RESTORE TABLE` command with an engine from the `MergeTree` family. +`storage_policy` setting is only applicable to the `RESTORE` command. The specified storage policy applies only to tables with an engine from the `MergeTree` family. ::: ### Usage examples From cd9bb8126ab68a1f69777669bf493c80f950afdc Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 2 Aug 2023 18:40:22 +0200 Subject: [PATCH 1481/2047] Update src/Functions/FunctionBinaryArithmetic.h MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/Functions/FunctionBinaryArithmetic.h | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 0dcd24cc266..4ad6b3b2ec6 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1162,15 +1162,6 @@ class FunctionBinaryArithmetic : public IFunction if (!return_type_array) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Return type for function {} must be array.", getName()); - return executeArray(arguments, result_type, input_rows_count); - - // if (typeid_cast(arguments[0].column.get())) - // return result_column; - // else - } - - ColumnPtr executeArray(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const - { if constexpr (is_multiply || is_division) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use multiplication or division on arrays"); From 98ecf7d43e94d9a370a2ad38ff7cd214b6b192f2 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 2 Aug 2023 18:40:33 +0200 Subject: [PATCH 1482/2047] Update src/Functions/FunctionBinaryArithmetic.h MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/Functions/FunctionBinaryArithmetic.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 4ad6b3b2ec6..dc0bf279e1d 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1195,7 +1195,7 @@ class FunctionBinaryArithmetic : public IFunction if (*typeid_cast(arguments[0].column.get())->getOffsets().data() != *typeid_cast(arguments[1].column.get())->getOffsets().data()) { - throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "Cannot apply operation for arguments of different sizes. Size of the first argument: {}, size of the second argument: {}", *typeid_cast(arguments[0].column.get())->getOffsets().data(), *typeid_cast(arguments[1].column.get())->getOffsets().data()); From 67d8f1c1c512c1de148784c3772d60f2305e48f4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Aug 2023 18:42:38 +0200 Subject: [PATCH 1483/2047] fix data race --- programs/server/Server.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 796ab583fe4..e6d5837dd0e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1650,6 +1650,7 @@ try database_catalog.initializeAndLoadTemporaryDatabase(); loadMetadataSystem(global_context); maybeConvertSystemDatabase(global_context); + startupSystemTables(); /// After attaching system databases we can initialize system log. global_context->initializeSystemLogs(); global_context->setSystemZooKeeperLogAfterInitializationIfNeeded(); @@ -1668,7 +1669,6 @@ try /// Then, load remaining databases loadMetadata(global_context, default_database); convertDatabasesEnginesIfNeed(global_context); - startupSystemTables(); database_catalog.startupBackgroundCleanup(); /// After loading validate that default database exists database_catalog.assertDatabaseExists(default_database); From d43e76b14720ebdb1e8ad2668be71306e9e38d45 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 2 Aug 2023 16:47:01 +0000 Subject: [PATCH 1484/2047] fix reading of empty Nested(Array(LowCardinality(...))) --- src/Interpreters/inplaceBlockConversions.cpp | 4 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 10 +-- src/Storages/MergeTree/IMergeTreeReader.h | 5 +- .../MergeTree/MergeTreeReaderCompact.cpp | 62 +++++++++++++------ .../MergeTree/MergeTreeReaderCompact.h | 7 ++- .../MergeTree/MergeTreeReaderInMemory.cpp | 2 +- ...2835_nested_array_lowcardinality.reference | 30 +++++++++ .../02835_nested_array_lowcardinality.sql | 49 +++++++++++++++ 8 files changed, 138 insertions(+), 31 deletions(-) create mode 100644 tests/queries/0_stateless/02835_nested_array_lowcardinality.reference create mode 100644 tests/queries/0_stateless/02835_nested_array_lowcardinality.sql diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 5bbd2667f55..4cac2f0e20c 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -306,7 +306,9 @@ void fillMissingColumns( return; size_t level = ISerialization::getArrayLevel(subpath); - assert(level < num_dimensions); + /// It can happen if element of Array is Map. + if (level >= num_dimensions) + return; auto stream_name = ISerialization::getFileNameForStream(*requested_column, subpath); auto it = offsets_columns.find(stream_name); diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index f9b97a6a05d..73fbe447e23 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -216,7 +216,7 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const } } -IMergeTreeReader::ColumnPositionLevel IMergeTreeReader::findColumnForOffsets(const NameAndTypePair & required_column) const +IMergeTreeReader::ColumnNameLevel IMergeTreeReader::findColumnForOffsets(const NameAndTypePair & required_column) const { auto get_offsets_streams = [](const auto & serialization, const auto & name_in_storage) { @@ -238,7 +238,7 @@ IMergeTreeReader::ColumnPositionLevel IMergeTreeReader::findColumnForOffsets(con auto required_offsets_streams = get_offsets_streams(getSerializationInPart(required_column), required_name_in_storage); size_t max_matched_streams = 0; - ColumnPositionLevel position_level; + ColumnNameLevel name_level; /// Find column that has maximal number of matching /// offsets columns with required_column. @@ -261,14 +261,14 @@ IMergeTreeReader::ColumnPositionLevel IMergeTreeReader::findColumnForOffsets(con it = current_it; } - if (i && (!position_level || i > max_matched_streams)) + if (i && (!name_level || i > max_matched_streams)) { max_matched_streams = i; - position_level.emplace(*data_part_info_for_read->getColumnPosition(part_column.name), it->second); + name_level.emplace(part_column.name, it->second); } } - return position_level; + return name_level; } void IMergeTreeReader::checkNumberOfColumns(size_t num_columns_to_read) const diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index fcab35fb4c2..97e17d56892 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -96,11 +96,12 @@ protected: MarkRanges all_mark_ranges; /// Position and level (of nesting). - using ColumnPositionLevel = std::optional>; + using ColumnNameLevel = std::optional>; + /// In case of part of the nested column does not exists, offsets should be /// read, but only the offsets for the current column, that is why it /// returns pair of size_t, not just one. - ColumnPositionLevel findColumnForOffsets(const NameAndTypePair & column) const; + ColumnNameLevel findColumnForOffsets(const NameAndTypePair & column) const; NameSet partially_read_columns; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index f65e66ff52d..308b33cc62d 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -130,7 +130,7 @@ void MergeTreeReaderCompact::fillColumnPositions() size_t columns_num = columns_to_read.size(); column_positions.resize(columns_num); - read_only_offsets.resize(columns_num); + columns_for_offsets.resize(columns_num); for (size_t i = 0; i < columns_num; ++i) { @@ -153,11 +153,11 @@ void MergeTreeReaderCompact::fillColumnPositions() { /// If array of Nested column is missing in part, /// we have to read its offsets if they exist. - auto position_level = findColumnForOffsets(column_to_read); - if (position_level.has_value()) + auto name_level_for_offsets = findColumnForOffsets(column_to_read); + if (name_level_for_offsets.has_value()) { - column_positions[i].emplace(position_level->first); - read_only_offsets[i].emplace(position_level->second); + column_positions[i] = data_part_info_for_read->getColumnPosition(name_level_for_offsets->first); + columns_for_offsets[i] = name_level_for_offsets; partially_read_columns.insert(column_to_read.name); } } @@ -203,7 +203,7 @@ size_t MergeTreeReaderCompact::readRows( auto & column = res_columns[pos]; size_t column_size_before_reading = column->size(); - readData(columns_to_read[pos], column, from_mark, current_task_last_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]); + readData(columns_to_read[pos], column, from_mark, current_task_last_mark, *column_positions[pos], rows_to_read, columns_for_offsets[pos]); size_t read_rows_in_column = column->size() - column_size_before_reading; if (read_rows_in_column != rows_to_read) @@ -239,23 +239,37 @@ size_t MergeTreeReaderCompact::readRows( void MergeTreeReaderCompact::readData( const NameAndTypePair & name_and_type, ColumnPtr & column, size_t from_mark, size_t current_task_last_mark, size_t column_position, size_t rows_to_read, - std::optional only_offsets_level) + ColumnNameLevel name_level_for_offsets) { const auto & [name, type] = name_and_type; + std::optional column_for_offsets; + + if (name_level_for_offsets.has_value()) + { + const auto & part_columns = data_part_info_for_read->getColumnsDescription(); + column_for_offsets = part_columns.getPhysical(name_level_for_offsets->first); + } adjustUpperBound(current_task_last_mark); /// Must go before seek. if (!isContinuousReading(from_mark, column_position)) seekToMark(from_mark, column_position); + /// If we read only offsets we have to read prefix anyway + /// to preserve correctness of serialization. + auto buffer_getter_for_prefix = [&](const auto &) -> ReadBuffer * + { + return data_buffer; + }; + auto buffer_getter = [&](const ISerialization::SubstreamPath & substream_path) -> ReadBuffer * { /// Offset stream from another column could be read, in case of current /// column does not exists (see findColumnForOffsets() in /// MergeTreeReaderCompact::fillColumnPositions()) - bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; - if (only_offsets_level.has_value()) + if (name_level_for_offsets.has_value()) { + bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; if (!is_offsets) return nullptr; @@ -275,7 +289,7 @@ void MergeTreeReaderCompact::readData( /// /// Here only_offsets_level is the level of the alternative stream, /// and substream_path.size() is the level of the current stream. - if (only_offsets_level.value() < ISerialization::getArrayLevel(substream_path)) + if (name_level_for_offsets->second < ISerialization::getArrayLevel(substream_path)) return nullptr; } @@ -284,21 +298,25 @@ void MergeTreeReaderCompact::readData( ISerialization::DeserializeBinaryBulkStatePtr state; ISerialization::DeserializeBinaryBulkSettings deserialize_settings; - deserialize_settings.getter = buffer_getter; deserialize_settings.avg_value_size_hint = avg_value_size_hints[name]; if (name_and_type.isSubcolumn()) { - const auto & type_in_storage = name_and_type.getTypeInStorage(); - const auto & name_in_storage = name_and_type.getNameInStorage(); + NameAndTypePair name_type_in_storage{name_and_type.getNameInStorage(), name_and_type.getTypeInStorage()}; - auto serialization = getSerializationInPart({name_in_storage, type_in_storage}); - ColumnPtr temp_column = type_in_storage->createColumn(*serialization); + /// In case of reading onlys offset use the correct serialization for reading of the prefix + auto serialization = getSerializationInPart(name_type_in_storage); + auto serialization_for_prefix = column_for_offsets ? getSerializationInPart(*column_for_offsets) : serialization; - serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); + ColumnPtr temp_column = name_type_in_storage.type->createColumn(*serialization); + + deserialize_settings.getter = buffer_getter_for_prefix; + serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state); + + deserialize_settings.getter = buffer_getter; serialization->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state, nullptr); - auto subcolumn = type_in_storage->getSubcolumn(name_and_type.getSubcolumnName(), temp_column); + auto subcolumn = name_type_in_storage.type->getSubcolumn(name_and_type.getSubcolumnName(), temp_column); /// TODO: Avoid extra copying. if (column->empty()) @@ -308,13 +326,19 @@ void MergeTreeReaderCompact::readData( } else { + /// In case of reading only offsets use the correct serialization for reading the prefix auto serialization = getSerializationInPart(name_and_type); - serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); + auto serialization_for_prefix = column_for_offsets ? getSerializationInPart(*column_for_offsets) : serialization; + + deserialize_settings.getter = buffer_getter_for_prefix; + serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state); + + deserialize_settings.getter = buffer_getter; serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state, nullptr); } /// The buffer is left in inconsistent state after reading single offsets - if (only_offsets_level.has_value()) + if (name_level_for_offsets.has_value()) last_read_granule.reset(); else last_read_granule.emplace(from_mark, column_position); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index f180d7508f7..ebe76d4f5cd 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -55,9 +55,10 @@ private: /// Positions of columns in part structure. using ColumnPositions = std::vector>; ColumnPositions column_positions; + /// Should we read full column or only it's offsets. /// Element of the vector is the level of the alternative stream. - std::vector> read_only_offsets; + std::vector columns_for_offsets; /// For asynchronous reading from remote fs. Same meaning as in MergeTreeReaderStream. std::optional last_right_offset; @@ -68,8 +69,8 @@ private: void seekToMark(size_t row_index, size_t column_index); void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, size_t from_mark, - size_t current_task_last_mark, size_t column_position, size_t rows_to_read, - std::optional only_offsets_level); + size_t current_task_last_mark, size_t column_position, + size_t rows_to_read, ColumnNameLevel name_level_for_offsets); /// Returns maximal value of granule size in compressed file from @mark_ranges. /// This value is used as size of read buffer. diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index fed8032fb17..ae1740525f7 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -42,7 +42,7 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( { if (auto offsets_position = findColumnForOffsets(column_to_read)) { - positions_for_offsets[column_to_read.name] = offsets_position->first; + positions_for_offsets[column_to_read.name] = *data_part_info_for_read->getColumnPosition(offsets_position->first); partially_read_columns.insert(column_to_read.name); } } diff --git a/tests/queries/0_stateless/02835_nested_array_lowcardinality.reference b/tests/queries/0_stateless/02835_nested_array_lowcardinality.reference new file mode 100644 index 00000000000..5bdb2788cc1 --- /dev/null +++ b/tests/queries/0_stateless/02835_nested_array_lowcardinality.reference @@ -0,0 +1,30 @@ +[] [] +['0'] [''] +['0','1'] ['',''] +['0','1','2'] ['','',''] +['0','1','2','3'] ['','','',''] +['0','1','2','3','4'] ['','','','',''] +['0','1','2','3','4','5'] ['','','','','',''] +['0','1','2','3','4','5','6'] ['','','','','','',''] +['0','1','2','3','4','5','6','7'] ['','','','','','','',''] +['0','1','2','3','4','5','6','7','8'] ['','','','','','','','',''] +[] [] +[[]] [[]] +[[],['0']] [[],['']] +[[],['0'],['0','1']] [[],[''],['','']] +[[],['0'],['0','1'],['0','1','2']] [[],[''],['',''],['','','']] +[[],['0'],['0','1'],['0','1','2'],[]] [[],[''],['',''],['','',''],[]] +[[],['0'],['0','1'],['0','1','2'],[],['0']] [[],[''],['',''],['','',''],[],['']] +[[],['0'],['0','1'],['0','1','2'],[],['0'],['0','1']] [[],[''],['',''],['','',''],[],[''],['','']] +[[],['0'],['0','1'],['0','1','2'],[],['0'],['0','1'],['0','1','2']] [[],[''],['',''],['','',''],[],[''],['',''],['','','']] +[[],['0'],['0','1'],['0','1','2'],[],['0'],['0','1'],['0','1','2'],[]] [[],[''],['',''],['','',''],[],[''],['',''],['','',''],[]] +[] [] +[{}] [{}] +[{},{'k0':0}] [{},{}] +[{},{'k0':0},{'k0':0,'k1':1}] [{},{},{}] +[{},{'k0':0},{'k0':0,'k1':1},{'k0':0,'k1':1,'k2':2}] [{},{},{},{}] +[{},{'k0':0},{'k0':0,'k1':1},{'k0':0,'k1':1,'k2':2},{}] [{},{},{},{},{}] +[{},{'k0':0},{'k0':0,'k1':1},{'k0':0,'k1':1,'k2':2},{},{'k0':0}] [{},{},{},{},{},{}] +[{},{'k0':0},{'k0':0,'k1':1},{'k0':0,'k1':1,'k2':2},{},{'k0':0},{'k0':0,'k1':1}] [{},{},{},{},{},{},{}] +[{},{'k0':0},{'k0':0,'k1':1},{'k0':0,'k1':1,'k2':2},{},{'k0':0},{'k0':0,'k1':1},{'k0':0,'k1':1,'k2':2}] [{},{},{},{},{},{},{},{}] +[{},{'k0':0},{'k0':0,'k1':1},{'k0':0,'k1':1,'k2':2},{},{'k0':0},{'k0':0,'k1':1},{'k0':0,'k1':1,'k2':2},{}] [{},{},{},{},{},{},{},{},{}] diff --git a/tests/queries/0_stateless/02835_nested_array_lowcardinality.sql b/tests/queries/0_stateless/02835_nested_array_lowcardinality.sql new file mode 100644 index 00000000000..36c1eb39cfd --- /dev/null +++ b/tests/queries/0_stateless/02835_nested_array_lowcardinality.sql @@ -0,0 +1,49 @@ +DROP TABLE IF EXISTS cool_table; + +CREATE TABLE IF NOT EXISTS cool_table +( + id UInt64, + n Nested(n UInt64, lc1 LowCardinality(String)) +) +ENGINE = MergeTree +ORDER BY id; + +INSERT INTO cool_table SELECT number, range(number), range(number) FROM numbers(10); + +ALTER TABLE cool_table ADD COLUMN IF NOT EXISTS `n.lc2` Array(LowCardinality(String)); + +SELECT n.lc1, n.lc2 FROM cool_table ORDER BY id; + +DROP TABLE IF EXISTS cool_table; + +CREATE TABLE IF NOT EXISTS cool_table +( + id UInt64, + n Nested(n UInt64, lc1 Array(LowCardinality(String))) +) +ENGINE = MergeTree +ORDER BY id; + +INSERT INTO cool_table SELECT number, range(number), arrayMap(x -> range(x % 4), range(number)) FROM numbers(10); + +ALTER TABLE cool_table ADD COLUMN IF NOT EXISTS `n.lc2` Array(Array(LowCardinality(String))); + +SELECT n.lc1, n.lc2 FROM cool_table ORDER BY id; + +DROP TABLE IF EXISTS cool_table; + +CREATE TABLE IF NOT EXISTS cool_table +( + id UInt64, + n Nested(n UInt64, lc1 Map(LowCardinality(String), UInt64)) +) +ENGINE = MergeTree +ORDER BY id; + +INSERT INTO cool_table SELECT number, range(number), arrayMap(x -> (arrayMap(y -> 'k' || toString(y), range(x % 4)), range(x % 4))::Map(LowCardinality(String), UInt64), range(number)) FROM numbers(10); + +ALTER TABLE cool_table ADD COLUMN IF NOT EXISTS `n.lc2` Array(Map(LowCardinality(String), UInt64)); + +SELECT n.lc1, n.lc2 FROM cool_table ORDER BY id; + +DROP TABLE IF EXISTS cool_table; From 638182ef34e64177246a9b2602f883336edd2705 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 2 Aug 2023 16:54:46 +0000 Subject: [PATCH 1485/2047] Test triggered error in remove redundant sorting during AST fuzzing ... but can't reproduce it locally --- .../02835_fuzz_remove_redundant_sorting.reference | 3 +++ .../0_stateless/02835_fuzz_remove_redundant_sorting.sql | 5 +++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.reference create mode 100644 tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.sql diff --git a/tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.reference b/tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.reference new file mode 100644 index 00000000000..5fda23e0114 --- /dev/null +++ b/tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.reference @@ -0,0 +1,3 @@ +\N + +\N diff --git a/tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.sql b/tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.sql new file mode 100644 index 00000000000..b5fac5a56a7 --- /dev/null +++ b/tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.sql @@ -0,0 +1,5 @@ +DROP TABLE IF EXISTS numbers500k; +CREATE TABLE numbers500k (`number` UInt32) ENGINE = TinyLog; +INSERT INTO numbers500k SELECT number FROM system.numbers LIMIT 500000; +SELECT intDiv(number, NULL) AS k FROM (SELECT * FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) PREWHERE 31 WHERE 65537 ORDER BY number DESC NULLS FIRST) GROUP BY GROUPING SETS ((k)) WITH TOTALS ORDER BY k ASC NULLS LAST LIMIT 2147483648; +DROP TABLE IF EXISTS numbers500k; From 469dd7f30033b4e7870d6999825f8a16fe106e59 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 19:06:31 +0200 Subject: [PATCH 1486/2047] Add the docs --- docs/en/interfaces/cli.md | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 8779dd1a544..36afb94433a 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -323,9 +323,9 @@ clickhouse-client clickhouse://192.168.1.15,192.168.1.25 `clickhouse-client` uses the first existing file of the following: - Defined in the `--config-file` parameter. -- `./clickhouse-client.xml` -- `~/.clickhouse-client/config.xml` -- `/etc/clickhouse-client/config.xml` +- `./clickhouse-client.xml`, `.yaml`, `.yml` +- `~/.clickhouse-client/config.xml`, `.yaml`, `.yml` +- `/etc/clickhouse-client/config.xml`, `.yaml`, `.yml` Example of a config file: @@ -342,6 +342,17 @@ Example of a config file: ``` +Or the same config in a YAML format: + +```yaml +user: username +password: 'password' +secure: true +openSSL: + client: + caConfig: '/etc/ssl/cert.pem' +``` + ### Query ID Format {#query-id-format} In interactive mode `clickhouse-client` shows query ID for every query. By default, the ID is formatted like this: From 6dcdc4cd380c773a9dd135c66cdb79995ba0550a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 2 Aug 2023 15:35:47 +0200 Subject: [PATCH 1487/2047] Remove mark_failed_report_pending, it's already in builds_report_config --- tests/ci/build_check.py | 39 --------------------------------------- 1 file changed, 39 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index eae9efdd548..b75650d6d60 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -9,12 +9,6 @@ import time from typing import List, Tuple from ci_config import CI_CONFIG, BuildConfig -from commit_status_helper import ( - NotSet, - get_commit_filtered_statuses, - get_commit, - post_commit_status, -) from docker_pull_helper import get_image_with_version from env_helper import ( GITHUB_JOB, @@ -24,8 +18,6 @@ from env_helper import ( S3_DOWNLOAD, TEMP_PATH, ) -from get_robot_token import get_best_robot_token -from github_helper import GitHub from pr_info import PRInfo from s3_helper import S3Helper from tee_popen import TeePopen @@ -242,34 +234,6 @@ def upload_master_static_binaries( print(f"::notice ::Binary static URL: {url}") -def mark_failed_reports_pending(build_name: str, pr_info: PRInfo) -> None: - try: - gh = GitHub(get_best_robot_token()) - commit = get_commit(gh, pr_info.sha) - statuses = get_commit_filtered_statuses(commit) - report_status = [ - name - for name, builds in CI_CONFIG["builds_report_config"].items() - if build_name in builds - ][0] - for status in statuses: - if status.context == report_status and status.state in ["failure", "error"]: - logging.info( - "Commit already have failed status for '%s', setting it to 'pending'", - report_status, - ) - post_commit_status( - commit, - "pending", - status.target_url or NotSet, - "Set to pending on rerun", - report_status, - pr_info, - ) - except: # we do not care about any exception here - logging.info("Failed to get or mark the reports status as pending, continue") - - def main(): logging.basicConfig(level=logging.INFO) @@ -300,9 +264,6 @@ def main(): # put them as github actions artifact (result) check_for_success_run(s3_helper, s3_path_prefix, build_name, build_config) - # If it's a latter running, we need to mark possible failed status - mark_failed_reports_pending(build_name, pr_info) - docker_image = get_image_with_version(IMAGES_PATH, IMAGE_NAME) image_version = docker_image.version From 89bb3f3410bbab491eead2adae2b25ffbfe82c70 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 2 Aug 2023 19:25:33 +0200 Subject: [PATCH 1488/2047] fix style-check --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 31a49808edf..ae105aba4c8 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -649,7 +649,7 @@ def test_read_subcolumns(cluster): assert res == "42\tcont/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" - + def test_read_from_not_existing_container(cluster): node = cluster.instances["node"] query = f"select * from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont_not_exists', 'test_table.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto')" From b4f0d0bc2edb5865128e16dd90db00d25df8747e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 21:05:39 +0300 Subject: [PATCH 1489/2047] Update pull_request.yml --- .github/workflows/pull_request.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 1d515633f61..6ec8c003491 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5194,7 +5194,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/sqltest REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME="SQLTest" + CHECK_NAME=SQLTest REPO_COPY=${{runner.temp}}/sqltest/ClickHouse EOF - name: Download json reports From fe09a36f4447e4a2cd141b3cac9fb2b0ef995b16 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 2 Aug 2023 18:07:54 +0000 Subject: [PATCH 1490/2047] fixed fuzzer --- src/Functions/FunctionBinaryArithmetic.h | 28 ++++++++++++++++-------- 1 file changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index dc0bf279e1d..1ce352f0661 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1169,7 +1169,7 @@ class FunctionBinaryArithmetic : public IFunction DataTypes data_types; ColumnsWithTypeAndName new_arguments {num_args}; - DataTypePtr t; + DataTypePtr result_array_type; const auto * left_const = typeid_cast(arguments[0].column.get()); const auto * right_const = typeid_cast(arguments[1].column.get()); @@ -1195,21 +1195,31 @@ class FunctionBinaryArithmetic : public IFunction if (*typeid_cast(arguments[0].column.get())->getOffsets().data() != *typeid_cast(arguments[1].column.get())->getOffsets().data()) { - throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Cannot apply operation for arguments of different sizes. Size of the first argument: {}, size of the second argument: {}", *typeid_cast(arguments[0].column.get())->getOffsets().data(), *typeid_cast(arguments[1].column.get())->getOffsets().data()); } - auto a = typeid_cast(arguments[0].column.get())->getData().getPtr(); - t = typeid_cast(arguments[0].type.get())->getNestedType(); - new_arguments[0] = {a, t, arguments[0].name}; + auto array_ptr = typeid_cast(arguments[0].column.get())->getData().getPtr(); + result_array_type = typeid_cast(arguments[0].type.get())->getNestedType(); + new_arguments[0] = {array_ptr, result_array_type, arguments[0].name}; - a = typeid_cast(arguments[1].column.get())->getData().getPtr(); - t = typeid_cast(arguments[1].type.get())->getNestedType(); - new_arguments[1] = {a, t, arguments[1].name}; + array_ptr = typeid_cast(arguments[1].column.get())->getData().getPtr(); + result_array_type = typeid_cast(arguments[1].type.get())->getNestedType(); + new_arguments[1] = {array_ptr, result_array_type, arguments[1].name}; - auto res = executeImpl(new_arguments, t, input_rows_count); + + result_array_type = typeid_cast(result_type.get())->getNestedType(); + + std::cerr << result_array_type->getName() << std::endl; + + const auto & offsets = typeid_cast(arguments[0].column.get())->getOffsets(); + size_t rows_count = 0; + if (!offsets.empty()) + rows_count = offsets.back(); + auto res = executeImpl(new_arguments, result_array_type, rows_count); + std::cerr << res->dumpStructure() << std::endl; return ColumnArray::create(res, typeid_cast(arguments[0].column.get())->getOffsetsPtr()); } From 0fbd2bb802388278067b2c42af527fff99fbf19f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 2 Aug 2023 20:19:08 +0200 Subject: [PATCH 1491/2047] Update FunctionBinaryArithmetic.h --- src/Functions/FunctionBinaryArithmetic.h | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 1ce352f0661..2dd68d9cc90 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1209,17 +1209,14 @@ class FunctionBinaryArithmetic : public IFunction result_array_type = typeid_cast(arguments[1].type.get())->getNestedType(); new_arguments[1] = {array_ptr, result_array_type, arguments[1].name}; - result_array_type = typeid_cast(result_type.get())->getNestedType(); - std::cerr << result_array_type->getName() << std::endl; - const auto & offsets = typeid_cast(arguments[0].column.get())->getOffsets(); size_t rows_count = 0; if (!offsets.empty()) rows_count = offsets.back(); auto res = executeImpl(new_arguments, result_array_type, rows_count); - std::cerr << res->dumpStructure() << std::endl; + return ColumnArray::create(res, typeid_cast(arguments[0].column.get())->getOffsetsPtr()); } From d2a264c9bb4e70137dab30dbe0493dba7518d746 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 2 Aug 2023 11:05:21 +0200 Subject: [PATCH 1492/2047] tests: use explicit address for console to avoid EADDRINUSE The problem is that MinIO uses dynamic port for console address by default, which may be below ip_local_port_range, and in this case it ca be reused in a short time, sicne first MinIO allocates the socket, then close this socket, and only after try to bind to it. And even though this is a problem of MinIO I'm not a go developer to fix it. v2: use long notation of the 127.0.0.1 (that version of MinIO on CI cannot handle 127.1, while 2023-07-21T21-12-44Z can) Signed-off-by: Azat Khuzhin --- docker/test/integration/runner/compose/docker_compose_minio.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/runner/compose/docker_compose_minio.yml b/docker/test/integration/runner/compose/docker_compose_minio.yml index 3eaf891ff8e..f2979566296 100644 --- a/docker/test/integration/runner/compose/docker_compose_minio.yml +++ b/docker/test/integration/runner/compose/docker_compose_minio.yml @@ -14,7 +14,7 @@ services: MINIO_ACCESS_KEY: minio MINIO_SECRET_KEY: minio123 MINIO_PROMETHEUS_AUTH_TYPE: public - command: server --address :9001 --certs-dir /certs /data1-1 + command: server --console-address 127.0.0.1:19001 --address :9001 --certs-dir /certs /data1-1 depends_on: - proxy1 - proxy2 From 344d648caba44f867eb3c528ac09237b2dd9182e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 2 Aug 2023 11:08:47 +0200 Subject: [PATCH 1493/2047] tests: adjust net.ipv4.ip_local_port_range for some services to avoid EADDRINUSE As it turns out, docker does not pass through the sysctls, so adjust this for know users of unprivileged ports (>32K): - HDFS - kafka Signed-off-by: Azat Khuzhin --- .../integration/runner/compose/docker_compose_hdfs.yml | 2 ++ .../integration/runner/compose/docker_compose_kafka.yml | 2 ++ .../runner/compose/docker_compose_kerberized_hdfs.yml | 4 ++++ .../runner/compose/docker_compose_kerberized_kafka.yml | 2 ++ tests/integration/conftest.py | 7 +------ 5 files changed, 11 insertions(+), 6 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_hdfs.yml b/docker/test/integration/runner/compose/docker_compose_hdfs.yml index f83eb93fea7..1cae54ad9e1 100644 --- a/docker/test/integration/runner/compose/docker_compose_hdfs.yml +++ b/docker/test/integration/runner/compose/docker_compose_hdfs.yml @@ -12,3 +12,5 @@ services: - type: ${HDFS_FS:-tmpfs} source: ${HDFS_LOGS:-} target: /usr/local/hadoop/logs + sysctls: + net.ipv4.ip_local_port_range: '55000 65535' diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 30d1b0bed3f..4ae3de3cbc7 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -31,6 +31,8 @@ services: - kafka_zookeeper security_opt: - label:disable + sysctls: + net.ipv4.ip_local_port_range: '55000 65535' schema-registry: image: confluentinc/cp-schema-registry:5.2.0 diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml index 1160192696d..e955a14eb3d 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml @@ -20,6 +20,8 @@ services: depends_on: - hdfskerberos entrypoint: /etc/bootstrap.sh -d + sysctls: + net.ipv4.ip_local_port_range: '55000 65535' hdfskerberos: image: clickhouse/kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG:-latest} @@ -29,3 +31,5 @@ services: - ${KERBERIZED_HDFS_DIR}/../../kerberos_image_config.sh:/config.sh - /dev/urandom:/dev/random expose: [88, 749] + sysctls: + net.ipv4.ip_local_port_range: '55000 65535' diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml index 86e920ff573..49d4c1db90f 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -48,6 +48,8 @@ services: - kafka_kerberos security_opt: - label:disable + sysctls: + net.ipv4.ip_local_port_range: '55000 65535' kafka_kerberos: image: clickhouse/kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG:-latest} diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index b5f7aababc9..cd69861df3f 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -17,14 +17,9 @@ def tune_local_port_range(): # Lots of services uses non privileged ports: # - hdfs -- 50020/50070/... # - minio - # - mysql - # - psql - # - # So instead of tuning all these thirdparty services, let's simply - # prohibit using such ports for outgoing connections, this should fix - # possible "Address already in use" errors. # # NOTE: 5K is not enough, and sometimes leads to EADDRNOTAVAIL error. + # NOTE: it is not inherited, so you may need to specify this in docker_compose_$SERVICE.yml run_and_check(["sysctl net.ipv4.ip_local_port_range='55000 65535'"], shell=True) From cd74da4c032a6134d484fb7cbab863992e3a999c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 2 Aug 2023 21:21:40 +0200 Subject: [PATCH 1494/2047] Remove unused code in StorageSystemStackTrace This columns mask is actually useless, since the main thing is done without out, in particular: - detect does the signal should be sent to thread - does the thread name should be read for this thread And this cannot be done with columns mask, because multiple columns depends on signals and thread names. Signed-off-by: Azat Khuzhin --- src/Storages/System/StorageSystemStackTrace.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index 887d7f2a5d4..8d703632c68 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -275,15 +275,6 @@ Pipe StorageSystemStackTrace::read( Block sample_block = storage_snapshot->metadata->getSampleBlock(); - std::vector columns_mask(sample_block.columns()); - for (size_t i = 0, size = columns_mask.size(); i < size; ++i) - { - if (names_set.contains(sample_block.getByPosition(i).name)) - { - columns_mask[i] = 1; - } - } - bool send_signal = names_set.contains("trace") || names_set.contains("query_id"); bool read_thread_names = names_set.contains("thread_name"); From 471296c098f42b7a6fa3ed59289ac5c3659c3ecb Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 2 Aug 2023 22:20:03 +0200 Subject: [PATCH 1495/2047] Update 02174_cte_scalar_cache.sql --- .../queries/0_stateless/02174_cte_scalar_cache.sql | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02174_cte_scalar_cache.sql b/tests/queries/0_stateless/02174_cte_scalar_cache.sql index 50a10834e64..86cfff21446 100644 --- a/tests/queries/0_stateless/02174_cte_scalar_cache.sql +++ b/tests/queries/0_stateless/02174_cte_scalar_cache.sql @@ -6,7 +6,7 @@ WITH ( SELECT sleep(0.0001) FROM system.one ) as a3, ( SELECT sleep(0.0001) FROM system.one ) as a4, ( SELECT sleep(0.0001) FROM system.one ) as a5 -SELECT '02177_CTE_GLOBAL_ON', a5 FROM system.numbers LIMIT 100 +SELECT '02177_CTE_GLOBAL_ON', a1, a2, a3, a4, a5 FROM system.numbers LIMIT 100 FORMAT Null SETTINGS enable_global_with_statement = 1; @@ -16,17 +16,17 @@ WITH ( SELECT sleep(0.0001) FROM system.one ) as a3, ( SELECT sleep(0.0001) FROM system.one ) as a4, ( SELECT sleep(0.0001) FROM system.one ) as a5 -SELECT '02177_CTE_GLOBAL_OFF', a5 FROM system.numbers LIMIT 100 +SELECT '02177_CTE_GLOBAL_OFF', a1, a2, a3, a4, a5 FROM system.numbers LIMIT 100 FORMAT Null SETTINGS enable_global_with_statement = 0; WITH - ( SELECT sleep(0.0001) FROM system.one ), - ( SELECT sleep(0.0001) FROM system.one ), - ( SELECT sleep(0.0001) FROM system.one ), - ( SELECT sleep(0.0001) FROM system.one ), + ( SELECT sleep(0.0001) FROM system.one ) as a1, + ( SELECT sleep(0.0001) FROM system.one ) as a2, + ( SELECT sleep(0.0001) FROM system.one ) as a3, + ( SELECT sleep(0.0001) FROM system.one ) as a4, ( SELECT sleep(0.0001) FROM system.one ) as a5 -SELECT '02177_CTE_NEW_ANALYZER', a5 FROM system.numbers LIMIT 100 +SELECT '02177_CTE_NEW_ANALYZER', a1, a2, a3, a4, a5 FROM system.numbers LIMIT 100 FORMAT Null SETTINGS allow_experimental_analyzer = 1; From 3f4faeab0b8e737767a508c24934ab6af242fa6a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 00:05:56 +0300 Subject: [PATCH 1496/2047] Update TableFunctionRemote.cpp --- src/TableFunctions/TableFunctionRemote.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 6c3622d6e33..fc5b021bf2a 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -114,7 +114,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr /// /// clusterAllReplicas() - same as cluster() - if ((!is_cluster_function && args.size() < 1) || args.size() > max_args) + if ((!is_cluster_function && args.empty()) || args.size() > max_args) throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); size_t arg_num = 0; @@ -133,7 +133,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (is_cluster_function) { - if (args.size() > 0) + if (!args.empty()) { args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); cluster_name = checkAndGetLiteralArgument(args[arg_num], "cluster_name"); From dbf14b5e2dd96bd8f76f6db9671353afa52c20be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 23:16:06 +0200 Subject: [PATCH 1497/2047] Fix wrong error code "BAD_GET" --- src/Common/ErrorCodes.cpp | 3 ++- src/Interpreters/Context.cpp | 6 +++--- src/TableFunctions/ITableFunctionCluster.h | 4 ++-- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 393486f805c..a3277821111 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -582,7 +582,8 @@ M(697, CANNOT_RESTORE_TO_NONENCRYPTED_DISK) \ M(698, INVALID_REDIS_STORAGE_TYPE) \ M(699, INVALID_REDIS_TABLE_STRUCTURE) \ - M(700, USER_SESSION_LIMIT_EXCEEDED) \ + M(700, USER_SESSION_LIMIT_EXCEEDED) \ + M(701, CLUSTER_DOESNT_EXIST) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a5ff7643294..8eba4a9c3a5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -166,7 +166,6 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int BAD_GET; extern const int UNKNOWN_DATABASE; extern const int UNKNOWN_TABLE; extern const int TABLE_ALREADY_EXISTS; @@ -181,6 +180,7 @@ namespace ErrorCodes extern const int UNKNOWN_FUNCTION; extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; + extern const int CLUSTER_DOESNT_EXIST; } #define SHUTDOWN(log, desc, ptr, method) do \ @@ -3073,7 +3073,7 @@ UInt16 Context::getServerPort(const String & port_name) const { auto it = shared->server_ports.find(port_name); if (it == shared->server_ports.end()) - throw Exception(ErrorCodes::BAD_GET, "There is no port named {}", port_name); + throw Exception(ErrorCodes::CLUSTER_DOESNT_EXIST, "There is no port named {}", port_name); else return it->second; } @@ -3082,7 +3082,7 @@ std::shared_ptr Context::getCluster(const std::string & cluster_name) c { if (auto res = tryGetCluster(cluster_name)) return res; - throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", cluster_name); + throw Exception(ErrorCodes::CLUSTER_DOESNT_EXIST, "Requested cluster '{}' not found", cluster_name); } diff --git a/src/TableFunctions/ITableFunctionCluster.h b/src/TableFunctions/ITableFunctionCluster.h index a8329684ee6..7e81d6d21b7 100644 --- a/src/TableFunctions/ITableFunctionCluster.h +++ b/src/TableFunctions/ITableFunctionCluster.h @@ -17,7 +17,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_GET; + extern const int CLUSTER_DOESNT_EXIST; extern const int LOGICAL_ERROR; } @@ -59,7 +59,7 @@ protected: cluster_name = checkAndGetLiteralArgument(args[0], "cluster_name"); if (!context->tryGetCluster(cluster_name)) - throw Exception(ErrorCodes::BAD_GET, "Requested cluster '{}' not found", cluster_name); + throw Exception(ErrorCodes::CLUSTER_DOESNT_EXIST, "Requested cluster '{}' not found", cluster_name); /// Just cut the first arg (cluster_name) and try to parse other table function arguments as is args.erase(args.begin()); From a915886b87ad053ca40155e53426088f91d21408 Mon Sep 17 00:00:00 2001 From: hendrik-m Date: Wed, 2 Aug 2023 23:16:45 +0200 Subject: [PATCH 1498/2047] adapt documentation and change all references of `clickhouse local` to `clickhouse-local` --- .../operations/utilities/clickhouse-local.md | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index 0443a80cf17..9c462a04a1d 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -34,21 +34,27 @@ The binary you just downloaded can run all sorts of ClickHouse tools and utiliti A common use of `clickhouse-local` is to run ad-hoc queries on files: where you don't have to insert the data into a table. `clickhouse-local` can stream the data from a file into a temporary table and execute your SQL. -If the file is sitting on the same machine as `clickhouse-local`, use the `file` table engine. The following `reviews.tsv` file contains a sampling of Amazon product reviews: +If the file is sitting on the same machine as `clickhouse-local`, you can simple specify the file to load. The following `reviews.tsv` file contains a sampling of Amazon product reviews: ```bash -./clickhouse local -q "SELECT * FROM file('reviews.tsv')" +./clickhouse-local -q "SELECT * FROM 'reviews.tsv'" +``` + +This command is a shortcut of: + +```bash +./clickhouse-local -q "SELECT * FROM file('reviews.tsv')" ``` ClickHouse knows the file uses a tab-separated format from filename extension. If you need to explicitly specify the format, simply add one of the [many ClickHouse input formats](../../interfaces/formats.md): ```bash -./clickhouse local -q "SELECT * FROM file('reviews.tsv', 'TabSeparated')" +./clickhouse-local -q "SELECT * FROM file('reviews.tsv', 'TabSeparated')" ``` The `file` table function creates a table, and you can use `DESCRIBE` to see the inferred schema: ```bash -./clickhouse local -q "DESCRIBE file('reviews.tsv')" +./clickhouse-local -q "DESCRIBE file('reviews.tsv')" ``` ```response @@ -72,7 +78,7 @@ review_date Nullable(Date) Let's find a product with the highest rating: ```bash -./clickhouse local -q "SELECT +./clickhouse-local -q "SELECT argMax(product_title,star_rating), max(star_rating) FROM file('reviews.tsv')" @@ -87,7 +93,7 @@ Monopoly Junior Board Game 5 If you have a file in S3, use `clickhouse-local` and the `s3` table function to query the file in place (without inserting the data into a ClickHouse table). We have a file named `house_0.parquet` in a public bucket that contains home prices of property sold in the United Kingdom. Let's see how many rows it has: ```bash -./clickhouse local -q " +./clickhouse-local -q " SELECT count() FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/house_parquet/house_0.parquet')" ``` @@ -101,7 +107,7 @@ The file has 2.7M rows: It's always useful to see what the inferred schema that ClickHouse determines from the file: ```bash -./clickhouse local -q "DESCRIBE s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/house_parquet/house_0.parquet')" +./clickhouse-local -q "DESCRIBE s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/house_parquet/house_0.parquet')" ``` ```response @@ -124,7 +130,7 @@ county Nullable(String) Let's see what the most expensive neighborhoods are: ```bash -./clickhouse local -q " +./clickhouse-local -q " SELECT town, district, @@ -171,7 +177,7 @@ $ clickhouse-local --structure "table_structure" --input-format "format_of_incom Basic usage (Mac): ``` bash -$ ./clickhouse local --structure "table_structure" --input-format "format_of_incoming_data" --query "query" +$ ./clickhouse-local --structure "table_structure" --input-format "format_of_incoming_data" --query "query" ``` :::note From aaf3eb4335c2c92d463e719e40ae2b1228a82486 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 23:18:31 +0200 Subject: [PATCH 1499/2047] Fix tests --- tests/queries/0_stateless/02725_database_hdfs.reference | 1 - tests/queries/0_stateless/02725_database_hdfs.sh | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02725_database_hdfs.reference b/tests/queries/0_stateless/02725_database_hdfs.reference index 9f63e757b27..932a0786921 100644 --- a/tests/queries/0_stateless/02725_database_hdfs.reference +++ b/tests/queries/0_stateless/02725_database_hdfs.reference @@ -5,7 +5,6 @@ test1 test2 Test 2: check exceptions BAD_ARGUMENTS -CANNOT_EXTRACT_TABLE_STRUCTURE BAD_ARGUMENTS BAD_ARGUMENTS CANNOT_EXTRACT_TABLE_STRUCTURE diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index dab2d37605c..c258042a917 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -38,14 +38,14 @@ echo "Test 2: check exceptions" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = HDFS('abacaba'); -""" | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test4; CREATE DATABASE test4 ENGINE = HDFS; USE test4; SELECT * FROM \"abacaba/file.tsv\" -""" | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" +""" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" ${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" From 5ec6a4695f2911758e8d64ee52e34149c4940112 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 23:22:15 +0200 Subject: [PATCH 1500/2047] Fix log paths --- tests/ci/sqltest.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index b752d4e4aee..7385716850f 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,11 +110,10 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "main.log": os.path.join(workspace_path, "main.log"), - "server.log.zst": os.path.join(workspace_path, "server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"), - "report.html": os.path.join(workspace_path, "report.html"), - "test.log": os.path.join(workspace_path, "test.log"), + "server.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.err.log.zst"), + "report.html": os.path.join(workspace_path, "/report.html"), + "test.log": os.path.join(workspace_path, "/test.log"), } s3_helper = S3Helper() From 48d736f8fdd7bfac6581de7702527445fd7b05bd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Aug 2023 22:34:18 +0200 Subject: [PATCH 1501/2047] select only required columns from system.databases --- src/Storages/SelectQueryInfo.h | 3 ++ src/Storages/System/IStorageSystemOneBlock.h | 13 +++++++- src/Storages/System/StorageSystemColumns.cpp | 17 ++-------- .../StorageSystemDataSkippingIndices.cpp | 15 ++------- .../System/StorageSystemDatabases.cpp | 24 ++++++++++---- src/Storages/System/StorageSystemDatabases.h | 2 ++ .../System/StorageSystemDetachedParts.cpp | 33 +++++-------------- .../System/StorageSystemPartsBase.cpp | 16 ++------- src/Storages/System/StorageSystemTables.cpp | 17 ++-------- .../System/getQueriedColumnsMaskAndHeader.cpp | 24 ++++++++++++++ .../System/getQueriedColumnsMaskAndHeader.h | 11 +++++++ 11 files changed, 86 insertions(+), 89 deletions(-) create mode 100644 src/Storages/System/getQueriedColumnsMaskAndHeader.cpp create mode 100644 src/Storages/System/getQueriedColumnsMaskAndHeader.h diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 0f75562e0c1..8a3451e0662 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -262,6 +262,9 @@ struct SelectQueryInfo // If limit is not 0, that means it's a trivial limit query. UInt64 limit = 0; + /// For IStorageSystemOneBlock + std::vector columns_mask; + InputOrderInfoPtr getInputOrderInfo() const { return input_order_info ? input_order_info : (projection ? projection->input_order_info : nullptr); diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 63b9a443f95..e09b27adf32 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include @@ -30,6 +32,8 @@ class IStorageSystemOneBlock : public IStorage protected: virtual void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const = 0; + virtual bool supportsColumnsMask() const { return false; } + public: explicit IStorageSystemOneBlock(const StorageID & table_id_) : IStorage(table_id_) { @@ -48,8 +52,15 @@ public: size_t /*num_streams*/) override { storage_snapshot->check(column_names); - Block sample_block = storage_snapshot->metadata->getSampleBlockWithVirtuals(getVirtuals()); + + if (supportsColumnsMask()) + { + auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); + query_info.columns_mask = std::move(columns_mask); + sample_block = std::move(header); + } + MutableColumns res_columns = sample_block.cloneEmptyColumns(); fillData(res_columns, context, query_info); diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index e4ca6a15138..b76ad07abeb 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -315,23 +316,9 @@ Pipe StorageSystemColumns::read( const size_t /*num_streams*/) { storage_snapshot->check(column_names); - - /// Create a mask of what columns are needed in the result. - - NameSet names_set(column_names.begin(), column_names.end()); - Block sample_block = storage_snapshot->metadata->getSampleBlock(); - Block header; - std::vector columns_mask(sample_block.columns()); - for (size_t i = 0, size = columns_mask.size(); i < size; ++i) - { - if (names_set.contains(sample_block.getByPosition(i).name)) - { - columns_mask[i] = 1; - header.insert(sample_block.getByPosition(i)); - } - } + auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); Block block_to_filter; Storages storages; diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index 2649cf71182..0c4eb197efd 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -185,21 +186,9 @@ Pipe StorageSystemDataSkippingIndices::read( size_t /* num_streams */) { storage_snapshot->check(column_names); - - NameSet names_set(column_names.begin(), column_names.end()); - Block sample_block = storage_snapshot->metadata->getSampleBlock(); - Block header; - std::vector columns_mask(sample_block.columns()); - for (size_t i = 0, size = columns_mask.size(); i < size; ++i) - { - if (names_set.contains(sample_block.getByPosition(i).name)) - { - columns_mask[i] = 1; - header.insert(sample_block.getByPosition(i)); - } - } + auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); MutableColumnPtr column = ColumnString::create(); diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp index 2fcc91e49bb..1fa94fab7bf 100644 --- a/src/Storages/System/StorageSystemDatabases.cpp +++ b/src/Storages/System/StorageSystemDatabases.cpp @@ -117,13 +117,23 @@ void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr c const auto & database = databases.at(database_name); - res_columns[0]->insert(database_name); - res_columns[1]->insert(database->getEngineName()); - res_columns[2]->insert(context->getPath() + database->getDataPath()); - res_columns[3]->insert(database->getMetadataPath()); - res_columns[4]->insert(database->getUUID()); - res_columns[5]->insert(getEngineFull(context, database)); - res_columns[6]->insert(database->getDatabaseComment()); + size_t src_index = 0; + size_t res_index = 0; + const auto & columns_mask = query_info.columns_mask; + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database_name); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database->getEngineName()); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(context->getPath() + database->getDataPath()); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database->getMetadataPath()); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database->getUUID()); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(getEngineFull(context, database)); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(database->getDatabaseComment()); } } diff --git a/src/Storages/System/StorageSystemDatabases.h b/src/Storages/System/StorageSystemDatabases.h index 37c5f97d497..29dd786ca0a 100644 --- a/src/Storages/System/StorageSystemDatabases.h +++ b/src/Storages/System/StorageSystemDatabases.h @@ -26,6 +26,8 @@ public: protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; + bool supportsColumnsMask() const override { return true; } + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; }; diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 97af4094e42..c5d2ba94e09 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -81,13 +82,11 @@ struct WorkerState class DetachedPartsSource : public ISource { public: - DetachedPartsSource(Block header_, std::shared_ptr state_, std::vector columns_mask_, UInt64 block_size_, - bool has_bytes_on_disk_column_) + DetachedPartsSource(Block header_, std::shared_ptr state_, std::vector columns_mask_, UInt64 block_size_) : ISource(std::move(header_)) , state(state_) , columns_mask(std::move(columns_mask_)) , block_size(block_size_) - , has_bytes_on_disk_column(has_bytes_on_disk_column_) {} String getName() const override { return "DataPartsSource"; } @@ -127,7 +126,6 @@ private: std::shared_ptr state; const std::vector columns_mask; const UInt64 block_size; - const bool has_bytes_on_disk_column; const size_t support_threads = 35; StoragesInfo current_info; @@ -149,9 +147,6 @@ private: void calculatePartSizeOnDisk(size_t begin, std::vector> & parts_sizes) { - if (!has_bytes_on_disk_column) - return; - WorkerState worker_state; for (auto p_id = begin; p_id < detached_parts.size(); ++p_id) @@ -211,7 +206,9 @@ private: auto begin = detached_parts.size() - rows; std::vector> parts_sizes(rows); - calculatePartSizeOnDisk(begin, parts_sizes); + constexpr size_t bytes_on_disk_col_idx = 4; + if (columns_mask[bytes_on_disk_col_idx]) + calculatePartSizeOnDisk(begin, parts_sizes); for (auto p_id = begin; p_id < detached_parts.size(); ++p_id) { @@ -229,7 +226,7 @@ private: new_columns[res_index++]->insert(p.dir_name); if (columns_mask[src_index++]) { - chassert(has_bytes_on_disk_column); + chassert(src_index - 1 == bytes_on_disk_col_idx); size_t bytes_on_disk = parts_sizes.at(p_id - begin).load(); new_columns[res_index++]->insert(bytes_on_disk); } @@ -285,21 +282,7 @@ Pipe StorageSystemDetachedParts::read( storage_snapshot->check(column_names); Block sample_block = storage_snapshot->metadata->getSampleBlock(); - NameSet names_set(column_names.begin(), column_names.end()); - - Block header; - std::vector columns_mask(sample_block.columns()); - - for (size_t i = 0; i < columns_mask.size(); ++i) - { - if (names_set.contains(sample_block.getByPosition(i).name)) - { - columns_mask[i] = 1; - header.insert(sample_block.getByPosition(i)); - } - } - - bool has_bytes_on_disk_column = names_set.contains("bytes_on_disk"); + auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names); auto state = std::make_shared(StoragesInfoStream(query_info, context)); @@ -307,7 +290,7 @@ Pipe StorageSystemDetachedParts::read( for (size_t i = 0; i < num_streams; ++i) { - auto source = std::make_shared(header.cloneEmpty(), state, columns_mask, max_block_size, has_bytes_on_disk_column); + auto source = std::make_shared(header.cloneEmpty(), state, columns_mask, max_block_size); pipe.addSource(std::move(source)); } diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 0979b9d9371..513af6cfc46 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -254,21 +255,10 @@ Pipe StorageSystemPartsBase::read( StoragesInfoStream stream(query_info, context); /// Create the result. - - NameSet names_set(column_names.begin(), column_names.end()); - Block sample = storage_snapshot->metadata->getSampleBlock(); - Block header; - std::vector columns_mask(sample.columns()); - for (size_t i = 0; i < sample.columns(); ++i) - { - if (names_set.contains(sample.getByPosition(i).name)) - { - columns_mask[i] = 1; - header.insert(sample.getByPosition(i)); - } - } + auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample, column_names); + MutableColumns res_columns = header.cloneEmptyColumns(); if (has_state_column) res_columns.push_back(ColumnString::create()); diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index e00d2d95568..60dfc3a75e8 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -587,23 +588,9 @@ Pipe StorageSystemTables::read( const size_t /*num_streams*/) { storage_snapshot->check(column_names); - - /// Create a mask of what columns are needed in the result. - - NameSet names_set(column_names.begin(), column_names.end()); - Block sample_block = storage_snapshot->metadata->getSampleBlock(); - Block res_block; - std::vector columns_mask(sample_block.columns()); - for (size_t i = 0, size = columns_mask.size(); i < size; ++i) - { - if (names_set.contains(sample_block.getByPosition(i).name)) - { - columns_mask[i] = 1; - res_block.insert(sample_block.getByPosition(i)); - } - } + auto [columns_mask, res_block] = getQueriedColumnsMaskAndHeader(sample_block, column_names); ColumnPtr filtered_databases_column = getFilteredDatabases(query_info, context); ColumnPtr filtered_tables_column = getFilteredTables(query_info.query, filtered_databases_column, context); diff --git a/src/Storages/System/getQueriedColumnsMaskAndHeader.cpp b/src/Storages/System/getQueriedColumnsMaskAndHeader.cpp new file mode 100644 index 00000000000..c29ccb590ed --- /dev/null +++ b/src/Storages/System/getQueriedColumnsMaskAndHeader.cpp @@ -0,0 +1,24 @@ +#include + +namespace DB +{ + +std::pair, Block> getQueriedColumnsMaskAndHeader(const Block & sample_block, const Names & column_names) +{ + std::vector columns_mask(sample_block.columns()); + Block header; + + NameSet names_set(column_names.begin(), column_names.end()); + for (size_t i = 0; i < columns_mask.size(); ++i) + { + if (names_set.contains(sample_block.getByPosition(i).name)) + { + columns_mask[i] = 1; + header.insert(sample_block.getByPosition(i)); + } + } + + return std::make_pair(columns_mask, header); +} + +} diff --git a/src/Storages/System/getQueriedColumnsMaskAndHeader.h b/src/Storages/System/getQueriedColumnsMaskAndHeader.h new file mode 100644 index 00000000000..0781a92fa60 --- /dev/null +++ b/src/Storages/System/getQueriedColumnsMaskAndHeader.h @@ -0,0 +1,11 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +std::pair, Block> getQueriedColumnsMaskAndHeader(const Block & sample_block, const Names & column_names); + +} From c4fb0a0cd520faeb61ac0cdb9b834062a558b1b4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Aug 2023 23:36:54 +0200 Subject: [PATCH 1502/2047] fix two minor issue --- src/Databases/DatabaseFactory.cpp | 3 --- src/Interpreters/loadMetadata.cpp | 3 +++ tests/queries/0_stateless/01114_database_atomic.reference | 1 + tests/queries/0_stateless/01114_database_atomic.sh | 2 ++ 4 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 9d90c61bb41..53d5245770e 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -107,9 +107,6 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m { cckMetadataPathForOrdinary(create, metadata_path); - /// Creates store/xxx/ for Atomic - fs::create_directories(fs::path(metadata_path).parent_path()); - DatabasePtr impl = getImpl(create, metadata_path, context); if (impl && context->hasQueryContext() && context->getSettingsRef().log_queries) diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 83af2684322..105603e1f04 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -250,6 +250,9 @@ static void loadSystemDatabaseImpl(ContextMutablePtr context, const String & dat { String path = context->getPath() + "metadata/" + database_name; String metadata_file = path + ".sql"; + if (fs::exists(metadata_file + ".tmp")) + fs::remove(metadata_file + ".tmp"); + if (fs::exists(fs::path(metadata_file))) { /// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted. diff --git a/tests/queries/0_stateless/01114_database_atomic.reference b/tests/queries/0_stateless/01114_database_atomic.reference index 10a39087c57..93e89e3a2ec 100644 --- a/tests/queries/0_stateless/01114_database_atomic.reference +++ b/tests/queries/0_stateless/01114_database_atomic.reference @@ -1,3 +1,4 @@ +2 CREATE DATABASE test_01114_1\nENGINE = Atomic CREATE DATABASE test_01114_2\nENGINE = Atomic CREATE DATABASE test_01114_3\nENGINE = Ordinary diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index decbe136fc4..1b295e5f36f 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -13,6 +13,8 @@ DROP DATABASE IF EXISTS test_01114_2; DROP DATABASE IF EXISTS test_01114_3; " +$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Ordinary" 2>&1| grep -Fac "UNKNOWN_DATABASE_ENGINE" + $CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_2" $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -q "CREATE DATABASE test_01114_3 ENGINE=Ordinary" From 619af0c2c66255e8c8aa4f8fd62b96c94fe35ee3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 2 Aug 2023 21:46:23 +0000 Subject: [PATCH 1503/2047] Automatic style fix --- tests/ci/sqltest.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 7385716850f..498448a7928 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,8 +110,12 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "server.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.err.log.zst"), + "server.log.zst": os.path.join( + workspace_path, "/var/log/clickhouse-server/server.log.zst" + ), + "server.err.log.zst": os.path.join( + workspace_path, "/var/log/clickhouse-server/server.err.log.zst" + ), "report.html": os.path.join(workspace_path, "/report.html"), "test.log": os.path.join(workspace_path, "/test.log"), } From 99c7f7b48ca61af6077fd99431194bde24f761b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 23:50:19 +0200 Subject: [PATCH 1504/2047] Fix the test --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index d1c5a40c04f..66d3df8829e 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -1,11 +1,11 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -pushd "${CLICKHOUSE_TMP}" || exit > /dev/null +pushd "${CLICKHOUSE_TMP}" > /dev/null || exit echo "max_block_size: 31337" > clickhouse-client.yaml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" @@ -19,4 +19,4 @@ echo "31339" > clickho ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.xml -popd || exit > /dev/null +popd > /dev/null || exit From bde87997155abcb3014b67d78bfde0a29905a5ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 00:16:04 +0200 Subject: [PATCH 1505/2047] Fix the test --- tests/queries/0_stateless/02012_get_server_port.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02012_get_server_port.sql b/tests/queries/0_stateless/02012_get_server_port.sql index cc7fecb0bf0..2cf2014cfcc 100644 --- a/tests/queries/0_stateless/02012_get_server_port.sql +++ b/tests/queries/0_stateless/02012_get_server_port.sql @@ -1,3 +1,3 @@ select getServerPort('tcp_port'); -select getServerPort('unknown'); -- { serverError 170 } +select getServerPort('unknown'); -- { serverError CLUSTER_DOESNT_EXIST } From 48b1a0b44f85034d53245d9dd941929378f10d83 Mon Sep 17 00:00:00 2001 From: Jiyoung Yoo Date: Mon, 31 Jul 2023 08:09:57 -0700 Subject: [PATCH 1506/2047] Fix for issue #40220 improve error message for table function remote --- src/TableFunctions/TableFunctionRemote.cpp | 13 +++++++++++-- .../02841_remote_parameter_parsing_error.reference | 0 .../02841_remote_parameter_parsing_error.sql | 14 ++++++++++++++ 3 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index e4442c565df..d88d4190191 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -159,7 +159,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name is missing"); } else { @@ -211,7 +211,16 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr } if (arg_num < args.size()) - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + { + if (is_cluster_function) + { + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); + } + } } if (!cluster_name.empty()) diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql new file mode 100644 index 00000000000..f2ac88b62fb --- /dev/null +++ b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql @@ -0,0 +1,14 @@ +-- Tags: shard + + +select * from remote('127.0.0.1'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', sys); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system.o); -- { serverError UNKNOWN_TABLE } +select * from remote('127.0.0.1', system.one, default); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, ''); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, default, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', '', key1); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, '', key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', pwd, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system, one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From dbaa2038b534c4d4760eb3659f7ec6f144cc3cb0 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 3 Aug 2023 02:40:06 +0200 Subject: [PATCH 1507/2047] Fix HTTP Header Filtering --- src/Common/HTTPHeaderFilter.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Common/HTTPHeaderFilter.cpp b/src/Common/HTTPHeaderFilter.cpp index d2dc9452bff..c75050fae8f 100644 --- a/src/Common/HTTPHeaderFilter.cpp +++ b/src/Common/HTTPHeaderFilter.cpp @@ -33,6 +33,9 @@ void HTTPHeaderFilter::setValuesFromConfig(const Poco::Util::AbstractConfigurati { std::lock_guard guard(mutex); + forbidden_headers.clear(); + forbidden_headers_regexp.clear(); + if (config.has("http_forbid_headers")) { std::vector keys; @@ -46,11 +49,6 @@ void HTTPHeaderFilter::setValuesFromConfig(const Poco::Util::AbstractConfigurati forbidden_headers.insert(config.getString("http_forbid_headers." + key)); } } - else - { - forbidden_headers.clear(); - forbidden_headers_regexp.clear(); - } } } From b9596f0e208ffec5fbeb325f99307c2b039b096c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 3 Aug 2023 03:01:33 +0200 Subject: [PATCH 1508/2047] Fix style --- src/Common/HTTPHeaderFilter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/HTTPHeaderFilter.cpp b/src/Common/HTTPHeaderFilter.cpp index c75050fae8f..622bc475d33 100644 --- a/src/Common/HTTPHeaderFilter.cpp +++ b/src/Common/HTTPHeaderFilter.cpp @@ -35,7 +35,7 @@ void HTTPHeaderFilter::setValuesFromConfig(const Poco::Util::AbstractConfigurati forbidden_headers.clear(); forbidden_headers_regexp.clear(); - + if (config.has("http_forbid_headers")) { std::vector keys; From b338e2fcecfddd9f690176e40482fa2e483556f6 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 3 Aug 2023 02:45:37 +0000 Subject: [PATCH 1509/2047] Added peak_memory_usage to final progress --- src/Common/ProgressIndication.cpp | 4 ++++ tests/queries/0_stateless/01921_test_progress_bar.py | 1 + 2 files changed, 5 insertions(+) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 5a1929d4ec2..e9e923c50d7 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -101,6 +101,10 @@ void ProgressIndication::writeFinalProgress() << formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)"; else std::cout << ". "; + + auto peak_memory_usage = getMemoryUsage().peak; + if (peak_memory_usage >= 0) + std::cout << "\nPeak memory usage: " << formatReadableSizeWithBinarySuffix(peak_memory_usage) << "."; } void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) diff --git a/tests/queries/0_stateless/01921_test_progress_bar.py b/tests/queries/0_stateless/01921_test_progress_bar.py index 89eecbc3987..54c7ae59894 100755 --- a/tests/queries/0_stateless/01921_test_progress_bar.py +++ b/tests/queries/0_stateless/01921_test_progress_bar.py @@ -17,3 +17,4 @@ with client(name="client1>", log=log) as client1: client1.send("SELECT number FROM numbers(1000) FORMAT Null") client1.expect("Progress: 1\.00 thousand rows, 8\.00 KB .*" + end_of_block) client1.expect("0 rows in set. Elapsed: [\\w]{1}\.[\\w]{3} sec.") + client1.expect("Peak memory usage: .*B" + end_of_block) From 06229c1ba6816978d92f6cddf892e283c8704a0a Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 3 Aug 2023 03:50:41 +0000 Subject: [PATCH 1510/2047] fix password leak in show create mysql table --- src/Databases/MySQL/DatabaseMySQL.cpp | 1 + .../test_mysql_database_engine/test.py | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 94e5ba1773e..a4e5c4d92ea 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -144,6 +144,7 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, Context auto table_storage_define = database_engine_define->clone(); { ASTStorage * ast_storage = table_storage_define->as(); + ast_storage->engine->kind = ASTFunction::Kind::TABLE_ENGINE; ASTs storage_children = ast_storage->children; auto storage_engine_arguments = ast_storage->engine->arguments; diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 18dde5307fd..1d713823dc8 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -1021,3 +1021,21 @@ def test_memory_leak(started_cluster): clickhouse_node.query("DROP DATABASE test_database") clickhouse_node.restart_clickhouse() + +def test_password_leak(started_cluster): + with contextlib.closing( + MySQLNodeInstance( + "root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port + ) + ) as mysql_node: + mysql_node.query("DROP DATABASE IF EXISTS test_database") + mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query( + "CREATE TABLE `test_database`.`test_table` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;" + ) + + clickhouse_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query( + "CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse') SETTINGS connection_auto_close = 1" + ) + assert "clickhouse" not in clickhouse_node.query("SHOW CREATE test_database.test_table") From 9bcb980511119c8668aca29d408c1ed9525a12d3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Aug 2023 06:13:18 +0200 Subject: [PATCH 1511/2047] tests: fix 01293_client_interactive_vertical_multiline flakiness (increase timeout) All other expect tests has timeout of 60. And yes, sometimes I believe that on CI client spawning can took > 10 seconds for debug build. Signed-off-by: Azat Khuzhin --- .../01293_client_interactive_vertical_multiline.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect index 3bfd454bb1f..25933777f9f 100755 --- a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect +++ b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect @@ -6,7 +6,7 @@ exp_internal -f $env(CLICKHOUSE_TMP)/$basename.debuglog 0 set history_file $env(CLICKHOUSE_TMP)/$basename.history log_user 0 -set timeout 10 +set timeout 60 match_max 100000 expect_after { From 8e3c98ca7a78614edfe185b19a6037b83b7da9bf Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 3 Aug 2023 11:25:26 +0300 Subject: [PATCH 1512/2047] Amend the docs --- docs/en/operations/backup.md | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 630e171661d..c3545a778b7 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -84,13 +84,9 @@ The BACKUP and RESTORE statements take a list of DATABASE and TABLE names, a des - `password` for the file on disk - `base_backup`: the destination of the previous backup of this source. For example, `Disk('backups', '1.zip')` - `structure_only`: if enabled, allows to only backup or restore the CREATE statements without the data of tables - - `storage_policy`: storage policy for the tables being restored. See [Using Multiple Block Devices for Data Storage](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). + - `storage_policy`: storage policy for the tables being restored. See [Using Multiple Block Devices for Data Storage](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). This setting is only applicable to the `RESTORE` command. The specified storage policy applies only to tables with an engine from the `MergeTree` family. - `s3_storage_class`: the storage class used for S3 backup. For example, `STANDARD` -:::note storage_policy -`storage_policy` setting is only applicable to the `RESTORE` command. The specified storage policy applies only to tables with an engine from the `MergeTree` family. -::: - ### Usage examples Backup and then restore a table: From c7fcc611b2b8446babbab41e7b9b6449c87a992a Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 3 Aug 2023 08:27:58 +0000 Subject: [PATCH 1513/2047] added TSAN option report_atomic_races=0 --- .../integration/test_profile_max_sessions_for_user/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 72addd79ec5..2930262f63e 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -51,7 +51,12 @@ instance = cluster.add_instance( "configs/server.key", ], user_configs=["configs/users.xml"], - env_variables={"UBSAN_OPTIONS": "print_stacktrace=1"}, + env_variables={ + "UBSAN_OPTIONS": "print_stacktrace=1", + # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 + "TSAN_OPTIONS": "report_atomic_races=0 " + + os.getenv("TSAN_OPTIONS", default=""), + }, ) From fc7f47d083a84c9cb7996256f400f170022ac928 Mon Sep 17 00:00:00 2001 From: Dani Pozo Date: Wed, 2 Aug 2023 18:03:27 +0200 Subject: [PATCH 1514/2047] Make it possible to throttle S3 requests on backups --- src/Backups/BackupIO_S3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 31a33ea1a79..ca619127edf 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -50,7 +50,7 @@ namespace context->getRemoteHostFilter(), static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false, /* get_request_throttler = */ {}, /* put_request_throttler = */ {}); + /* for_disk_s3 = */ false, settings.request_settings.get_request_throttler, settings.request_settings.put_request_throttler); client_configuration.endpointOverride = s3_uri.endpoint; client_configuration.maxConnections = static_cast(context->getSettingsRef().s3_max_connections); From cc7f771093e98de0fc4f01a067f6d68c184549b0 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 3 Aug 2023 09:15:45 +0000 Subject: [PATCH 1515/2047] Fix for postgres as well --- .../PostgreSQL/DatabasePostgreSQL.cpp | 1 + src/Databases/SQLite/DatabaseSQLite.cpp | 1 + .../test_mysql_database_engine/test.py | 5 +++- .../test_postgresql_database_engine/test.py | 30 +++++++++++++++++++ 4 files changed, 36 insertions(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 812a0d8717e..1116c972f94 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -390,6 +390,7 @@ ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, Co auto create_table_query = std::make_shared(); auto table_storage_define = database_engine_define->clone(); + table_storage_define->as()->engine->kind = ASTFunction::Kind::TABLE_ENGINE; create_table_query->set(create_table_query->storage, table_storage_define); auto columns_declare_list = std::make_shared(); diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index 1cba9d1dc26..d031fd8e420 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -187,6 +187,7 @@ ASTPtr DatabaseSQLite::getCreateTableQueryImpl(const String & table_name, Contex } auto table_storage_define = database_engine_define->clone(); ASTStorage * ast_storage = table_storage_define->as(); + ast_storage->engine->kind = ASTFunction::Kind::TABLE_ENGINE; auto storage_engine_arguments = ast_storage->engine->arguments; auto table_id = storage->getStorageID(); /// Add table_name to engine arguments diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 1d713823dc8..00b5eb9e8aa 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -1022,6 +1022,7 @@ def test_memory_leak(started_cluster): clickhouse_node.query("DROP DATABASE test_database") clickhouse_node.restart_clickhouse() + def test_password_leak(started_cluster): with contextlib.closing( MySQLNodeInstance( @@ -1038,4 +1039,6 @@ def test_password_leak(started_cluster): clickhouse_node.query( "CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse') SETTINGS connection_auto_close = 1" ) - assert "clickhouse" not in clickhouse_node.query("SHOW CREATE test_database.test_table") + assert "clickhouse" not in clickhouse_node.query( + "SHOW CREATE test_database.test_table" + ) diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py index 59a464f9020..b14d234f7eb 100644 --- a/tests/integration/test_postgresql_database_engine/test.py +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -400,6 +400,36 @@ def test_datetime(started_cluster): assert "DateTime64(6)" in node1.query("show create table pg.test") +def test_postgresql_password_leak(started_cluster): + conn = get_postgres_conn( + started_cluster.postgres_ip, started_cluster.postgres_port, database=True + ) + cursor = conn.cursor() + + cursor.execute("CREATE SCHEMA test_schema") + cursor.execute("CREATE TABLE test_schema.table1 (a integer)") + cursor.execute("CREATE TABLE table2 (a integer)") + + node1.query( + "CREATE DATABASE postgres_database ENGINE = PostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword', 'test_schema')" + ) + + node1.query( + "CREATE DATABASE postgres_database2 ENGINE = PostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')" + ) + + assert "mysecretpassword" not in node1.query("SHOW CREATE postgres_database.table1") + assert "mysecretpassword" not in node1.query( + "SHOW CREATE postgres_database2.table2" + ) + + node1.query("DROP DATABASE postgres_database") + node1.query("DROP DATABASE postgres_database2") + + cursor.execute("DROP SCHEMA test_schema CASCADE") + cursor.execute("DROP TABLE table2") + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 5cdc50b9ba3c85f448ff673ef2dd8f604874b734 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 3 Aug 2023 11:31:15 +0200 Subject: [PATCH 1516/2047] Fixed style --- src/Storages/StorageAzureBlob.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 2080ec92bff..7c9de198435 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1240,7 +1240,7 @@ ColumnsDescription StorageAzureBlob::getTableStructureFromData( std::shared_ptr file_iterator; if (distributed_processing) { - file_iterator = std::make_shared(ctx , + file_iterator = std::make_shared(ctx, ctx->getReadTaskCallback()); } else if (configuration.withGlobs()) From e507221b7c56db0d19850de81c596af0c2b47512 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 Aug 2023 09:35:45 +0000 Subject: [PATCH 1517/2047] Fixing a test. --- src/Common/Allocator.h | 21 ++++++++++++--------- src/Common/CurrentMemoryTracker.cpp | 10 ++-------- src/Common/CurrentMemoryTracker.h | 1 - src/Common/MemoryTracker.cpp | 8 ++++++-- src/Common/MemoryTracker.h | 2 +- 5 files changed, 21 insertions(+), 21 deletions(-) diff --git a/src/Common/Allocator.h b/src/Common/Allocator.h index f94062ce7a5..570e4e60610 100644 --- a/src/Common/Allocator.h +++ b/src/Common/Allocator.h @@ -139,15 +139,16 @@ public: && alignment <= MALLOC_MIN_ALIGNMENT) { /// Resize malloc'd memory region with no special alignment requirement. - auto trace = CurrentMemoryTracker::realloc(old_size, new_size); - trace.onFree(buf, old_size); + auto trace_free = CurrentMemoryTracker::free(old_size); + auto trace_alloc = CurrentMemoryTracker::alloc(new_size); + trace_free.onFree(buf, old_size); void * new_buf = ::realloc(buf, new_size); if (nullptr == new_buf) DB::throwFromErrno(fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); buf = new_buf; - trace.onAlloc(buf, new_size); + trace_alloc.onAlloc(buf, new_size); if constexpr (clear_memory) if (new_size > old_size) @@ -156,8 +157,9 @@ public: else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD) { /// Resize mmap'd memory region. - auto trace = CurrentMemoryTracker::realloc(old_size, new_size); - trace.onFree(buf, old_size); + auto trace_free = CurrentMemoryTracker::free(old_size); + auto trace_alloc = CurrentMemoryTracker::alloc(new_size); + trace_free.onFree(buf, old_size); // On apple and freebsd self-implemented mremap used (common/mremap.h) buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE, @@ -166,17 +168,18 @@ public: DB::throwFromErrno(fmt::format("Allocator: Cannot mremap memory chunk from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_MREMAP); - trace.onAlloc(buf, new_size); /// No need for zero-fill, because mmap guarantees it. + trace_alloc.onAlloc(buf, new_size); } else if (new_size < MMAP_THRESHOLD) { /// Small allocs that requires a copy. Assume there's enough memory in system. Call CurrentMemoryTracker once. - auto trace = CurrentMemoryTracker::realloc(old_size, new_size); - trace.onFree(buf, old_size); + auto trace_free = CurrentMemoryTracker::free(old_size); + auto trace_alloc = CurrentMemoryTracker::alloc(new_size); + trace_free.onFree(buf, old_size); void * new_buf = allocNoTrack(new_size, alignment); - trace.onAlloc(new_buf, new_size); + trace_alloc.onAlloc(buf, new_size); memcpy(new_buf, buf, std::min(old_size, new_size)); freeNoTrack(buf, old_size); buf = new_buf; diff --git a/src/Common/CurrentMemoryTracker.cpp b/src/Common/CurrentMemoryTracker.cpp index 68afd89aef6..02c7dc6e224 100644 --- a/src/Common/CurrentMemoryTracker.cpp +++ b/src/Common/CurrentMemoryTracker.cpp @@ -72,7 +72,7 @@ AllocationTrace CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory return memory_tracker->allocImpl(size, throw_if_memory_exceeded); } - return AllocationTrace(memory_tracker->getSampleProbability()); + return AllocationTrace(memory_tracker->getSampleProbability(size)); } return AllocationTrace(0); @@ -96,12 +96,6 @@ AllocationTrace CurrentMemoryTracker::allocNoThrow(Int64 size) return allocImpl(size, throw_if_memory_exceeded); } -AllocationTrace CurrentMemoryTracker::realloc(Int64 old_size, Int64 new_size) -{ - Int64 addition = new_size - old_size; - return addition > 0 ? alloc(addition) : free(-addition); -} - AllocationTrace CurrentMemoryTracker::free(Int64 size) { if (auto * memory_tracker = getMemoryTracker()) @@ -122,7 +116,7 @@ AllocationTrace CurrentMemoryTracker::free(Int64 size) return memory_tracker->free(size); } - return AllocationTrace(memory_tracker->getSampleProbability()); + return AllocationTrace(memory_tracker->getSampleProbability(size)); } return AllocationTrace(0); diff --git a/src/Common/CurrentMemoryTracker.h b/src/Common/CurrentMemoryTracker.h index ae925a26565..18a1e3f49b1 100644 --- a/src/Common/CurrentMemoryTracker.h +++ b/src/Common/CurrentMemoryTracker.h @@ -9,7 +9,6 @@ struct CurrentMemoryTracker /// Call the following functions before calling of corresponding operations with memory allocators. [[nodiscard]] static AllocationTrace alloc(Int64 size); [[nodiscard]] static AllocationTrace allocNoThrow(Int64 size); - [[nodiscard]] static AllocationTrace realloc(Int64 old_size, Int64 new_size); /// This function should be called after memory deallocation. [[nodiscard]] static AllocationTrace free(Int64 size); diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 25d80d96956..ccb21d6b66a 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -565,13 +565,17 @@ void MemoryTracker::setOrRaiseProfilerLimit(Int64 value) ; } -double MemoryTracker::getSampleProbability() +double MemoryTracker::getSampleProbability(UInt64 size) { if (sample_probability >= 0) + { + if (!isSizeOkForSampling(size)) + return 0; return sample_probability; + } if (auto * loaded_next = parent.load(std::memory_order_relaxed)) - return loaded_next->getSampleProbability(); + return loaded_next->getSampleProbability(size); return 0; } diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index ee773cae73e..5041dc2af41 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -176,7 +176,7 @@ public: sample_probability = value; } - double getSampleProbability(); + double getSampleProbability(UInt64 size); void setSampleMinAllocationSize(UInt64 value) { From 7da85cc0de3e3a2b63ccae5639231ddb31f0fe31 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Thu, 3 Aug 2023 11:04:07 +0200 Subject: [PATCH 1518/2047] MaterializedMySQL: Add tests for unquoted utf8 column names in DML --- .../materialized_with_ddl.py | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 389d430622d..604f7a62428 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -1671,22 +1671,24 @@ def utf8mb4_name_test(clickhouse_node, mysql_node, service_name): f"CREATE TABLE `{db}`.`{table}` (id INT(11) NOT NULL PRIMARY KEY, `{table}` DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" ) mysql_node.query(f"INSERT INTO `{db}`.`{table}` VALUES(1, now())") + mysql_node.query(f"INSERT INTO `{db}`.`{table}`(id, `{table}`) VALUES(2, now())") mysql_node.query( f"CREATE TABLE {db}.{table}_unquoted (id INT(11) NOT NULL PRIMARY KEY, {table} DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" ) mysql_node.query(f"INSERT INTO {db}.{table}_unquoted VALUES(1, now())") + mysql_node.query(f"INSERT INTO {db}.{table}_unquoted(id, {table}) VALUES(2, now())") clickhouse_node.query( f"CREATE DATABASE `{db}` ENGINE = MaterializedMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')" ) check_query( clickhouse_node, - f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}`", - "1\n", + f"/* expect: 2 */ SELECT COUNT() FROM `{db}`.`{table}`", + "2\n", ) check_query( clickhouse_node, - f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}_unquoted`", - "1\n", + f"/* expect: 2 */ SELECT COUNT() FROM `{db}`.`{table}_unquoted`", + "2\n", ) # Inc sync @@ -1694,20 +1696,24 @@ def utf8mb4_name_test(clickhouse_node, mysql_node, service_name): f"CREATE TABLE `{db}`.`{table}2` (id INT(11) NOT NULL PRIMARY KEY, `{table}` DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" ) mysql_node.query(f"INSERT INTO `{db}`.`{table}2` VALUES(1, now())") + mysql_node.query(f"INSERT INTO `{db}`.`{table}2`(id, `{table}`) VALUES(2, now())") check_query( clickhouse_node, - f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}2`", - "1\n", + f"/* expect: 2 */ SELECT COUNT() FROM `{db}`.`{table}2`", + "2\n", ) mysql_node.query( f"CREATE TABLE {db}.{table}2_unquoted (id INT(11) NOT NULL PRIMARY KEY, {table} DATETIME) ENGINE=InnoDB DEFAULT CHARACTER SET utf8mb4" ) mysql_node.query(f"INSERT INTO {db}.{table}2_unquoted VALUES(1, now())") + mysql_node.query( + f"INSERT INTO {db}.{table}2_unquoted(id, {table}) VALUES(2, now())" + ) check_query( clickhouse_node, - f"/* expect: 1 */ SELECT COUNT() FROM `{db}`.`{table}2_unquoted`", - "1\n", + f"/* expect: 2 */ SELECT COUNT() FROM `{db}`.`{table}2_unquoted`", + "2\n", ) clickhouse_node.query(f"DROP DATABASE IF EXISTS `{db}`") From 659c907d3884ac92d80cd5210a8886eba13c87d5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Aug 2023 09:55:00 +0000 Subject: [PATCH 1519/2047] Update version_date.tsv and changelogs after v23.7.2.25-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.7.2.25-stable.md | 31 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 5 files changed, 35 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.7.2.25-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index c9800e4e66d..bee4c9d7f1e 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.7.1.2470" +ARG VERSION="23.7.2.25" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index f558338b23c..efad16509ea 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.7.1.2470" +ARG VERSION="23.7.2.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 156de034a7f..036b159dc03 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -23,7 +23,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.7.1.2470" +ARG VERSION="23.7.2.25" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.7.2.25-stable.md b/docs/changelogs/v23.7.2.25-stable.md new file mode 100644 index 00000000000..267083d8e03 --- /dev/null +++ b/docs/changelogs/v23.7.2.25-stable.md @@ -0,0 +1,31 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.7.2.25-stable (8dd1107b032) FIXME as compared to v23.7.1.2470-stable (a70127baecc) + +#### Backward Incompatible Change +* Backported in [#52850](https://github.com/ClickHouse/ClickHouse/issues/52850): If a dynamic disk contains a name, it should be specified as `disk = disk(name = 'disk_name'`, ...) in disk function arguments. In previous version it could be specified as `disk = disk_(...)`, which is no longer supported. [#52820](https://github.com/ClickHouse/ClickHouse/pull/52820) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### Build/Testing/Packaging Improvement +* Backported in [#52913](https://github.com/ClickHouse/ClickHouse/issues/52913): Add `clickhouse-keeper-client` symlink to the clickhouse-server package. [#51882](https://github.com/ClickHouse/ClickHouse/pull/51882) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix binary arithmetic for Nullable(IPv4) [#51642](https://github.com/ClickHouse/ClickHouse/pull/51642) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Support IPv4 and IPv6 as dictionary attributes [#51756](https://github.com/ClickHouse/ClickHouse/pull/51756) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* init and destroy ares channel on demand.. [#52634](https://github.com/ClickHouse/ClickHouse/pull/52634) ([Arthur Passos](https://github.com/arthurpassos)). +* Fix crash in function `tuple` with one sparse column argument [#52659](https://github.com/ClickHouse/ClickHouse/pull/52659) ([Anton Popov](https://github.com/CurtizJ)). +* Fix data race in Keeper reconfiguration [#52804](https://github.com/ClickHouse/ClickHouse/pull/52804) ([Antonio Andelic](https://github.com/antonio2368)). +* clickhouse-keeper: fix implementation of server with poll() [#52833](https://github.com/ClickHouse/ClickHouse/pull/52833) ([Andy Fiddaman](https://github.com/citrus-it)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Rename setting disable_url_encoding to enable_url_encoding and add a test [#52656](https://github.com/ClickHouse/ClickHouse/pull/52656) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix bugs and better test for SYSTEM STOP LISTEN [#52680](https://github.com/ClickHouse/ClickHouse/pull/52680) ([Nikolay Degterinsky](https://github.com/evillique)). +* Increase min protocol version for sparse serialization [#52835](https://github.com/ClickHouse/ClickHouse/pull/52835) ([Anton Popov](https://github.com/CurtizJ)). +* Docker improvements [#52869](https://github.com/ClickHouse/ClickHouse/pull/52869) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 1eabc65a10f..3976c676eb5 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.7.2.25-stable 2023-08-03 v23.7.1.2470-stable 2023-07-27 v23.6.2.18-stable 2023-07-09 v23.6.1.1524-stable 2023-06-30 From 9e0d27dc4d4d7f01446364eb1f4746e347fe5705 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 3 Aug 2023 13:16:32 +0300 Subject: [PATCH 1520/2047] don't create empty parts on drop partittion if we have a transaction (#52945) --- src/Storages/StorageMergeTree.cpp | 71 +++++++++- src/Storages/StorageMergeTree.h | 1 + tests/integration/test_transactions/test.py | 3 - .../01168_mutations_isolation.reference | 24 ++-- .../0_stateless/01168_mutations_isolation.sh | 9 +- ...alter_partition_isolation_stress.reference | 8 ++ ...69_old_alter_partition_isolation_stress.sh | 123 ++++++++++++++++++ .../01172_transaction_counters.reference | 4 - ...421_truncate_isolation_no_merges.reference | 12 -- 9 files changed, 214 insertions(+), 41 deletions(-) create mode 100644 tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.reference create mode 100755 tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a279291aef1..14ea2ff4afe 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1738,15 +1738,24 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont { /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. - auto merge_blocker = stopMergesAndWait(); waitForOutdatedPartsToBeLoaded(); + auto merge_blocker = stopMergesAndWait(); Stopwatch watch; ProfileEventsScope profile_events_scope; auto txn = query_context->getCurrentTransaction(); - MergeTreeData::Transaction transaction(*this, txn.get()); + if (txn) { + auto data_parts_lock = lockParts(); + auto parts_to_remove = getVisibleDataPartsVectorUnlocked(query_context, data_parts_lock); + removePartsFromWorkingSet(txn.get(), parts_to_remove, true, data_parts_lock); + LOG_INFO(log, "Removed {} parts: [{}]", parts_to_remove.size(), fmt::join(getPartsNames(parts_to_remove), ", ")); + } + else + { + MergeTreeData::Transaction transaction(*this, txn.get()); + auto operation_data_parts_lock = lockOperationsWithParts(); auto parts = getVisibleDataPartsVector(query_context); @@ -1790,8 +1799,15 @@ void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPt /// It's important to create it outside of lock scope because /// otherwise it can lock parts in destructor and deadlock is possible. auto txn = query_context->getCurrentTransaction(); - MergeTreeData::Transaction transaction(*this, txn.get()); + if (txn) { + if (auto part = outdatePart(txn.get(), part_name, /*force=*/ true)) + dropPartsImpl({part}, detach); + } + else + { + MergeTreeData::Transaction transaction(*this, txn.get()); + auto operation_data_parts_lock = lockOperationsWithParts(); auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); @@ -1848,8 +1864,26 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont /// It's important to create it outside of lock scope because /// otherwise it can lock parts in destructor and deadlock is possible. auto txn = query_context->getCurrentTransaction(); - MergeTreeData::Transaction transaction(*this, txn.get()); + if (txn) { + DataPartsVector parts_to_remove; + { + auto data_parts_lock = lockParts(); + if (partition_ast && partition_ast->all) + parts_to_remove = getVisibleDataPartsVectorUnlocked(query_context, data_parts_lock); + else + { + String partition_id = getPartitionIDFromQuery(partition, query_context, &data_parts_lock); + parts_to_remove = getVisibleDataPartsVectorInPartition(query_context, partition_id, data_parts_lock); + } + removePartsFromWorkingSet(txn.get(), parts_to_remove, true, data_parts_lock); + } + dropPartsImpl(std::move(parts_to_remove), detach); + } + else + { + MergeTreeData::Transaction transaction(*this, txn.get()); + auto operation_data_parts_lock = lockOperationsWithParts(); DataPartsVector parts; @@ -1864,12 +1898,14 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont } if (detach) + { for (const auto & part : parts) { auto metadata_snapshot = getInMemoryMetadataPtr(); LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory()); part->makeCloneInDetached("", metadata_snapshot); } + } auto future_parts = initCoverageWithNewEmptyParts(parts); @@ -1898,6 +1934,33 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont clearEmptyParts(); } +void StorageMergeTree::dropPartsImpl(DataPartsVector && parts_to_remove, bool detach) +{ + auto metadata_snapshot = getInMemoryMetadataPtr(); + + if (detach) + { + /// If DETACH clone parts to detached/ directory + /// NOTE: no race with background cleanup until we hold pointers to parts + for (const auto & part : parts_to_remove) + { + LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory()); + part->makeCloneInDetached("", metadata_snapshot); + } + } + + if (deduplication_log) + { + for (const auto & part : parts_to_remove) + deduplication_log->dropPart(part->info); + } + + if (detach) + LOG_INFO(log, "Detached {} parts: [{}]", parts_to_remove.size(), fmt::join(getPartsNames(parts_to_remove), ", ")); + else + LOG_INFO(log, "Removed {} parts: [{}]", parts_to_remove.size(), fmt::join(getPartsNames(parts_to_remove), ", ")); +} + PartitionCommandsResultInfo StorageMergeTree::attachPartition( const ASTPtr & partition, const StorageMetadataPtr & /* metadata_snapshot */, bool attach_part, ContextPtr local_context) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 936ba1b7f18..c77e5140d75 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -237,6 +237,7 @@ private: void dropPartNoWaitNoThrow(const String & part_name) override; void dropPart(const String & part_name, bool detach, ContextPtr context) override; void dropPartition(const ASTPtr & partition, bool detach, ContextPtr context) override; + void dropPartsImpl(DataPartsVector && parts_to_remove, bool detach); PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, ContextPtr context) override; void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr context) override; diff --git a/tests/integration/test_transactions/test.py b/tests/integration/test_transactions/test.py index a12d30915dd..46660581223 100644 --- a/tests/integration/test_transactions/test.py +++ b/tests/integration/test_transactions/test.py @@ -105,8 +105,6 @@ def test_rollback_unfinished_on_restart1(start_cluster): "0_4_4_0_7\t0\ttid3\tcsn18446744073709551615_\ttid0\tcsn0_\n" "0_8_8_0\t0\ttid5\tcsn18446744073709551615_\ttid0\tcsn0_\n" "1_1_1_0\t0\ttid0\tcsn1_\ttid1\tcsn_1\n" - "1_1_1_1\t1\ttid1\tcsn_1\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" - "1_1_1_1_7\t0\ttid3\tcsn18446744073709551615_\ttid0\tcsn0_\n" "1_3_3_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" "1_3_3_0_7\t0\ttid3\tcsn18446744073709551615_\ttid0\tcsn0_\n" "1_5_5_0\t1\ttid6\tcsn_6\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" @@ -194,6 +192,5 @@ def test_rollback_unfinished_on_restart2(start_cluster): "0_4_4_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" "0_5_5_0\t0\ttid5\tcsn18446744073709551615_\ttid0\tcsn0_\n" "1_1_1_0\t0\ttid0\tcsn1_\ttid1\tcsn_1\n" - "1_1_1_1\t1\ttid1\tcsn_1\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" "1_3_3_0\t1\ttid2\tcsn_2\t(0,0,'00000000-0000-0000-0000-000000000000')\tcsn0_\n" ) diff --git a/tests/queries/0_stateless/01168_mutations_isolation.reference b/tests/queries/0_stateless/01168_mutations_isolation.reference index 44da63385ca..00859ce99b9 100644 --- a/tests/queries/0_stateless/01168_mutations_isolation.reference +++ b/tests/queries/0_stateless/01168_mutations_isolation.reference @@ -21,20 +21,20 @@ tx7 7 20 all_1_1_0_13 tx7 7 40 all_14_14_0 tx7 7 60 all_7_7_0_13 tx7 7 80 all_12_12_0_13 -tx7 8 20 all_1_14_2_13 -tx7 8 40 all_1_14_2_13 -tx7 8 60 all_1_14_2_13 -tx7 8 80 all_1_14_2_13 +tx7 8 20 all_1_14_1_13 +tx7 8 40 all_1_14_1_13 +tx7 8 60 all_1_14_1_13 +tx7 8 80 all_1_14_1_13 Serialization error INVALID_TRANSACTION -tx11 9 21 all_1_14_2_17 -tx11 9 41 all_1_14_2_17 -tx11 9 61 all_1_14_2_17 -tx11 9 81 all_1_14_2_17 +tx11 9 21 all_1_14_1_17 +tx11 9 41 all_1_14_1_17 +tx11 9 61 all_1_14_1_17 +tx11 9 81 all_1_14_1_17 1 1 RUNNING -tx14 10 22 all_1_14_2_18 -tx14 10 42 all_1_14_2_18 -tx14 10 62 all_1_14_2_18 -tx14 10 82 all_1_14_2_18 +tx14 10 22 all_1_14_1_18 +tx14 10 42 all_1_14_1_18 +tx14 10 62 all_1_14_1_18 +tx14 10 82 all_1_14_1_18 11 2 all_2_2_0 11 10 all_1_1_0_3 diff --git a/tests/queries/0_stateless/01168_mutations_isolation.sh b/tests/queries/0_stateless/01168_mutations_isolation.sh index 2b76e5742ac..c1d70189673 100755 --- a/tests/queries/0_stateless/01168_mutations_isolation.sh +++ b/tests/queries/0_stateless/01168_mutations_isolation.sh @@ -53,9 +53,6 @@ tx 6 "alter table mt update n=n*10 wh tx 6 "insert into mt values (40)" tx 6 "commit" -function accept_both_parts() { - sed 's/all_1_14_1_1/all_1_14_2_1/g' -} tx 7 "begin transaction" tx 7 "select 7, n, _part from mt order by n" @@ -64,7 +61,7 @@ tx_async 8 "alter table mt update n = 0 whe $CLICKHOUSE_CLIENT -q "kill mutation where database=currentDatabase() and mutation_id='mutation_15.txt' format Null" 2>&1| grep -Fv "probably it finished" tx_sync 8 "rollback" tx 7 "optimize table mt final" -tx 7 "select 8, n, _part from mt order by n" | accept_both_parts +tx 7 "select 8, n, _part from mt order by n" tx 10 "begin transaction" tx 10 "alter table mt update n = 0 where 1" | grep -Eo "Serialization error" | uniq tx 7 "alter table mt update n=n+1 where 1" @@ -74,7 +71,7 @@ tx 7 "commit" tx_async 11 "begin transaction" -tx_async 11 "select 9, n, _part from mt order by n" | accept_both_parts +tx_async 11 "select 9, n, _part from mt order by n" tx_async 12 "begin transaction" tx_async 11 "alter table mt update n=n+1 where 1" >/dev/null tx_async 12 "alter table mt update n=n+1 where 1" >/dev/null @@ -91,7 +88,7 @@ $CLICKHOUSE_CLIENT -q "kill transaction where tid=$tid_to_kill format Null" tx_sync 13 "rollback" tx 14 "begin transaction" -tx 14 "select 10, n, _part from mt order by n" | accept_both_parts +tx 14 "select 10, n, _part from mt order by n" $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table mt" diff --git a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.reference b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.reference new file mode 100644 index 00000000000..12b941eab50 --- /dev/null +++ b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.reference @@ -0,0 +1,8 @@ +1 1 +2 1 +3 1 +4 1 +1 +10 100 +1 1 1 +2 1 1 diff --git a/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh new file mode 100755 index 00000000000..32ad78dead6 --- /dev/null +++ b/tests/queries/0_stateless/01169_old_alter_partition_isolation_stress.sh @@ -0,0 +1,123 @@ +#!/usr/bin/env bash +# Tags: long, no-replicated-database, no-ordinary-database + +# shellcheck disable=SC2015 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src"; +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS dst"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE src (n UInt64, type UInt8) ENGINE=MergeTree ORDER BY type SETTINGS old_parts_lifetime=0"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE dst (n UInt64, type UInt8) ENGINE=MergeTree ORDER BY type SETTINGS old_parts_lifetime=0"; + +function thread_insert() +{ + set -e + val=1 + while true; do + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + INSERT INTO src VALUES /* ($val, 1) */ ($val, 1); + INSERT INTO src VALUES /* ($val, 2) */ ($val, 2); + COMMIT;" + val=$((val+1)) + sleep 0.$RANDOM; + done +} + + +# NOTE +# ALTER PARTITION query stops merges, +# but serialization error is still possible if some merge was assigned (and committed) between BEGIN and ALTER. +function thread_partition_src_to_dst() +{ + set -e + count=0 + sum=0 + for i in {1..20}; do + out=$( + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + INSERT INTO src VALUES /* ($i, 3) */ ($i, 3); + INSERT INTO dst SELECT * FROM src; + ALTER TABLE src DROP PARTITION ID 'all'; + SET throw_on_unsupported_query_inside_transaction=0; + SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=3) != ($count + 1, $sum + $i)) FORMAT Null; + COMMIT;" 2>&1) ||: + + echo "$out" | grep -Fv "SERIALIZATION_ERROR" | grep -F "Received from " && $CLICKHOUSE_CLIENT --multiquery --query " + begin transaction; + set transaction snapshot 3; + select $i, 'src', type, n, _part from src order by type, n; + select $i, 'dst', type, n, _part from dst order by type, n; + rollback" ||: + echo "$out" | grep -Fa "SERIALIZATION_ERROR" >/dev/null || count=$((count+1)) + echo "$out" | grep -Fa "SERIALIZATION_ERROR" >/dev/null || sum=$((sum+i)) + done +} + +function thread_partition_dst_to_src() +{ + set -e + for i in {1..20}; do + action="ROLLBACK" + if (( i % 2 )); then + action="COMMIT" + fi + $CLICKHOUSE_CLIENT --multiquery --query " + SYSTEM STOP MERGES dst; + ALTER TABLE dst DROP PARTITION ID 'nonexistent'; -- STOP MERGES doesn't wait for started merges to finish, so we use this trick + SYSTEM SYNC TRANSACTION LOG; + BEGIN TRANSACTION; + INSERT INTO dst VALUES /* ($i, 4) */ ($i, 4); + INSERT INTO src SELECT * FROM dst; + ALTER TABLE dst DROP PARTITION ID 'all'; + SET throw_on_unsupported_query_inside_transaction=0; + SYSTEM START MERGES dst; + SELECT throwIf((SELECT (count(), sum(n)) FROM merge(currentDatabase(), '') WHERE type=4) != (toUInt8($i/2 + 1), (select sum(number) from numbers(1, $i) where number % 2 or number=$i))) FORMAT Null; + $action;" + done +} + +function thread_select() +{ + set -e + while true; do + $CLICKHOUSE_CLIENT --multiquery --query " + BEGIN TRANSACTION; + -- no duplicates + SELECT type, throwIf(count(n) != countDistinct(n)) FROM src GROUP BY type FORMAT Null; + SELECT type, throwIf(count(n) != countDistinct(n)) FROM dst GROUP BY type FORMAT Null; + -- rows inserted by thread_insert moved together + SET throw_on_unsupported_query_inside_transaction=0; + SELECT _table, throwIf(arraySort(groupArrayIf(n, type=1)) != arraySort(groupArrayIf(n, type=2))) FROM merge(currentDatabase(), '') GROUP BY _table FORMAT Null; + -- all rows are inserted in insert_thread + SELECT type, throwIf(count(n) != max(n)), throwIf(sum(n) != max(n)*(max(n)+1)/2) FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type FORMAT Null; + COMMIT;" + done +} + +thread_insert & PID_1=$! +thread_select & PID_2=$! + +thread_partition_src_to_dst & PID_3=$! +thread_partition_dst_to_src & PID_4=$! +wait $PID_3 && wait $PID_4 + +kill -TERM $PID_1 +kill -TERM $PID_2 +wait +wait_for_queries_to_finish + +$CLICKHOUSE_CLIENT -q "SELECT type, count(n) = countDistinct(n) FROM merge(currentDatabase(), '') GROUP BY type ORDER BY type" +$CLICKHOUSE_CLIENT -q "SELECT DISTINCT arraySort(groupArrayIf(n, type=1)) = arraySort(groupArrayIf(n, type=2)) FROM merge(currentDatabase(), '') GROUP BY _table ORDER BY _table" +$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM merge(currentDatabase(), '') WHERE type=4" +$CLICKHOUSE_CLIENT -q "SELECT type, count(n) == max(n), sum(n) == max(n)*(max(n)+1)/2 FROM merge(currentDatabase(), '') WHERE type IN (1, 2) GROUP BY type ORDER BY type" + + +$CLICKHOUSE_CLIENT --query "DROP TABLE src"; +$CLICKHOUSE_CLIENT --query "DROP TABLE dst"; diff --git a/tests/queries/0_stateless/01172_transaction_counters.reference b/tests/queries/0_stateless/01172_transaction_counters.reference index d088d747ee8..24083d7d40b 100644 --- a/tests/queries/0_stateless/01172_transaction_counters.reference +++ b/tests/queries/0_stateless/01172_transaction_counters.reference @@ -29,13 +29,9 @@ 4 1 Commit 1 1 1 0 5 1 Begin 1 1 1 1 5 1 AddPart 1 1 1 1 all_5_5_0 -5 1 AddPart 1 1 1 1 all_1_1_1 5 1 LockPart 1 1 1 1 all_1_1_0 -5 1 AddPart 1 1 1 1 all_3_3_1 5 1 LockPart 1 1 1 1 all_3_3_0 -5 1 AddPart 1 1 1 1 all_4_4_1 5 1 LockPart 1 1 1 1 all_4_4_0 -5 1 AddPart 1 1 1 1 all_5_5_1 5 1 LockPart 1 1 1 1 all_5_5_0 5 1 UnlockPart 1 1 1 1 all_1_1_0 5 1 UnlockPart 1 1 1 1 all_3_3_0 diff --git a/tests/queries/0_stateless/02421_truncate_isolation_no_merges.reference b/tests/queries/0_stateless/02421_truncate_isolation_no_merges.reference index a89ce339f6c..9f725a55439 100644 --- a/tests/queries/0_stateless/02421_truncate_isolation_no_merges.reference +++ b/tests/queries/0_stateless/02421_truncate_isolation_no_merges.reference @@ -5,19 +5,13 @@ tx21 3 UNKNOWN_TABLE concurrent_insert 2 -all_1_1_1 0 -all_2_2_1 0 -all_3_3_1 0 -all_4_4_1 0 all_5_5_0 1 -all_6_6_1 0 concurrent_drop_part_before SERIALIZATION_ERROR INVALID_TRANSACTION 1 3 all_1_1_0 1 -all_2_2_1 0 all_3_3_0 1 read_from_snapshot tx51 3 @@ -28,15 +22,9 @@ tx51 3 concurrent_drop_part_after NO_SUCH_DATA_PART INVALID_TRANSACTION -all_1_1_1 0 -all_2_2_1 0 -all_3_3_1 0 NewPart all_1_1_0 -NewPart all_1_1_1 NewPart all_2_2_0 -NewPart all_2_2_1 NewPart all_3_3_0 -NewPart all_3_3_1 concurrent_truncate_notx_after tx71 3 tx71 0 From 9cdd4fe41afc686a6069f4c16b495704920b4c9a Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 28 Jun 2023 13:39:48 +0000 Subject: [PATCH 1521/2047] Add info about aquired space in cache to not enough space error --- src/Interpreters/Cache/FileCache.cpp | 16 +++++++++++++++- src/Interpreters/Cache/FileCache.h | 18 +++++++++++++++++- src/Interpreters/Cache/FileSegment.cpp | 14 +++++++++----- src/Interpreters/Cache/FileSegment.h | 8 ++------ .../Cache/WriteBufferToFileSegment.cpp | 19 +++++++++++++++++-- 5 files changed, 60 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 9507622d386..c9bd39a0a2c 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -587,7 +587,7 @@ KeyMetadata::iterator FileCache::addFileSegment( } } -bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) +bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCacheReserveStat & reserve_stat) { ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheReserveMicroseconds); @@ -653,6 +653,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) { chassert(segment_metadata->file_segment->assertCorrectness()); + auto & stat_by_kind = reserve_stat.stat_by_kind[segment_metadata->file_segment->getKind()]; if (segment_metadata->releasable()) { const auto & key = segment_metadata->file_segment->key(); @@ -661,9 +662,18 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) it = to_delete.emplace(key, locked_key.getKeyMetadata()).first; it->second.add(segment_metadata); + stat_by_kind.releasable_size += segment_metadata->size(); + ++stat_by_kind.releasable_count; + freeable_space += segment_metadata->size(); ++freeable_count; } + else + { + stat_by_kind.non_releasable_size += segment_metadata->size(); + ++stat_by_kind.non_releasable_count; + } + return PriorityIterationResult::CONTINUE; }; @@ -718,6 +728,10 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) return is_overflow; }; + /// If we have enough space in query_priority, we are not interested about stat there anymore. + /// Clean the stat before iterating main_priority to avoid calculating any segment stat twice. + reserve_stat.stat_by_kind.clear(); + if (is_main_priority_overflow()) { main_priority->iterate( diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 2e6a5094758..d020f6d35f7 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -30,6 +30,22 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +/// Track acquired space in cache during reservation +/// to make error messages when no space left more informative. +struct FileCacheReserveStat +{ + struct Stat + { + size_t releasable_size; + size_t releasable_count; + + size_t non_releasable_size; + size_t non_releasable_count; + }; + + std::unordered_map stat_by_kind; +}; + /// Local cache for remote filesystem files, represented as a set of non-overlapping non-empty file segments. /// Different caching algorithms are implemented using IFileCachePriority. class FileCache : private boost::noncopyable @@ -106,7 +122,7 @@ public: size_t getMaxFileSegmentSize() const { return max_file_segment_size; } - bool tryReserve(FileSegment & file_segment, size_t size); + bool tryReserve(FileSegment & file_segment, size_t size, FileCacheReserveStat & stat); FileSegmentsHolderPtr getSnapshot(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 6afccafe94a..d280b382b95 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -478,7 +478,7 @@ LockedKeyPtr FileSegment::lockKeyMetadata(bool assert_exists) const return metadata->tryLock(); } -bool FileSegment::reserve(size_t size_to_reserve) +bool FileSegment::reserve(size_t size_to_reserve, FileCacheReserveStat * reserve_stat) { if (!size_to_reserve) throw Exception(ErrorCodes::LOGICAL_ERROR, "Zero space reservation is not allowed"); @@ -514,9 +514,8 @@ bool FileSegment::reserve(size_t size_to_reserve) size_t already_reserved_size = reserved_size - expected_downloaded_size; - bool reserved = already_reserved_size >= size_to_reserve; - if (reserved) - return reserved; + if (already_reserved_size >= size_to_reserve) + return true; size_to_reserve = size_to_reserve - already_reserved_size; @@ -525,7 +524,12 @@ bool FileSegment::reserve(size_t size_to_reserve) if (is_unbound && is_file_segment_size_exceeded) segment_range.right = range().left + expected_downloaded_size + size_to_reserve; - reserved = cache->tryReserve(*this, size_to_reserve); + /// if reserve_stat is not passed then use dummy stat and discard the result. + FileCacheReserveStat dummy_stat; + if (!reserve_stat) + reserve_stat = &dummy_stat; + + bool reserved = cache->tryReserve(*this, size_to_reserve, *reserve_stat); if (!reserved) setDownloadFailedUnlocked(lockFileSegment()); diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 44d95816915..82ab7b7e4f2 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -26,6 +26,7 @@ namespace DB { class ReadBufferFromFileBase; +struct FileCacheReserveStat; /* * FileSegmentKind is used to specify the eviction policy for file segments. @@ -243,12 +244,7 @@ public: /// Try to reserve exactly `size` bytes (in addition to the getDownloadedSize() bytes already downloaded). /// Returns true if reservation was successful, false otherwise. - bool reserve(size_t size_to_reserve); - - /// Try to reserve at max `size_to_reserve` bytes. - /// Returns actual size reserved. It can be less than size_to_reserve in non strict mode. - /// In strict mode throws an error on attempt to reserve space too much space. - size_t tryReserve(size_t size_to_reserve, bool strict = false); + bool reserve(size_t size_to_reserve, FileCacheReserveStat * reserve_stat = nullptr); /// Write data into reserved space. void write(const char * from, size_t size, size_t offset); diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index d50289a5728..85597ce881a 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -44,11 +45,25 @@ void WriteBufferToFileSegment::nextImpl() size_t bytes_to_write = offset(); + FileCacheReserveStat reserve_stat; /// In case of an error, we don't need to finalize the file segment /// because it will be deleted soon and completed in the holder's destructor. - bool ok = file_segment->reserve(bytes_to_write); + bool ok = file_segment->reserve(bytes_to_write, &reserve_stat); + if (!ok) - throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Failed to reserve space for the file cache ({})", file_segment->getInfoForLog()); + { + String reserve_stat_msg; + for (const auto & [kind, stat] : reserve_stat.stat_by_kind) + reserve_stat_msg += fmt::format("{} hold {}, can release {}; ", + toString(kind), ReadableSize(stat.non_releasable_size), ReadableSize(stat.releasable_size)); + + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Failed to reserve {} bytes for {}: {}(segment info: {})", + bytes_to_write, + file_segment->getKind() == FileSegmentKind::Temporary ? "temporary file" : "the file in cache", + reserve_stat_msg, + file_segment->getInfoForLog() + ); + } try { From 4747b7b0bea62ef881358b35b6ac3d1af5410cee Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Aug 2023 10:29:06 +0000 Subject: [PATCH 1522/2047] upd test_temporary_data_in_cache --- tests/integration/test_temporary_data_in_cache/test.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_temporary_data_in_cache/test.py b/tests/integration/test_temporary_data_in_cache/test.py index e6142c7eef1..ed06a70cf5a 100644 --- a/tests/integration/test_temporary_data_in_cache/test.py +++ b/tests/integration/test_temporary_data_in_cache/test.py @@ -2,6 +2,7 @@ # pylint: disable=redefined-outer-name import pytest +import fnmatch from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException @@ -68,7 +69,9 @@ def test_cache_evicted_by_temporary_data(start_cluster): "max_bytes_before_external_sort": "4M", }, ) - assert "Failed to reserve space for the file cache" in str(exc.value) + assert fnmatch.fnmatch( + str(exc.value), "*Failed to reserve * for temporary file*" + ), exc.value # Some data evicted from cache by temporary data cache_size_after_eviction = get_cache_size() @@ -104,6 +107,8 @@ def test_cache_evicted_by_temporary_data(start_cluster): "SELECT randomPrintableASCII(1024) FROM numbers(32 * 1024) FORMAT TSV", params={"buffer_size": 0, "wait_end_of_query": 1}, ) - assert "Failed to reserve space for the file cache" in str(exc.value) + assert fnmatch.fnmatch( + str(exc.value), "*Failed to reserve * for temporary file*" + ), exc.value q("DROP TABLE IF EXISTS t1") From 7684e680081970ebc6107f958e31e6e0b888b310 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 3 Aug 2023 13:34:43 +0300 Subject: [PATCH 1523/2047] Add system stop pulling replication log (#52881) * add system stop pulling replication log * add const * fix another stupid test * Update test.py --- docs/en/sql-reference/statements/system.md | 16 ++++++++++ src/Access/Common/AccessType.h | 1 + src/Access/tests/gtest_access_rights_ops.cpp | 2 +- src/Interpreters/ActionLocksManager.cpp | 1 + src/Interpreters/InterpreterSystemQuery.cpp | 32 +++++++++++++++---- src/Interpreters/loadMetadata.cpp | 8 ++--- src/Parsers/ASTSystemQuery.cpp | 4 ++- src/Parsers/ASTSystemQuery.h | 2 ++ src/Parsers/ParserSystemQuery.cpp | 2 ++ src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++- .../integration/test_grant_and_revoke/test.py | 2 +- .../01271_show_privileges.reference | 1 + .../02117_show_create_table_system.reference | 6 ++-- 13 files changed, 67 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index fb601cd5d35..59970dbeccd 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -314,6 +314,22 @@ Provides possibility to start background fetch tasks from replication queues whi SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] ``` +### STOP PULLING REPLICATION LOG + +Stops loading new entries from replication log to replication queue in a `ReplicatedMergeTree` table. + +``` sql +SYSTEM STOP PULLING REPLICATION LOG [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] +``` + +### START PULLING REPLICATION LOG + +Cancels `SYSTEM STOP PULLING REPLICATION LOG`. + +``` sql +SYSTEM START PULLING REPLICATION LOG [ON CLUSTER cluster_name] [[db.]replicated_merge_tree_family_table_name] +``` + ### SYNC REPLICA Wait until a `ReplicatedMergeTree` table will be synced with other replicas in a cluster, but no more than `receive_timeout` seconds. diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index b253a0e13ce..06507fd85c8 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -168,6 +168,7 @@ enum class AccessType M(SYSTEM_TTL_MERGES, "SYSTEM STOP TTL MERGES, SYSTEM START TTL MERGES, STOP TTL MERGES, START TTL MERGES", TABLE, SYSTEM) \ M(SYSTEM_FETCHES, "SYSTEM STOP FETCHES, SYSTEM START FETCHES, STOP FETCHES, START FETCHES", TABLE, SYSTEM) \ M(SYSTEM_MOVES, "SYSTEM STOP MOVES, SYSTEM START MOVES, STOP MOVES, START MOVES", TABLE, SYSTEM) \ + M(SYSTEM_PULLING_REPLICATION_LOG, "SYSTEM STOP PULLING REPLICATION LOG, SYSTEM START PULLING REPLICATION LOG", TABLE, SYSTEM) \ M(SYSTEM_DISTRIBUTED_SENDS, "SYSTEM STOP DISTRIBUTED SENDS, SYSTEM START DISTRIBUTED SENDS, STOP DISTRIBUTED SENDS, START DISTRIBUTED SENDS", TABLE, SYSTEM_SENDS) \ M(SYSTEM_REPLICATED_SENDS, "SYSTEM STOP REPLICATED SENDS, SYSTEM START REPLICATED SENDS, STOP REPLICATED SENDS, START REPLICATED SENDS", TABLE, SYSTEM_SENDS) \ M(SYSTEM_SENDS, "SYSTEM STOP SENDS, SYSTEM START SENDS, STOP SENDS, START SENDS", GROUP, SYSTEM) \ diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index c2e9501f58c..91d79be918b 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -51,7 +51,7 @@ TEST(AccessRights, Union) "CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, " "TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, " "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " - "SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, " + "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*, GRANT NAMED COLLECTION ADMIN ON db1"); } diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index 7b57b8803cd..d7480d45524 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -16,6 +16,7 @@ namespace ActionLocks extern const StorageActionBlockType DistributedSend = 5; extern const StorageActionBlockType PartsTTLMerge = 6; extern const StorageActionBlockType PartsMove = 7; + extern const StorageActionBlockType PullReplicationLog = 8; } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index b62840945c4..9c77b40f803 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -89,13 +89,14 @@ namespace ErrorCodes namespace ActionLocks { - extern StorageActionBlockType PartsMerge; - extern StorageActionBlockType PartsFetch; - extern StorageActionBlockType PartsSend; - extern StorageActionBlockType ReplicationQueue; - extern StorageActionBlockType DistributedSend; - extern StorageActionBlockType PartsTTLMerge; - extern StorageActionBlockType PartsMove; + extern const StorageActionBlockType PartsMerge; + extern const StorageActionBlockType PartsFetch; + extern const StorageActionBlockType PartsSend; + extern const StorageActionBlockType ReplicationQueue; + extern const StorageActionBlockType DistributedSend; + extern const StorageActionBlockType PartsTTLMerge; + extern const StorageActionBlockType PartsMove; + extern const StorageActionBlockType PullReplicationLog; } @@ -155,6 +156,8 @@ AccessType getRequiredAccessType(StorageActionBlockType action_type) return AccessType::SYSTEM_TTL_MERGES; else if (action_type == ActionLocks::PartsMove) return AccessType::SYSTEM_MOVES; + else if (action_type == ActionLocks::PullReplicationLog) + return AccessType::SYSTEM_PULLING_REPLICATION_LOG; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown action type: {}", std::to_string(action_type)); } @@ -513,6 +516,12 @@ BlockIO InterpreterSystemQuery::execute() case Type::START_DISTRIBUTED_SENDS: startStopAction(ActionLocks::DistributedSend, true); break; + case Type::STOP_PULLING_REPLICATION_LOG: + startStopAction(ActionLocks::PullReplicationLog, false); + break; + case Type::START_PULLING_REPLICATION_LOG: + startStopAction(ActionLocks::PullReplicationLog, true); + break; case Type::DROP_REPLICA: dropReplica(query); break; @@ -1090,6 +1099,15 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_MOVES, query.getDatabase(), query.getTable()); break; } + case Type::STOP_PULLING_REPLICATION_LOG: + case Type::START_PULLING_REPLICATION_LOG: + { + if (!query.table) + required_access.emplace_back(AccessType::SYSTEM_PULLING_REPLICATION_LOG); + else + required_access.emplace_back(AccessType::SYSTEM_PULLING_REPLICATION_LOG, query.getDatabase(), query.getTable()); + break; + } case Type::STOP_FETCHES: case Type::START_FETCHES: { diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 83af2684322..84326e40bf6 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -45,10 +45,10 @@ namespace ErrorCodes namespace ActionLocks { - extern StorageActionBlockType PartsMerge; - extern StorageActionBlockType PartsFetch; - extern StorageActionBlockType PartsSend; - extern StorageActionBlockType DistributedSend; + extern const StorageActionBlockType PartsMerge; + extern const StorageActionBlockType PartsFetch; + extern const StorageActionBlockType PartsSend; + extern const StorageActionBlockType DistributedSend; } static void executeCreateQuery( diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 3385844cb4f..fb10474a4d4 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -162,7 +162,9 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, || type == Type::STOP_REPLICATION_QUEUES || type == Type::START_REPLICATION_QUEUES || type == Type::STOP_DISTRIBUTED_SENDS - || type == Type::START_DISTRIBUTED_SENDS) + || type == Type::START_DISTRIBUTED_SENDS + || type == Type::STOP_PULLING_REPLICATION_LOG + || type == Type::START_PULLING_REPLICATION_LOG) { if (table) print_database_table(); diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index f97f4465621..9d15e7cacee 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -80,6 +80,8 @@ public: UNFREEZE, ENABLE_FAILPOINT, DISABLE_FAILPOINT, + STOP_PULLING_REPLICATION_LOG, + START_PULLING_REPLICATION_LOG, END }; diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 3af0d62f769..40fc1acae69 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -379,6 +379,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & case Type::START_REPLICATED_SENDS: case Type::STOP_REPLICATION_QUEUES: case Type::START_REPLICATION_QUEUES: + case Type::STOP_PULLING_REPLICATION_LOG: + case Type::START_PULLING_REPLICATION_LOG: if (!parseQueryWithOnCluster(res, pos, expected)) return false; parseDatabaseAndTableAsAST(pos, expected, res->database, res->table); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2c2cea0af2b..23683ec2313 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -197,6 +197,7 @@ namespace ActionLocks extern const StorageActionBlockType ReplicationQueue; extern const StorageActionBlockType PartsTTLMerge; extern const StorageActionBlockType PartsMove; + extern const StorageActionBlockType PullReplicationLog; } @@ -4340,7 +4341,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) { auto zookeeper = getZooKeeper(); - LOG_DEBUG(log, "Cleaning up last parent node for partition {}", partition_id); + LOG_DEBUG(log, "Cleaning up last part node for partition {}", partition_id); /// The name of the previous part for which the quorum was reached. const String quorum_last_part_path = fs::path(zookeeper_path) / "quorum" / "last_part"; @@ -4361,6 +4362,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) if (!parts_with_quorum.added_parts.contains(partition_id)) { /// There is no information about interested part. + LOG_TEST(log, "There is no information about the partition"); break; } @@ -4378,6 +4380,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) else if (code == Coordination::Error::ZNONODE) { /// Node is deleted. It is impossible, but it is Ok. + LOG_WARNING(log, "The last part node {} was deleted", quorum_last_part_path); break; } else if (code == Coordination::Error::ZBADVERSION) @@ -8169,6 +8172,9 @@ ActionLock StorageReplicatedMergeTree::getActionLock(StorageActionBlockType acti if (action_type == ActionLocks::PartsMove) return parts_mover.moves_blocker.cancel(); + if (action_type == ActionLocks::PullReplicationLog) + return queue.pull_log_blocker.cancel(); + return {}; } diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index ee5d4b5df93..8a8eb11d91f 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -188,7 +188,7 @@ def test_grant_all_on_table(): instance.query("SHOW GRANTS FOR B") == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, " "DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, " - "SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, " + "SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, " "SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n" ) instance.query("REVOKE ALL ON test.table FROM B", user="A") diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index db0f2d8235b..a3e15f0793c 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -119,6 +119,7 @@ SYSTEM MERGES ['SYSTEM STOP MERGES','SYSTEM START MERGES','STOP MERGES','START M SYSTEM TTL MERGES ['SYSTEM STOP TTL MERGES','SYSTEM START TTL MERGES','STOP TTL MERGES','START TTL MERGES'] TABLE SYSTEM SYSTEM FETCHES ['SYSTEM STOP FETCHES','SYSTEM START FETCHES','STOP FETCHES','START FETCHES'] TABLE SYSTEM SYSTEM MOVES ['SYSTEM STOP MOVES','SYSTEM START MOVES','STOP MOVES','START MOVES'] TABLE SYSTEM +SYSTEM PULLING REPLICATION LOG ['SYSTEM STOP PULLING REPLICATION LOG','SYSTEM START PULLING REPLICATION LOG'] TABLE SYSTEM SYSTEM DISTRIBUTED SENDS ['SYSTEM STOP DISTRIBUTED SENDS','SYSTEM START DISTRIBUTED SENDS','STOP DISTRIBUTED SENDS','START DISTRIBUTED SENDS'] TABLE SYSTEM SENDS SYSTEM REPLICATED SENDS ['SYSTEM STOP REPLICATED SENDS','SYSTEM START REPLICATED SENDS','STOP REPLICATED SENDS','START REPLICATED SENDS'] TABLE SYSTEM SENDS SYSTEM SENDS ['SYSTEM STOP SENDS','SYSTEM START SENDS','STOP SENDS','START SENDS'] \N SYSTEM diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index acff6e0efb2..5b678537248 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -298,7 +298,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM PULLING REPLICATION LOG' = 121, 'SYSTEM DISTRIBUTED SENDS' = 122, 'SYSTEM REPLICATED SENDS' = 123, 'SYSTEM SENDS' = 124, 'SYSTEM REPLICATION QUEUES' = 125, 'SYSTEM DROP REPLICA' = 126, 'SYSTEM SYNC REPLICA' = 127, 'SYSTEM RESTART REPLICA' = 128, 'SYSTEM RESTORE REPLICA' = 129, 'SYSTEM WAIT LOADING PARTS' = 130, 'SYSTEM SYNC DATABASE REPLICA' = 131, 'SYSTEM SYNC TRANSACTION LOG' = 132, 'SYSTEM SYNC FILE CACHE' = 133, 'SYSTEM FLUSH DISTRIBUTED' = 134, 'SYSTEM FLUSH LOGS' = 135, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 136, 'SYSTEM FLUSH' = 137, 'SYSTEM THREAD FUZZER' = 138, 'SYSTEM UNFREEZE' = 139, 'SYSTEM FAILPOINT' = 140, 'SYSTEM LISTEN' = 141, 'SYSTEM' = 142, 'dictGet' = 143, 'displaySecretsInShowAndSelect' = 144, 'addressToLine' = 145, 'addressToLineWithInlines' = 146, 'addressToSymbol' = 147, 'demangle' = 148, 'INTROSPECTION' = 149, 'FILE' = 150, 'URL' = 151, 'REMOTE' = 152, 'MONGO' = 153, 'REDIS' = 154, 'MEILISEARCH' = 155, 'MYSQL' = 156, 'POSTGRES' = 157, 'SQLITE' = 158, 'ODBC' = 159, 'JDBC' = 160, 'HDFS' = 161, 'S3' = 162, 'HIVE' = 163, 'AZURE' = 164, 'SOURCES' = 165, 'CLUSTER' = 166, 'ALL' = 167, 'NONE' = 168), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -586,10 +586,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM PULLING REPLICATION LOG' = 121, 'SYSTEM DISTRIBUTED SENDS' = 122, 'SYSTEM REPLICATED SENDS' = 123, 'SYSTEM SENDS' = 124, 'SYSTEM REPLICATION QUEUES' = 125, 'SYSTEM DROP REPLICA' = 126, 'SYSTEM SYNC REPLICA' = 127, 'SYSTEM RESTART REPLICA' = 128, 'SYSTEM RESTORE REPLICA' = 129, 'SYSTEM WAIT LOADING PARTS' = 130, 'SYSTEM SYNC DATABASE REPLICA' = 131, 'SYSTEM SYNC TRANSACTION LOG' = 132, 'SYSTEM SYNC FILE CACHE' = 133, 'SYSTEM FLUSH DISTRIBUTED' = 134, 'SYSTEM FLUSH LOGS' = 135, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 136, 'SYSTEM FLUSH' = 137, 'SYSTEM THREAD FUZZER' = 138, 'SYSTEM UNFREEZE' = 139, 'SYSTEM FAILPOINT' = 140, 'SYSTEM LISTEN' = 141, 'SYSTEM' = 142, 'dictGet' = 143, 'displaySecretsInShowAndSelect' = 144, 'addressToLine' = 145, 'addressToLineWithInlines' = 146, 'addressToSymbol' = 147, 'demangle' = 148, 'INTROSPECTION' = 149, 'FILE' = 150, 'URL' = 151, 'REMOTE' = 152, 'MONGO' = 153, 'REDIS' = 154, 'MEILISEARCH' = 155, 'MYSQL' = 156, 'POSTGRES' = 157, 'SQLITE' = 158, 'ODBC' = 159, 'JDBC' = 160, 'HDFS' = 161, 'S3' = 162, 'HIVE' = 163, 'AZURE' = 164, 'SOURCES' = 165, 'CLUSTER' = 166, 'ALL' = 167, 'NONE' = 168), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5, 'NAMED_COLLECTION' = 6)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM DISTRIBUTED SENDS' = 121, 'SYSTEM REPLICATED SENDS' = 122, 'SYSTEM SENDS' = 123, 'SYSTEM REPLICATION QUEUES' = 124, 'SYSTEM DROP REPLICA' = 125, 'SYSTEM SYNC REPLICA' = 126, 'SYSTEM RESTART REPLICA' = 127, 'SYSTEM RESTORE REPLICA' = 128, 'SYSTEM WAIT LOADING PARTS' = 129, 'SYSTEM SYNC DATABASE REPLICA' = 130, 'SYSTEM SYNC TRANSACTION LOG' = 131, 'SYSTEM SYNC FILE CACHE' = 132, 'SYSTEM FLUSH DISTRIBUTED' = 133, 'SYSTEM FLUSH LOGS' = 134, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 135, 'SYSTEM FLUSH' = 136, 'SYSTEM THREAD FUZZER' = 137, 'SYSTEM UNFREEZE' = 138, 'SYSTEM FAILPOINT' = 139, 'SYSTEM LISTEN' = 140, 'SYSTEM' = 141, 'dictGet' = 142, 'displaySecretsInShowAndSelect' = 143, 'addressToLine' = 144, 'addressToLineWithInlines' = 145, 'addressToSymbol' = 146, 'demangle' = 147, 'INTROSPECTION' = 148, 'FILE' = 149, 'URL' = 150, 'REMOTE' = 151, 'MONGO' = 152, 'REDIS' = 153, 'MEILISEARCH' = 154, 'MYSQL' = 155, 'POSTGRES' = 156, 'SQLITE' = 157, 'ODBC' = 158, 'JDBC' = 159, 'HDFS' = 160, 'S3' = 161, 'HIVE' = 162, 'AZURE' = 163, 'SOURCES' = 164, 'CLUSTER' = 165, 'ALL' = 166, 'NONE' = 167)) + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER NAMED COLLECTION' = 41, 'ALTER TABLE' = 42, 'ALTER DATABASE' = 43, 'ALTER VIEW REFRESH' = 44, 'ALTER VIEW MODIFY QUERY' = 45, 'ALTER VIEW' = 46, 'ALTER' = 47, 'CREATE DATABASE' = 48, 'CREATE TABLE' = 49, 'CREATE VIEW' = 50, 'CREATE DICTIONARY' = 51, 'CREATE TEMPORARY TABLE' = 52, 'CREATE ARBITRARY TEMPORARY TABLE' = 53, 'CREATE FUNCTION' = 54, 'CREATE NAMED COLLECTION' = 55, 'CREATE' = 56, 'DROP DATABASE' = 57, 'DROP TABLE' = 58, 'DROP VIEW' = 59, 'DROP DICTIONARY' = 60, 'DROP FUNCTION' = 61, 'DROP NAMED COLLECTION' = 62, 'DROP' = 63, 'UNDROP TABLE' = 64, 'TRUNCATE' = 65, 'OPTIMIZE' = 66, 'BACKUP' = 67, 'KILL QUERY' = 68, 'KILL TRANSACTION' = 69, 'MOVE PARTITION BETWEEN SHARDS' = 70, 'CREATE USER' = 71, 'ALTER USER' = 72, 'DROP USER' = 73, 'CREATE ROLE' = 74, 'ALTER ROLE' = 75, 'DROP ROLE' = 76, 'ROLE ADMIN' = 77, 'CREATE ROW POLICY' = 78, 'ALTER ROW POLICY' = 79, 'DROP ROW POLICY' = 80, 'CREATE QUOTA' = 81, 'ALTER QUOTA' = 82, 'DROP QUOTA' = 83, 'CREATE SETTINGS PROFILE' = 84, 'ALTER SETTINGS PROFILE' = 85, 'DROP SETTINGS PROFILE' = 86, 'SHOW USERS' = 87, 'SHOW ROLES' = 88, 'SHOW ROW POLICIES' = 89, 'SHOW QUOTAS' = 90, 'SHOW SETTINGS PROFILES' = 91, 'SHOW ACCESS' = 92, 'ACCESS MANAGEMENT' = 93, 'SHOW NAMED COLLECTIONS' = 94, 'SHOW NAMED COLLECTIONS SECRETS' = 95, 'NAMED COLLECTION' = 96, 'NAMED COLLECTION ADMIN' = 97, 'SYSTEM SHUTDOWN' = 98, 'SYSTEM DROP DNS CACHE' = 99, 'SYSTEM DROP MARK CACHE' = 100, 'SYSTEM DROP UNCOMPRESSED CACHE' = 101, 'SYSTEM DROP MMAP CACHE' = 102, 'SYSTEM DROP QUERY CACHE' = 103, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 104, 'SYSTEM DROP FILESYSTEM CACHE' = 105, 'SYSTEM DROP SCHEMA CACHE' = 106, 'SYSTEM DROP S3 CLIENT CACHE' = 107, 'SYSTEM DROP CACHE' = 108, 'SYSTEM RELOAD CONFIG' = 109, 'SYSTEM RELOAD USERS' = 110, 'SYSTEM RELOAD DICTIONARY' = 111, 'SYSTEM RELOAD MODEL' = 112, 'SYSTEM RELOAD FUNCTION' = 113, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 114, 'SYSTEM RELOAD' = 115, 'SYSTEM RESTART DISK' = 116, 'SYSTEM MERGES' = 117, 'SYSTEM TTL MERGES' = 118, 'SYSTEM FETCHES' = 119, 'SYSTEM MOVES' = 120, 'SYSTEM PULLING REPLICATION LOG' = 121, 'SYSTEM DISTRIBUTED SENDS' = 122, 'SYSTEM REPLICATED SENDS' = 123, 'SYSTEM SENDS' = 124, 'SYSTEM REPLICATION QUEUES' = 125, 'SYSTEM DROP REPLICA' = 126, 'SYSTEM SYNC REPLICA' = 127, 'SYSTEM RESTART REPLICA' = 128, 'SYSTEM RESTORE REPLICA' = 129, 'SYSTEM WAIT LOADING PARTS' = 130, 'SYSTEM SYNC DATABASE REPLICA' = 131, 'SYSTEM SYNC TRANSACTION LOG' = 132, 'SYSTEM SYNC FILE CACHE' = 133, 'SYSTEM FLUSH DISTRIBUTED' = 134, 'SYSTEM FLUSH LOGS' = 135, 'SYSTEM FLUSH ASYNC INSERT QUEUE' = 136, 'SYSTEM FLUSH' = 137, 'SYSTEM THREAD FUZZER' = 138, 'SYSTEM UNFREEZE' = 139, 'SYSTEM FAILPOINT' = 140, 'SYSTEM LISTEN' = 141, 'SYSTEM' = 142, 'dictGet' = 143, 'displaySecretsInShowAndSelect' = 144, 'addressToLine' = 145, 'addressToLineWithInlines' = 146, 'addressToSymbol' = 147, 'demangle' = 148, 'INTROSPECTION' = 149, 'FILE' = 150, 'URL' = 151, 'REMOTE' = 152, 'MONGO' = 153, 'REDIS' = 154, 'MEILISEARCH' = 155, 'MYSQL' = 156, 'POSTGRES' = 157, 'SQLITE' = 158, 'ODBC' = 159, 'JDBC' = 160, 'HDFS' = 161, 'S3' = 162, 'HIVE' = 163, 'AZURE' = 164, 'SOURCES' = 165, 'CLUSTER' = 166, 'ALL' = 167, 'NONE' = 168)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' From 141a3df727dba39c20546141e57c3c202c5c9d2b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 3 Aug 2023 12:35:14 +0200 Subject: [PATCH 1524/2047] fix --- src/Databases/MySQL/DatabaseMySQL.cpp | 2 ++ src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 94e5ba1773e..434e702125c 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -77,6 +77,8 @@ DatabaseMySQL::DatabaseMySQL( throw; } + fs::create_directories(metadata_path); + thread = ThreadFromGlobalPool{&DatabaseMySQL::cleanOutdatedTables, this}; } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 812a0d8717e..3a2ea8c97eb 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -54,6 +54,7 @@ DatabasePostgreSQL::DatabasePostgreSQL( , cache_tables(cache_tables_) , log(&Poco::Logger::get("DatabasePostgreSQL(" + dbname_ + ")")) { + fs::create_directories(metadata_path); cleaner_task = getContext()->getSchedulePool().createTask("PostgreSQLCleanerTask", [this]{ removeOutdatedTables(); }); cleaner_task->deactivate(); } From d9f136ce0ca640f65f37b6097366901d120db77c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 3 Aug 2023 10:15:02 +0000 Subject: [PATCH 1525/2047] Add tests for globs --- src/IO/Archives/LibArchiveReader.cpp | 7 +- src/Storages/StorageFile.cpp | 3 + ...ence => 02661_read_from_archive.reference} | 68 ++--- .../0_stateless/02661_read_from_archive.sh | 36 +++ ...661_read_from_archive_with_globs.reference | 264 ++++++++++++++++++ .../02661_read_from_archive_with_globs.sh | 46 +++ .../02661_select_from_table_in_archive.sh | 59 ---- 7 files changed, 390 insertions(+), 93 deletions(-) rename tests/queries/0_stateless/{02661_select_from_table_in_archive.reference => 02661_read_from_archive.reference} (52%) create mode 100755 tests/queries/0_stateless/02661_read_from_archive.sh create mode 100644 tests/queries/0_stateless/02661_read_from_archive_with_globs.reference create mode 100755 tests/queries/0_stateless/02661_read_from_archive_with_globs.sh delete mode 100755 tests/queries/0_stateless/02661_select_from_table_in_archive.sh diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 0e0d035d98b..148de3bffdd 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -254,7 +254,8 @@ template LibArchiveReader::FileInfo LibArchiveReader::getFileInfo(const String & filename) { Handle handle(path_to_archive); - handle.locateFile(filename); + if (!handle.locateFile(filename)) + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't unpack archive {}: file not found", path_to_archive); return handle.getFileInfo(); } @@ -278,7 +279,9 @@ template std::unique_ptr LibArchiveReader::readFile(NameFilter filter) { Handle handle(path_to_archive); - handle.locateFile(filter); + if (!handle.locateFile(filter)) + throw Exception( + ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't unpack archive {}: no file found satisfying the filter", path_to_archive); return std::make_unique(std::move(handle), path_to_archive); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 3d87793d06c..5fdb049fc4b 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1403,6 +1403,9 @@ SinkToStoragePtr StorageFile::write( ContextPtr context, bool /*async_insert*/) { + if (!use_table_fd && !paths_to_archive.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Writing to archives is not supported"); + if (format_name == "Distributed") throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is not implemented for Distributed format"); diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.reference b/tests/queries/0_stateless/02661_read_from_archive.reference similarity index 52% rename from tests/queries/0_stateless/02661_select_from_table_in_archive.reference rename to tests/queries/0_stateless/02661_read_from_archive.reference index 67f1d859da5..eeeb4edb6ba 100644 --- a/tests/queries/0_stateless/02661_select_from_table_in_archive.reference +++ b/tests/queries/0_stateless/02661_read_from_archive.reference @@ -3,50 +3,54 @@ Running for zip files 3 4 1 2 3 4 -1 -3 -1 -3 -1 -3 -1 -3 +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +OK Running for tar.gz files 1 2 3 4 1 2 3 4 -1 -3 -1 -3 -1 -3 -1 -3 +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +OK Running for tar files 1 2 3 4 1 2 3 4 -1 -3 -1 -3 -1 -3 -1 -3 +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +OK Running for 7z files 1 2 3 4 1 2 3 4 -1 -3 -1 -3 -1 -3 -1 -3 +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +OK diff --git a/tests/queries/0_stateless/02661_read_from_archive.sh b/tests/queries/0_stateless/02661_read_from_archive.sh new file mode 100755 index 00000000000..d7dea772844 --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +source ${CUR_DIR}/02661_read_from_archive.lib + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS 02661_archive_table" + +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +echo -e "1,2\n3,4" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv + +function run_archive_test() { + echo "Running for $1 files" + + eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null" + eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null" + + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1..2}.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" + + $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1::nonexistent.csv')" 2>&1 | grep -q "CANNOT_UNPACK_ARCHIVE" && echo "OK" || echo "FAIL" + + rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1 + rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.$1 +} + +run_archive_test "zip" "zip" +run_archive_test "tar.gz" "tar -cvzf" +run_archive_test "tar" "tar -cvf" +run_archive_test "7z" "7z a" + +rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv diff --git a/tests/queries/0_stateless/02661_read_from_archive_with_globs.reference b/tests/queries/0_stateless/02661_read_from_archive_with_globs.reference new file mode 100644 index 00000000000..00063fecc54 --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive_with_globs.reference @@ -0,0 +1,264 @@ +Running for zip files +archive{1,2} data{1,3}.csv +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +archive3 data*.csv +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +archive* *.csv +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +archive* {2..3}.csv +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +OK +Running for tar.gz files +archive{1,2} data{1,3}.csv +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +archive3 data*.csv +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +archive* *.csv +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +archive* {2..3}.csv +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +OK +Running for tar files +archive{1,2} data{1,3}.csv +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +archive3 data*.csv +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +archive* *.csv +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +archive* {2..3}.csv +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +OK +Running for 7z files +archive{1,2} data{1,3}.csv +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +archive3 data*.csv +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +archive* *.csv +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +archive* {2..3}.csv +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +OK diff --git a/tests/queries/0_stateless/02661_read_from_archive_with_globs.sh b/tests/queries/0_stateless/02661_read_from_archive_with_globs.sh new file mode 100755 index 00000000000..2ec763b456e --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive_with_globs.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +source ${CUR_DIR}/02661_read_from_archive.lib + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS 02661_archive_table" + +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +echo -e "1,2\n3,4" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data1.csv +echo -e "5,6\n7,8" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data2.csv +echo -e "9,10\n11,12" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data3.csv + +function run_archive_test() { + echo "Running for $1 files" + + eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive1.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data1.csv ${CLICKHOUSE_TEST_UNIQUE_NAME}_data2.csv > /dev/null" + eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive2.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data1.csv ${CLICKHOUSE_TEST_UNIQUE_NAME}_data3.csv > /dev/null" + eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive3.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data2.csv ${CLICKHOUSE_TEST_UNIQUE_NAME}_data3.csv > /dev/null" + + echo "archive{1,2} data{1,3}.csv" + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive{1,2}.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data{1,3}.csv" + echo "archive3 data*.csv" + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive3.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data*.csv" + echo "archive* *.csv" + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive*.$1 :: *.csv" + echo "archive* {2..3}.csv" + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive*.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data{2..3}.csv" + + $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive3.$1::{2..3}.csv')" 2>&1 | grep -q "CANNOT_UNPACK_ARCHIVE" && echo "OK" || echo "FAIL" + + rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive1.$1 + rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive2.$1 + rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive3.$1 +} + +run_archive_test "zip" "zip" +run_archive_test "tar.gz" "tar -cvzf" +run_archive_test "tar" "tar -cvf" +run_archive_test "7z" "7z a" + +rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_data{1..3}.csv \ No newline at end of file diff --git a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh b/tests/queries/0_stateless/02661_select_from_table_in_archive.sh deleted file mode 100755 index 6e6342e3d54..00000000000 --- a/tests/queries/0_stateless/02661_select_from_table_in_archive.sh +++ /dev/null @@ -1,59 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS 02661_archive_table" - -user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -echo -e "1,2\n3,4" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv - -function read_archive_file() { - $CLICKHOUSE_LOCAL --query "SELECT $1 FROM file('${user_files_path}/$2')" - $CLICKHOUSE_CLIENT --query "CREATE TABLE 02661_archive_table Engine=File('CSV', '${user_files_path}/$2')" - $CLICKHOUSE_CLIENT --query "SELECT $1 FROM 02661_archive_table" - $CLICKHOUSE_CLIENT --query "DROP TABLE 02661_archive_table" -} - -function run_archive_test() { - echo "Running for $1 files" - read_archive_file "*" "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" - read_archive_file "c1" "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1..2}.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" -} - -zip ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null -zip ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null - -run_archive_test "zip" - -rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.zip -rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.zip - -tar -cvzf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar.gz ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv &> /dev/null -tar -cvzf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar.gz ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv &> /dev/null - -run_archive_test "tar.gz" - -rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar.gz -rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar.gz - -tar -cvf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv &> /dev/null -tar -cvf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv &> /dev/null - -run_archive_test "tar" - -rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.tar -rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.tar - -7z a ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.7z ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null -7z a ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.7z ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null - -run_archive_test "7z" - -rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.7z -rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.7z - -rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv From 84df903958afa1cad2c6ba652c373a186131d7f2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 Aug 2023 10:48:24 +0000 Subject: [PATCH 1526/2047] Decrease a num of tries for a couple of too slow tests for debug. --- tests/queries/0_stateless/00719_parallel_ddl_db.sh | 2 +- .../0_stateless/02450_kill_distributed_query_deadlock.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00719_parallel_ddl_db.sh b/tests/queries/0_stateless/00719_parallel_ddl_db.sh index 31ea1dbbe58..004590c21df 100755 --- a/tests/queries/0_stateless/00719_parallel_ddl_db.sh +++ b/tests/queries/0_stateless/00719_parallel_ddl_db.sh @@ -11,7 +11,7 @@ ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl" function query() { - for _ in {1..100}; do + for _ in {1..50}; do ${CLICKHOUSE_CLIENT} --query "CREATE DATABASE IF NOT EXISTS parallel_ddl" ${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS parallel_ddl" done diff --git a/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh b/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh index abcf1bf4c5b..d15acba3837 100755 --- a/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh +++ b/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Test that running distributed query and cancel it ASAP, # this can trigger a hung/deadlock in ProcessorList. -for i in {1..100}; do +for i in {1..50}; do query_id="$CLICKHOUSE_TEST_UNIQUE_NAME-$i" $CLICKHOUSE_CLIENT --format Null --query_id "$query_id" -q "select * from remote('127.{1|2|3|4|5|6}', numbers(1e12))" 2>/dev/null & while :; do From 15b05745fbd2adc0ebcd62ecee6081da806a7b98 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 3 Aug 2023 11:32:00 +0000 Subject: [PATCH 1527/2047] Fix: adapt query to avoid issue with analyzer See #52982 --- .../0_stateless/02835_fuzz_remove_redundant_sorting.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.sql b/tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.sql index b5fac5a56a7..bdbc5594189 100644 --- a/tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.sql +++ b/tests/queries/0_stateless/02835_fuzz_remove_redundant_sorting.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS numbers500k; -CREATE TABLE numbers500k (`number` UInt32) ENGINE = TinyLog; +CREATE TABLE numbers500k (`number` UInt32) ENGINE = MergeTree() ORDER BY tuple(); INSERT INTO numbers500k SELECT number FROM system.numbers LIMIT 500000; -SELECT intDiv(number, NULL) AS k FROM (SELECT * FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) PREWHERE 31 WHERE 65537 ORDER BY number DESC NULLS FIRST) GROUP BY GROUPING SETS ((k)) WITH TOTALS ORDER BY k ASC NULLS LAST LIMIT 2147483648; +SELECT intDiv(number, NULL) AS k FROM (SELECT * FROM remote('127.0.0.{2,3}', currentDatabase(), numbers500k) PREWHERE 31 WHERE 65537 > 0 ORDER BY number DESC NULLS FIRST) GROUP BY GROUPING SETS ((k)) WITH TOTALS ORDER BY k ASC NULLS LAST LIMIT 2147483648; DROP TABLE IF EXISTS numbers500k; From 27701ed24e68157b85f0f104afc069d364751428 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 3 Aug 2023 11:44:24 +0000 Subject: [PATCH 1528/2047] Small fixes --- src/IO/Archives/LibArchiveReader.cpp | 2 -- src/IO/Archives/ZipArchiveReader.cpp | 1 - tests/queries/0_stateless/02661_read_from_archive.sh | 3 ++- .../queries/0_stateless/02661_read_from_archive_with_globs.sh | 3 ++- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 148de3bffdd..d819547c9bd 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -14,8 +14,6 @@ namespace ErrorCodes { extern const int CANNOT_UNPACK_ARCHIVE; extern const int LOGICAL_ERROR; - extern const int SEEK_POSITION_OUT_OF_BOUND; - extern const int NOT_IMPLEMENTED; extern const int CANNOT_READ_ALL_DATA; extern const int UNSUPPORTED_METHOD; } diff --git a/src/IO/Archives/ZipArchiveReader.cpp b/src/IO/Archives/ZipArchiveReader.cpp index 84a8001e70e..181174ef6ec 100644 --- a/src/IO/Archives/ZipArchiveReader.cpp +++ b/src/IO/Archives/ZipArchiveReader.cpp @@ -1,5 +1,4 @@ #include -#include "mz_compat.h" #if USE_MINIZIP #include diff --git a/tests/queries/0_stateless/02661_read_from_archive.sh b/tests/queries/0_stateless/02661_read_from_archive.sh index d7dea772844..6d69c9e80c7 100755 --- a/tests/queries/0_stateless/02661_read_from_archive.sh +++ b/tests/queries/0_stateless/02661_read_from_archive.sh @@ -5,7 +5,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -source ${CUR_DIR}/02661_read_from_archive.lib +# shellcheck source=./02661_read_from_archive.lib +. "$CUR_DIR"/02661_read_from_archive.lib $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS 02661_archive_table" diff --git a/tests/queries/0_stateless/02661_read_from_archive_with_globs.sh b/tests/queries/0_stateless/02661_read_from_archive_with_globs.sh index 2ec763b456e..12d07112a63 100755 --- a/tests/queries/0_stateless/02661_read_from_archive_with_globs.sh +++ b/tests/queries/0_stateless/02661_read_from_archive_with_globs.sh @@ -5,7 +5,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -source ${CUR_DIR}/02661_read_from_archive.lib +# shellcheck source=./02661_read_from_archive.lib +. "$CUR_DIR"/02661_read_from_archive.lib $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS 02661_archive_table" From 16dfb028b68b6c18185582d534c3b1d948ef88f4 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 16 Jul 2023 13:40:47 +0200 Subject: [PATCH 1529/2047] Remove creation of a unnecessary temporary ContextAccess on login. --- src/Access/AccessControl.cpp | 18 +++++++++++++ src/Access/AccessControl.h | 11 ++++++++ src/Interpreters/Context.cpp | 49 +++++++++++------------------------- src/Interpreters/Context.h | 6 ++--- 4 files changed, 46 insertions(+), 38 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index bf0a2a0fbba..05cba7f8510 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -729,6 +730,14 @@ std::shared_ptr AccessControl::getEnabledRoles( } +std::shared_ptr AccessControl::getEnabledRolesInfo( + const std::vector & current_roles, + const std::vector & current_roles_with_admin_option) const +{ + return getEnabledRoles(current_roles, current_roles_with_admin_option)->getRolesInfo(); +} + + std::shared_ptr AccessControl::getEnabledRowPolicies(const UUID & user_id, const boost::container::flat_set & enabled_roles) const { return row_policy_cache->getEnabledRowPolicies(user_id, enabled_roles); @@ -772,6 +781,15 @@ std::shared_ptr AccessControl::getEnabledSettings( return settings_profiles_cache->getEnabledSettings(user_id, settings_from_user, enabled_roles, settings_from_enabled_roles); } +std::shared_ptr AccessControl::getEnabledSettingsInfo( + const UUID & user_id, + const SettingsProfileElements & settings_from_user, + const boost::container::flat_set & enabled_roles, + const SettingsProfileElements & settings_from_enabled_roles) const +{ + return getEnabledSettings(user_id, settings_from_user, enabled_roles, settings_from_enabled_roles)->getInfo(); +} + std::shared_ptr AccessControl::getSettingsProfileInfo(const UUID & profile_id) { return settings_profiles_cache->getSettingsProfileInfo(profile_id); diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 74816090f88..c7b94955a47 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -29,6 +29,7 @@ class ContextAccessParams; struct User; using UserPtr = std::shared_ptr; class EnabledRoles; +struct EnabledRolesInfo; class RoleCache; class EnabledRowPolicies; class RowPolicyCache; @@ -187,6 +188,10 @@ public: const std::vector & current_roles, const std::vector & current_roles_with_admin_option) const; + std::shared_ptr getEnabledRolesInfo( + const std::vector & current_roles, + const std::vector & current_roles_with_admin_option) const; + std::shared_ptr getEnabledRowPolicies( const UUID & user_id, const boost::container::flat_set & enabled_roles) const; @@ -209,6 +214,12 @@ public: const boost::container::flat_set & enabled_roles, const SettingsProfileElements & settings_from_enabled_roles) const; + std::shared_ptr getEnabledSettingsInfo( + const UUID & user_id, + const SettingsProfileElements & settings_from_user, + const boost::container::flat_set & enabled_roles, + const SettingsProfileElements & settings_from_enabled_roles) const; + std::shared_ptr getSettingsProfileInfo(const UUID & profile_id); const ExternalAuthenticators & getExternalAuthenticators() const; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a5ff7643294..b140d8cab2d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1089,52 +1089,31 @@ ConfigurationPtr Context::getUsersConfig() return shared->users_config; } -void Context::setUser(const UUID & user_id_, bool set_current_profiles_, bool set_current_roles_, bool set_current_database_) +void Context::setUser(const UUID & user_id_) { /// Prepare lists of user's profiles, constraints, settings, roles. + /// NOTE: AccessControl::read() and other AccessControl's functions may require some IO work, + /// so Context::getLock() must be unlocked while we're doing this. - std::shared_ptr user; - std::shared_ptr temp_access; - if (set_current_profiles_ || set_current_roles_ || set_current_database_) - { - std::optional params; - { - auto lock = getLock(); - params.emplace(ContextAccessParams{user_id_, /* full_access= */ false, /* use_default_roles = */ true, {}, settings, current_database, client_info }); - } - /// `temp_access` is used here only to extract information about the user, not to actually check access. - /// NOTE: AccessControl::getContextAccess() may require some IO work, so Context::getLock() must be unlocked while we're doing this. - temp_access = getAccessControl().getContextAccess(*params); - user = temp_access->getUser(); - } - - std::shared_ptr profiles; - if (set_current_profiles_) - profiles = temp_access->getDefaultProfileInfo(); - - std::optional> roles; - if (set_current_roles_) - roles = user->granted_roles.findGranted(user->default_roles); - - String database; - if (set_current_database_) - database = user->default_database; + auto user = getAccessControl().read(user_id_); + auto default_roles = user->granted_roles.findGranted(user->default_roles); + auto enabled_roles = getAccessControl().getEnabledRolesInfo(default_roles, {}); + auto enabled_profiles = getAccessControl().getEnabledSettingsInfo(user_id_, user->settings, enabled_roles->enabled_roles, enabled_roles->settings_from_enabled_roles); + const auto & database = user->default_database; /// Apply user's profiles, constraints, settings, roles. + auto lock = getLock(); setUserID(user_id_); - if (profiles) - { - /// A profile can specify a value and a readonly constraint for same setting at the same time, - /// so we shouldn't check constraints here. - setCurrentProfiles(*profiles, /* check_constraints= */ false); - } + /// A profile can specify a value and a readonly constraint for same setting at the same time, + /// so we shouldn't check constraints here. + setCurrentProfiles(*enabled_profiles, /* check_constraints= */ false); - if (roles) - setCurrentRoles(*roles); + setCurrentRoles(default_roles); + /// It's optional to specify the DEFAULT DATABASE in the user's definition. if (!database.empty()) setCurrentDatabase(database); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 676eb8412e5..753e4dbf6f1 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -534,12 +534,10 @@ public: /// Sets the current user assuming that he/she is already authenticated. /// WARNING: This function doesn't check password! - void setUser(const UUID & user_id_, bool set_current_profiles_ = true, bool set_current_roles_ = true, bool set_current_database_ = true); + void setUser(const UUID & user_id_); UserPtr getUser() const; - void setUserID(const UUID & user_id_); std::optional getUserID() const; - String getUserName() const; void setCurrentRoles(const std::vector & current_roles_); @@ -1195,6 +1193,8 @@ private: void initGlobal(); + void setUserID(const UUID & user_id_); + template void checkAccessImpl(const Args &... args) const; From 7a112459d3d52aa32962eb8d4c22d9f59f070ea6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 27 Jul 2023 16:23:19 +0200 Subject: [PATCH 1530/2047] Implement passing current roles to Context::setUser(). --- src/Interpreters/Context.cpp | 9 +++++---- src/Interpreters/Context.h | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b140d8cab2d..e6b9b12b6ca 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1089,15 +1089,16 @@ ConfigurationPtr Context::getUsersConfig() return shared->users_config; } -void Context::setUser(const UUID & user_id_) +void Context::setUser(const UUID & user_id_, const std::optional> & current_roles_) { /// Prepare lists of user's profiles, constraints, settings, roles. /// NOTE: AccessControl::read() and other AccessControl's functions may require some IO work, /// so Context::getLock() must be unlocked while we're doing this. auto user = getAccessControl().read(user_id_); - auto default_roles = user->granted_roles.findGranted(user->default_roles); - auto enabled_roles = getAccessControl().getEnabledRolesInfo(default_roles, {}); + + auto new_current_roles = current_roles_ ? user->granted_roles.findGranted(*current_roles_) : user->granted_roles.findGranted(user->default_roles); + auto enabled_roles = getAccessControl().getEnabledRolesInfo(new_current_roles, {}); auto enabled_profiles = getAccessControl().getEnabledSettingsInfo(user_id_, user->settings, enabled_roles->enabled_roles, enabled_roles->settings_from_enabled_roles); const auto & database = user->default_database; @@ -1111,7 +1112,7 @@ void Context::setUser(const UUID & user_id_) /// so we shouldn't check constraints here. setCurrentProfiles(*enabled_profiles, /* check_constraints= */ false); - setCurrentRoles(default_roles); + setCurrentRoles(new_current_roles); /// It's optional to specify the DEFAULT DATABASE in the user's definition. if (!database.empty()) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 753e4dbf6f1..531207de60e 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -534,7 +534,7 @@ public: /// Sets the current user assuming that he/she is already authenticated. /// WARNING: This function doesn't check password! - void setUser(const UUID & user_id_); + void setUser(const UUID & user_id_, const std::optional> & current_roles_ = {}); UserPtr getUser() const; std::optional getUserID() const; From dc740db3fb9cbd7a289424108ebe652badb01b05 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 13:46:57 +0200 Subject: [PATCH 1531/2047] Fix test 00061_storage_buffer --- tests/queries/1_stateful/00061_storage_buffer.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00061_storage_buffer.sql b/tests/queries/1_stateful/00061_storage_buffer.sql index e1f67abda20..e3cda3de36d 100644 --- a/tests/queries/1_stateful/00061_storage_buffer.sql +++ b/tests/queries/1_stateful/00061_storage_buffer.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS test.hits_dst; DROP TABLE IF EXISTS test.hits_buffer; CREATE TABLE test.hits_dst AS test.hits; -CREATE TABLE test.hits_buffer AS test.hits_dst ENGINE = Buffer(test, hits_dst, 8, 1, 10, 10000, 100000, 10000000, 100000000); +CREATE TABLE test.hits_buffer AS test.hits_dst ENGINE = Buffer(test, hits_dst, 8, 600, 600, 1000000, 1000000, 100000000, 1000000000); INSERT INTO test.hits_buffer SELECT * FROM test.hits WHERE CounterID = 800784; SELECT count() FROM test.hits_buffer; From f427c779ebf854e1aa842063f9d497a48b746bdc Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 20 Jul 2023 09:46:22 +0200 Subject: [PATCH 1532/2047] Disable using fs cache for backup/restore. --- src/Backups/BackupEntryFromImmutableFile.cpp | 8 ++-- src/Backups/BackupEntryFromImmutableFile.h | 4 +- .../BackupEntryWithChecksumCalculation.cpp | 15 +++--- .../BackupEntryWithChecksumCalculation.h | 4 +- src/Backups/BackupEntryWrappedWith.h | 4 +- src/Backups/BackupFactory.h | 4 ++ src/Backups/BackupFileInfo.cpp | 25 ++++++---- src/Backups/BackupFileInfo.h | 5 +- src/Backups/BackupIO_Default.cpp | 13 +++--- src/Backups/BackupIO_Default.h | 5 +- src/Backups/BackupIO_Disk.cpp | 8 ++-- src/Backups/BackupIO_Disk.h | 4 +- src/Backups/BackupIO_File.cpp | 8 ++-- src/Backups/BackupIO_File.h | 4 +- src/Backups/BackupIO_S3.cpp | 21 +++++++-- src/Backups/BackupIO_S3.h | 4 +- src/Backups/BackupSettings.cpp | 1 + src/Backups/BackupSettings.h | 4 ++ src/Backups/BackupsWorker.cpp | 46 +++++++++++++++++-- src/Backups/BackupsWorker.h | 3 +- src/Backups/IBackupEntriesLazyBatch.cpp | 4 +- src/Backups/IBackupEntry.h | 4 +- src/Backups/registerBackupEngineS3.cpp | 19 +++++++- .../registerBackupEnginesFileAndDisk.cpp | 8 ++-- src/Interpreters/Context.cpp | 8 ---- src/Interpreters/Context.h | 3 -- src/Storages/MergeTree/MergeTreeData.cpp | 1 - 27 files changed, 152 insertions(+), 85 deletions(-) diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index 93d555065ec..77ebf6232d4 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -57,7 +57,7 @@ UInt64 BackupEntryFromImmutableFile::getSize() const return *file_size; } -UInt128 BackupEntryFromImmutableFile::getChecksum() const +UInt128 BackupEntryFromImmutableFile::getChecksum(const ReadSettings & read_settings) const { { std::lock_guard lock{size_and_checksum_mutex}; @@ -73,7 +73,7 @@ UInt128 BackupEntryFromImmutableFile::getChecksum() const } } - auto calculated_checksum = BackupEntryWithChecksumCalculation::getChecksum(); + auto calculated_checksum = BackupEntryWithChecksumCalculation::getChecksum(read_settings); { std::lock_guard lock{size_and_checksum_mutex}; @@ -86,13 +86,13 @@ UInt128 BackupEntryFromImmutableFile::getChecksum() const } } -std::optional BackupEntryFromImmutableFile::getPartialChecksum(size_t prefix_length) const +std::optional BackupEntryFromImmutableFile::getPartialChecksum(size_t prefix_length, const ReadSettings & read_settings) const { if (prefix_length == 0) return 0; if (prefix_length >= getSize()) - return getChecksum(); + return getChecksum(read_settings); /// For immutable files we don't use partial checksums. return std::nullopt; diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index 37bc6b43cd3..9e3dc8ebb31 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -27,8 +27,8 @@ public: std::unique_ptr getReadBuffer(const ReadSettings & read_settings) const override; UInt64 getSize() const override; - UInt128 getChecksum() const override; - std::optional getPartialChecksum(size_t prefix_length) const override; + UInt128 getChecksum(const ReadSettings & read_settings) const override; + std::optional getPartialChecksum(size_t prefix_length, const ReadSettings & read_settings) const override; DataSourceDescription getDataSourceDescription() const override { return data_source_description; } bool isEncryptedByDisk() const override { return copy_encrypted; } diff --git a/src/Backups/BackupEntryWithChecksumCalculation.cpp b/src/Backups/BackupEntryWithChecksumCalculation.cpp index 610b46238ba..a507e1b0a84 100644 --- a/src/Backups/BackupEntryWithChecksumCalculation.cpp +++ b/src/Backups/BackupEntryWithChecksumCalculation.cpp @@ -6,7 +6,7 @@ namespace DB { template -UInt128 BackupEntryWithChecksumCalculation::getChecksum() const +UInt128 BackupEntryWithChecksumCalculation::getChecksum(const ReadSettings & read_settings) const { { std::lock_guard lock{checksum_calculation_mutex}; @@ -26,7 +26,7 @@ UInt128 BackupEntryWithChecksumCalculation::getChecksum() const } else { - auto read_buffer = this->getReadBuffer(ReadSettings{}.adjustBufferSize(size)); + auto read_buffer = this->getReadBuffer(read_settings.adjustBufferSize(size)); HashingReadBuffer hashing_read_buffer(*read_buffer); hashing_read_buffer.ignoreAll(); calculated_checksum = hashing_read_buffer.getHash(); @@ -37,23 +37,20 @@ UInt128 BackupEntryWithChecksumCalculation::getChecksum() const } template -std::optional BackupEntryWithChecksumCalculation::getPartialChecksum(size_t prefix_length) const +std::optional BackupEntryWithChecksumCalculation::getPartialChecksum(size_t prefix_length, const ReadSettings & read_settings) const { if (prefix_length == 0) return 0; size_t size = this->getSize(); if (prefix_length >= size) - return this->getChecksum(); + return this->getChecksum(read_settings); std::lock_guard lock{checksum_calculation_mutex}; - ReadSettings read_settings; - if (calculated_checksum) - read_settings.adjustBufferSize(calculated_checksum ? prefix_length : size); - - auto read_buffer = this->getReadBuffer(read_settings); + auto read_buffer = this->getReadBuffer(read_settings.adjustBufferSize(calculated_checksum ? prefix_length : size)); HashingReadBuffer hashing_read_buffer(*read_buffer); + hashing_read_buffer.ignore(prefix_length); auto partial_checksum = hashing_read_buffer.getHash(); diff --git a/src/Backups/BackupEntryWithChecksumCalculation.h b/src/Backups/BackupEntryWithChecksumCalculation.h index 32701ab9952..99ed4a32462 100644 --- a/src/Backups/BackupEntryWithChecksumCalculation.h +++ b/src/Backups/BackupEntryWithChecksumCalculation.h @@ -11,8 +11,8 @@ template class BackupEntryWithChecksumCalculation : public Base { public: - UInt128 getChecksum() const override; - std::optional getPartialChecksum(size_t prefix_length) const override; + UInt128 getChecksum(const ReadSettings & read_settings) const override; + std::optional getPartialChecksum(size_t prefix_length, const ReadSettings & read_settings) const override; private: mutable std::optional calculated_checksum; diff --git a/src/Backups/BackupEntryWrappedWith.h b/src/Backups/BackupEntryWrappedWith.h index f865d529206..7f04c135921 100644 --- a/src/Backups/BackupEntryWrappedWith.h +++ b/src/Backups/BackupEntryWrappedWith.h @@ -17,8 +17,8 @@ public: std::unique_ptr getReadBuffer(const ReadSettings & read_settings) const override { return entry->getReadBuffer(read_settings); } UInt64 getSize() const override { return entry->getSize(); } - UInt128 getChecksum() const override { return entry->getChecksum(); } - std::optional getPartialChecksum(size_t prefix_length) const override { return entry->getPartialChecksum(prefix_length); } + UInt128 getChecksum(const ReadSettings & read_settings) const override { return entry->getChecksum(read_settings); } + std::optional getPartialChecksum(size_t prefix_length, const ReadSettings & read_settings) const override { return entry->getPartialChecksum(prefix_length, read_settings); } DataSourceDescription getDataSourceDescription() const override { return entry->getDataSourceDescription(); } bool isEncryptedByDisk() const override { return entry->isEncryptedByDisk(); } bool isFromFile() const override { return entry->isFromFile(); } diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index a79c6d354fc..ecdbd5cffbc 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -37,6 +39,8 @@ public: std::optional backup_uuid; bool deduplicate_files = true; bool allow_s3_native_copy = true; + ReadSettings read_settings; + WriteSettings write_settings; }; static BackupFactory & instance(); diff --git a/src/Backups/BackupFileInfo.cpp b/src/Backups/BackupFileInfo.cpp index d539ada55c4..f595c02ddc5 100644 --- a/src/Backups/BackupFileInfo.cpp +++ b/src/Backups/BackupFileInfo.cpp @@ -57,12 +57,12 @@ namespace /// Calculate checksum for backup entry if it's empty. /// Also able to calculate additional checksum of some prefix. - ChecksumsForNewEntry calculateNewEntryChecksumsIfNeeded(const BackupEntryPtr & entry, size_t prefix_size) + ChecksumsForNewEntry calculateNewEntryChecksumsIfNeeded(const BackupEntryPtr & entry, size_t prefix_size, const ReadSettings & read_settings) { ChecksumsForNewEntry res; /// The partial checksum should be calculated before the full checksum to enable optimization in BackupEntryWithChecksumCalculation. - res.prefix_checksum = entry->getPartialChecksum(prefix_size); - res.full_checksum = entry->getChecksum(); + res.prefix_checksum = entry->getPartialChecksum(prefix_size, read_settings); + res.full_checksum = entry->getChecksum(read_settings); return res; } @@ -93,7 +93,12 @@ String BackupFileInfo::describe() const } -BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const BackupEntryPtr & backup_entry, const BackupPtr & base_backup, Poco::Logger * log) +BackupFileInfo buildFileInfoForBackupEntry( + const String & file_name, + const BackupEntryPtr & backup_entry, + const BackupPtr & base_backup, + const ReadSettings & read_settings, + Poco::Logger * log) { auto adjusted_path = removeLeadingSlash(file_name); @@ -126,7 +131,7 @@ BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const Backu /// File with the same name but smaller size exist in previous backup if (check_base == CheckBackupResult::HasPrefix) { - auto checksums = calculateNewEntryChecksumsIfNeeded(backup_entry, base_backup_file_info->first); + auto checksums = calculateNewEntryChecksumsIfNeeded(backup_entry, base_backup_file_info->first, read_settings); info.checksum = checksums.full_checksum; /// We have prefix of this file in backup with the same checksum. @@ -146,7 +151,7 @@ BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const Backu { /// We have full file or have nothing, first of all let's get checksum /// of current file - auto checksums = calculateNewEntryChecksumsIfNeeded(backup_entry, 0); + auto checksums = calculateNewEntryChecksumsIfNeeded(backup_entry, 0, read_settings); info.checksum = checksums.full_checksum; if (info.checksum == base_backup_file_info->second) @@ -169,7 +174,7 @@ BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const Backu } else { - auto checksums = calculateNewEntryChecksumsIfNeeded(backup_entry, 0); + auto checksums = calculateNewEntryChecksumsIfNeeded(backup_entry, 0, read_settings); info.checksum = checksums.full_checksum; } @@ -188,7 +193,7 @@ BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const Backu return info; } -BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entries, const BackupPtr & base_backup, ThreadPool & thread_pool) +BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entries, const BackupPtr & base_backup, const ReadSettings & read_settings, ThreadPool & thread_pool) { BackupFileInfos infos; infos.resize(backup_entries.size()); @@ -210,7 +215,7 @@ BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entr ++num_active_jobs; } - auto job = [&mutex, &num_active_jobs, &event, &exception, &infos, &backup_entries, &base_backup, &thread_group, i, log](bool async) + auto job = [&mutex, &num_active_jobs, &event, &exception, &infos, &backup_entries, &read_settings, &base_backup, &thread_group, i, log](bool async) { SCOPE_EXIT_SAFE({ std::lock_guard lock{mutex}; @@ -237,7 +242,7 @@ BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entr return; } - infos[i] = buildFileInfoForBackupEntry(name, entry, base_backup, log); + infos[i] = buildFileInfoForBackupEntry(name, entry, base_backup, read_settings, log); } catch (...) { diff --git a/src/Backups/BackupFileInfo.h b/src/Backups/BackupFileInfo.h index a925a1e81ac..63da6f23427 100644 --- a/src/Backups/BackupFileInfo.h +++ b/src/Backups/BackupFileInfo.h @@ -13,6 +13,7 @@ class IBackupEntry; using BackupPtr = std::shared_ptr; using BackupEntryPtr = std::shared_ptr; using BackupEntries = std::vector>; +struct ReadSettings; /// Information about a file stored in a backup. @@ -66,9 +67,9 @@ struct BackupFileInfo using BackupFileInfos = std::vector; /// Builds a BackupFileInfo for a specified backup entry. -BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const BackupEntryPtr & backup_entry, const BackupPtr & base_backup, Poco::Logger * log); +BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const BackupEntryPtr & backup_entry, const BackupPtr & base_backup, const ReadSettings & read_settings, Poco::Logger * log); /// Builds a vector of BackupFileInfos for specified backup entries. -BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entries, const BackupPtr & base_backup, ThreadPool & thread_pool); +BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entries, const BackupPtr & base_backup, const ReadSettings & read_settings, ThreadPool & thread_pool); } diff --git a/src/Backups/BackupIO_Default.cpp b/src/Backups/BackupIO_Default.cpp index b36cb22498d..5ac522695ce 100644 --- a/src/Backups/BackupIO_Default.cpp +++ b/src/Backups/BackupIO_Default.cpp @@ -4,17 +4,16 @@ #include #include #include -#include #include namespace DB { -BackupReaderDefault::BackupReaderDefault(Poco::Logger * log_, const ContextPtr & context_) +BackupReaderDefault::BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_) : log(log_) - , read_settings(context_->getBackupReadSettings()) - , write_settings(context_->getWriteSettings()) + , read_settings(read_settings_) + , write_settings(write_settings_) , write_buffer_size(DBMS_DEFAULT_BUFFER_SIZE) { } @@ -37,10 +36,10 @@ void BackupReaderDefault::copyFileToDisk(const String & path_in_backup, size_t f write_buffer->finalize(); } -BackupWriterDefault::BackupWriterDefault(Poco::Logger * log_, const ContextPtr & context_) +BackupWriterDefault::BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_) : log(log_) - , read_settings(context_->getBackupReadSettings()) - , write_settings(context_->getWriteSettings()) + , read_settings(read_settings_) + , write_settings(write_settings_) , write_buffer_size(DBMS_DEFAULT_BUFFER_SIZE) { } diff --git a/src/Backups/BackupIO_Default.h b/src/Backups/BackupIO_Default.h index ad7bdf15d9f..b4888fecd2f 100644 --- a/src/Backups/BackupIO_Default.h +++ b/src/Backups/BackupIO_Default.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB @@ -19,7 +18,7 @@ enum class WriteMode; class BackupReaderDefault : public IBackupReader { public: - BackupReaderDefault(Poco::Logger * log_, const ContextPtr & context_); + BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_); ~BackupReaderDefault() override = default; /// The function copyFileToDisk() can be much faster than reading the file with readFile() and then writing it to some disk. @@ -46,7 +45,7 @@ protected: class BackupWriterDefault : public IBackupWriter { public: - BackupWriterDefault(Poco::Logger * log_, const ContextPtr & context_); + BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_); ~BackupWriterDefault() override = default; bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index 1514b4c24c7..21b3afbddf8 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -8,8 +8,8 @@ namespace DB { -BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & root_path_, const ContextPtr & context_) - : BackupReaderDefault(&Poco::Logger::get("BackupReaderDisk"), context_) +BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_) + : BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderDisk")) , disk(disk_) , root_path(root_path_) , data_source_description(disk->getDataSourceDescription()) @@ -56,8 +56,8 @@ void BackupReaderDisk::copyFileToDisk(const String & path_in_backup, size_t file } -BackupWriterDisk::BackupWriterDisk(const DiskPtr & disk_, const String & root_path_, const ContextPtr & context_) - : BackupWriterDefault(&Poco::Logger::get("BackupWriterDisk"), context_) +BackupWriterDisk::BackupWriterDisk(const DiskPtr & disk_, const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_) + : BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterDisk")) , disk(disk_) , root_path(root_path_) , data_source_description(disk->getDataSourceDescription()) diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index faf4ef03447..70d31eacc1a 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -13,7 +13,7 @@ using DiskPtr = std::shared_ptr; class BackupReaderDisk : public BackupReaderDefault { public: - BackupReaderDisk(const DiskPtr & disk_, const String & root_path_, const ContextPtr & context_); + BackupReaderDisk(const DiskPtr & disk_, const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_); ~BackupReaderDisk() override; bool fileExists(const String & file_name) override; @@ -33,7 +33,7 @@ private: class BackupWriterDisk : public BackupWriterDefault { public: - BackupWriterDisk(const DiskPtr & disk_, const String & root_path_, const ContextPtr & context_); + BackupWriterDisk(const DiskPtr & disk_, const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_); ~BackupWriterDisk() override; bool fileExists(const String & file_name) override; diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index e1a3f336521..2bedb5470fb 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -16,8 +16,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -BackupReaderFile::BackupReaderFile(const String & root_path_, const ContextPtr & context_) - : BackupReaderDefault(&Poco::Logger::get("BackupReaderFile"), context_) +BackupReaderFile::BackupReaderFile(const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_) + : BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderFile")) , root_path(root_path_) , data_source_description(DiskLocal::getLocalDataSourceDescription(root_path)) { @@ -74,8 +74,8 @@ void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file } -BackupWriterFile::BackupWriterFile(const String & root_path_, const ContextPtr & context_) - : BackupWriterDefault(&Poco::Logger::get("BackupWriterFile"), context_) +BackupWriterFile::BackupWriterFile(const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_) + : BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterFile")) , root_path(root_path_) , data_source_description(DiskLocal::getLocalDataSourceDescription(root_path)) { diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index fd2c0b07158..6bb4b11e134 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -11,7 +11,7 @@ namespace DB class BackupReaderFile : public BackupReaderDefault { public: - explicit BackupReaderFile(const String & root_path_, const ContextPtr & context_); + explicit BackupReaderFile(const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_); bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; @@ -29,7 +29,7 @@ private: class BackupWriterFile : public BackupWriterDefault { public: - BackupWriterFile(const String & root_path_, const ContextPtr & context_); + BackupWriterFile(const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_); bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 31a33ea1a79..56402187703 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -101,8 +101,14 @@ namespace BackupReaderS3::BackupReaderS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_) - : BackupReaderDefault(&Poco::Logger::get("BackupReaderS3"), context_) + const S3::URI & s3_uri_, + const String & access_key_id_, + const String & secret_access_key_, + bool allow_s3_native_copy, + const ReadSettings & read_settings_, + const WriteSettings & write_settings_, + const ContextPtr & context_) + : BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderS3")) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) @@ -178,8 +184,15 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s BackupWriterS3::BackupWriterS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const String & storage_class_name, const ContextPtr & context_) - : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) + const S3::URI & s3_uri_, + const String & access_key_id_, + const String & secret_access_key_, + bool allow_s3_native_copy, + const String & storage_class_name, + const ReadSettings & read_settings_, + const WriteSettings & write_settings_, + const ContextPtr & context_) + : BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterS3")) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 8015dade60d..a29c91498ec 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -17,7 +17,7 @@ namespace DB class BackupReaderS3 : public BackupReaderDefault { public: - BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_); + BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); ~BackupReaderS3() override; bool fileExists(const String & file_name) override; @@ -38,7 +38,7 @@ private: class BackupWriterS3 : public BackupWriterDefault { public: - BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const String & storage_class_name, const ContextPtr & context_); + BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const String & storage_class_name, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); ~BackupWriterS3() override; bool fileExists(const String & file_name) override; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 3138959191e..43259e6958a 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -27,6 +27,7 @@ namespace ErrorCodes M(Bool, decrypt_files_from_encrypted_disks) \ M(Bool, deduplicate_files) \ M(Bool, allow_s3_native_copy) \ + M(Bool, read_from_filesystem_cache_if_exists_otherwise_bypass_cache) \ M(UInt64, shard_num) \ M(UInt64, replica_num) \ M(Bool, internal) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index dabfe9a600f..2a950ef1b4f 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -44,6 +44,10 @@ struct BackupSettings /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) bool allow_s3_native_copy = true; + /// Allow to use the filesystem cache in passive mode - benefit from the existing cache entries, + /// but don't put more entries into the cache. + bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; + /// 1-based shard index to store in the backup. 0 means all shards. /// Can only be used with BACKUP ON CLUSTER. size_t shard_num = 0; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 287560d1e5c..8a87f168a0d 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -178,6 +178,42 @@ namespace { return status == BackupStatus::RESTORING; } + + /// We use slightly different read and write settings for backup/restore + /// with a separate throttler and limited usage of filesystem cache. + ReadSettings getReadSettingsForBackup(const ContextPtr & context, const BackupSettings & backup_settings) + { + auto read_settings = context->getReadSettings(); + read_settings.remote_throttler = context->getBackupsThrottler(); + read_settings.local_throttler = context->getBackupsThrottler(); + read_settings.enable_filesystem_cache = false; + read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = backup_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache; + return read_settings; + } + + WriteSettings getWriteSettingsForBackup(const ContextPtr & context) + { + auto write_settings = context->getWriteSettings(); + write_settings.enable_filesystem_cache_on_write_operations = false; + return write_settings; + } + + ReadSettings getReadSettingsForRestore(const ContextPtr & context) + { + auto read_settings = context->getReadSettings(); + read_settings.remote_throttler = context->getBackupsThrottler(); + read_settings.local_throttler = context->getBackupsThrottler(); + read_settings.enable_filesystem_cache = false; + read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = false; + return read_settings; + } + + WriteSettings getWriteSettingsForRestore(const ContextPtr & context) + { + auto write_settings = context->getWriteSettings(); + write_settings.enable_filesystem_cache_on_write_operations = false; + return write_settings; + } } @@ -350,6 +386,8 @@ void BackupsWorker::doBackup( backup_create_params.backup_uuid = backup_settings.backup_uuid; backup_create_params.deduplicate_files = backup_settings.deduplicate_files; backup_create_params.allow_s3_native_copy = backup_settings.allow_s3_native_copy; + backup_create_params.read_settings = getReadSettingsForBackup(context, backup_settings); + backup_create_params.write_settings = getWriteSettingsForBackup(context); BackupMutablePtr backup = BackupFactory::instance().createBackup(backup_create_params); /// Write the backup. @@ -383,7 +421,7 @@ void BackupsWorker::doBackup( } /// Write the backup entries to the backup. - buildFileInfosForBackupEntries(backup, backup_entries, backup_coordination); + buildFileInfosForBackupEntries(backup, backup_entries, backup_create_params.read_settings, backup_coordination); writeBackupEntries(backup, std::move(backup_entries), backup_id, backup_coordination, backup_settings.internal); /// We have written our backup entries, we need to tell other hosts (they could be waiting for it). @@ -433,12 +471,12 @@ void BackupsWorker::doBackup( } -void BackupsWorker::buildFileInfosForBackupEntries(const BackupPtr & backup, const BackupEntries & backup_entries, std::shared_ptr backup_coordination) +void BackupsWorker::buildFileInfosForBackupEntries(const BackupPtr & backup, const BackupEntries & backup_entries, const ReadSettings & read_settings, std::shared_ptr backup_coordination) { LOG_TRACE(log, "{}", Stage::BUILDING_FILE_INFOS); backup_coordination->setStage(Stage::BUILDING_FILE_INFOS, ""); backup_coordination->waitForStage(Stage::BUILDING_FILE_INFOS); - backup_coordination->addFileInfos(::DB::buildFileInfosForBackupEntries(backup_entries, backup->getBaseBackup(), *backups_thread_pool)); + backup_coordination->addFileInfos(::DB::buildFileInfosForBackupEntries(backup_entries, backup->getBaseBackup(), read_settings, *backups_thread_pool)); } @@ -650,6 +688,8 @@ void BackupsWorker::doRestore( backup_open_params.base_backup_info = restore_settings.base_backup_info; backup_open_params.password = restore_settings.password; backup_open_params.allow_s3_native_copy = restore_settings.allow_s3_native_copy; + backup_open_params.read_settings = getReadSettingsForRestore(context); + backup_open_params.write_settings = getWriteSettingsForRestore(context); BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); String current_database = context->getCurrentDatabase(); diff --git a/src/Backups/BackupsWorker.h b/src/Backups/BackupsWorker.h index cbfadc24b7b..ab4359ec257 100644 --- a/src/Backups/BackupsWorker.h +++ b/src/Backups/BackupsWorker.h @@ -24,6 +24,7 @@ using BackupPtr = std::shared_ptr; class IBackupEntry; using BackupEntries = std::vector>>; using DataRestoreTasks = std::vector>; +struct ReadSettings; /// Manager of backups and restores: executes backups and restores' threads in the background. /// Keeps information about backups and restores started in this session. @@ -107,7 +108,7 @@ private: bool called_async); /// Builds file infos for specified backup entries. - void buildFileInfosForBackupEntries(const BackupPtr & backup, const BackupEntries & backup_entries, std::shared_ptr backup_coordination); + void buildFileInfosForBackupEntries(const BackupPtr & backup, const BackupEntries & backup_entries, const ReadSettings & read_settings, std::shared_ptr backup_coordination); /// Write backup entries to an opened backup. void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, const OperationID & backup_id, std::shared_ptr backup_coordination, bool internal); diff --git a/src/Backups/IBackupEntriesLazyBatch.cpp b/src/Backups/IBackupEntriesLazyBatch.cpp index 7c6bb891981..4974d9f6702 100644 --- a/src/Backups/IBackupEntriesLazyBatch.cpp +++ b/src/Backups/IBackupEntriesLazyBatch.cpp @@ -19,8 +19,8 @@ public: std::unique_ptr getReadBuffer(const ReadSettings & read_settings) const override { return getInternalBackupEntry()->getReadBuffer(read_settings); } UInt64 getSize() const override { return getInternalBackupEntry()->getSize(); } - UInt128 getChecksum() const override { return getInternalBackupEntry()->getChecksum(); } - std::optional getPartialChecksum(size_t prefix_length) const override { return getInternalBackupEntry()->getPartialChecksum(prefix_length); } + UInt128 getChecksum(const ReadSettings & read_settings) const override { return getInternalBackupEntry()->getChecksum(read_settings); } + std::optional getPartialChecksum(size_t prefix_length, const ReadSettings & read_settings) const override { return getInternalBackupEntry()->getPartialChecksum(prefix_length, read_settings); } DataSourceDescription getDataSourceDescription() const override { return getInternalBackupEntry()->getDataSourceDescription(); } bool isEncryptedByDisk() const override { return getInternalBackupEntry()->isEncryptedByDisk(); } bool isFromFile() const override { return getInternalBackupEntry()->isFromFile(); } diff --git a/src/Backups/IBackupEntry.h b/src/Backups/IBackupEntry.h index 7e952e9b568..1b72b4358ba 100644 --- a/src/Backups/IBackupEntry.h +++ b/src/Backups/IBackupEntry.h @@ -21,11 +21,11 @@ public: virtual UInt64 getSize() const = 0; /// Returns the checksum of the data. - virtual UInt128 getChecksum() const = 0; + virtual UInt128 getChecksum(const ReadSettings & read_settings) const = 0; /// Returns a partial checksum, i.e. the checksum calculated for a prefix part of the data. /// Can return nullopt if the partial checksum is too difficult to calculate. - virtual std::optional getPartialChecksum(size_t /* prefix_length */) const { return {}; } + virtual std::optional getPartialChecksum(size_t /* prefix_length */, const ReadSettings &) const { return {}; } /// Returns a read buffer for reading the data. virtual std::unique_ptr getReadBuffer(const ReadSettings & read_settings) const = 0; diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 5b6f7825157..451e98b1290 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -107,12 +107,27 @@ void registerBackupEngineS3(BackupFactory & factory) if (params.open_mode == IBackup::OpenMode::READ) { - auto reader = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.allow_s3_native_copy, params.context); + auto reader = std::make_shared(S3::URI{s3_uri}, + access_key_id, + secret_access_key, + params.allow_s3_native_copy, + params.read_settings, + params.write_settings, + params.context); + return std::make_unique(backup_name_for_logging, archive_params, params.base_backup_info, reader, params.context); } else { - auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.allow_s3_native_copy, params.s3_storage_class, params.context); + auto writer = std::make_shared(S3::URI{s3_uri}, + access_key_id, + secret_access_key, + params.allow_s3_native_copy, + params.s3_storage_class, + params.read_settings, + params.write_settings, + params.context); + return std::make_unique( backup_name_for_logging, archive_params, diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index daae9627759..a498e287f15 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -169,18 +169,18 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) { std::shared_ptr reader; if (engine_name == "File") - reader = std::make_shared(path, params.context); + reader = std::make_shared(path, params.read_settings, params.write_settings); else - reader = std::make_shared(disk, path, params.context); + reader = std::make_shared(disk, path, params.read_settings, params.write_settings); return std::make_unique(backup_name_for_logging, archive_params, params.base_backup_info, reader, params.context); } else { std::shared_ptr writer; if (engine_name == "File") - writer = std::make_shared(path, params.context); + writer = std::make_shared(path, params.read_settings, params.write_settings); else - writer = std::make_shared(disk, path, params.context); + writer = std::make_shared(disk, path, params.read_settings, params.write_settings); return std::make_unique( backup_name_for_logging, archive_params, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a5ff7643294..119cf3ac0c9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4550,14 +4550,6 @@ ReadSettings Context::getReadSettings() const return res; } -ReadSettings Context::getBackupReadSettings() const -{ - ReadSettings read_settings = getReadSettings(); - read_settings.remote_throttler = getBackupsThrottler(); - read_settings.local_throttler = getBackupsThrottler(); - return read_settings; -} - WriteSettings Context::getWriteSettings() const { WriteSettings res; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 676eb8412e5..cf6b1cca38d 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1168,9 +1168,6 @@ public: /** Get settings for reading from filesystem. */ ReadSettings getReadSettings() const; - /** Get settings for reading from filesystem for BACKUPs. */ - ReadSettings getBackupReadSettings() const; - /** Get settings for writing to filesystem. */ WriteSettings getWriteSettings() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5ef3899929e..7524dc2ffc3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5153,7 +5153,6 @@ MergeTreeData::PartsBackupEntries MergeTreeData::backupParts( MergeTreeData::PartsBackupEntries res; std::map> temp_dirs; TableLockHolder table_lock; - ReadSettings read_settings = local_context->getBackupReadSettings(); for (const auto & part : data_parts) { From e127d27453713be645e826191ca57854afc3748a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 23 Jul 2023 11:23:36 +0200 Subject: [PATCH 1533/2047] Disable using fs cache for backup/restore #2. --- src/Backups/BackupEntriesCollector.cpp | 2 ++ src/Backups/BackupEntriesCollector.h | 3 +++ src/Backups/BackupEntryFromSmallFile.cpp | 16 ++++++++-------- src/Backups/BackupEntryFromSmallFile.h | 4 ++-- src/Backups/BackupsWorker.cpp | 4 ++-- .../MergeTree/DataPartStorageOnDiskBase.cpp | 3 ++- .../MergeTree/DataPartStorageOnDiskBase.h | 1 + src/Storages/MergeTree/IDataPartStorage.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 3 +++ src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageLog.cpp | 3 ++- src/Storages/StorageMemory.cpp | 8 +++++++- src/Storages/StorageMergeTree.cpp | 3 ++- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- src/Storages/StorageStripeLog.cpp | 3 ++- 15 files changed, 40 insertions(+), 19 deletions(-) diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 485d59eff38..2c7985f2baa 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -77,10 +77,12 @@ BackupEntriesCollector::BackupEntriesCollector( const ASTBackupQuery::Elements & backup_query_elements_, const BackupSettings & backup_settings_, std::shared_ptr backup_coordination_, + const ReadSettings & read_settings_, const ContextPtr & context_) : backup_query_elements(backup_query_elements_) , backup_settings(backup_settings_) , backup_coordination(backup_coordination_) + , read_settings(read_settings_) , context(context_) , on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000)) , consistent_metadata_snapshot_timeout(context->getConfigRef().getUInt64("backups.consistent_metadata_snapshot_timeout", 600000)) diff --git a/src/Backups/BackupEntriesCollector.h b/src/Backups/BackupEntriesCollector.h index be6ca8d1ebe..54d82088129 100644 --- a/src/Backups/BackupEntriesCollector.h +++ b/src/Backups/BackupEntriesCollector.h @@ -30,6 +30,7 @@ public: BackupEntriesCollector(const ASTBackupQuery::Elements & backup_query_elements_, const BackupSettings & backup_settings_, std::shared_ptr backup_coordination_, + const ReadSettings & read_settings_, const ContextPtr & context_); ~BackupEntriesCollector(); @@ -40,6 +41,7 @@ public: const BackupSettings & getBackupSettings() const { return backup_settings; } std::shared_ptr getBackupCoordination() const { return backup_coordination; } + const ReadSettings & getReadSettings() const { return read_settings; } ContextPtr getContext() const { return context; } /// Adds a backup entry which will be later returned by run(). @@ -93,6 +95,7 @@ private: const ASTBackupQuery::Elements backup_query_elements; const BackupSettings backup_settings; std::shared_ptr backup_coordination; + const ReadSettings read_settings; ContextPtr context; std::chrono::milliseconds on_cluster_first_sync_timeout; std::chrono::milliseconds consistent_metadata_snapshot_timeout; diff --git a/src/Backups/BackupEntryFromSmallFile.cpp b/src/Backups/BackupEntryFromSmallFile.cpp index d0a99056b59..55a851bdf8b 100644 --- a/src/Backups/BackupEntryFromSmallFile.cpp +++ b/src/Backups/BackupEntryFromSmallFile.cpp @@ -11,17 +11,17 @@ namespace DB { namespace { - String readFile(const String & file_path) + String readFile(const String & file_path, const ReadSettings & read_settings) { - auto buf = createReadBufferFromFileBase(file_path, /* settings= */ {}); + auto buf = createReadBufferFromFileBase(file_path, read_settings); String s; readStringUntilEOF(s, *buf); return s; } - String readFile(const DiskPtr & disk, const String & file_path, bool copy_encrypted) + String readFile(const DiskPtr & disk, const String & file_path, const ReadSettings & read_settings, bool copy_encrypted) { - auto buf = copy_encrypted ? disk->readEncryptedFile(file_path, {}) : disk->readFile(file_path); + auto buf = copy_encrypted ? disk->readEncryptedFile(file_path, read_settings) : disk->readFile(file_path, read_settings); String s; readStringUntilEOF(s, *buf); return s; @@ -29,19 +29,19 @@ namespace } -BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_) +BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_, const ReadSettings & read_settings_) : file_path(file_path_) , data_source_description(DiskLocal::getLocalDataSourceDescription(file_path_)) - , data(readFile(file_path_)) + , data(readFile(file_path_, read_settings_)) { } -BackupEntryFromSmallFile::BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_, bool copy_encrypted_) +BackupEntryFromSmallFile::BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_, const ReadSettings & read_settings_, bool copy_encrypted_) : disk(disk_) , file_path(file_path_) , data_source_description(disk_->getDataSourceDescription()) , copy_encrypted(copy_encrypted_ && data_source_description.is_encrypted) - , data(readFile(disk_, file_path, copy_encrypted)) + , data(readFile(disk_, file_path, read_settings_, copy_encrypted)) { } diff --git a/src/Backups/BackupEntryFromSmallFile.h b/src/Backups/BackupEntryFromSmallFile.h index d6651ab8cb5..0c4b9ea15e7 100644 --- a/src/Backups/BackupEntryFromSmallFile.h +++ b/src/Backups/BackupEntryFromSmallFile.h @@ -13,8 +13,8 @@ using DiskPtr = std::shared_ptr; class BackupEntryFromSmallFile : public BackupEntryWithChecksumCalculation { public: - explicit BackupEntryFromSmallFile(const String & file_path_); - BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_, bool copy_encrypted_ = false); + explicit BackupEntryFromSmallFile(const String & file_path_, const ReadSettings & read_settings_); + BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_, const ReadSettings & read_settings_, bool copy_encrypted_ = false); std::unique_ptr getReadBuffer(const ReadSettings &) const override; UInt64 getSize() const override { return data.size(); } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 8a87f168a0d..a5c4a5314dc 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -186,7 +186,7 @@ namespace auto read_settings = context->getReadSettings(); read_settings.remote_throttler = context->getBackupsThrottler(); read_settings.local_throttler = context->getBackupsThrottler(); - read_settings.enable_filesystem_cache = false; + read_settings.enable_filesystem_cache = backup_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache; read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = backup_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache; return read_settings; } @@ -416,7 +416,7 @@ void BackupsWorker::doBackup( /// Prepare backup entries. BackupEntries backup_entries; { - BackupEntriesCollector backup_entries_collector{backup_query->elements, backup_settings, backup_coordination, context}; + BackupEntriesCollector backup_entries_collector{backup_query->elements, backup_settings, backup_coordination, backup_create_params.read_settings, context}; backup_entries = backup_entries_collector.run(); } diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index a0492f5f38e..618dbd845ae 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -331,6 +331,7 @@ void DataPartStorageOnDiskBase::backup( const NameSet & files_without_checksums, const String & path_in_backup, const BackupSettings & backup_settings, + const ReadSettings & read_settings, bool make_temporary_hard_links, BackupEntries & backup_entries, TemporaryFilesOnDisks * temp_dirs) const @@ -382,7 +383,7 @@ void DataPartStorageOnDiskBase::backup( if (files_without_checksums.contains(filepath)) { - backup_entries.emplace_back(filepath_in_backup, std::make_unique(disk, filepath_on_disk, copy_encrypted)); + backup_entries.emplace_back(filepath_in_backup, std::make_unique(disk, filepath_on_disk, read_settings, copy_encrypted)); continue; } diff --git a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h index 648bc908f59..6c2987e4971 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.h +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.h @@ -55,6 +55,7 @@ public: const NameSet & files_without_checksums, const String & path_in_backup, const BackupSettings & backup_settings, + const ReadSettings & read_settings, bool make_temporary_hard_links, BackupEntries & backup_entries, TemporaryFilesOnDisks * temp_dirs) const override; diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 8dbf5caa168..19af6085547 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -221,6 +221,7 @@ public: const NameSet & files_without_checksums, const String & path_in_backup, const BackupSettings & backup_settings, + const ReadSettings & read_settings, bool make_temporary_hard_links, BackupEntries & backup_entries, TemporaryFilesOnDisks * temp_dirs) const = 0; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7524dc2ffc3..0cfcd815cce 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5148,6 +5148,7 @@ MergeTreeData::PartsBackupEntries MergeTreeData::backupParts( const DataPartsVector & data_parts, const String & data_path_in_backup, const BackupSettings & backup_settings, + const ReadSettings & read_settings, const ContextPtr & local_context) { MergeTreeData::PartsBackupEntries res; @@ -5186,6 +5187,7 @@ MergeTreeData::PartsBackupEntries MergeTreeData::backupParts( part->getFileNamesWithoutChecksums(), data_path_in_backup, backup_settings, + read_settings, make_temporary_hard_links, backup_entries_from_part, &temp_dirs); @@ -5198,6 +5200,7 @@ MergeTreeData::PartsBackupEntries MergeTreeData::backupParts( projection_part->getFileNamesWithoutChecksums(), fs::path{data_path_in_backup} / part->name, backup_settings, + read_settings, make_temporary_hard_links, backup_entries_from_part, &temp_dirs); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5e6b043c31c..9ee61134740 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1336,7 +1336,7 @@ protected: using PartsBackupEntries = std::vector; /// Makes backup entries to backup the parts of this table. - PartsBackupEntries backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup, const BackupSettings & backup_settings, const ContextPtr & local_context); + PartsBackupEntries backupParts(const DataPartsVector & data_parts, const String & data_path_in_backup, const BackupSettings & backup_settings, const ReadSettings & read_settings, const ContextPtr & local_context); class RestoredPartsHolder; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index d02a51fab22..87aa71f3e8d 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -949,6 +949,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c fs::path temp_dir = temp_dir_owner->getRelativePath(); disk->createDirectories(temp_dir); + const auto & read_settings = backup_entries_collector.getReadSettings(); bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks; /// *.bin @@ -980,7 +981,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c /// sizes.json String files_info_path = file_checker.getPath(); backup_entries_collector.addBackupEntry( - data_path_in_backup_fs / fileName(files_info_path), std::make_unique(disk, files_info_path, copy_encrypted)); + data_path_in_backup_fs / fileName(files_info_path), std::make_unique(disk, files_info_path, read_settings, copy_encrypted)); /// columns.txt backup_entries_collector.addBackupEntry( diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index c990d488969..2ef1d8d3183 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -277,11 +277,13 @@ namespace const std::shared_ptr blocks_, const String & data_path_in_backup, const DiskPtr & temp_disk_, + const ReadSettings & read_settings_, UInt64 max_compress_block_size_) : context(context_) , metadata_snapshot(metadata_snapshot_) , blocks(blocks_) , temp_disk(temp_disk_) + , read_settings(read_settings_) , max_compress_block_size(max_compress_block_size_) { fs::path data_path_in_backup_fs = data_path_in_backup; @@ -371,7 +373,7 @@ namespace file_checker.update(temp_dir / fs::path{file_paths[i]}.filename()); } file_checker.save(); - backup_entries[sizes_json_pos] = {file_paths[sizes_json_pos], std::make_shared(temp_disk, sizes_json_path)}; + backup_entries[sizes_json_pos] = {file_paths[sizes_json_pos], std::make_shared(temp_disk, sizes_json_path, read_settings)}; } /// We don't need to keep `blocks` any longer. @@ -386,6 +388,7 @@ namespace std::shared_ptr blocks; DiskPtr temp_disk; std::optional temp_dir_owner; + ReadSettings read_settings; UInt64 max_compress_block_size; Strings file_paths; size_t data_bin_pos, index_mrk_pos, columns_txt_pos, count_txt_pos, sizes_json_pos; @@ -395,13 +398,16 @@ namespace void StorageMemory::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & /* partitions */) { auto temp_disk = backup_entries_collector.getContext()->getGlobalTemporaryVolume()->getDisk(0); + const auto & read_settings = backup_entries_collector.getReadSettings(); auto max_compress_block_size = backup_entries_collector.getContext()->getSettingsRef().max_compress_block_size; + backup_entries_collector.addBackupEntries(std::make_shared( backup_entries_collector.getContext(), getInMemoryMetadataPtr(), data.get(), data_path_in_backup, temp_disk, + read_settings, max_compress_block_size)->getBackupEntries()); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 14ea2ff4afe..ad9013d9f13 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2253,6 +2253,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_ void StorageMergeTree::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional & partitions) { const auto & backup_settings = backup_entries_collector.getBackupSettings(); + const auto & read_settings = backup_entries_collector.getReadSettings(); auto local_context = backup_entries_collector.getContext(); DataPartsVector data_parts; @@ -2265,7 +2266,7 @@ void StorageMergeTree::backupData(BackupEntriesCollector & backup_entries_collec for (const auto & data_part : data_parts) min_data_version = std::min(min_data_version, data_part->info.getDataVersion() + 1); - auto parts_backup_entries = backupParts(data_parts, data_path_in_backup, backup_settings, local_context); + auto parts_backup_entries = backupParts(data_parts, data_path_in_backup, backup_settings, read_settings, local_context); for (auto & part_backup_entries : parts_backup_entries) backup_entries_collector.addBackupEntries(std::move(part_backup_entries.backup_entries)); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 23683ec2313..7fce373e26b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9826,6 +9826,7 @@ void StorageReplicatedMergeTree::backupData( /// because we need to coordinate them with other replicas (other replicas can have better parts). const auto & backup_settings = backup_entries_collector.getBackupSettings(); + const auto & read_settings = backup_entries_collector.getReadSettings(); auto local_context = backup_entries_collector.getContext(); DataPartsVector data_parts; @@ -9834,7 +9835,7 @@ void StorageReplicatedMergeTree::backupData( else data_parts = getVisibleDataPartsVector(local_context); - auto parts_backup_entries = backupParts(data_parts, /* data_path_in_backup */ "", backup_settings, local_context); + auto parts_backup_entries = backupParts(data_parts, /* data_path_in_backup */ "", backup_settings, read_settings, local_context); auto coordination = backup_entries_collector.getBackupCoordination(); String shared_id = getTableSharedID(); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 2f3b5f25ee4..0bfef5ed5e5 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -547,6 +547,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec fs::path temp_dir = temp_dir_owner->getRelativePath(); disk->createDirectories(temp_dir); + const auto & read_settings = backup_entries_collector.getReadSettings(); bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks; /// data.bin @@ -576,7 +577,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec /// sizes.json String files_info_path = file_checker.getPath(); backup_entries_collector.addBackupEntry( - data_path_in_backup_fs / fileName(files_info_path), std::make_unique(disk, files_info_path, copy_encrypted)); + data_path_in_backup_fs / fileName(files_info_path), std::make_unique(disk, files_info_path, read_settings, copy_encrypted)); /// columns.txt backup_entries_collector.addBackupEntry( From 1cd79020790b153a7cdbda09159cc18e6de77a7e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 23 Jul 2023 11:24:01 +0200 Subject: [PATCH 1534/2047] Add test. --- .../configs/disk_s3.xml | 15 +++ .../test_backup_restore_s3/test.py | 100 +++++++++++++++++- 2 files changed, 110 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_backup_restore_s3/configs/disk_s3.xml b/tests/integration/test_backup_restore_s3/configs/disk_s3.xml index c1fd059bc67..d635e39e13f 100644 --- a/tests/integration/test_backup_restore_s3/configs/disk_s3.xml +++ b/tests/integration/test_backup_restore_s3/configs/disk_s3.xml @@ -21,6 +21,13 @@ minio123 33554432 + + cache + disk_s3 + /tmp/s3_cache/ + 1000000000 + 1 +
@@ -37,11 +44,19 @@ + + +
+ disk_s3_cache +
+
+
default disk_s3 disk_s3_plain + disk_s3_cache diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 8701bf0d832..21018924143 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -57,29 +57,42 @@ def get_events_for_query(query_id: str) -> Dict[str, int]: } +def format_settings(settings): + if not settings: + return "" + return "SETTINGS " + ",".join(f"{k}={v}" for k, v in settings.items()) + + def check_backup_and_restore( storage_policy, backup_destination, size=1000, - backup_name=None, + backup_settings=None, + restore_settings=None, + insert_settings=None, + optimize_table=True, ): + optimize_table_query = "OPTIMIZE TABLE data FINAL;" if optimize_table else "" + node.query( f""" DROP TABLE IF EXISTS data SYNC; CREATE TABLE data (key Int, value String, array Array(String)) Engine=MergeTree() ORDER BY tuple() SETTINGS storage_policy='{storage_policy}'; - INSERT INTO data SELECT * FROM generateRandom('key Int, value String, array Array(String)') LIMIT {size}; - OPTIMIZE TABLE data FINAL; + INSERT INTO data SELECT * FROM generateRandom('key Int, value String, array Array(String)') LIMIT {size} {format_settings(insert_settings)}; + {optimize_table_query} """ ) + try: backup_query_id = uuid.uuid4().hex node.query( - f"BACKUP TABLE data TO {backup_destination}", query_id=backup_query_id + f"BACKUP TABLE data TO {backup_destination} {format_settings(backup_settings)}", + query_id=backup_query_id, ) restore_query_id = uuid.uuid4().hex node.query( f""" - RESTORE TABLE data AS data_restored FROM {backup_destination}; + RESTORE TABLE data AS data_restored FROM {backup_destination} {format_settings(restore_settings)}; """, query_id=restore_query_id, ) @@ -114,6 +127,7 @@ def check_system_tables(): expected_disks = ( ("default", "local"), ("disk_s3", "s3"), + ("disk_s3_cache", "s3"), ("disk_s3_other_bucket", "s3"), ("disk_s3_plain", "s3_plain"), ) @@ -312,3 +326,79 @@ def test_incremental_backup_append_table_def(): assert node.query("SELECT count(), sum(x) FROM data") == "100\t4950\n" assert "parts_to_throw_insert = 100" in node.query("SHOW CREATE TABLE data") + + +@pytest.mark.parametrize( + "in_cache_initially, allow_backup_read_cache, allow_s3_native_copy", + [ + (False, True, False), + (True, False, False), + (True, True, False), + (True, True, True), + ], +) +def test_backup_with_fs_cache( + in_cache_initially, allow_backup_read_cache, allow_s3_native_copy +): + storage_policy = "policy_s3_cache" + + backup_name = new_backup_name() + backup_destination = ( + f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" + ) + + insert_settings = { + "enable_filesystem_cache_on_write_operations": int(in_cache_initially) + } + + backup_settings = { + "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": int( + allow_backup_read_cache + ), + "allow_s3_native_copy": int(allow_s3_native_copy), + } + + restore_settings = {"allow_s3_native_copy": int(allow_s3_native_copy)} + + backup_events, restore_events = check_backup_and_restore( + storage_policy, + backup_destination, + size=10, + insert_settings=insert_settings, + optimize_table=False, + backup_settings=backup_settings, + restore_settings=restore_settings, + ) + + #print(f"backup_events = {backup_events}") + #print(f"restore_events = {restore_events}") + + # BACKUP never updates the filesystem cache but it may read it if `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` allows that. + if allow_backup_read_cache and in_cache_initially: + assert backup_events["CachedReadBufferReadFromCacheBytes"] > 0 + assert not "CachedReadBufferReadFromSourceBytes" in backup_events + elif allow_backup_read_cache: + assert not "CachedReadBufferReadFromCacheBytes" in backup_events + assert backup_events["CachedReadBufferReadFromSourceBytes"] > 0 + else: + assert not "CachedReadBufferReadFromCacheBytes" in backup_events + assert not "CachedReadBufferReadFromSourceBytes" in backup_events + + assert not "CachedReadBufferCacheWriteBytes" in backup_events + assert not "CachedWriteBufferCacheWriteBytes" in backup_events + + # RESTORE doesn't use the filesystem cache during write operations. + # However while attaching parts it may use the cache while reading such files as "columns.txt" or "checksums.txt" or "primary.idx", + # see IMergeTreeDataPart::loadColumnsChecksumsIndexes() + if "CachedReadBufferReadFromSourceBytes" in restore_events: + assert ( + restore_events["CachedReadBufferReadFromSourceBytes"] + == restore_events["CachedReadBufferCacheWriteBytes"] + ) + + assert not "CachedReadBufferReadFromCacheBytes" in restore_events + + # "format_version.txt" is written when a table is created, + # see MergeTreeData::initializeDirectoriesAndFormatVersion() + if "CachedWriteBufferCacheWriteBytes" in restore_events: + assert restore_events["CachedWriteBufferCacheWriteBytes"] <= 1 From 556ecf5c1c44121bf7b96e945b4241eb445c826f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 23 Jul 2023 13:00:17 +0200 Subject: [PATCH 1535/2047] Renamed backup setting "read_from_filesystem_cache_if_exists_otherwise_bypass_cache" -> "read_from_filesystem_cache". --- src/Backups/BackupSettings.cpp | 2 +- src/Backups/BackupSettings.h | 2 +- src/Backups/BackupsWorker.cpp | 4 ++-- tests/integration/test_backup_restore_s3/test.py | 8 +++----- 4 files changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 43259e6958a..650e817f4c3 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -27,7 +27,7 @@ namespace ErrorCodes M(Bool, decrypt_files_from_encrypted_disks) \ M(Bool, deduplicate_files) \ M(Bool, allow_s3_native_copy) \ - M(Bool, read_from_filesystem_cache_if_exists_otherwise_bypass_cache) \ + M(Bool, read_from_filesystem_cache) \ M(UInt64, shard_num) \ M(UInt64, replica_num) \ M(Bool, internal) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 2a950ef1b4f..68024ea1cbf 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -46,7 +46,7 @@ struct BackupSettings /// Allow to use the filesystem cache in passive mode - benefit from the existing cache entries, /// but don't put more entries into the cache. - bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true; + bool read_from_filesystem_cache = true; /// 1-based shard index to store in the backup. 0 means all shards. /// Can only be used with BACKUP ON CLUSTER. diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index a5c4a5314dc..90e76ef9b46 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -186,8 +186,8 @@ namespace auto read_settings = context->getReadSettings(); read_settings.remote_throttler = context->getBackupsThrottler(); read_settings.local_throttler = context->getBackupsThrottler(); - read_settings.enable_filesystem_cache = backup_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache; - read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = backup_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache; + read_settings.enable_filesystem_cache = backup_settings.read_from_filesystem_cache; + read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = backup_settings.read_from_filesystem_cache; return read_settings; } diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 21018924143..b6d4b4231d8 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -352,9 +352,7 @@ def test_backup_with_fs_cache( } backup_settings = { - "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": int( - allow_backup_read_cache - ), + "read_from_filesystem_cache": int(allow_backup_read_cache), "allow_s3_native_copy": int(allow_s3_native_copy), } @@ -370,8 +368,8 @@ def test_backup_with_fs_cache( restore_settings=restore_settings, ) - #print(f"backup_events = {backup_events}") - #print(f"restore_events = {restore_events}") + # print(f"backup_events = {backup_events}") + # print(f"restore_events = {restore_events}") # BACKUP never updates the filesystem cache but it may read it if `read_from_filesystem_cache_if_exists_otherwise_bypass_cache` allows that. if allow_backup_read_cache and in_cache_initially: From 6ac61b1fdd51e226bf0b3d76f778dbcc14f14849 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 24 Jul 2023 18:13:44 +0200 Subject: [PATCH 1536/2047] Fix tests. --- src/Backups/tests/gtest_backup_entries.cpp | 14 +++++++------- tests/integration/test_backup_restore_s3/test.py | 1 - 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Backups/tests/gtest_backup_entries.cpp b/src/Backups/tests/gtest_backup_entries.cpp index 75972b35ba4..2d5b993b95d 100644 --- a/src/Backups/tests/gtest_backup_entries.cpp +++ b/src/Backups/tests/gtest_backup_entries.cpp @@ -69,14 +69,14 @@ protected: static String getChecksum(const BackupEntryPtr & backup_entry) { - return getHexUIntUppercase(backup_entry->getChecksum()); + return getHexUIntUppercase(backup_entry->getChecksum({})); } static const constexpr std::string_view NO_CHECKSUM = "no checksum"; static String getPartialChecksum(const BackupEntryPtr & backup_entry, size_t prefix_length) { - auto partial_checksum = backup_entry->getPartialChecksum(prefix_length); + auto partial_checksum = backup_entry->getPartialChecksum(prefix_length, {}); if (!partial_checksum) return String{NO_CHECKSUM}; return getHexUIntUppercase(*partial_checksum); @@ -218,7 +218,7 @@ TEST_F(BackupEntriesTest, PartialChecksumBeforeFullChecksum) TEST_F(BackupEntriesTest, BackupEntryFromSmallFile) { writeFile(local_disk, "a.txt"); - auto entry = std::make_shared(local_disk, "a.txt"); + auto entry = std::make_shared(local_disk, "a.txt", ReadSettings{}); local_disk->removeFile("a.txt"); @@ -239,7 +239,7 @@ TEST_F(BackupEntriesTest, DecryptedEntriesFromEncryptedDisk) std::pair test_cases[] = {{std::make_shared(encrypted_disk, "a.txt"), false}, {std::make_shared(encrypted_disk, "a.txt"), true}, - {std::make_shared(encrypted_disk, "a.txt"), true}}; + {std::make_shared(encrypted_disk, "a.txt", ReadSettings{}), true}}; for (const auto & [entry, partial_checksum_allowed] : test_cases) { EXPECT_EQ(entry->getSize(), 9); @@ -258,7 +258,7 @@ TEST_F(BackupEntriesTest, DecryptedEntriesFromEncryptedDisk) BackupEntryPtr entries[] = {std::make_shared(encrypted_disk, "empty.txt"), std::make_shared(encrypted_disk, "empty.txt"), - std::make_shared(encrypted_disk, "empty.txt")}; + std::make_shared(encrypted_disk, "empty.txt", ReadSettings{})}; for (const auto & entry : entries) { EXPECT_EQ(entry->getSize(), 0); @@ -288,7 +288,7 @@ TEST_F(BackupEntriesTest, EncryptedEntriesFromEncryptedDisk) BackupEntryPtr entries[] = {std::make_shared(encrypted_disk, "a.txt", /* copy_encrypted= */ true), std::make_shared(encrypted_disk, "a.txt", /* copy_encrypted= */ true), - std::make_shared(encrypted_disk, "a.txt", /* copy_encrypted= */ true)}; + std::make_shared(encrypted_disk, "a.txt", ReadSettings{}, /* copy_encrypted= */ true)}; auto encrypted_checksum = getChecksum(entries[0]); EXPECT_NE(encrypted_checksum, NO_CHECKSUM); @@ -322,7 +322,7 @@ TEST_F(BackupEntriesTest, EncryptedEntriesFromEncryptedDisk) BackupEntryPtr entries[] = {std::make_shared(encrypted_disk, "empty.txt", /* copy_encrypted= */ true), std::make_shared(encrypted_disk, "empty.txt", /* copy_encrypted= */ true), - std::make_shared(encrypted_disk, "empty.txt", /* copy_encrypted= */ true)}; + std::make_shared(encrypted_disk, "empty.txt", ReadSettings{}, /* copy_encrypted= */ true)}; for (const auto & entry : entries) { EXPECT_EQ(entry->getSize(), 0); diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index b6d4b4231d8..f8ec39d240b 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -198,7 +198,6 @@ def test_backup_to_s3_multipart(): storage_policy, backup_destination, size=1000000, - backup_name=backup_name, ) assert node.contains_in_log( f"copyDataToS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}" From d9e45e11c7d2bb28509d050141d8e7ad986b52e3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 13:59:39 +0200 Subject: [PATCH 1537/2047] Remove `test_host_regexp_multiple_ptr_records_concurrent`, CC @arthurpassos --- .../__init__.py | 0 .../configs/config.xml | 4 - .../configs/host_regexp.xml | 11 --- .../configs/listen_host.xml | 5 -- .../coredns_config/Corefile | 8 -- .../coredns_config/example.com | 1 - .../scripts/stress_test.py | 62 ------------- .../test.py | 88 ------------------- 8 files changed, 179 deletions(-) delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records_concurrent/__init__.py delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/config.xml delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/host_regexp.xml delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/listen_host.xml delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/Corefile delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/example.com delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records_concurrent/test.py diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/__init__.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/config.xml b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/config.xml deleted file mode 100644 index 42a1f962705..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/config.xml +++ /dev/null @@ -1,4 +0,0 @@ - - 1 - 250 - diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/host_regexp.xml b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/host_regexp.xml deleted file mode 100644 index 7a2141e6c7e..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/host_regexp.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - - - test1\.example\.com$ - - default - - - \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/listen_host.xml b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/listen_host.xml deleted file mode 100644 index 58ef55cd3f3..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/configs/listen_host.xml +++ /dev/null @@ -1,5 +0,0 @@ - - :: - 0.0.0.0 - 1 - diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/Corefile b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/Corefile deleted file mode 100644 index 3edf37dafa5..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/Corefile +++ /dev/null @@ -1,8 +0,0 @@ -. { - hosts /example.com { - reload "20ms" - fallthrough - } - forward . 127.0.0.11 - log -} diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/example.com b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/example.com deleted file mode 100644 index 9beb415c290..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/coredns_config/example.com +++ /dev/null @@ -1 +0,0 @@ -filled in runtime, but needs to exist in order to be volume mapped in docker \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py deleted file mode 100644 index fe69d72c1c7..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/scripts/stress_test.py +++ /dev/null @@ -1,62 +0,0 @@ -import pycurl -import threading -from io import BytesIO -import sys - -client_ip = sys.argv[1] -server_ip = sys.argv[2] - -mutex = threading.Lock() -success_counter = 0 -number_of_threads = 100 -number_of_iterations = 100 - - -def perform_request(): - buffer = BytesIO() - crl = pycurl.Curl() - crl.setopt(pycurl.INTERFACE, client_ip) - crl.setopt(crl.WRITEDATA, buffer) - crl.setopt(crl.URL, f"http://{server_ip}:8123/?query=select+1&user=test_dns") - - crl.perform() - - # End curl session - crl.close() - - str_response = buffer.getvalue().decode("iso-8859-1") - expected_response = "1\n" - - mutex.acquire() - - global success_counter - - if str_response == expected_response: - success_counter += 1 - - mutex.release() - - -def perform_multiple_requests(n): - for request_number in range(n): - perform_request() - - -threads = [] - - -for i in range(number_of_threads): - thread = threading.Thread( - target=perform_multiple_requests, args=(number_of_iterations,) - ) - thread.start() - threads.append(thread) - -for thread in threads: - thread.join() - - -if success_counter == number_of_threads * number_of_iterations: - exit(0) - -exit(1) diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/test.py deleted file mode 100644 index d73e8813e79..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/test.py +++ /dev/null @@ -1,88 +0,0 @@ -import pytest -import socket -from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check -from time import sleep -import os - -DOCKER_COMPOSE_PATH = get_docker_compose_path() -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - -cluster = ClickHouseCluster(__file__) - -ch_server = cluster.add_instance( - "clickhouse-server", - with_coredns=True, - main_configs=["configs/config.xml", "configs/listen_host.xml"], - user_configs=["configs/host_regexp.xml"], -) - -client = cluster.add_instance( - "clickhouse-client", -) - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def check_ptr_record(ip, hostname): - try: - host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip) - if hostname.lower() == host.lower(): - return True - except socket.herror: - pass - return False - - -def setup_dns_server(ip): - domains_string = "test3.example.com test2.example.com test1.example.com" - example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' - run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) - - # DNS server takes time to reload the configuration. - for try_num in range(10): - if all(check_ptr_record(ip, host) for host in domains_string.split()): - break - sleep(1) - - -def setup_ch_server(dns_server_ip): - ch_server.exec_in_container( - (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) - ) - ch_server.exec_in_container( - (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) - ) - ch_server.query("SYSTEM DROP DNS CACHE") - - -def build_endpoint_v4(ip): - return f"'http://{ip}:8123/?query=SELECT+1&user=test_dns'" - - -def build_endpoint_v6(ip): - return build_endpoint_v4(f"[{ip}]") - - -def test_host_regexp_multiple_ptr_v4(started_cluster): - server_ip = cluster.get_instance_ip("clickhouse-server") - client_ip = cluster.get_instance_ip("clickhouse-client") - dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) - - setup_dns_server(client_ip) - setup_ch_server(dns_server_ip) - - current_dir = os.path.dirname(__file__) - client.copy_file_to_container( - os.path.join(current_dir, "scripts", "stress_test.py"), "stress_test.py" - ) - - client.exec_in_container(["python3", f"stress_test.py", client_ip, server_ip]) From 3a9f9e12979a5bb55485554c45337389f274a256 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 15:05:52 +0300 Subject: [PATCH 1538/2047] Revert "Implementing new commands for keeper-client" --- .../utilities/clickhouse-keeper-client.md | 4 - programs/keeper-client/Commands.cpp | 184 +----------------- programs/keeper-client/Commands.h | 78 +------- programs/keeper-client/KeeperClient.cpp | 4 - programs/keeper-client/Parser.cpp | 1 - tests/integration/test_keeper_client/test.py | 130 +++---------- 6 files changed, 36 insertions(+), 365 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index 37eb0bb71ff..77f816fe428 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -51,7 +51,3 @@ keeper foo bar - `rmr ` -- Recursively deletes path. Confirmation required - `flwc ` -- Executes four-letter-word command - `help` -- Prints this message -- `get_stat [path]` -- Returns the node's stat (default `.`) -- `find_super_nodes [path]` -- Finds nodes with number of children larger than some threshold for the given path (default `.`) -- `delete_stable_backups` -- Deletes ClickHouse nodes used for backups that are now inactive -- `find_big_family [path] [n]` -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index fd0a00d59db..05928a0d20b 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -1,6 +1,5 @@ #include "Commands.h" -#include #include "KeeperClient.h" @@ -25,18 +24,8 @@ void LSCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con else path = client->cwd; - auto children = client->zookeeper->getChildren(path); - std::sort(children.begin(), children.end()); - - bool need_space = false; - for (const auto & child : children) - { - if (std::exchange(need_space, true)) - std::cout << " "; - - std::cout << child; - } - + for (const auto & child : client->zookeeper->getChildren(path)) + std::cout << child << " "; std::cout << "\n"; } @@ -141,173 +130,6 @@ void GetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co std::cout << client->zookeeper->get(client->getAbsolutePath(query->args[0].safeGet())) << "\n"; } -bool GetStatCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const -{ - String arg; - if (!parseKeeperPath(pos, expected, arg)) - return true; - - node->args.push_back(std::move(arg)); - return true; -} - -void GetStatCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const -{ - Coordination::Stat stat; - String path; - if (!query->args.empty()) - path = client->getAbsolutePath(query->args[0].safeGet()); - else - path = client->cwd; - - client->zookeeper->get(path, &stat); - - std::cout << "cZxid = " << stat.czxid << "\n"; - std::cout << "mZxid = " << stat.mzxid << "\n"; - std::cout << "pZxid = " << stat.pzxid << "\n"; - std::cout << "ctime = " << stat.ctime << "\n"; - std::cout << "mtime = " << stat.mtime << "\n"; - std::cout << "version = " << stat.version << "\n"; - std::cout << "cversion = " << stat.cversion << "\n"; - std::cout << "aversion = " << stat.aversion << "\n"; - std::cout << "ephemeralOwner = " << stat.ephemeralOwner << "\n"; - std::cout << "dataLength = " << stat.dataLength << "\n"; - std::cout << "numChildren = " << stat.numChildren << "\n"; -} - -bool FindSuperNodes::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const -{ - ASTPtr threshold; - if (!ParserUnsignedInteger{}.parse(pos, threshold, expected)) - return false; - - node->args.push_back(threshold->as().value); - - String path; - if (!parseKeeperPath(pos, expected, path)) - path = "."; - - node->args.push_back(std::move(path)); - return true; -} - -void FindSuperNodes::execute(const ASTKeeperQuery * query, KeeperClient * client) const -{ - auto threshold = query->args[0].safeGet(); - auto path = client->getAbsolutePath(query->args[1].safeGet()); - - Coordination::Stat stat; - client->zookeeper->get(path, &stat); - - if (stat.numChildren >= static_cast(threshold)) - { - std::cout << static_cast(path) << "\t" << stat.numChildren << "\n"; - return; - } - - auto children = client->zookeeper->getChildren(path); - std::sort(children.begin(), children.end()); - for (const auto & child : children) - { - auto next_query = *query; - next_query.args[1] = DB::Field(path / child); - execute(&next_query, client); - } -} - -bool DeleteStableBackups::parse(IParser::Pos & /* pos */, std::shared_ptr & /* node */, Expected & /* expected */) const -{ - return true; -} - -void DeleteStableBackups::execute(const ASTKeeperQuery * /* query */, KeeperClient * client) const -{ - client->askConfirmation( - "You are going to delete all inactive backups in /clickhouse/backups.", - [client] - { - fs::path backup_root = "/clickhouse/backups"; - auto backups = client->zookeeper->getChildren(backup_root); - std::sort(backups.begin(), backups.end()); - - for (const auto & child : backups) - { - auto backup_path = backup_root / child; - std::cout << "Found backup " << backup_path << ", checking if it's active\n"; - - String stage_path = backup_path / "stage"; - auto stages = client->zookeeper->getChildren(stage_path); - - bool is_active = false; - for (const auto & stage : stages) - { - if (startsWith(stage, "alive")) - { - is_active = true; - break; - } - } - - if (is_active) - { - std::cout << "Backup " << backup_path << " is active, not going to delete\n"; - continue; - } - - std::cout << "Backup " << backup_path << " is not active, deleting it\n"; - client->zookeeper->removeRecursive(backup_path); - } - }); -} - -bool FindBigFamily::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const -{ - String path; - if (!parseKeeperPath(pos, expected, path)) - path = "."; - - node->args.push_back(std::move(path)); - - ASTPtr count; - if (ParserUnsignedInteger{}.parse(pos, count, expected)) - node->args.push_back(count->as().value); - else - node->args.push_back(UInt64(10)); - - return true; -} - -void FindBigFamily::execute(const ASTKeeperQuery * query, KeeperClient * client) const -{ - auto path = client->getAbsolutePath(query->args[0].safeGet()); - auto n = query->args[1].safeGet(); - - std::vector> result; - - std::queue queue; - queue.push(path); - while (!queue.empty()) - { - auto next_path = queue.front(); - queue.pop(); - - auto children = client->zookeeper->getChildren(next_path); - std::transform(children.cbegin(), children.cend(), children.begin(), [&](const String & child) { return next_path / child; }); - - auto response = client->zookeeper->get(children); - - for (size_t i = 0; i < response.size(); ++i) - { - result.emplace_back(response[i].stat.numChildren, children[i]); - queue.push(children[i]); - } - } - - std::sort(result.begin(), result.end(), std::greater()); - for (UInt64 i = 0; i < std::min(result.size(), static_cast(n)); ++i) - std::cout << std::get<1>(result[i]) << "\t" << std::get<0>(result[i]) << "\n"; -} - bool RMCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { String arg; @@ -348,7 +170,7 @@ bool HelpCommand::parse(IParser::Pos & /* pos */, std::shared_ptrgenerateHelpString() << "\n"; + std::cout << pair.second->getHelpMessage() << "\n"; } bool FourLetterWordCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index 093920cb10d..e4debd53e42 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -21,12 +21,6 @@ public: virtual String getName() const = 0; virtual ~IKeeperClientCommand() = default; - - String generateHelpString() const - { - return fmt::vformat(getHelpMessage(), fmt::make_format_args(getName())); - } - }; using Command = std::shared_ptr; @@ -40,7 +34,7 @@ class LSCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} [path] -- Lists the nodes for the given path (default: cwd)"; } + String getHelpMessage() const override { return "ls [path] -- Lists the nodes for the given path (default: cwd)"; } }; class CDCommand : public IKeeperClientCommand @@ -51,7 +45,7 @@ class CDCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} [path] -- Change the working path (default `.`)"; } + String getHelpMessage() const override { return "cd [path] -- Change the working path (default `.`)"; } }; class SetCommand : public IKeeperClientCommand @@ -64,7 +58,7 @@ class SetCommand : public IKeeperClientCommand String getHelpMessage() const override { - return "{} [version] -- Updates the node's value. Only update if version matches (default: -1)"; + return "set [version] -- Updates the node's value. Only update if version matches (default: -1)"; } }; @@ -76,7 +70,7 @@ class CreateCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} -- Creates new node"; } + String getHelpMessage() const override { return "create -- Creates new node"; } }; class GetCommand : public IKeeperClientCommand @@ -87,63 +81,9 @@ class GetCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} -- Returns the node's value"; } + String getHelpMessage() const override { return "get -- Returns the node's value"; } }; -class GetStatCommand : public IKeeperClientCommand -{ - String getName() const override { return "get_stat"; } - - bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; - - void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - - String getHelpMessage() const override { return "{} [path] -- Returns the node's stat (default `.`)"; } -}; - -class FindSuperNodes : public IKeeperClientCommand -{ - String getName() const override { return "find_super_nodes"; } - - bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; - - void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - - String getHelpMessage() const override - { - return "{} [path] -- Finds nodes with number of children larger than some threshold for the given path (default `.`)"; - } -}; - -class DeleteStableBackups : public IKeeperClientCommand -{ - String getName() const override { return "delete_stable_backups"; } - - bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; - - void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - - String getHelpMessage() const override - { - return "{} -- Deletes ClickHouse nodes used for backups that are now inactive"; - } -}; - -class FindBigFamily : public IKeeperClientCommand -{ - String getName() const override { return "find_big_family"; } - - bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; - - void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - - String getHelpMessage() const override - { - return "{} [path] [n] -- Returns the top n nodes with the biggest family in the subtree (default path = `.` and n = 10)"; - } -}; - - class RMCommand : public IKeeperClientCommand { String getName() const override { return "rm"; } @@ -152,7 +92,7 @@ class RMCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} -- Remove the node"; } + String getHelpMessage() const override { return "remove -- Remove the node"; } }; class RMRCommand : public IKeeperClientCommand @@ -163,7 +103,7 @@ class RMRCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} -- Recursively deletes path. Confirmation required"; } + String getHelpMessage() const override { return "rmr -- Recursively deletes path. Confirmation required"; } }; class HelpCommand : public IKeeperClientCommand @@ -174,7 +114,7 @@ class HelpCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} -- Prints this message"; } + String getHelpMessage() const override { return "help -- Prints this message"; } }; class FourLetterWordCommand : public IKeeperClientCommand @@ -185,7 +125,7 @@ class FourLetterWordCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} -- Executes four-letter-word command"; } + String getHelpMessage() const override { return "flwc -- Executes four-letter-word command"; } }; } diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 561a1f41f7a..f41dca1e27a 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -177,10 +177,6 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), - std::make_shared(), - std::make_shared(), - std::make_shared(), - std::make_shared(), std::make_shared(), std::make_shared(), std::make_shared(), diff --git a/programs/keeper-client/Parser.cpp b/programs/keeper-client/Parser.cpp index fe46058fcc1..3420ccb2219 100644 --- a/programs/keeper-client/Parser.cpp +++ b/programs/keeper-client/Parser.cpp @@ -58,7 +58,6 @@ bool KeeperParser::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; String command_name(pos->begin, pos->end); - std::transform(command_name.begin(), command_name.end(), command_name.begin(), [](unsigned char c) { return std::tolower(c); }); Command command; auto iter = KeeperClient::commands.find(command_name); diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index 3187ce10d2a..00c7908eeed 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -1,7 +1,6 @@ import pytest from helpers.client import CommandRequest from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) @@ -14,7 +13,7 @@ node = cluster.add_instance( ) -@pytest.fixture(scope="module", autouse=True) +@pytest.fixture(scope="module") def started_cluster(): try: cluster.start() @@ -24,122 +23,41 @@ def started_cluster(): cluster.shutdown() -def keeper_query(query: str): - return CommandRequest( +def test_base_commands(started_cluster): + _ = started_cluster + + command = CommandRequest( [ - cluster.server_bin_path, + started_cluster.server_bin_path, "keeper-client", "--host", str(cluster.get_instance_ip("zoo1")), "--port", str(cluster.zookeeper_port), "-q", - query, + "create test_create_zk_node1 testvalue1;create test_create_zk_node_2 testvalue2;get test_create_zk_node1;", ], stdin="", ) - -def test_big_family(): - command = keeper_query( - "create test_big_family foo;" - "create test_big_family/1 foo;" - "create test_big_family/1/1 foo;" - "create test_big_family/1/2 foo;" - "create test_big_family/1/3 foo;" - "create test_big_family/1/4 foo;" - "create test_big_family/1/5 foo;" - "create test_big_family/2 foo;" - "create test_big_family/2/1 foo;" - "create test_big_family/2/2 foo;" - "create test_big_family/2/3 foo;" - "find_big_family test_big_family;" - ) - - assert command.get_answer() == TSV( - [ - ["/test_big_family/1", "5"], - ["/test_big_family/2", "3"], - ["/test_big_family/2/3", "0"], - ["/test_big_family/2/2", "0"], - ["/test_big_family/2/1", "0"], - ["/test_big_family/1/5", "0"], - ["/test_big_family/1/4", "0"], - ["/test_big_family/1/3", "0"], - ["/test_big_family/1/2", "0"], - ["/test_big_family/1/1", "0"], - ] - ) - - command = keeper_query("find_big_family test_big_family 1;") - - assert command.get_answer() == TSV( - [ - ["/test_big_family/1", "5"], - ] - ) - - -def test_find_super_nodes(): - command = keeper_query( - "create test_find_super_nodes foo;" - "create test_find_super_nodes/1 foo;" - "create test_find_super_nodes/1/1 foo;" - "create test_find_super_nodes/1/2 foo;" - "create test_find_super_nodes/1/3 foo;" - "create test_find_super_nodes/1/4 foo;" - "create test_find_super_nodes/1/5 foo;" - "create test_find_super_nodes/2 foo;" - "create test_find_super_nodes/2/1 foo;" - "create test_find_super_nodes/2/2 foo;" - "create test_find_super_nodes/2/3 foo;" - "create test_find_super_nodes/2/4 foo;" - "cd test_find_super_nodes;" - "find_super_nodes 4;" - ) - - assert command.get_answer() == TSV( - [ - ["/test_find_super_nodes/1", "5"], - ["/test_find_super_nodes/2", "4"], - ] - ) - - -def test_delete_stable_backups(): - command = keeper_query( - "create /clickhouse/backups foo;" - "create /clickhouse/backups/1 foo;" - "create /clickhouse/backups/1/stage foo;" - "create /clickhouse/backups/1/stage/alive123 foo;" - "create /clickhouse/backups/2 foo;" - "create /clickhouse/backups/2/stage foo;" - "create /clickhouse/backups/2/stage/dead123 foo;" - "delete_stable_backups;" - "y;" - "ls clickhouse/backups;" - ) - - assert command.get_answer() == ( - "You are going to delete all inactive backups in /clickhouse/backups. Continue?\n" - 'Found backup "/clickhouse/backups/1", checking if it\'s active\n' - 'Backup "/clickhouse/backups/1" is active, not going to delete\n' - 'Found backup "/clickhouse/backups/2", checking if it\'s active\n' - 'Backup "/clickhouse/backups/2" is not active, deleting it\n' - "1\n" - ) - - -def test_base_commands(): - command = keeper_query( - "create test_create_zk_node1 testvalue1;" - "create test_create_zk_node_2 testvalue2;" - "get test_create_zk_node1;" - ) - assert command.get_answer() == "testvalue1\n" -def test_four_letter_word_commands(): - command = keeper_query("ruok") +def test_four_letter_word_commands(started_cluster): + _ = started_cluster + + command = CommandRequest( + [ + started_cluster.server_bin_path, + "keeper-client", + "--host", + str(cluster.get_instance_ip("zoo1")), + "--port", + str(cluster.zookeeper_port), + "-q", + "ruok", + ], + stdin="", + ) + assert command.get_answer() == "imok\n" From cc8c2c88bf160750545f1c33a00343ecf9e8d37f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 3 Aug 2023 14:25:37 +0200 Subject: [PATCH 1539/2047] Remove assertion from test_no_ttl_merges_in_busy_pool --- tests/integration/test_concurrent_ttl_merges/test.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/integration/test_concurrent_ttl_merges/test.py b/tests/integration/test_concurrent_ttl_merges/test.py index 96264e53522..3a3981d65ba 100644 --- a/tests/integration/test_concurrent_ttl_merges/test.py +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -103,10 +103,6 @@ def test_no_ttl_merges_in_busy_pool(started_cluster): rows_count.append(int(node1.query("SELECT count() FROM test_ttl").strip())) time.sleep(0.5) - # at least several seconds we didn't run any TTL merges and rows count equal - # to the original value - assert sum([1 for count in rows_count if count == 30]) > 4 - assert_eq_with_retry(node1, "SELECT COUNT() FROM test_ttl", "0") node1.query("DROP TABLE test_ttl SYNC") From 2a681f1577b70c7e0f250fdfa3e2ea2d9239d172 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 3 Aug 2023 14:32:03 +0200 Subject: [PATCH 1540/2047] Removed unused function and fixed build after merge --- src/Storages/StorageAzureBlob.h | 1 - src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp | 4 ++-- src/TableFunctions/TableFunctionAzureBlobStorageCluster.h | 3 ++- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 99907ba3da7..a3bacebfffe 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -204,7 +204,6 @@ public: { return {callback(), {}}; } - size_t getTotalSize() const override { return 0; } private: ReadTaskCallback callback; diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp index e4c5d25492b..eee585967c2 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp @@ -17,7 +17,7 @@ namespace DB StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( const ASTPtr & /*function*/, ContextPtr context, - const std::string & table_name, ColumnsDescription /*cached_columns*/) const + const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const { StoragePtr storage; ColumnsDescription columns; @@ -32,7 +32,7 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( columns = structure_hint; } - auto client = StorageAzureBlob::createClient(configuration); + auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); auto settings = StorageAzureBlob::createSettings(context); if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h index c263fd6ca0c..58f79328f63 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h @@ -44,7 +44,8 @@ protected: const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, - ColumnsDescription cached_columns) const override; + ColumnsDescription cached_columns, + bool is_insert_query) const override; const char * getStorageTypeName() const override { return "AzureBlobStorageCluster"; } }; From a98fae936d69b66518256e5067e3d1defa3a8048 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 14:33:29 +0200 Subject: [PATCH 1541/2047] Fix `test_dictionary_custom_settings` --- tests/integration/test_dictionary_custom_settings/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py index 715219ceb87..6b5ea643998 100644 --- a/tests/integration/test_dictionary_custom_settings/test.py +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -2,6 +2,8 @@ import os import pytest from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + DICTIONARY_FILES = [ "configs/dictionaries/FileSourceConfig.xml", @@ -78,5 +80,7 @@ def test_work(start_cluster): assert caught_exception.find("Limit for result exceeded") != -1 - assert query("SELECT dictGetString('test_http', 'first', toUInt64(1))") == "\\'a\n" - assert query("SELECT dictGetString('test_http', 'second', toUInt64(1))") == '"b\n' + # It is possible that the HTTP server takes long time to start accepting connections + + assert_eq_with_retry(instance, "SELECT dictGetString('test_http', 'first', toUInt64(1))", "\\'a\n") + assert_eq_with_retry(instance, "SELECT dictGetString('test_http', 'second', toUInt64(1))", '"b\n') From 8499956321dede2b89bd6f627d493edee7b5d486 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 2 Aug 2023 18:27:14 +0200 Subject: [PATCH 1542/2047] Refactor CI_CONFIG from dict to dataclasses --- tests/ci/build_check.py | 28 +- tests/ci/build_download_helper.py | 2 +- tests/ci/build_report_check.py | 2 +- tests/ci/ci_config.py | 620 +++++++++-------------- tests/ci/commit_status_helper.py | 3 +- tests/ci/download_binary.py | 8 +- tests/ci/performance_comparison_check.py | 11 +- 7 files changed, 267 insertions(+), 407 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index b75650d6d60..592c27c2c68 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -38,11 +38,11 @@ BUILD_LOG_NAME = "build_log.log" def _can_export_binaries(build_config: BuildConfig) -> bool: - if build_config["package_type"] != "deb": + if build_config.package_type != "deb": return False - if build_config["sanitizer"] != "": + if build_config.sanitizer != "": return True - if build_config["debug_build"]: + if build_config.debug_build: return True return False @@ -55,26 +55,26 @@ def get_packager_cmd( image_version: str, official: bool, ) -> str: - package_type = build_config["package_type"] - comp = build_config["compiler"] + package_type = build_config.package_type + comp = build_config.compiler cmake_flags = "-DENABLE_CLICKHOUSE_SELF_EXTRACTING=1" cmd = ( f"cd {packager_path} && CMAKE_FLAGS='{cmake_flags}' ./packager --output-dir={output_path} " f"--package-type={package_type} --compiler={comp}" ) - if build_config["debug_build"]: + if build_config.debug_build: cmd += " --debug-build" - if build_config["sanitizer"]: - cmd += f" --sanitizer={build_config['sanitizer']}" - if build_config["tidy"] == "enable": + if build_config.sanitizer: + cmd += f" --sanitizer={build_config.sanitizer}" + if build_config.tidy: cmd += " --clang-tidy" cmd += " --cache=sccache" cmd += " --s3-rw-access" cmd += f" --s3-bucket={S3_BUILDS_BUCKET}" - if "additional_pkgs" in build_config and build_config["additional_pkgs"]: + if build_config.additional_pkgs: cmd += " --additional-pkgs" cmd += f" --docker-image-version={image_version}" @@ -180,7 +180,7 @@ def create_json_artifact( result = { "log_url": log_url, "build_urls": build_urls, - "build_config": build_config, + "build_config": build_config.__dict__, "elapsed_seconds": elapsed, "status": success, "job_name": GITHUB_JOB, @@ -220,7 +220,7 @@ def upload_master_static_binaries( build_output_path: str, ) -> None: """Upload binary artifacts to a static S3 links""" - static_binary_name = build_config.get("static_binary_name", False) + static_binary_name = build_config.static_binary_name if pr_info.number != 0: return elif not static_binary_name: @@ -240,7 +240,7 @@ def main(): stopwatch = Stopwatch() build_name = sys.argv[1] - build_config = CI_CONFIG["build_config"][build_name] + build_config = CI_CONFIG.build_config[build_name] if not os.path.exists(TEMP_PATH): os.makedirs(TEMP_PATH) @@ -270,8 +270,6 @@ def main(): logging.info("Got version from repo %s", version.string) official_flag = pr_info.number == 0 - if "official" in build_config: - official_flag = build_config["official"] version_type = "testing" if "release" in pr_info.labels or "release-lts" in pr_info.labels: diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 47c11ee0911..ec4cf8f9bfa 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -91,7 +91,7 @@ def get_gh_api( def get_build_name_for_check(check_name: str) -> str: - return CI_CONFIG["tests_config"][check_name]["required_build"] # type: ignore + return CI_CONFIG.test_configs[check_name].required_build def read_build_urls(build_name: str, reports_path: str) -> List[str]: diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 295b6cf9740..a134cb19346 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -149,7 +149,7 @@ def main(): logging.info("Check is already finished according to github status, exiting") sys.exit(0) - builds_for_check = CI_CONFIG["builds_report_config"][build_check_name] + builds_for_check = CI_CONFIG.builds_report_config[build_check_name] required_builds = required_builds or len(builds_for_check) # Collect reports from json artifacts diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 1e921f4a0cc..9d170fe8ed6 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -1,180 +1,184 @@ #!/usr/bin/env python3 +import logging + from dataclasses import dataclass -from typing import Callable, Dict, TypeVar +from typing import Callable, Dict, List, Literal -ConfValue = TypeVar("ConfValue", str, bool) -BuildConfig = Dict[str, ConfValue] -CI_CONFIG = { - "build_config": { - "package_release": { - "compiler": "clang-16", - "debug_build": False, - "sanitizer": "", - "package_type": "deb", - "static_binary_name": "amd64", - "additional_pkgs": True, - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "package_aarch64": { - "compiler": "clang-16-aarch64", - "debug_build": False, - "sanitizer": "", - "package_type": "deb", - "static_binary_name": "aarch64", - "additional_pkgs": True, - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "package_asan": { - "compiler": "clang-16", - "debug_build": False, - "sanitizer": "address", - "package_type": "deb", - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "package_ubsan": { - "compiler": "clang-16", - "debug_build": False, - "sanitizer": "undefined", - "package_type": "deb", - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "package_tsan": { - "compiler": "clang-16", - "debug_build": False, - "sanitizer": "thread", - "package_type": "deb", - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "package_msan": { - "compiler": "clang-16", - "debug_build": False, - "sanitizer": "memory", - "package_type": "deb", - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "package_debug": { - "compiler": "clang-16", - "debug_build": True, - "sanitizer": "", - "package_type": "deb", - "tidy": "disable", - "with_coverage": False, - "comment": "Note: sparse checkout was used", - }, - "binary_release": { - "compiler": "clang-16", - "debug_build": False, - "sanitizer": "", - "package_type": "binary", - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "binary_tidy": { - "compiler": "clang-16", - "debug_build": True, - "sanitizer": "", - "package_type": "binary", - "static_binary_name": "debug-amd64", - "tidy": "enable", - "with_coverage": False, - "comment": "clang-tidy is used for static analysis", - }, - "binary_darwin": { - "compiler": "clang-16-darwin", - "debug_build": False, - "sanitizer": "", - "package_type": "binary", - "static_binary_name": "macos", - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "binary_aarch64": { - "compiler": "clang-16-aarch64", - "debug_build": False, - "sanitizer": "", - "package_type": "binary", - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "binary_aarch64_v80compat": { - "compiler": "clang-16-aarch64-v80compat", - "debug_build": False, - "sanitizer": "", - "package_type": "binary", - "static_binary_name": "aarch64v80compat", - "tidy": "disable", - "with_coverage": False, - "comment": "For ARMv8.1 and older", - }, - "binary_freebsd": { - "compiler": "clang-16-freebsd", - "debug_build": False, - "sanitizer": "", - "package_type": "binary", - "static_binary_name": "freebsd", - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "binary_darwin_aarch64": { - "compiler": "clang-16-darwin-aarch64", - "debug_build": False, - "sanitizer": "", - "package_type": "binary", - "static_binary_name": "macos-aarch64", - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "binary_ppc64le": { - "compiler": "clang-16-ppc64le", - "debug_build": False, - "sanitizer": "", - "package_type": "binary", - "static_binary_name": "powerpc64le", - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, - "binary_amd64_compat": { - "compiler": "clang-16-amd64-compat", - "debug_build": False, - "sanitizer": "", - "package_type": "binary", - "static_binary_name": "amd64compat", - "tidy": "disable", - "with_coverage": False, - "comment": "SSE2-only build", - }, - "binary_riscv64": { - "compiler": "clang-16-riscv64", - "debug_build": False, - "sanitizer": "", - "package_type": "binary", - "static_binary_name": "riscv64", - "tidy": "disable", - "with_coverage": False, - "comment": "", - }, +@dataclass +class BuildConfig: + compiler: str + package_type: Literal["deb", "binary"] + additional_pkgs: bool = False + debug_build: bool = False + sanitizer: str = "" + tidy: bool = False + comment: str = "" + static_binary_name: str = "" + + +@dataclass +class TestConfig: + required_build: str + force_tests: bool = False + + +BuildConfigs = Dict[str, BuildConfig] +BuildsReportConfig = Dict[str, List[str]] +TestConfigs = Dict[str, TestConfig] + + +@dataclass +class CiConfig: + build_config: BuildConfigs + builds_report_config: BuildsReportConfig + test_configs: TestConfigs + + def validate(self) -> None: + errors = [] + # All build configs must belong to build_report_config + for build_name in self.build_config.keys(): + build_in_reports = False + for report_config in self.builds_report_config.values(): + if build_name in report_config: + build_in_reports = True + break + if not build_in_reports: + logging.error( + "Build name %s does not belong to build reports", build_name + ) + errors.append( + f"Build name {build_name} does not belong to build reports" + ) + # And otherwise + for build_report_name, build_names in self.builds_report_config.items(): + missed_names = [ + name for name in build_names if name not in self.build_config.keys() + ] + if missed_names: + logging.error( + "The following names of the build report '%s' " + "are missed in build_config: %s", + build_report_name, + missed_names, + ) + errors.append( + f"The following names of the build report '{build_report_name}' " + f"are missed in build_config: {missed_names}", + ) + # And finally, all of tests' requirements must be in the builds + for test_name, test_config in self.test_configs.items(): + if test_config.required_build not in self.build_config.keys(): + logging.error( + "The requierment '%s' for '%s' is not found in builds", + test_config, + test_name, + ) + errors.append( + f"The requierment '{test_config}' for " + f"'{test_name}' is not found in builds" + ) + + if errors: + raise KeyError("config contains errors", errors) + + +CI_CONFIG = CiConfig( + build_config={ + "package_release": BuildConfig( + compiler="clang-16", + package_type="deb", + static_binary_name="amd64", + additional_pkgs=True, + ), + "package_aarch64": BuildConfig( + compiler="clang-16-aarch64", + package_type="deb", + static_binary_name="aarch64", + additional_pkgs=True, + ), + "package_asan": BuildConfig( + compiler="clang-16", + sanitizer="address", + package_type="deb", + ), + "package_ubsan": BuildConfig( + compiler="clang-16", + sanitizer="undefined", + package_type="deb", + ), + "package_tsan": BuildConfig( + compiler="clang-16", + sanitizer="thread", + package_type="deb", + ), + "package_msan": BuildConfig( + compiler="clang-16", + sanitizer="memory", + package_type="deb", + ), + "package_debug": BuildConfig( + compiler="clang-16", + debug_build=True, + package_type="deb", + comment="Note: sparse checkout was used", + ), + "binary_release": BuildConfig( + compiler="clang-16", + package_type="binary", + ), + "binary_tidy": BuildConfig( + compiler="clang-16", + debug_build=True, + package_type="binary", + static_binary_name="debug-amd64", + tidy=True, + comment="clang-tidy is used for static analysis", + ), + "binary_darwin": BuildConfig( + compiler="clang-16-darwin", + package_type="binary", + static_binary_name="macos", + ), + "binary_aarch64": BuildConfig( + compiler="clang-16-aarch64", + package_type="binary", + ), + "binary_aarch64_v80compat": BuildConfig( + compiler="clang-16-aarch64-v80compat", + package_type="binary", + static_binary_name="aarch64v80compat", + comment="For ARMv8.1 and older", + ), + "binary_freebsd": BuildConfig( + compiler="clang-16-freebsd", + package_type="binary", + static_binary_name="freebsd", + ), + "binary_darwin_aarch64": BuildConfig( + compiler="clang-16-darwin-aarch64", + package_type="binary", + static_binary_name="macos-aarch64", + ), + "binary_ppc64le": BuildConfig( + compiler="clang-16-ppc64le", + package_type="binary", + static_binary_name="powerpc64le", + ), + "binary_amd64_compat": BuildConfig( + compiler="clang-16-amd64-compat", + package_type="binary", + static_binary_name="amd64compat", + comment="SSE2-only build", + ), + "binary_riscv64": BuildConfig( + compiler="clang-16-riscv64", + package_type="binary", + static_binary_name="riscv64", + ), }, - "builds_report_config": { + builds_report_config={ "ClickHouse build check": [ "package_release", "package_aarch64", @@ -197,213 +201,79 @@ CI_CONFIG = { "binary_amd64_compat", ], }, - "tests_config": { - # required_build - build name for artifacts - # force_tests - force success status for tests - "Install packages (amd64)": { - "required_build": "package_release", - }, - "Install packages (arm64)": { - "required_build": "package_aarch64", - }, - "Stateful tests (asan)": { - "required_build": "package_asan", - }, - "Stateful tests (tsan)": { - "required_build": "package_tsan", - }, - "Stateful tests (msan)": { - "required_build": "package_msan", - }, - "Stateful tests (ubsan)": { - "required_build": "package_ubsan", - }, - "Stateful tests (debug)": { - "required_build": "package_debug", - }, - "Stateful tests (release)": { - "required_build": "package_release", - }, - "Stateful tests (aarch64)": { - "required_build": "package_aarch64", - }, - "Stateful tests (release, DatabaseOrdinary)": { - "required_build": "package_release", - }, - "Stateful tests (release, DatabaseReplicated)": { - "required_build": "package_release", - }, + test_configs={ + "Install packages (amd64)": TestConfig("package_release"), + "Install packages (arm64)": TestConfig("package_aarch64"), + "Stateful tests (asan)": TestConfig("package_asan"), + "Stateful tests (tsan)": TestConfig("package_tsan"), + "Stateful tests (msan)": TestConfig("package_msan"), + "Stateful tests (ubsan)": TestConfig("package_ubsan"), + "Stateful tests (debug)": TestConfig("package_debug"), + "Stateful tests (release)": TestConfig("package_release"), + "Stateful tests (aarch64)": TestConfig("package_aarch64"), + "Stateful tests (release, DatabaseOrdinary)": TestConfig("package_release"), + "Stateful tests (release, DatabaseReplicated)": TestConfig("package_release"), # Stateful tests for parallel replicas - "Stateful tests (release, ParallelReplicas)": { - "required_build": "package_release", - }, - "Stateful tests (debug, ParallelReplicas)": { - "required_build": "package_debug", - }, - "Stateful tests (asan, ParallelReplicas)": { - "required_build": "package_asan", - }, - "Stateful tests (msan, ParallelReplicas)": { - "required_build": "package_msan", - }, - "Stateful tests (ubsan, ParallelReplicas)": { - "required_build": "package_ubsan", - }, - "Stateful tests (tsan, ParallelReplicas)": { - "required_build": "package_tsan", - }, + "Stateful tests (release, ParallelReplicas)": TestConfig("package_release"), + "Stateful tests (debug, ParallelReplicas)": TestConfig("package_debug"), + "Stateful tests (asan, ParallelReplicas)": TestConfig("package_asan"), + "Stateful tests (msan, ParallelReplicas)": TestConfig("package_msan"), + "Stateful tests (ubsan, ParallelReplicas)": TestConfig("package_ubsan"), + "Stateful tests (tsan, ParallelReplicas)": TestConfig("package_tsan"), # End stateful tests for parallel replicas - "Stateless tests (asan)": { - "required_build": "package_asan", - }, - "Stateless tests (tsan)": { - "required_build": "package_tsan", - }, - "Stateless tests (msan)": { - "required_build": "package_msan", - }, - "Stateless tests (ubsan)": { - "required_build": "package_ubsan", - }, - "Stateless tests (debug)": { - "required_build": "package_debug", - }, - "Stateless tests (release)": { - "required_build": "package_release", - }, - "Stateless tests (aarch64)": { - "required_build": "package_aarch64", - }, - "Stateless tests (release, wide parts enabled)": { - "required_build": "package_release", - }, - "Stateless tests (release, analyzer)": { - "required_build": "package_release", - }, - "Stateless tests (release, DatabaseOrdinary)": { - "required_build": "package_release", - }, - "Stateless tests (release, DatabaseReplicated)": { - "required_build": "package_release", - }, - "Stateless tests (release, s3 storage)": { - "required_build": "package_release", - }, - "Stateless tests (debug, s3 storage)": { - "required_build": "package_debug", - }, - "Stateless tests (tsan, s3 storage)": { - "required_build": "package_tsan", - }, - "Stress test (asan)": { - "required_build": "package_asan", - }, - "Stress test (tsan)": { - "required_build": "package_tsan", - }, - "Stress test (ubsan)": { - "required_build": "package_ubsan", - }, - "Stress test (msan)": { - "required_build": "package_msan", - }, - "Stress test (debug)": { - "required_build": "package_debug", - }, - "Upgrade check (asan)": { - "required_build": "package_asan", - }, - "Upgrade check (tsan)": { - "required_build": "package_tsan", - }, - "Upgrade check (msan)": { - "required_build": "package_msan", - }, - "Upgrade check (debug)": { - "required_build": "package_debug", - }, - "Integration tests (asan)": { - "required_build": "package_asan", - }, - "Integration tests (asan, analyzer)": { - "required_build": "package_asan", - }, - "Integration tests (tsan)": { - "required_build": "package_tsan", - }, - "Integration tests (release)": { - "required_build": "package_release", - }, - "Integration tests (msan)": { - "required_build": "package_msan", - }, - "Integration tests flaky check (asan)": { - "required_build": "package_asan", - }, - "Compatibility check (amd64)": { - "required_build": "package_release", - }, - "Compatibility check (aarch64)": { - "required_build": "package_aarch64", - }, - "Unit tests (release)": { - "required_build": "binary_release", - }, - "Unit tests (asan)": { - "required_build": "package_asan", - }, - "Unit tests (msan)": { - "required_build": "package_msan", - }, - "Unit tests (tsan)": { - "required_build": "package_tsan", - }, - "Unit tests (ubsan)": { - "required_build": "package_ubsan", - }, - "AST fuzzer (debug)": { - "required_build": "package_debug", - }, - "AST fuzzer (asan)": { - "required_build": "package_asan", - }, - "AST fuzzer (msan)": { - "required_build": "package_msan", - }, - "AST fuzzer (tsan)": { - "required_build": "package_tsan", - }, - "AST fuzzer (ubsan)": { - "required_build": "package_ubsan", - }, - "Stateless tests flaky check (asan)": { - "required_build": "package_asan", - }, - "ClickHouse Keeper Jepsen": { - "required_build": "binary_release", - }, - "ClickHouse Server Jepsen": { - "required_build": "binary_release", - }, - "Performance Comparison": { - "required_build": "package_release", - "test_grep_exclude_filter": "", - }, - "Performance Comparison Aarch64": { - "required_build": "package_aarch64", - "test_grep_exclude_filter": "", - }, - "SQLancer (release)": { - "required_build": "package_release", - }, - "SQLancer (debug)": { - "required_build": "package_debug", - }, - "Sqllogic test (release)": { - "required_build": "package_release", - }, + "Stateless tests (asan)": TestConfig("package_asan"), + "Stateless tests (tsan)": TestConfig("package_tsan"), + "Stateless tests (msan)": TestConfig("package_msan"), + "Stateless tests (ubsan)": TestConfig("package_ubsan"), + "Stateless tests (debug)": TestConfig("package_debug"), + "Stateless tests (release)": TestConfig("package_release"), + "Stateless tests (aarch64)": TestConfig("package_aarch64"), + "Stateless tests (release, wide parts enabled)": TestConfig("package_release"), + "Stateless tests (release, analyzer)": TestConfig("package_release"), + "Stateless tests (release, DatabaseOrdinary)": TestConfig("package_release"), + "Stateless tests (release, DatabaseReplicated)": TestConfig("package_release"), + "Stateless tests (release, s3 storage)": TestConfig("package_release"), + "Stateless tests (debug, s3 storage)": TestConfig("package_debug"), + "Stateless tests (tsan, s3 storage)": TestConfig("package_tsan"), + "Stress test (asan)": TestConfig("package_asan"), + "Stress test (tsan)": TestConfig("package_tsan"), + "Stress test (ubsan)": TestConfig("package_ubsan"), + "Stress test (msan)": TestConfig("package_msan"), + "Stress test (debug)": TestConfig("package_debug"), + "Upgrade check (asan)": TestConfig("package_asan"), + "Upgrade check (tsan)": TestConfig("package_tsan"), + "Upgrade check (msan)": TestConfig("package_msan"), + "Upgrade check (debug)": TestConfig("package_debug"), + "Integration tests (asan)": TestConfig("package_asan"), + "Integration tests (asan, analyzer)": TestConfig("package_asan"), + "Integration tests (tsan)": TestConfig("package_tsan"), + "Integration tests (release)": TestConfig("package_release"), + "Integration tests (msan)": TestConfig("package_msan"), + "Integration tests flaky check (asan)": TestConfig("package_asan"), + "Compatibility check (amd64)": TestConfig("package_release"), + "Compatibility check (aarch64)": TestConfig("package_aarch64"), + "Unit tests (release)": TestConfig("binary_release"), + "Unit tests (asan)": TestConfig("package_asan"), + "Unit tests (msan)": TestConfig("package_msan"), + "Unit tests (tsan)": TestConfig("package_tsan"), + "Unit tests (ubsan)": TestConfig("package_ubsan"), + "AST fuzzer (debug)": TestConfig("package_debug"), + "AST fuzzer (asan)": TestConfig("package_asan"), + "AST fuzzer (msan)": TestConfig("package_msan"), + "AST fuzzer (tsan)": TestConfig("package_tsan"), + "AST fuzzer (ubsan)": TestConfig("package_ubsan"), + "Stateless tests flaky check (asan)": TestConfig("package_asan"), + "ClickHouse Keeper Jepsen": TestConfig("binary_release"), + "ClickHouse Server Jepsen": TestConfig("binary_release"), + "Performance Comparison": TestConfig("package_release"), + "Performance Comparison Aarch64": TestConfig("package_aarch64"), + "SQLancer (release)": TestConfig("package_release"), + "SQLancer (debug)": TestConfig("package_debug"), + "Sqllogic test (release)": TestConfig("package_release"), }, -} # type: dict +) +CI_CONFIG.validate() + # checks required by Mergeable Check REQUIRED_CHECKS = [ diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index efe149b0aa4..945bcfe05ed 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -51,7 +51,8 @@ class RerunHelper: def override_status(status: str, check_name: str, invert: bool = False) -> str: - if CI_CONFIG["tests_config"].get(check_name, {}).get("force_tests", False): + test_config = CI_CONFIG.test_configs.get(check_name) + if test_config and test_config.force_tests: return "success" if invert: diff --git a/tests/ci/download_binary.py b/tests/ci/download_binary.py index c57780daa36..034e65f204d 100755 --- a/tests/ci/download_binary.py +++ b/tests/ci/download_binary.py @@ -9,7 +9,7 @@ import os from pathlib import Path from build_download_helper import download_build_with_progress -from ci_config import CI_CONFIG, BuildConfig +from ci_config import CI_CONFIG from env_helper import RUNNER_TEMP, S3_ARTIFACT_DOWNLOAD_TEMPLATE from git_helper import Git, commit from version_helper import get_version_from_repo, version_arg @@ -62,9 +62,9 @@ def main(): temp_path.mkdir(parents=True, exist_ok=True) for build in args.build_names: # check if it's in CI_CONFIG - config = CI_CONFIG["build_config"][build] # type: BuildConfig - if args.rename: - path = temp_path / f"clickhouse-{config['static_binary_name']}" + config = CI_CONFIG.build_config[build] + if args.rename and config.static_binary_name: + path = temp_path / f"clickhouse-{config.static_binary_name}" else: path = temp_path / "clickhouse" diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index 41ace95c350..70d37b24c4e 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -71,7 +71,7 @@ if __name__ == "__main__": reports_path = os.getenv("REPORTS_PATH", "./reports") check_name = sys.argv[1] - required_build = CI_CONFIG["tests_config"][check_name]["required_build"] + required_build = CI_CONFIG.test_configs[check_name].required_build if not os.path.exists(temp_path): os.makedirs(temp_path) @@ -121,15 +121,6 @@ if __name__ == "__main__": ) sys.exit(0) - test_grep_exclude_filter = CI_CONFIG["tests_config"][check_name][ - "test_grep_exclude_filter" - ] - if test_grep_exclude_filter: - docker_env += f" -e CHPC_TEST_GREP_EXCLUDE={test_grep_exclude_filter}" - logging.info( - "Fill fliter our performance tests by grep -v %s", test_grep_exclude_filter - ) - rerun_helper = RerunHelper(commit, check_name_with_group) if rerun_helper.is_already_finished_by_status(): logging.info("Check is already finished according to github status, exiting") From 5c7788b1547647e0e0696dac85ea73e84ad0f306 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 3 Aug 2023 12:42:50 +0000 Subject: [PATCH 1543/2047] Lib file --- .../queries/0_stateless/02661_read_from_archive.lib | 13 +++++++++++++ 1 file changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/02661_read_from_archive.lib diff --git a/tests/queries/0_stateless/02661_read_from_archive.lib b/tests/queries/0_stateless/02661_read_from_archive.lib new file mode 100644 index 00000000000..fc76c19e8de --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive.lib @@ -0,0 +1,13 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function read_archive_file() { + $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${user_files_path}/$1') ORDER BY 1, 2" + $CLICKHOUSE_CLIENT --query "CREATE TABLE 02661_archive_table Engine=File('CSV', '${user_files_path}/$1')" + $CLICKHOUSE_CLIENT --query "SELECT * FROM 02661_archive_table ORDER BY 1, 2" + $CLICKHOUSE_CLIENT --query "DROP TABLE 02661_archive_table" +} \ No newline at end of file From dadf4678ee9d678459c96cd13b8231e33dc02cc3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Aug 2023 12:50:40 +0000 Subject: [PATCH 1544/2047] Automatic style fix --- tests/integration/test_dictionary_custom_settings/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py index 6b5ea643998..eb394da8bb6 100644 --- a/tests/integration/test_dictionary_custom_settings/test.py +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -82,5 +82,9 @@ def test_work(start_cluster): # It is possible that the HTTP server takes long time to start accepting connections - assert_eq_with_retry(instance, "SELECT dictGetString('test_http', 'first', toUInt64(1))", "\\'a\n") - assert_eq_with_retry(instance, "SELECT dictGetString('test_http', 'second', toUInt64(1))", '"b\n') + assert_eq_with_retry( + instance, "SELECT dictGetString('test_http', 'first', toUInt64(1))", "\\'a\n" + ) + assert_eq_with_retry( + instance, "SELECT dictGetString('test_http', 'second', toUInt64(1))", '"b\n' + ) From c279b00c52729190e1bce4cc54a7a9268b14c2e1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Aug 2023 12:55:27 +0000 Subject: [PATCH 1545/2047] Automatic style fix --- tests/integration/test_dictionary_custom_settings/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py index 6b5ea643998..eb394da8bb6 100644 --- a/tests/integration/test_dictionary_custom_settings/test.py +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -82,5 +82,9 @@ def test_work(start_cluster): # It is possible that the HTTP server takes long time to start accepting connections - assert_eq_with_retry(instance, "SELECT dictGetString('test_http', 'first', toUInt64(1))", "\\'a\n") - assert_eq_with_retry(instance, "SELECT dictGetString('test_http', 'second', toUInt64(1))", '"b\n') + assert_eq_with_retry( + instance, "SELECT dictGetString('test_http', 'first', toUInt64(1))", "\\'a\n" + ) + assert_eq_with_retry( + instance, "SELECT dictGetString('test_http', 'second', toUInt64(1))", '"b\n' + ) From 63a46d7cb0d51745856a9f044c36d6b43ba50029 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 3 Aug 2023 21:05:31 +0800 Subject: [PATCH 1546/2047] fix pg integration test --- tests/integration/test_postgresql_database_engine/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py index b14d234f7eb..bdf1056765e 100644 --- a/tests/integration/test_postgresql_database_engine/test.py +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -406,14 +406,17 @@ def test_postgresql_password_leak(started_cluster): ) cursor = conn.cursor() + cursor.execute("DROP SCHEMA IF EXISTS test_schema CASCADE") cursor.execute("CREATE SCHEMA test_schema") cursor.execute("CREATE TABLE test_schema.table1 (a integer)") cursor.execute("CREATE TABLE table2 (a integer)") + node1.query("DROP DATABASE IF EXISTS postgres_database") node1.query( "CREATE DATABASE postgres_database ENGINE = PostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword', 'test_schema')" ) + node1.query("DROP DATABASE IF EXISTS postgres_database2") node1.query( "CREATE DATABASE postgres_database2 ENGINE = PostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')" ) From 65e8ee8fb7d30b0765c8bfd3d58d9def64e002bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 15:28:18 +0200 Subject: [PATCH 1547/2047] Fix paths --- docker/test/sqltest/run.sh | 4 +++- tests/ci/sqltest.py | 8 ++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 42aeef9df15..cba1c1dab1f 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -42,8 +42,10 @@ done # Run the test pushd sqltest/standards/2016/ /test.py -mv report.html test.log / +mv report.html test.log /workspace popd zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.err.log + +mv /var/log/clickhouse-server/clickhouse-server.log.zst /var/log/clickhouse-server/clickhouse-server.err.log.zst /workspace diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 7385716850f..d76baf22bb3 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,10 +110,10 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "server.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.err.log.zst"), - "report.html": os.path.join(workspace_path, "/report.html"), - "test.log": os.path.join(workspace_path, "/test.log"), + "server.log.zst": os.path.join(workspace_path, "/workspace/server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), + "report.html": os.path.join(workspace_path, "/workspace/report.html"), + "test.log": os.path.join(workspace_path, "/workspace/test.log"), } s3_helper = S3Helper() From a96b0457e04543876b35ca1a5223ea6007bbf9bb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Aug 2023 13:43:45 +0000 Subject: [PATCH 1548/2047] Automatic style fix --- tests/ci/sqltest.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index d76baf22bb3..69af21f1761 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -111,7 +111,9 @@ def main(): paths = { "run.log": run_log_path, "server.log.zst": os.path.join(workspace_path, "/workspace/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), + "server.err.log.zst": os.path.join( + workspace_path, "/workspace/server.err.log.zst" + ), "report.html": os.path.join(workspace_path, "/workspace/report.html"), "test.log": os.path.join(workspace_path, "/workspace/test.log"), } From f0eb22ac5f5cd64dec77550e417db06793fe0d94 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 3 Aug 2023 14:20:19 +0000 Subject: [PATCH 1549/2047] Fix build for S3Queue --- src/Storages/S3Queue/S3QueueSource.cpp | 23 ++----- src/Storages/S3Queue/S3QueueSource.h | 8 +-- src/Storages/S3Queue/StorageS3Queue.cpp | 83 ++----------------------- 3 files changed, 16 insertions(+), 98 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 57d2d6304b0..8d1d8e7b143 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -148,22 +148,12 @@ StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next( return KeyWithInfo(); } -Block StorageS3QueueSource::getHeader(Block sample_block, const std::vector & requested_virtual_columns) -{ - for (const auto & virtual_column : requested_virtual_columns) - sample_block.insert({virtual_column.type->createColumn(), virtual_column.type, virtual_column.name}); - - return sample_block; -} - StorageS3QueueSource::StorageS3QueueSource( - const std::vector & requested_virtual_columns_, + const ReadFromFormatInfo & info, const String & format_, String name_, - const Block & sample_block_, ContextPtr context_, std::optional format_settings_, - const ColumnsDescription & columns_, UInt64 max_block_size_, const S3Settings::RequestSettings & request_settings_, String compression_hint_, @@ -174,28 +164,27 @@ StorageS3QueueSource::StorageS3QueueSource( std::shared_ptr files_metadata_, const S3QueueAction & action_, const size_t download_thread_num_) - : ISource(getHeader(sample_block_, requested_virtual_columns_)) + : ISource(info.source_header) , WithContext(context_) , name(std::move(name_)) , bucket(bucket_) , version_id(version_id_) , format(format_) - , columns_desc(columns_) + , columns_desc(info.columns_description) , request_settings(request_settings_) , client(client_) , files_metadata(files_metadata_) - , requested_virtual_columns(requested_virtual_columns_) + , requested_virtual_columns(info.requested_virtual_columns) + , requested_columns(info.requested_columns) , file_iterator(file_iterator_) , action(action_) { internal_source = std::make_shared( - requested_virtual_columns_, + info, format_, name_, - sample_block_, context_, format_settings_, - columns_, max_block_size_, request_settings_, compression_hint_, diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index a85fce46ad8..523b8c0e81f 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -11,6 +11,7 @@ # include # include # include +# include # include # include @@ -67,13 +68,11 @@ public: static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); StorageS3QueueSource( - const std::vector & requested_virtual_columns_, + const ReadFromFormatInfo & info, const String & format, String name_, - const Block & sample_block, ContextPtr context_, std::optional format_settings_, - const ColumnsDescription & columns_, UInt64 max_block_size_, const S3Settings::RequestSettings & request_settings_, String compression_hint_, @@ -105,7 +104,8 @@ private: using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; - std::vector requested_virtual_columns; + NamesAndTypesList requested_virtual_columns; + NamesAndTypesList requested_columns; std::shared_ptr file_iterator; const S3QueueAction action; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 87bff398172..673c0dde1d2 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -33,6 +33,7 @@ # include # include # include +# include # include @@ -187,7 +188,7 @@ StorageS3Queue::StorageS3Queue( bool StorageS3Queue::supportsSubcolumns() const { - return FormatFactory::instance().checkIfFormatSupportsSubcolumns(configuration.format); + return true; } bool StorageS3Queue::supportsSubsetOfColumns() const @@ -213,55 +214,18 @@ Pipe StorageS3Queue::read( auto query_configuration = updateConfigurationAndGetCopy(local_context); - Pipes pipes; - - std::unordered_set column_names_set(column_names.begin(), column_names.end()); - std::vector requested_virtual_columns; - - for (const auto & virtual_column : getVirtuals()) - { - if (column_names_set.contains(virtual_column.name)) - requested_virtual_columns.push_back(virtual_column); - } - std::shared_ptr iterator_wrapper = createFileIterator(local_context, query_info.query); - ColumnsDescription columns_description; - Block block_for_format; - if (supportsSubsetOfColumns()) - { - auto fetch_columns = column_names; - const auto & virtuals = getVirtuals(); - std::erase_if( - fetch_columns, - [&](const String & col) - { - return std::any_of( - virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); - }); - - if (fetch_columns.empty()) - fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); - - columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); - block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); - } - else - { - columns_description = storage_snapshot->metadata->getColumns(); - block_for_format = storage_snapshot->metadata->getSampleBlock(); - } + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; return Pipe(std::make_shared( - requested_virtual_columns, + read_from_format_info, configuration.format, getName(), - block_for_format, local_context, format_settings, - columns_description, max_block_size, query_configuration.request_settings, configuration.compression_method, @@ -425,52 +389,17 @@ void StorageS3Queue::streamToViews() auto column_names = block_io.pipeline.getHeader().getNames(); // Create a stream for each consumer and join them in a union stream - std::vector requested_virtual_columns; - - for (const auto & virtual_column : getVirtuals()) - { - requested_virtual_columns.push_back(virtual_column); - } std::shared_ptr iterator_wrapper = createFileIterator(s3queue_context, nullptr); - ColumnsDescription columns_description; - Block block_for_format; - if (supportsSubsetOfColumns()) - { - auto fetch_columns = column_names; - const auto & virtuals = getVirtuals(); - std::erase_if( - fetch_columns, - [&](const String & col) - { - return std::any_of( - virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); - }); - - if (fetch_columns.empty()) - fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); - - columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); - block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); - } - else - { - columns_description = storage_snapshot->metadata->getColumns(); - block_for_format = storage_snapshot->metadata->getSampleBlock(); - } - + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); const size_t max_download_threads = s3queue_context->getSettingsRef().max_download_threads; - Pipes pipes; - auto pipe = Pipe(std::make_shared( - requested_virtual_columns, + read_from_format_info, configuration.format, getName(), - block_for_format, s3queue_context, format_settings, - columns_description, block_size, query_configuration.request_settings, configuration.compression_method, From 13d5ef91641fc02e737a729e520eff483f4dd837 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Aug 2023 14:43:03 +0000 Subject: [PATCH 1550/2047] Convert sparse to full in CreateSetAndFilterOnTheFlyStep --- .../CreateSetAndFilterOnTheFlyTransform.cpp | 9 +++++++-- .../02841_join_filter_set_sparse.reference | 2 ++ .../02841_join_filter_set_sparse.sql | 19 +++++++++++++++++++ 3 files changed, 28 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02841_join_filter_set_sparse.reference create mode 100644 tests/queries/0_stateless/02841_join_filter_set_sparse.sql diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 59c4b9a6a87..83a75318d61 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -35,7 +36,11 @@ Columns getColumnsByIndices(const Chunk & chunk, const std::vector & ind Columns columns; const Columns & all_cols = chunk.getColumns(); for (const auto & index : indices) - columns.push_back(all_cols.at(index)); + { + auto col = recursiveRemoveSparse(all_cols.at(index)); + columns.push_back(std::move(col)); + } + return columns; } @@ -149,7 +154,7 @@ IProcessor::Status FilterBySetOnTheFlyTransform::prepare() LOG_DEBUG(log, "Finished {} by [{}]: consumed {} rows in total, {} rows bypassed, result {} rows, {:.2f}% filtered", Poco::toLower(getDescription()), fmt::join(column_names, ", "), stat.consumed_rows, stat.consumed_rows_before_set, stat.result_rows, - 100 - 100.0 * stat.result_rows / stat.consumed_rows); + stat.consumed_rows > 0 ? (100 - 100.0 * stat.result_rows / stat.consumed_rows) : 0); } else { diff --git a/tests/queries/0_stateless/02841_join_filter_set_sparse.reference b/tests/queries/0_stateless/02841_join_filter_set_sparse.reference new file mode 100644 index 00000000000..1777e2e42f7 --- /dev/null +++ b/tests/queries/0_stateless/02841_join_filter_set_sparse.reference @@ -0,0 +1,2 @@ +3428033 +3428033 diff --git a/tests/queries/0_stateless/02841_join_filter_set_sparse.sql b/tests/queries/0_stateless/02841_join_filter_set_sparse.sql new file mode 100644 index 00000000000..6eab29675bb --- /dev/null +++ b/tests/queries/0_stateless/02841_join_filter_set_sparse.sql @@ -0,0 +1,19 @@ + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (s String) ENGINE = MergeTree ORDER BY s +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.5; + +INSERT INTO t1 SELECT if (number % 13 = 0, toString(number), '') FROM numbers(2000); + +CREATE TABLE t2 (s String) ENGINE = MergeTree ORDER BY s +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.5; + +INSERT INTO t2 SELECT if (number % 14 = 0, toString(number), '') FROM numbers(2000); + +SELECT countIf(ignore(*) == 0) FROM t1 JOIN t2 ON t1.s = t2.s; + +SET join_algorithm = 'full_sorting_merge', max_rows_in_set_to_optimize_join = 100_000; + +SELECT countIf(ignore(*) == 0) FROM t1 JOIN t2 ON t1.s = t2.s; From e9872d0e72eb8aa643e2a9e8a3a413fbe0494502 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 3 Aug 2023 14:44:14 +0000 Subject: [PATCH 1551/2047] fixed issues --- src/Functions/FunctionBinaryArithmetic.h | 55 +++++++++--------------- 1 file changed, 21 insertions(+), 34 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 2dd68d9cc90..33057a16dd0 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -74,6 +74,7 @@ namespace ErrorCodes extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int UNSUPPORTED_METHOD; extern const int BAD_ARGUMENTS; + extern const int SIZES_OF_ARRAYS_DONT_MATCH; } namespace traits_ @@ -1140,21 +1141,6 @@ class FunctionBinaryArithmetic : public IFunction return function->execute(arguments, result_type, input_rows_count); } - template - ColumnPtr executeArrayPlusMinus(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, - size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const - { - auto function = function_builder->build(arguments); - return function->execute(arguments, result_type, input_rows_count); - } - - static ColumnPtr callFunctionNotEquals(ColumnWithTypeAndName first, ColumnWithTypeAndName second, ContextPtr context) - { - ColumnsWithTypeAndName args{first, second}; - auto eq_func = FunctionFactory::instance().get("notEquals", context)->build(args); - return eq_func->execute(args, eq_func->getResultType(), args.front().column->size()); - } - ColumnPtr executeArrayImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { const auto * return_type_array = checkAndGetDataType(result_type.get()); @@ -1162,8 +1148,8 @@ class FunctionBinaryArithmetic : public IFunction if (!return_type_array) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Return type for function {} must be array.", getName()); - if constexpr (is_multiply || is_division) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use multiplication or division on arrays"); + if constexpr (!is_plus && !is_minus) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use this operation on arrays"); auto num_args = arguments.size(); DataTypes data_types; @@ -1191,23 +1177,29 @@ class FunctionBinaryArithmetic : public IFunction return executeImpl(new_arguments, result_type, input_rows_count); } + const auto * left_array_col = typeid_cast(arguments[0].column.get()); + const auto * right_array_col = typeid_cast(arguments[1].column.get()); + const auto & left_offsets = left_array_col->getOffsets(); + const auto & right_offsets = right_array_col->getOffsets(); + + chassert(left_offsets.size() == right_offsets.size() && "Unexpected difference in number of offsets"); /// Unpacking non-const arrays and checking sizes of them. - if (*typeid_cast(arguments[0].column.get())->getOffsets().data() != - *typeid_cast(arguments[1].column.get())->getOffsets().data()) + for(auto offset_index = 0U; offset_index < left_offsets.size(); ++offset_index) { - throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, - "Cannot apply operation for arguments of different sizes. Size of the first argument: {}, size of the second argument: {}", - *typeid_cast(arguments[0].column.get())->getOffsets().data(), - *typeid_cast(arguments[1].column.get())->getOffsets().data()); + if (left_offsets[offset_index] != right_offsets[offset_index]) + { + throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, + "Cannot apply operation for arguments of different sizes. Size of the first argument: {}, size of the second argument: {}", + *left_array_col->getOffsets().data(), + *right_array_col ->getOffsets().data()); + } } - auto array_ptr = typeid_cast(arguments[0].column.get())->getData().getPtr(); - result_array_type = typeid_cast(arguments[0].type.get())->getNestedType(); - new_arguments[0] = {array_ptr, result_array_type, arguments[0].name}; + const auto & left_array_type = typeid_cast(arguments[0].type.get())->getNestedType(); + new_arguments[0] = {left_array_col->getDataPtr(), left_array_type, arguments[0].name}; - array_ptr = typeid_cast(arguments[1].column.get())->getData().getPtr(); - result_array_type = typeid_cast(arguments[1].type.get())->getNestedType(); - new_arguments[1] = {array_ptr, result_array_type, arguments[1].name}; + const auto & right_array_type = typeid_cast(arguments[1].type.get())->getNestedType(); + new_arguments[1] = {right_array_col->getDataPtr(), right_array_type, arguments[1].name}; result_array_type = typeid_cast(result_type.get())->getNestedType(); @@ -1431,11 +1423,6 @@ public: return std::make_shared(getReturnTypeImplStatic(new_arguments, context)); } - if (isArray(arguments[0]) || isArray(arguments[1])) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Cannot execute arguments of different type. Type of the first argument: {}, type of the second argument: {}", - arguments[0]->getName(), arguments[1]->getName()); - /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1], context)) { From 7d00a2d29aca31caa2db285302c78cc9b2154981 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 3 Aug 2023 15:16:20 +0000 Subject: [PATCH 1552/2047] Fix 02483_elapsed_time --- tests/queries/0_stateless/02483_elapsed_time.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02483_elapsed_time.sh b/tests/queries/0_stateless/02483_elapsed_time.sh index e3b983129fb..fdb23d6da01 100755 --- a/tests/queries/0_stateless/02483_elapsed_time.sh +++ b/tests/queries/0_stateless/02483_elapsed_time.sh @@ -32,7 +32,7 @@ OK_QUERY_JSON=" WITH ( SELECT sleepEachRow(1.0) ) AS sub -SELECT * +SELECT *, sub FROM ( SELECT * @@ -50,7 +50,7 @@ WITH ( SELECT * FROM ( - SELECT * + SELECT *, sub FROM system.one ) FORMAT XML From da7f71d76155292ea88bf6e04ff7cb937c71c9d0 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 3 Aug 2023 17:20:16 +0200 Subject: [PATCH 1553/2047] Update FunctionBinaryArithmetic.h --- src/Functions/FunctionBinaryArithmetic.h | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 33057a16dd0..92da9515678 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -71,9 +71,7 @@ namespace ErrorCodes extern const int DECIMAL_OVERFLOW; extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int UNSUPPORTED_METHOD; - extern const int BAD_ARGUMENTS; extern const int SIZES_OF_ARRAYS_DONT_MATCH; } @@ -1181,10 +1179,10 @@ class FunctionBinaryArithmetic : public IFunction const auto * right_array_col = typeid_cast(arguments[1].column.get()); const auto & left_offsets = left_array_col->getOffsets(); const auto & right_offsets = right_array_col->getOffsets(); - + chassert(left_offsets.size() == right_offsets.size() && "Unexpected difference in number of offsets"); /// Unpacking non-const arrays and checking sizes of them. - for(auto offset_index = 0U; offset_index < left_offsets.size(); ++offset_index) + for (auto offset_index = 0U; offset_index < left_offsets.size(); ++offset_index) { if (left_offsets[offset_index] != right_offsets[offset_index]) { From a0fa3cc73bbbec2879873608607fc18f51a6556a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 3 Aug 2023 18:55:15 +0300 Subject: [PATCH 1554/2047] EXPLAIN actions for JOIN step --- src/Interpreters/ConcurrentHashJoin.h | 1 + src/Interpreters/DirectJoin.h | 1 + src/Interpreters/FullSortingMergeJoin.h | 1 + src/Interpreters/GraceHashJoin.h | 1 + src/Interpreters/HashJoin.h | 1 + src/Interpreters/IJoin.h | 2 + src/Interpreters/JoinSwitcher.h | 1 + src/Interpreters/MergeJoin.h | 1 + src/Interpreters/TableJoin.h | 2 +- src/Processors/QueryPlan/JoinStep.cpp | 33 +++++ src/Processors/QueryPlan/JoinStep.h | 3 + .../02835_join_step_explain.reference | 116 ++++++++++++++++++ .../0_stateless/02835_join_step_explain.sql | 31 +++++ 13 files changed, 193 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02835_join_step_explain.reference create mode 100644 tests/queries/0_stateless/02835_join_step_explain.sql diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 1283879971d..85e0c5a0ae7 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -36,6 +36,7 @@ public: explicit ConcurrentHashJoin(ContextPtr context_, std::shared_ptr table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_ = false); ~ConcurrentHashJoin() override = default; + std::string getName() const override { return "ConcurrentHashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & block, bool check_limits) override; void checkTypesOfKeys(const Block & block) const override; diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index e55ac278705..5f664314818 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -30,6 +30,7 @@ public: std::shared_ptr storage_, const Block & right_sample_block_with_storage_column_names_); + std::string getName() const override { return "DirectKeyValueJoin"; } virtual const TableJoin & getTableJoin() const override { return *table_join; } virtual bool addBlockToJoin(const Block &, bool) override; diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index a6b53a51c04..3fc9f8920ed 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -28,6 +28,7 @@ public: LOG_TRACE(&Poco::Logger::get("FullSortingMergeJoin"), "Will use full sorting merge join"); } + std::string getName() const override { return "FullSortingMergeJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & /* block */, bool /* check_limits */) override diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index ce519892b0e..44949440467 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -60,6 +60,7 @@ public: ~GraceHashJoin() override; + std::string getName() const override { return "GraceHashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } void initialize(const Block & sample_block) override; diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 56dea98c1f1..9f55945816c 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -151,6 +151,7 @@ public: ~HashJoin() override; + std::string getName() const override { return "HashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } /** Add block of data from right hand of JOIN to the map. diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 97b119bd795..493a5dd2126 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -48,6 +48,8 @@ class IJoin public: virtual ~IJoin() = default; + virtual std::string getName() const = 0; + virtual const TableJoin & getTableJoin() const = 0; /// Add block of data from right hand of JOIN. diff --git a/src/Interpreters/JoinSwitcher.h b/src/Interpreters/JoinSwitcher.h index fb5066b2d04..1d2ebc6b456 100644 --- a/src/Interpreters/JoinSwitcher.h +++ b/src/Interpreters/JoinSwitcher.h @@ -18,6 +18,7 @@ class JoinSwitcher : public IJoin public: JoinSwitcher(std::shared_ptr table_join_, const Block & right_sample_block_); + std::string getName() const override { return "JoinSwitcher"; } const TableJoin & getTableJoin() const override { return *table_join; } /// Add block of data from right hand of JOIN into current join object. diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 03a661c5b8a..98fae1d419f 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -22,6 +22,7 @@ class MergeJoin : public IJoin public: MergeJoin(std::shared_ptr table_join_, const Block & right_sample_block); + std::string getName() const override { return "PartialMergeJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & block, bool check_limits) override; void checkTypesOfKeys(const Block & block) const override; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 5d14a57759f..75626764bda 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -331,7 +331,7 @@ public: const ColumnsWithTypeAndName & right_sample_columns); void setAsofInequality(ASOFJoinInequality inequality) { asof_inequality = inequality; } - ASOFJoinInequality getAsofInequality() { return asof_inequality; } + ASOFJoinInequality getAsofInequality() const { return asof_inequality; } ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 33fa7955e0d..858a01a437d 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -2,6 +2,9 @@ #include #include #include +#include +#include +#include #include namespace DB @@ -62,6 +65,36 @@ void JoinStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } +void JoinStep::describeActions(FormatSettings & settings) const +{ + String prefix(settings.offset, ' '); + + const auto & table_join = join->getTableJoin(); + settings.out << prefix << "Kind: " << toString(table_join.kind()) << '\n'; + settings.out << prefix << "Strictness: " << toString(table_join.strictness()) << '\n'; + settings.out << prefix << "Type: " << join->getName() << '\n'; + + if (table_join.strictness() == JoinStrictness::Asof) + settings.out << prefix << "ASOF inequality: " << toString(table_join.getAsofInequality()) << '\n'; + + if (!table_join.getClauses().empty()) + settings.out << prefix << "Clauses: " << table_join.formatClauses(table_join.getClauses(), true /*short_format*/) << '\n'; +} + +void JoinStep::describeActions(JSONBuilder::JSONMap & map) const +{ + const auto & table_join = join->getTableJoin(); + map.add("Kind", toString(table_join.kind())); + map.add("Strictness", toString(table_join.strictness())); + map.add("Type", join->getName()); + + if (table_join.strictness() == JoinStrictness::Asof) + map.add("ASOF inequality", toString(table_join.getAsofInequality())); + + if (!table_join.getClauses().empty()) + map.add("Clauses", table_join.formatClauses(table_join.getClauses(), true /*short_format*/)); +} + void JoinStep::updateInputStream(const DataStream & new_input_stream_, size_t idx) { if (idx == 0) diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index e7185f36588..369ee9bec8b 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -27,6 +27,9 @@ public: void describePipeline(FormatSettings & settings) const override; + void describeActions(JSONBuilder::JSONMap & map) const override; + void describeActions(FormatSettings & settings) const override; + const JoinPtr & getJoin() const { return join; } bool allowPushDownToRight() const; diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference new file mode 100644 index 00000000000..77c91b3d7e9 --- /dev/null +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -0,0 +1,116 @@ +Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Header: id UInt64 + value_1 String + rhs.id UInt64 + rhs.value_1 String +Actions: INPUT : 0 -> id_0 UInt64 : 0 + INPUT : 1 -> value_1_1 String : 1 + INPUT : 2 -> value_1_3 String : 2 + INPUT : 3 -> id_2 UInt64 : 3 + ALIAS id_0 :: 0 -> id UInt64 : 4 + ALIAS value_1_1 :: 1 -> value_1 String : 0 + ALIAS value_1_3 :: 2 -> rhs.value_1 String : 1 + ALIAS id_2 :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 + Join (JOIN FillRightFirst) + Header: id_0 UInt64 + value_1_1 String + value_1_3 String + id_2 UInt64 + Kind: INNER + Strictness: ALL + Type: HashJoin + Clauses: [(id_0) = (id_2)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_0 UInt64 + value_1_1 String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + ALIAS id :: 0 -> id_0 UInt64 : 2 + ALIAS value_1 :: 1 -> value_1_1 String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value_1 String + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_2 UInt64 + value_1_3 String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + ALIAS id :: 0 -> id_2 UInt64 : 2 + ALIAS value_1 :: 1 -> value_1_3 String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value_1 String + ReadType: Default + Parts: 1 + Granules: 1 +-- +Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Header: id UInt64 + value_1 String + rhs.id UInt64 + rhs.value_1 String +Actions: INPUT : 0 -> id_0 UInt64 : 0 + INPUT : 1 -> value_1_1 String : 1 + INPUT :: 2 -> value_2_4 UInt64 : 2 + INPUT : 3 -> value_1_3 String : 3 + INPUT :: 4 -> value_2_5 UInt64 : 4 + INPUT : 5 -> id_2 UInt64 : 5 + ALIAS id_0 :: 0 -> id UInt64 : 6 + ALIAS value_1_1 :: 1 -> value_1 String : 0 + ALIAS value_1_3 :: 3 -> rhs.value_1 String : 1 + ALIAS id_2 :: 5 -> rhs.id UInt64 : 3 +Positions: 6 0 3 1 + Join (JOIN FillRightFirst) + Header: id_0 UInt64 + value_1_1 String + value_2_4 UInt64 + value_1_3 String + value_2_5 UInt64 + id_2 UInt64 + Kind: INNER + Strictness: ASOF + Type: HashJoin + ASOF inequality: LESS + Clauses: [(id_0, value_2_4) = (id_2, value_2_5)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_0 UInt64 + value_1_1 String + value_2_4 UInt64 + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 UInt64 : 2 + ALIAS id :: 0 -> id_0 UInt64 : 3 + ALIAS value_1 :: 1 -> value_1_1 String : 0 + ALIAS value_2 :: 2 -> value_2_4 UInt64 : 1 + Positions: 3 0 1 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value_1 String + value_2 UInt64 + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_2 UInt64 + value_1_3 String + value_2_5 UInt64 + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 UInt64 : 2 + ALIAS id :: 0 -> id_2 UInt64 : 3 + ALIAS value_1 :: 1 -> value_1_3 String : 0 + ALIAS value_2 :: 2 -> value_2_5 UInt64 : 1 + Positions: 3 0 1 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value_1 String + value_2 UInt64 + ReadType: Default + Parts: 1 + Granules: 1 diff --git a/tests/queries/0_stateless/02835_join_step_explain.sql b/tests/queries/0_stateless/02835_join_step_explain.sql new file mode 100644 index 00000000000..d0475fa14b6 --- /dev/null +++ b/tests/queries/0_stateless/02835_join_step_explain.sql @@ -0,0 +1,31 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS test_table_1; +CREATE TABLE test_table_1 +( + id UInt64, + value_1 String, + value_2 UInt64 +) ENGINE=MergeTree ORDER BY id; + +DROP TABLE IF EXISTS test_table_2; +CREATE TABLE test_table_2 +( + id UInt64, + value_1 String, + value_2 UInt64 +) ENGINE=MergeTree ORDER BY id; + +INSERT INTO test_table_1 VALUES (0, 'Value', 0); +INSERT INTO test_table_2 VALUES (0, 'Value', 0); + +EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 +FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 +FROM test_table_1 AS lhs ASOF JOIN test_table_2 AS rhs ON lhs.id = rhs.id AND lhs.value_2 < rhs.value_2; + +DROP TABLE test_table_1; +DROP TABLE test_table_2; From 49f76b8600371948cf77c8fd59d2ab1c12fcd251 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 3 Aug 2023 18:02:32 +0200 Subject: [PATCH 1555/2047] Fix flaky test --- .../0_stateless/01605_adaptive_granularity_block_borders.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index 4623c456475..f12a61055c4 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -1,4 +1,5 @@ --- Tags: no-random-merge-tree-settings +-- Tags: no-random-merge-tree-settings, no-tsan, no-debug +-- no-tsan: too slow SET use_uncompressed_cache = 0; SET allow_prefetched_read_pool_for_remote_filesystem=0; From ca67576720f7be4e120ee3ed5f3c4a35b3255b52 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 3 Aug 2023 18:06:00 +0200 Subject: [PATCH 1556/2047] Do not use inline cache in server images --- tests/ci/docker_server.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index a434d3cc841..89bd7b7755b 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -238,7 +238,7 @@ def build_and_push_image( result = [] # type: TestResults if os != "ubuntu": tag += f"-{os}" - init_args = ["docker", "buildx", "build", "--build-arg BUILDKIT_INLINE_CACHE=1"] + init_args = ["docker", "buildx", "build"] if push: init_args.append("--push") init_args.append("--output=type=image,push-by-digest=true") From 14a8ced90640519dea8843597435f376145aa88c Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 3 Aug 2023 12:29:31 -0400 Subject: [PATCH 1557/2047] Fix default port --- docs/en/operations/utilities/clickhouse-keeper-client.md | 6 +++--- programs/keeper-client/KeeperClient.cpp | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index 37eb0bb71ff..c9fea3597fd 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -11,7 +11,7 @@ A client application to interact with clickhouse-keeper by its native protocol. - `-q QUERY`, `--query=QUERY` — Query to execute. If this parameter is not passed, `clickhouse-keeper-client` will start in interactive mode. - `-h HOST`, `--host=HOST` — Server host. Default value: `localhost`. -- `-p N`, `--port=N` — Server port. Default value: 2181 +- `-p N`, `--port=N` — Server port. Default value: 9181 - `--connection-timeout=TIMEOUT` — Set connection timeout in seconds. Default value: 10s. - `--session-timeout=TIMEOUT` — Set session timeout in seconds. Default value: 10s. - `--operation-timeout=TIMEOUT` — Set operation timeout in seconds. Default value: 10s. @@ -21,8 +21,8 @@ A client application to interact with clickhouse-keeper by its native protocol. ## Example {#clickhouse-keeper-client-example} ```bash -./clickhouse-keeper-client -h localhost:2181 --connection-timeout 30 --session-timeout 30 --operation-timeout 30 -Connected to ZooKeeper at [::1]:2181 with session_id 137 +./clickhouse-keeper-client -h localhost:9181 --connection-timeout 30 --session-timeout 30 --operation-timeout 30 +Connected to ZooKeeper at [::1]:9181 with session_id 137 / :) ls keeper foo bar / :) cd keeper diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 561a1f41f7a..7ac9f61ebf9 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -131,7 +131,7 @@ void KeeperClient::defineOptions(Poco::Util::OptionSet & options) .binding("host")); options.addOption( - Poco::Util::Option("port", "p", "server port. default `2181`") + Poco::Util::Option("port", "p", "server port. default `9181`") .argument("") .binding("port")); @@ -303,7 +303,7 @@ int KeeperClient::main(const std::vector & /* args */) } auto host = config().getString("host", "localhost"); - auto port = config().getString("port", "2181"); + auto port = config().getString("port", "9181"); zk_args.hosts = {host + ":" + port}; zk_args.connection_timeout_ms = config().getInt("connection-timeout", 10) * 1000; zk_args.session_timeout_ms = config().getInt("session-timeout", 10) * 1000; From 388feb953d97b244b162c433e2983cbe4e2d73f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 19:28:59 +0200 Subject: [PATCH 1558/2047] Fix paths --- tests/ci/sqltest.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index d76baf22bb3..be22a1c9312 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,10 +110,10 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "server.log.zst": os.path.join(workspace_path, "/workspace/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), - "report.html": os.path.join(workspace_path, "/workspace/report.html"), - "test.log": os.path.join(workspace_path, "/workspace/test.log"), + "server.log.zst": os.path.join(workspace_path, "server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"), + "report.html": os.path.join(workspace_path, "report.html"), + "test.log": os.path.join(workspace_path, "test.log"), } s3_helper = S3Helper() From 8a7870f2504b3f3f1eed257d5aa3fc87e868a317 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 3 Aug 2023 20:24:29 +0200 Subject: [PATCH 1559/2047] Add a test to broken tests (Analyzer) --- tests/analyzer_integration_broken_tests.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 02f70c8a6df..1b0c44de1ed 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -201,3 +201,4 @@ test_backward_compatibility/test_data_skipping_indices.py::test_index test_backward_compatibility/test_convert_ordinary.py::test_convert_ordinary_to_atomic test_backward_compatibility/test_memory_bound_aggregation.py::test_backward_compatability test_odbc_interaction/test.py::test_postgres_insert +test_merge_tree_azure_blob_storage/test.py::test_table_manipulations From 921b1eecb52bb081fb90184207ce790b4b29b7c3 Mon Sep 17 00:00:00 2001 From: hendrik-m Date: Thu, 3 Aug 2023 18:43:18 +0200 Subject: [PATCH 1560/2047] revert changes of `clickhouse local` to `clickhouse-local` --- .../operations/utilities/clickhouse-local.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index 9c462a04a1d..0b17afb7e12 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -37,24 +37,24 @@ A common use of `clickhouse-local` is to run ad-hoc queries on files: where you If the file is sitting on the same machine as `clickhouse-local`, you can simple specify the file to load. The following `reviews.tsv` file contains a sampling of Amazon product reviews: ```bash -./clickhouse-local -q "SELECT * FROM 'reviews.tsv'" +./clickhouse local -q "SELECT * FROM 'reviews.tsv'" ``` This command is a shortcut of: ```bash -./clickhouse-local -q "SELECT * FROM file('reviews.tsv')" +./clickhouse local -q "SELECT * FROM file('reviews.tsv')" ``` ClickHouse knows the file uses a tab-separated format from filename extension. If you need to explicitly specify the format, simply add one of the [many ClickHouse input formats](../../interfaces/formats.md): ```bash -./clickhouse-local -q "SELECT * FROM file('reviews.tsv', 'TabSeparated')" +./clickhouse local -q "SELECT * FROM file('reviews.tsv', 'TabSeparated')" ``` The `file` table function creates a table, and you can use `DESCRIBE` to see the inferred schema: ```bash -./clickhouse-local -q "DESCRIBE file('reviews.tsv')" +./clickhouse local -q "DESCRIBE file('reviews.tsv')" ``` ```response @@ -78,7 +78,7 @@ review_date Nullable(Date) Let's find a product with the highest rating: ```bash -./clickhouse-local -q "SELECT +./clickhouse local -q "SELECT argMax(product_title,star_rating), max(star_rating) FROM file('reviews.tsv')" @@ -93,7 +93,7 @@ Monopoly Junior Board Game 5 If you have a file in S3, use `clickhouse-local` and the `s3` table function to query the file in place (without inserting the data into a ClickHouse table). We have a file named `house_0.parquet` in a public bucket that contains home prices of property sold in the United Kingdom. Let's see how many rows it has: ```bash -./clickhouse-local -q " +./clickhouse local -q " SELECT count() FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/house_parquet/house_0.parquet')" ``` @@ -107,7 +107,7 @@ The file has 2.7M rows: It's always useful to see what the inferred schema that ClickHouse determines from the file: ```bash -./clickhouse-local -q "DESCRIBE s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/house_parquet/house_0.parquet')" +./clickhouse local -q "DESCRIBE s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/house_parquet/house_0.parquet')" ``` ```response @@ -130,7 +130,7 @@ county Nullable(String) Let's see what the most expensive neighborhoods are: ```bash -./clickhouse-local -q " +./clickhouse local -q " SELECT town, district, @@ -177,7 +177,7 @@ $ clickhouse-local --structure "table_structure" --input-format "format_of_incom Basic usage (Mac): ``` bash -$ ./clickhouse-local --structure "table_structure" --input-format "format_of_incoming_data" --query "query" +$ ./clickhouse local --structure "table_structure" --input-format "format_of_incoming_data" --query "query" ``` :::note From 72e6b1d2d4a23b3529f04fb753f7a016a7318f02 Mon Sep 17 00:00:00 2001 From: hendrik-m Date: Thu, 3 Aug 2023 21:10:53 +0200 Subject: [PATCH 1561/2047] use tryReadQuotedStringInto --- src/Parsers/ExpressionElementParsers.cpp | 7 +------ .../02816_clickhouse_local_table_name_expressions.sh | 2 +- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 52f87e2e048..e99661e4c83 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -251,12 +251,7 @@ bool ParserTableAsStringLiteralIdentifier::parseImpl(Pos & pos, ASTPtr & node, E ReadBufferFromMemory in(pos->begin, pos->size()); String s; - try - { - readQuotedStringWithSQLStyle(s, in); - } - catch (const Exception &) - { + if (!tryReadQuotedStringInto(s, in)) { expected.add(pos, "string literal"); return false; } diff --git a/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.sh b/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.sh index 73e64d1e175..5cf3b1c88fd 100755 --- a/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.sh +++ b/tests/queries/0_stateless/02816_clickhouse_local_table_name_expressions.sh @@ -46,7 +46,7 @@ $CLICKHOUSE_LOCAL -q "SELECT * FROM '${dir}/tmp.csv'" ################# echo "Test 3a: check literal no parsing overflow" -$CLICKHOUSE_LOCAL -q "SELECT * FROM '${dir}/tmp.csv''bad'" 2>&1 | grep -c "UNKNOWN_TABLE" +$CLICKHOUSE_LOCAL -q "SELECT * FROM '${dir}/tmp.csv''bad'" 2>&1 | grep -c "SYNTAX_ERROR" ################# echo "Test 3b: check literal empty" From 6dc74fb5ccf0c6ef6538e60835667b57b6d4f060 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 3 Aug 2023 12:38:43 -0700 Subject: [PATCH 1562/2047] Implement big-endian support for transform --- .../Serializations/SerializationArray.cpp | 4 ++-- .../Serializations/SerializationNumber.cpp | 8 ++++---- src/Functions/transform.cpp | 14 ++++++++------ 3 files changed, 14 insertions(+), 12 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index cedcca870dd..e01c1aea0e9 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -129,7 +129,7 @@ namespace for (size_t i = offset; i < end; ++i) { ColumnArray::Offset current_offset = offset_values[i]; - writeIntBinary(current_offset - prev_offset, ostr); + writeBinaryLittleEndian(current_offset - prev_offset, ostr); prev_offset = current_offset; } } @@ -145,7 +145,7 @@ namespace while (i < initial_size + limit && !istr.eof()) { ColumnArray::Offset current_size = 0; - readIntBinary(current_size, istr); + readBinaryLittleEndian(current_size, istr); if (unlikely(current_size > MAX_ARRAY_SIZE)) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size is too large: {}", current_size); diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 8cabaec753d..0294a1c8a67 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -106,28 +106,28 @@ void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & { /// ColumnVector::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64 typename ColumnVector::ValueType x = static_cast::ValueType>(field.get()); - writeBinary(x, ostr); + writeBinaryLittleEndian(x, ostr); } template void SerializationNumber::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); field = NearestFieldType(x); } template void SerializationNumber::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeBinary(assert_cast &>(column).getData()[row_num], ostr); + writeBinaryLittleEndian(assert_cast &>(column).getData()[row_num], ostr); } template void SerializationNumber::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); assert_cast &>(column).getData().push_back(x); } diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index e03701327b1..16326dd5a44 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -764,9 +764,8 @@ namespace } /// Note: Doesn't check the duplicates in the `from` array. - - WhichDataType which(from_type); - if (isNativeNumber(which) || which.isDecimal32() || which.isDecimal64()) + /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 + if (WhichDataType which(from_type); isNativeNumber(which) || which.isDecimal32() || which.isDecimal64()) { cache.table_num_to_idx = std::make_unique(); auto & table = *cache.table_num_to_idx; @@ -774,10 +773,13 @@ namespace { if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i])) { - /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 - StringRef ref = cache.from_column->getDataAt(i); UInt64 key = 0; - memcpy(&key, ref.data, ref.size); + auto * dst = reinterpret_cast(&key); + const auto ref = cache.from_column->getDataAt(i); + if constexpr (std::endian::native == std::endian::big) + dst += sizeof(key) - ref.size; + + memcpy(dst, ref.data, ref.size); table[key] = i; } } From b53084ce43e1714ae89df29dca692ef53e184838 Mon Sep 17 00:00:00 2001 From: hendrik-m Date: Thu, 3 Aug 2023 22:07:05 +0200 Subject: [PATCH 1563/2047] fix style --- src/Parsers/ExpressionElementParsers.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index e99661e4c83..587908eb49c 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -251,7 +251,8 @@ bool ParserTableAsStringLiteralIdentifier::parseImpl(Pos & pos, ASTPtr & node, E ReadBufferFromMemory in(pos->begin, pos->size()); String s; - if (!tryReadQuotedStringInto(s, in)) { + if (!tryReadQuotedStringInto(s, in)) + { expected.add(pos, "string literal"); return false; } From 1e7daca85b84447a4a5577cb545d42fe31486df6 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 3 Aug 2023 22:06:33 +0000 Subject: [PATCH 1564/2047] Added new tests for session_log plus bugfix --- src/Interpreters/Session.cpp | 44 ++- src/Interpreters/Session.h | 2 + src/Interpreters/SessionLog.cpp | 14 +- src/Interpreters/SessionLog.h | 9 +- src/Server/HTTPHandler.cpp | 3 +- .../test.py | 5 +- tests/integration/test_session_log/.gitignore | 1 + .../integration/test_session_log/__init__.py | 0 .../test_session_log/configs/log.xml | 9 + .../test_session_log/configs/ports.xml | 9 + .../test_session_log/configs/session_log.xml | 9 + .../test_session_log/configs/users.xml | 23 ++ .../protos/clickhouse_grpc.proto | 1 + tests/integration/test_session_log/test.py | 289 ++++++++++++++++++ .../02833_concurrrent_sessions.reference | 46 +++ .../0_stateless/02833_concurrrent_sessions.sh | 138 +++++++++ .../02834_remote_session_log.reference | 13 + .../0_stateless/02834_remote_session_log.sh | 56 ++++ .../02835_drop_user_during_session.reference | 8 + .../02835_drop_user_during_session.sh | 112 +++++++ 20 files changed, 764 insertions(+), 27 deletions(-) create mode 100644 tests/integration/test_session_log/.gitignore create mode 100644 tests/integration/test_session_log/__init__.py create mode 100644 tests/integration/test_session_log/configs/log.xml create mode 100644 tests/integration/test_session_log/configs/ports.xml create mode 100644 tests/integration/test_session_log/configs/session_log.xml create mode 100644 tests/integration/test_session_log/configs/users.xml create mode 120000 tests/integration/test_session_log/protos/clickhouse_grpc.proto create mode 100644 tests/integration/test_session_log/test.py create mode 100644 tests/queries/0_stateless/02833_concurrrent_sessions.reference create mode 100755 tests/queries/0_stateless/02833_concurrrent_sessions.sh create mode 100644 tests/queries/0_stateless/02834_remote_session_log.reference create mode 100755 tests/queries/0_stateless/02834_remote_session_log.sh create mode 100644 tests/queries/0_stateless/02835_drop_user_during_session.reference create mode 100755 tests/queries/0_stateless/02835_drop_user_during_session.sh diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index de2a779b740..0a6435cff75 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -520,6 +520,8 @@ ContextMutablePtr Session::makeSessionContext() {}, session_context->getSettingsRef().max_sessions_for_user); + recordLoginSucess(session_context); + return session_context; } @@ -582,6 +584,8 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: { session_name_ }, max_sessions_for_user); + recordLoginSucess(session_context); + return session_context; } @@ -655,24 +659,38 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (user_id) user = query_context->getUser(); - if (!notified_session_log_about_login) - { - if (auto session_log = getSessionLog()) - { - session_log->addLoginSuccess( - auth_id, - named_session ? std::optional(named_session->key.second) : std::nullopt, - *query_context, - user); - - notified_session_log_about_login = true; - } - } + /// Interserver does not create session context + recordLoginSucess(query_context); return query_context; } +void Session::recordLoginSucess(ContextPtr login_context) const +{ + if (notified_session_log_about_login) + return; + + if (!login_context) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session or query context must be created"); + + if (auto session_log = getSessionLog()) + { + const auto & settings = login_context->getSettingsRef(); + const auto access = login_context->getAccess(); + + session_log->addLoginSuccess(auth_id, + named_session ? named_session->key.second : "", + settings, + access, + getClientInfo(), + user); + + notified_session_log_about_login = true; + } +} + + void Session::releaseSessionID() { if (!named_session) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 51c0e3c71fa..81ef987b428 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -97,6 +97,8 @@ public: private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; + void recordLoginSucess(ContextPtr login_context) const; + mutable bool notified_session_log_about_login = false; const UUID auth_id; diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 0a8a7fc18c5..b847eaf9824 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -199,12 +199,13 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(auth_failure_reason.data(), auth_failure_reason.length()); } -void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user) +void SessionLog::addLoginSuccess(const UUID & auth_id, + const String & session_id, + const Settings & settings, + const ContextAccessPtr & access, + const ClientInfo & client_info, + const UserPtr & login_user) { - const auto access = login_context.getAccess(); - const auto & settings = login_context.getSettingsRef(); - const auto & client_info = login_context.getClientInfo(); - DB::SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS); log_entry.client_info = client_info; @@ -215,8 +216,7 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional ses } log_entry.external_auth_server = login_user ? login_user->auth_data.getLDAPServerName() : ""; - if (session_id) - log_entry.session_id = *session_id; + log_entry.session_id = session_id; if (const auto roles_info = access->getRolesInfo()) log_entry.roles = roles_info->getCurrentRolesNames(); diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index 1282ac09c4d..8757bc12270 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -20,6 +20,7 @@ enum SessionLogElementType : int8_t class ContextAccess; struct User; using UserPtr = std::shared_ptr; +using ContextAccessPtr = std::shared_ptr; /** A struct which will be inserted as row into session_log table. * @@ -72,7 +73,13 @@ class SessionLog : public SystemLog using SystemLog::SystemLog; public: - void addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user); + void addLoginSuccess(const UUID & auth_id, + const String & session_id, + const Settings & settings, + const ContextAccessPtr & access, + const ClientInfo & client_info, + const UserPtr & login_user); + void addLoginFailure(const UUID & auth_id, const ClientInfo & info, const std::optional & user, const Exception & reason); void addLogOut(const UUID & auth_id, const UserPtr & login_user, const ClientInfo & client_info); }; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index ff5690a3b07..10d5e7a0242 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -561,8 +561,7 @@ void HTTPHandler::processQuery( session->makeSessionContext(); } - auto client_info = session->getClientInfo(); - auto context = session->makeQueryContext(std::move(client_info)); + auto context = session->makeQueryContext(); /// This parameter is used to tune the behavior of output formats (such as Native) for compatibility. if (params.has("client_protocol_version")) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 72addd79ec5..5b21aee1426 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -27,10 +27,7 @@ proto_dir = os.path.join(SCRIPT_DIR, "./protos") gen_dir = os.path.join(SCRIPT_DIR, "./_gen") os.makedirs(gen_dir, exist_ok=True) run_and_check( - "python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \ - {proto_dir}/clickhouse_grpc.proto".format( - proto_dir=proto_dir, gen_dir=gen_dir - ), + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", shell=True, ) diff --git a/tests/integration/test_session_log/.gitignore b/tests/integration/test_session_log/.gitignore new file mode 100644 index 00000000000..edf565ec632 --- /dev/null +++ b/tests/integration/test_session_log/.gitignore @@ -0,0 +1 @@ +_gen diff --git a/tests/integration/test_session_log/__init__.py b/tests/integration/test_session_log/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_session_log/configs/log.xml b/tests/integration/test_session_log/configs/log.xml new file mode 100644 index 00000000000..7a079b81e69 --- /dev/null +++ b/tests/integration/test_session_log/configs/log.xml @@ -0,0 +1,9 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/ports.xml b/tests/integration/test_session_log/configs/ports.xml new file mode 100644 index 00000000000..fbaefc16b3a --- /dev/null +++ b/tests/integration/test_session_log/configs/ports.xml @@ -0,0 +1,9 @@ + + 5433 + 9001 + 9100 + + + false + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml new file mode 100644 index 00000000000..a0e4e3e2216 --- /dev/null +++ b/tests/integration/test_session_log/configs/session_log.xml @@ -0,0 +1,9 @@ + + + system + session_log
+ + toYYYYMM(event_date) + 7500 +
+
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml new file mode 100644 index 00000000000..0416dfadc8a --- /dev/null +++ b/tests/integration/test_session_log/configs/users.xml @@ -0,0 +1,23 @@ + + + + 0 + + + + + + + pass + + + pass + + + pass + + + pass + + + \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto new file mode 120000 index 00000000000..25d15f11e3b --- /dev/null +++ b/tests/integration/test_session_log/protos/clickhouse_grpc.proto @@ -0,0 +1 @@ +../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py new file mode 100644 index 00000000000..2748f1be9b1 --- /dev/null +++ b/tests/integration/test_session_log/test.py @@ -0,0 +1,289 @@ +import os + +import grpc +import pymysql.connections +import psycopg2 as py_psql +import pytest +import random +import sys +import threading + +from helpers.cluster import ClickHouseCluster, run_and_check + +POSTGRES_SERVER_PORT = 5433 +MYSQL_SERVER_PORT = 9001 +GRPC_PORT = 9100 +SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_ENCODING = "utf-8" + +# Use grpcio-tools to generate *pb2.py files from *.proto. +proto_dir = os.path.join(SCRIPT_DIR, "./protos") +gen_dir = os.path.join(SCRIPT_DIR, "./_gen") +os.makedirs(gen_dir, exist_ok=True) +run_and_check( + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", + shell=True, +) + +sys.path.append(gen_dir) + +import clickhouse_grpc_pb2 +import clickhouse_grpc_pb2_grpc + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "node", + main_configs=[ + "configs/ports.xml", + "configs/log.xml", + "configs/session_log.xml", + ], + user_configs=["configs/users.xml"], + # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 + env_variables={ + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") + }, +) + + +def grpc_get_url(): + return f"{instance.ip_address}:{GRPC_PORT}" + + +def grpc_create_insecure_channel(): + channel = grpc.insecure_channel(grpc_get_url()) + grpc.channel_ready_future(channel).result(timeout=2) + return channel + + +session_id_counter = 0 + + +def next_session_id(): + global session_id_counter + session_id = session_id_counter + session_id_counter += 1 + return str(session_id) + + +def grpc_query(query, user_, pass_, raise_exception): + try: + query_info = clickhouse_grpc_pb2.QueryInfo( + query=query, + session_id=next_session_id(), + user_name=user_, + password=pass_, + ) + channel = grpc_create_insecure_channel() + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) + result = stub.ExecuteQuery(query_info) + if result and result.HasField("exception"): + raise Exception(result.exception.display_text) + + return result.output.decode(DEFAULT_ENCODING) + except Exception: + assert raise_exception + + +def postgres_query(query, user_, pass_, raise_exception): + try: + client = py_psql.connect( + host=instance.ip_address, + port=POSTGRES_SERVER_PORT, + user=user_, + password=pass_, + database="default", + ) + cursor = client.cursor() + cursor.execute(query) + cursor.fetchall() + except Exception: + assert raise_exception + + +def mysql_query(query, user_, pass_, raise_exception): + try: + client = pymysql.connections.Connection( + host=instance.ip_address, + user=user_, + password=pass_, + database="default", + port=MYSQL_SERVER_PORT, + ) + cursor = client.cursor(pymysql.cursors.DictCursor) + if raise_exception: + with pytest.raises(Exception): + cursor.execute(query) + else: + cursor.execute(query) + cursor.fetchall() + except Exception: + assert raise_exception + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_grpc_session(started_cluster): + grpc_query("SELECT 1", "grpc_user", "pass", False) + grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) + grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "grpc_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" + ) + assert logout_records == "grpc_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "grpc_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_mysql_session(started_cluster): + mysql_query("SELECT 1", "mysql_user", "pass", False) + mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) + mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "mysql_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" + ) + assert logout_records == "mysql_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "mysql_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_postgres_session(started_cluster): + postgres_query("SELECT 1", "postgres_user", "pass", False) + postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) + postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "postgres_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" + ) + assert logout_records == "postgres_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "postgres_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_parallel_sessions(started_cluster): + thread_list = [] + for _ in range(10): + # Sleep time does not dignificantly matters here, + # test should pass even without sleeping + for function in [postgres_query, grpc_query, mysql_query]: + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "pass", + False, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "wrong_pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "wrong_parallel_user", + "pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + + for thread in thread_list: + thread.join() + + instance.query("SYSTEM FLUSH LOGS") + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" + ) + assert port_0_sessions == "90\n" + + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" + ) + assert port_0_sessions == "0\n" + + address_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" + ) + assert address_0_sessions == "0\n" + + grpc_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" + ) + assert grpc_sessions == "30\n" + + mysql_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" + ) + assert mysql_sessions == "30\n" + + postgres_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" + ) + assert postgres_sessions == "30\n" + + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "30\n" + + logout_failure_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" + ) + assert logout_failure_sessions == "30\n" diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference new file mode 100644 index 00000000000..c77928f6c99 --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.reference @@ -0,0 +1,46 @@ +sessions: +360 +port_0_sessions: +0 +address_0_sessions: +0 +tcp_sessions +90 +http_sessions +90 +http_with_session_id_sessions +90 +my_sql_sessions +90 +Corresponding LoginSuccess/Logout +15 +LoginFailure +15 +Corresponding LoginSuccess/Logout +15 +LoginFailure +15 +Corresponding LoginSuccess/Logout +15 +LoginFailure +15 +Corresponding LoginSuccess/Logout +15 +LoginFailure +15 +Corresponding LoginSuccess/Logout +15 +LoginFailure +15 +Corresponding LoginSuccess/Logout +15 +LoginFailure +15 +Corresponding LoginSuccess/Logout +15 +LoginFailure +15 +Corresponding LoginSuccess/Logout +15 +LoginFailure +15 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh new file mode 100755 index 00000000000..40baeae87a4 --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -0,0 +1,138 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +readonly TCP_USERS=( "TCP_USER_${PID}"_{1,2} ) +readonly HTTP_USERS=( "HTTP_USER_${PID}"_{1,2} ) +readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "HTTP_WITH_SESSION_ID_USER_${PID}"_{1,2} ) +readonly MYSQL_USERS=( "MYSQL_USER_${PID}"_{1,2} ) +readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) + +readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" + +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; +done + +# All _session functions execute in separate threads +# These functions tries to create a session with succesful login and logout, +# sleep small random amount of time to make concurency more intesive +# and try to login with invalid password. +# Test actually not timing dependent, it should work even without sleep at all. +function tcp_session() +{ + local user=$1 + local i=0 + while (( (i++) < 15 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep 0.03${RANDOM}" --user="${user}" --password="pass" + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' + done +} + +function http_session() +{ + local user=$1 + local i=0 + while (( (i++) < 15 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep 0.03${RANDOM}" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" + done +} + +function http_with_session_id_session() +{ + local user=$1 + local i=0 + while (( (i++) < 15 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.03${RANDOM}" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" + done +} + +function mysql_session() +{ + local user=$1 + local i=0 + while (( (i++) < 15 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.03${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" + + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; +export -f mysql_session; + +for user in "${TCP_USERS[@]}"; do + timeout 30s bash -c "tcp_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_USERS[@]}"; do + timeout 30s bash -c "http_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do + timeout 30s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & +done + +for user in "${MYSQL_USERS[@]}"; do + timeout 30s bash -c "mysql_session ${user}" >/dev/null 2>&1 & +done + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" + +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" + +echo "tcp_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" +echo "http_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "http_with_session_id_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "my_sql_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" + echo "Corresponding LoginSuccess/Logout" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" + echo "LoginFailure" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" + done diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference new file mode 100644 index 00000000000..e2680982ab0 --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.reference @@ -0,0 +1,13 @@ +0 +0 +0 +0 +client_port 0 connections: +0 +client_address '::' connections: +0 +login failures: +0 +TCP Login and logout count is equal +HTTP Login and logout count is equal +MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh new file mode 100755 index 00000000000..37b45ee8da3 --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ +readonly TEST_USER=$"02834_USER_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "client_port 0 connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" + +echo "client_address '::' connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" + +echo "login failures:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" + +# remote(...) function sometimes reuse old cached sessions for query execution. +# This makes LoginSuccess/Logout entries count unsable but success and logouts must always match. + +for interface in 'TCP' 'HTTP' 'MySQL' +do + LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` + CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` + + if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then + echo "${interface} Login and logout count is equal" + else + TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` + echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" + fi +done + +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference new file mode 100644 index 00000000000..7252faab8c6 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.reference @@ -0,0 +1,8 @@ +port_0_sessions: +0 +address_0_sessions: +0 +Corresponding LoginSuccess/Logout +9 +LoginFailure +0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh new file mode 100755 index 00000000000..f6ee4c30473 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -0,0 +1,112 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +readonly TEST_USER="02835_USER_${PID}" +readonly TEST_ROLE="02835_ROLE_${PID}" +readonly TEST_PROFILE="02835_PROFILE_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +function tcp_session() +{ + local user=$1 + ${CLICKHOUSE_CLIENT} -q "SELECT * FROM system.numbers" --user="${user}" +} + +function http_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT * FROM system.numbers" +} + +function http_with_session_id_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT * FROM system.numbers" +} + +# Busy-waits until user $1, specified amount of queries ($2) will run simultaneosly. +function wait_for_queries_start() +{ + local user=$1 + local queries_count=$2 + # 5 seconds waiting + counter=0 retries=50 + I=0 + while [[ $counter -lt $retries ]]; do + I=$((I + 1)) + result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") + if [[ $result == "${queries_count}" ]]; then + break; + fi + sleep 0.1 + ((++counter)) + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +# DROP USE CASE +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; + +timeout 5s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 5s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 5s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP ROLE CASE +${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" + +timeout 5s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 5s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 5s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP PROFILE CASE +${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" + +timeout 5s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 5s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 5s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" +echo "Corresponding LoginSuccess/Logout" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" +echo "LoginFailure" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From 24cdaf305d21e386bd8c3534e09833c14d790092 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 4 Aug 2023 06:09:29 +0000 Subject: [PATCH 1565/2047] Reduced test flakiness --- tests/integration/test_session_log/test.py | 4 +- .../02833_concurrrent_sessions.reference | 42 +++++++------------ .../0_stateless/02833_concurrrent_sessions.sh | 39 +++++++++-------- .../0_stateless/02834_remote_session_log.sh | 4 +- .../02835_drop_user_during_session.sh | 35 ++++++++-------- 5 files changed, 58 insertions(+), 66 deletions(-) diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py index 2748f1be9b1..dbb39993ce3 100644 --- a/tests/integration/test_session_log/test.py +++ b/tests/integration/test_session_log/test.py @@ -207,8 +207,8 @@ def test_postgres_session(started_cluster): def test_parallel_sessions(started_cluster): thread_list = [] for _ in range(10): - # Sleep time does not dignificantly matters here, - # test should pass even without sleeping + # Sleep time does not significantly matter here, + # test should pass even without sleeping. for function in [postgres_query, grpc_query, mysql_query]: thread = threading.Thread( target=function, diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference index c77928f6c99..bfe507e8eac 100644 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.reference +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.reference @@ -1,46 +1,34 @@ sessions: -360 +150 port_0_sessions: 0 address_0_sessions: 0 tcp_sessions -90 +60 http_sessions -90 +30 http_with_session_id_sessions -90 +30 my_sql_sessions -90 +30 Corresponding LoginSuccess/Logout -15 +10 LoginFailure -15 +10 Corresponding LoginSuccess/Logout -15 +10 LoginFailure -15 +10 Corresponding LoginSuccess/Logout -15 +10 LoginFailure -15 +10 Corresponding LoginSuccess/Logout -15 +10 LoginFailure -15 +10 Corresponding LoginSuccess/Logout -15 +10 LoginFailure -15 -Corresponding LoginSuccess/Logout -15 -LoginFailure -15 -Corresponding LoginSuccess/Logout -15 -LoginFailure -15 -Corresponding LoginSuccess/Logout -15 -LoginFailure -15 +10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh index 40baeae87a4..fd8731cd1af 100755 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel +# If tests run in parallel, results can become flaky. +# Because each test starts many processes and waits for the query to run. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -7,10 +9,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) readonly PID=$$ -readonly TCP_USERS=( "TCP_USER_${PID}"_{1,2} ) -readonly HTTP_USERS=( "HTTP_USER_${PID}"_{1,2} ) -readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "HTTP_WITH_SESSION_ID_USER_${PID}"_{1,2} ) -readonly MYSQL_USERS=( "MYSQL_USER_${PID}"_{1,2} ) +# Each user uses a separate thread. +readonly TCP_USERS=( "TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users +readonly HTTP_USERS=( "HTTP_USER_${PID}" ) +readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "HTTP_WITH_SESSION_ID_USER_${PID}" ) +readonly MYSQL_USERS=( "MYSQL_USER_${PID}") readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" @@ -27,18 +30,18 @@ for user in "${ALL_USERS[@]}"; do ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; done -# All _session functions execute in separate threads -# These functions tries to create a session with succesful login and logout, -# sleep small random amount of time to make concurency more intesive -# and try to login with invalid password. -# Test actually not timing dependent, it should work even without sleep at all. +# All _session functions execute in separate threads. +# These functions try to create a session with successful login and logout. +# Sleep a small, random amount of time to make concurrency more intense. +# and try to login with an invalid password. +# test is actually not timing dependent. it should work even without sleep at all. function tcp_session() { local user=$1 local i=0 - while (( (i++) < 15 )); do + while (( (i++) < 10 )); do # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep 0.03${RANDOM}" --user="${user}" --password="pass" + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.02${RANDOM})" --user="${user}" --password="pass" # login failure ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' done @@ -48,9 +51,9 @@ function http_session() { local user=$1 local i=0 - while (( (i++) < 15 )); do + while (( (i++) < 10 )); do # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep 0.03${RANDOM}" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.02${RANDOM})" # login failure ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" @@ -61,9 +64,9 @@ function http_with_session_id_session() { local user=$1 local i=0 - while (( (i++) < 15 )); do + while (( (i++) < 10 )); do # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.03${RANDOM}" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.02${RANDOM}" # login failure ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" @@ -74,9 +77,9 @@ function mysql_session() { local user=$1 local i=0 - while (( (i++) < 15 )); do + while (( (i++) < 10 )); do # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.03${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.02${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" # login failure ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh index 37b45ee8da3..64f20a70cb1 100755 --- a/tests/queries/0_stateless/02834_remote_session_log.sh +++ b/tests/queries/0_stateless/02834_remote_session_log.sh @@ -37,8 +37,8 @@ ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '$ echo "login failures:" ${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" -# remote(...) function sometimes reuse old cached sessions for query execution. -# This makes LoginSuccess/Logout entries count unsable but success and logouts must always match. +# remote(...) function sometimes reuses old cached sessions for query execution. +# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. for interface in 'TCP' 'HTTP' 'MySQL' do diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh index f6ee4c30473..9bf687fd60c 100755 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -1,4 +1,7 @@ #!/usr/bin/env bash +# Tags: no-parallel +# If tests run in parallel, results can become flaky. +# Because each test starts many processes and waits for the query to run. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -14,31 +17,29 @@ readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, function tcp_session() { local user=$1 - ${CLICKHOUSE_CLIENT} -q "SELECT * FROM system.numbers" --user="${user}" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" } function http_session() { local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT * FROM system.numbers" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" } function http_with_session_id_session() { local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT * FROM system.numbers" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" } -# Busy-waits until user $1, specified amount of queries ($2) will run simultaneosly. +# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. function wait_for_queries_start() { local user=$1 local queries_count=$2 - # 5 seconds waiting - counter=0 retries=50 - I=0 + # 10 seconds waiting + counter=0 retries=100 while [[ $counter -lt $retries ]]; do - I=$((I + 1)) result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") if [[ $result == "${queries_count}" ]]; then break; @@ -58,9 +59,9 @@ export -f tcp_session; export -f http_session; export -f http_with_session_id_session; -timeout 5s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 5s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 5s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & wait_for_queries_start $TEST_USER 3 ${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" @@ -72,9 +73,9 @@ wait ${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" ${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" -timeout 5s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 5s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 5s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & wait_for_queries_start $TEST_USER 3 ${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" @@ -88,9 +89,9 @@ wait ${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" ${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" -timeout 5s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 5s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 5s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & wait_for_queries_start $TEST_USER 3 ${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" From be067e0ddf1752ae4d68de1742384b01a0f02859 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 4 Aug 2023 06:25:31 +0000 Subject: [PATCH 1566/2047] removed no-parallel --- tests/queries/0_stateless/02835_drop_user_during_session.sh | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh index 9bf687fd60c..cd31caf870e 100755 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -1,7 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel -# If tests run in parallel, results can become flaky. -# Because each test starts many processes and waits for the query to run. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From eda396eb7c321f7f93a91b4b504586c9e13d778a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 4 Aug 2023 09:20:01 +0200 Subject: [PATCH 1567/2047] Fix completion for clickhouse-keeper-client Previously some symbols had been counted as word breakers, and this breaks the completion since after such symbol the completion will start from scratch. One of the most annoying was the "-" sign. But actually all those word breakers that is used for SQL completion is not required for keeper client, so let's leave only necessary. Signed-off-by: Azat Khuzhin --- programs/keeper-client/KeeperClient.cpp | 10 +++++++++- src/Client/ClientBase.cpp | 17 +++++++++++++++-- src/Client/LineReader.cpp | 7 +++++-- src/Client/LineReader.h | 3 +-- src/Client/ReplxxLineReader.cpp | 6 ++++-- src/Client/ReplxxLineReader.h | 3 +++ 6 files changed, 37 insertions(+), 9 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index f41dca1e27a..b483c1a746c 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -266,8 +266,16 @@ void KeeperClient::runInteractive() LineReader::Patterns query_extenders = {"\\"}; LineReader::Patterns query_delimiters = {}; + char word_break_characters[] = " \t\v\f\a\b\r\n/"; - ReplxxLineReader lr(suggest, history_file, false, query_extenders, query_delimiters, {}); + ReplxxLineReader lr( + suggest, + history_file, + /* multiline= */ false, + query_extenders, + query_delimiters, + word_break_characters, + /* highlighter_= */ {}); lr.enableBracketedPaste(); while (true) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 06dabf96c28..15204288c56 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2313,15 +2313,28 @@ void ClientBase::runInteractive() LineReader::Patterns query_extenders = {"\\"}; LineReader::Patterns query_delimiters = {";", "\\G", "\\G;"}; + char word_break_characters[] = " \t\v\f\a\b\r\n`~!@#$%^&*()-=+[{]}\\|;:'\",<.>/?"; #if USE_REPLXX replxx::Replxx::highlighter_callback_t highlight_callback{}; if (config().getBool("highlight", true)) highlight_callback = highlight; - ReplxxLineReader lr(*suggest, history_file, config().has("multiline"), query_extenders, query_delimiters, highlight_callback); + ReplxxLineReader lr( + *suggest, + history_file, + config().has("multiline"), + query_extenders, + query_delimiters, + word_break_characters, + highlight_callback); #else - LineReader lr(history_file, config().has("multiline"), query_extenders, query_delimiters); + LineReader lr( + history_file, + config().has("multiline"), + query_extenders, + query_delimiters, + word_break_characters); #endif static const std::initializer_list> backslash_aliases = diff --git a/src/Client/LineReader.cpp b/src/Client/LineReader.cpp index 82dbe03e5d3..77b4185ec3b 100644 --- a/src/Client/LineReader.cpp +++ b/src/Client/LineReader.cpp @@ -66,7 +66,7 @@ void addNewWords(Words & to, const Words & from, Compare comp) namespace DB { -replxx::Replxx::completions_t LineReader::Suggest::getCompletions(const String & prefix, size_t prefix_length) +replxx::Replxx::completions_t LineReader::Suggest::getCompletions(const String & prefix, size_t prefix_length, const char * word_break_characters) { std::string_view last_word; @@ -135,7 +135,10 @@ void LineReader::Suggest::addWords(Words && new_words) } LineReader::LineReader(const String & history_file_path_, bool multiline_, Patterns extenders_, Patterns delimiters_) - : history_file_path(history_file_path_), multiline(multiline_), extenders(std::move(extenders_)), delimiters(std::move(delimiters_)) + : history_file_path(history_file_path_) + , multiline(multiline_) + , extenders(std::move(extenders_)) + , delimiters(std::move(delimiters_)) { /// FIXME: check extender != delimiter } diff --git a/src/Client/LineReader.h b/src/Client/LineReader.h index ff297dd03b4..fc19eaa5667 100644 --- a/src/Client/LineReader.h +++ b/src/Client/LineReader.h @@ -21,7 +21,7 @@ public: using Callback = std::function; /// Get vector for the matched range of words if any. - replxx::Replxx::completions_t getCompletions(const String & prefix, size_t prefix_length); + replxx::Replxx::completions_t getCompletions(const String & prefix, size_t prefix_length, const char * word_break_characters); void addWords(Words && new_words); void setCompletionsCallback(Callback && callback) { custom_completions_callback = callback; } @@ -65,7 +65,6 @@ protected: }; const String history_file_path; - static constexpr char word_break_characters[] = " \t\v\f\a\b\r\n`~!@#$%^&*()-=+[{]}\\|;:'\",<.>/?"; String input; diff --git a/src/Client/ReplxxLineReader.cpp b/src/Client/ReplxxLineReader.cpp index e691105ecba..49f44e3d0f9 100644 --- a/src/Client/ReplxxLineReader.cpp +++ b/src/Client/ReplxxLineReader.cpp @@ -287,8 +287,10 @@ ReplxxLineReader::ReplxxLineReader( bool multiline_, Patterns extenders_, Patterns delimiters_, + const char word_break_characters_[], replxx::Replxx::highlighter_callback_t highlighter_) : LineReader(history_file_path_, multiline_, std::move(extenders_), std::move(delimiters_)), highlighter(std::move(highlighter_)) + , word_break_characters(word_break_characters_) , editor(getEditor()) { using namespace std::placeholders; @@ -326,9 +328,9 @@ ReplxxLineReader::ReplxxLineReader( rx.install_window_change_handler(); - auto callback = [&suggest] (const String & context, size_t context_size) + auto callback = [&suggest, this] (const String & context, size_t context_size) { - return suggest.getCompletions(context, context_size); + return suggest.getCompletions(context, context_size, word_break_characters); }; rx.set_completion_callback(callback); diff --git a/src/Client/ReplxxLineReader.h b/src/Client/ReplxxLineReader.h index 5cb8e48eb86..9cc2853f77a 100644 --- a/src/Client/ReplxxLineReader.h +++ b/src/Client/ReplxxLineReader.h @@ -15,6 +15,7 @@ public: bool multiline, Patterns extenders_, Patterns delimiters_, + const char word_break_characters_[], replxx::Replxx::highlighter_callback_t highlighter_); ~ReplxxLineReader() override; @@ -33,6 +34,8 @@ private: replxx::Replxx rx; replxx::Replxx::highlighter_callback_t highlighter; + const char * word_break_characters; + // used to call flock() to synchronize multiple clients using same history file int history_file_fd = -1; bool bracketed_paste_enabled = false; From 94fcae58b91b145a5a09322be24421c1e4f2ea39 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 4 Aug 2023 09:49:51 +0200 Subject: [PATCH 1568/2047] clickhouse-keeper-client: fix version parsing for set command Previously: / :) set /test foo 1 DB::Exception: Bad get: has UInt64, requested Int64 Signed-off-by: Azat Khuzhin --- programs/keeper-client/Commands.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 05928a0d20b..3afd49e9855 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -77,7 +77,7 @@ void SetCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co client->zookeeper->set( client->getAbsolutePath(query->args[0].safeGet()), query->args[1].safeGet(), - static_cast(query->args[2].safeGet())); + static_cast(query->args[2].get())); } bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const From 8e7eb7f3fa739ef4d02594ca25fccb815a706a73 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Aug 2023 08:59:55 +0000 Subject: [PATCH 1569/2047] Fix tests --- src/IO/Archives/LibArchiveReader.cpp | 169 ++++++++++-------- src/IO/Archives/LibArchiveReader.h | 25 ++- src/TableFunctions/ITableFunctionFileLike.cpp | 3 +- src/TableFunctions/TableFunctionFile.cpp | 1 + 4 files changed, 110 insertions(+), 88 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index d819547c9bd..1686b12f37a 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -1,9 +1,11 @@ #include #include #include +#include #include +#include namespace DB { @@ -18,12 +20,11 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } - -template -class LibArchiveReader::Handle +class LibArchiveReader::Handle { public: - explicit Handle(const String & path_to_archive_) : path_to_archive(path_to_archive_) + explicit Handle(std::string path_to_archive_, bool lock_on_reading_) + : path_to_archive(path_to_archive_), lock_on_reading(lock_on_reading_) { current_archive = open(path_to_archive); current_entry = archive_entry_new(); @@ -40,11 +41,7 @@ public: ~Handle() { - if (current_archive) - { - archive_read_close(current_archive); - archive_read_free(current_archive); - } + close(current_archive); } bool locateFile(const std::string & filename) @@ -58,7 +55,7 @@ public: int err = ARCHIVE_OK; while (true) { - err = archive_read_next_header(current_archive, ¤t_entry); + err = readNextHeader(current_archive, ¤t_entry); if (err == ARCHIVE_RETRY) continue; @@ -80,59 +77,37 @@ public: int err = ARCHIVE_OK; do { - err = archive_read_next_header(current_archive, ¤t_entry); + err = readNextHeader(current_archive, ¤t_entry); } while (err == ARCHIVE_RETRY); checkError(err); return err == ARCHIVE_OK; } - static struct archive * open(const String & path_to_archive) - { - auto * archive = archive_read_new(); - archive_read_support_filter_all(archive); - archive_read_support_format_all(archive); - if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) - throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open {} archive: {}", ArchiveInfo::name, quoteString(path_to_archive)); - - return archive; - } - std::vector getAllFiles(NameFilter filter) { auto * archive = open(path_to_archive); + SCOPE_EXIT( + close(archive); + ); + auto * entry = archive_entry_new(); std::vector files; - int error = archive_read_next_header(archive, &entry); + int error = readNextHeader(archive, &entry); while (error == ARCHIVE_OK || error == ARCHIVE_RETRY) { std::string name = archive_entry_pathname(entry); if (!filter || filter(name)) files.push_back(std::move(name)); - error = archive_read_next_header(archive, &entry); + error = readNextHeader(archive, &entry); } - archive_read_close(archive); - archive_read_free(archive); - checkError(error); return files; } - void checkError(int error) - { - if (error == ARCHIVE_FATAL) - throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Failed to read archive while fetching all files: {}", archive_error_string(current_archive)); - } - - void resetFileInfo() - { - file_name.reset(); - file_info.reset(); - } - const String & getFileName() const { if (!file_name) @@ -157,13 +132,67 @@ public: struct archive * current_archive; struct archive_entry * current_entry; private: + void checkError(int error) const + { + if (error == ARCHIVE_FATAL) + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Failed to read archive while fetching all files: {}", archive_error_string(current_archive)); + } + + void resetFileInfo() + { + file_name.reset(); + file_info.reset(); + } + + static struct archive * open(const String & path_to_archive) + { + auto * archive = archive_read_new(); + try + { + archive_read_support_filter_all(archive); + archive_read_support_format_all(archive); + if (archive_read_open_filename(archive, path_to_archive.c_str(), 10240) != ARCHIVE_OK) + throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't open archive: {}", quoteString(path_to_archive)); + } + catch (...) + { + close(archive); + throw; + } + + return archive; + } + + static void close(struct archive * archive) + { + if (archive) + { + archive_read_close(archive); + archive_read_free(archive); + } + } + + int readNextHeader(struct archive * archive, struct archive_entry ** entry) const + { + std::unique_lock lock(Handle::read_lock, std::defer_lock); + if (lock_on_reading) + lock.lock(); + + return archive_read_next_header(archive, entry); + } + const String path_to_archive; + + /// for some archive types when we are reading headers static variables are used + /// which are not thread-safe + const bool lock_on_reading = false; + static inline std::mutex read_lock; + mutable std::optional file_name; mutable std::optional file_info; }; -template -class LibArchiveReader::FileEnumeratorImpl : public FileEnumerator +class LibArchiveReader::FileEnumeratorImpl : public FileEnumerator { public: explicit FileEnumeratorImpl(Handle handle_) : handle(std::move(handle_)) {} @@ -178,8 +207,7 @@ private: Handle handle; }; -template -class LibArchiveReader::ReadBufferFromLibArchive : public ReadBufferFromFileBase +class LibArchiveReader::ReadBufferFromLibArchive : public ReadBufferFromFileBase { public: explicit ReadBufferFromLibArchive(Handle handle_, std::string path_to_archive_) @@ -228,63 +256,55 @@ private: size_t total_bytes_read = 0; }; -template -LibArchiveReader::LibArchiveReader(const String & path_to_archive_) : path_to_archive(path_to_archive_) +LibArchiveReader::LibArchiveReader(std::string archive_name_, bool lock_on_reading_, std::string path_to_archive_) + : archive_name(std::move(archive_name_)), lock_on_reading(lock_on_reading_), path_to_archive(std::move(path_to_archive_)) {} -template -LibArchiveReader::~LibArchiveReader() = default; +LibArchiveReader::~LibArchiveReader() = default; -template -const std::string & LibArchiveReader::getPath() const +const std::string & LibArchiveReader::getPath() const { return path_to_archive; } -template -bool LibArchiveReader::fileExists(const String & filename) +bool LibArchiveReader::fileExists(const String & filename) { - Handle handle(path_to_archive); + Handle handle(path_to_archive, lock_on_reading); return handle.locateFile(filename); } -template -LibArchiveReader::FileInfo LibArchiveReader::getFileInfo(const String & filename) +LibArchiveReader::FileInfo LibArchiveReader::getFileInfo(const String & filename) { - Handle handle(path_to_archive); + Handle handle(path_to_archive, lock_on_reading); if (!handle.locateFile(filename)) throw Exception(ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't unpack archive {}: file not found", path_to_archive); return handle.getFileInfo(); } -template -std::unique_ptr::FileEnumerator> LibArchiveReader::firstFile() +std::unique_ptr LibArchiveReader::firstFile() { - Handle handle(path_to_archive); + Handle handle(path_to_archive, lock_on_reading); if (!handle.nextFile()) return nullptr; return std::make_unique(std::move(handle)); } -template -std::unique_ptr LibArchiveReader::readFile(const String & filename) +std::unique_ptr LibArchiveReader::readFile(const String & filename) { return readFile([&](const std::string & file) { return file == filename; }); } -template -std::unique_ptr LibArchiveReader::readFile(NameFilter filter) +std::unique_ptr LibArchiveReader::readFile(NameFilter filter) { - Handle handle(path_to_archive); + Handle handle(path_to_archive, lock_on_reading); if (!handle.locateFile(filter)) throw Exception( ErrorCodes::CANNOT_UNPACK_ARCHIVE, "Couldn't unpack archive {}: no file found satisfying the filter", path_to_archive); return std::make_unique(std::move(handle), path_to_archive); } -template -std::unique_ptr LibArchiveReader::readFile(std::unique_ptr enumerator) +std::unique_ptr LibArchiveReader::readFile(std::unique_ptr enumerator) { if (!dynamic_cast(enumerator.get())) throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong enumerator passed to readFile()"); @@ -293,8 +313,7 @@ std::unique_ptr LibArchiveReader::readFile( return std::make_unique(std::move(handle), path_to_archive); } -template std::unique_ptr::FileEnumerator> -LibArchiveReader::nextFile(std::unique_ptr read_buffer) +std::unique_ptr LibArchiveReader::nextFile(std::unique_ptr read_buffer) { if (!dynamic_cast(read_buffer.get())) throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong ReadBuffer passed to nextFile()"); @@ -305,28 +324,22 @@ LibArchiveReader::nextFile(std::unique_ptr read_buffer) return std::make_unique(std::move(handle)); } -template -std::vector LibArchiveReader::getAllFiles() +std::vector LibArchiveReader::getAllFiles() { return getAllFiles({}); } -template -std::vector LibArchiveReader::getAllFiles(NameFilter filter) +std::vector LibArchiveReader::getAllFiles(NameFilter filter) { - Handle handle(path_to_archive); + Handle handle(path_to_archive, lock_on_reading); return handle.getAllFiles(filter); } -template -void LibArchiveReader::setPassword(const String & /*password_*/) +void LibArchiveReader::setPassword(const String & /*password_*/) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to {} archive", ArchiveInfo::name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not set password to {} archive", archive_name); } -template class LibArchiveReader; -template class LibArchiveReader; - #endif } diff --git a/src/IO/Archives/LibArchiveReader.h b/src/IO/Archives/LibArchiveReader.h index 86127fa6953..700e8f70d04 100644 --- a/src/IO/Archives/LibArchiveReader.h +++ b/src/IO/Archives/LibArchiveReader.h @@ -16,13 +16,9 @@ class ReadBufferFromFileBase; class SeekableReadBuffer; /// Implementation of IArchiveReader for reading archives using libarchive. -template class LibArchiveReader : public IArchiveReader { public: - /// Constructs an archive's reader that will read from a file in the local filesystem. - explicit LibArchiveReader(const String & path_to_archive_); - ~LibArchiveReader() override; const std::string & getPath() const override; @@ -52,18 +48,31 @@ public: /// Sets password used to decrypt the contents of the files in the archive. void setPassword(const String & password_) override; +protected: + /// Constructs an archive's reader that will read from a file in the local filesystem. + LibArchiveReader(std::string archive_name_, bool lock_on_reading_, std::string path_to_archive_); + private: class ReadBufferFromLibArchive; class Handle; class FileEnumeratorImpl; + const std::string archive_name; + const bool lock_on_reading; const String path_to_archive; }; -struct TarArchiveInfo { static constexpr std::string_view name = "tar"; }; -using TarArchiveReader = LibArchiveReader; -struct SevenZipArchiveInfo { static constexpr std::string_view name = "7z"; }; -using SevenZipArchiveReader = LibArchiveReader; +class TarArchiveReader : public LibArchiveReader +{ +public: + explicit TarArchiveReader(std::string path_to_archive) : LibArchiveReader("tar", /*lock_on_reading_=*/ true, std::move(path_to_archive)) { } +}; + +class SevenZipArchiveReader : public LibArchiveReader +{ +public: + explicit SevenZipArchiveReader(std::string path_to_archive) : LibArchiveReader("7z", /*lock_on_reading_=*/ false, std::move(path_to_archive)) { } +}; #endif diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index d99d0856da6..487826dc363 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -24,8 +24,7 @@ namespace ErrorCodes void ITableFunctionFileLike::parseFirstArguments(const ASTPtr & arg, const ContextPtr &) { - String path = checkAndGetLiteralArgument(arg, "source"); - StorageFile::parseFileSource(std::move(path), filename, path_to_archive); + filename = checkAndGetLiteralArgument(arg, "source"); } String ITableFunctionFileLike::getFormatFromFirstArgument() diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index 4b0e71ba60c..56a6839ddbb 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -25,6 +25,7 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr if (context->getApplicationType() != Context::ApplicationType::LOCAL) { ITableFunctionFileLike::parseFirstArguments(arg, context); + StorageFile::parseFileSource(std::move(filename), filename, path_to_archive); return; } From 58bea8d0675b07b5144d67d846e4ca9163a4dc59 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Elena=20Torr=C3=B3?= Date: Fri, 4 Aug 2023 11:23:41 +0200 Subject: [PATCH 1570/2047] Format 's3_upload_part_size_multiply_parts_count_threshold' info --- docs/en/engines/table-engines/integrations/s3.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 051945538b2..c1752ea488c 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -237,7 +237,7 @@ The following settings can be set before query execution or placed into configur - `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited). - `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. - `s3_upload_part_size_multiply_factor` - Multiply `s3_min_upload_part_size` by this factor each time `s3_multiply_parts_count_threshold` parts were uploaded from a single write to S3. Default values is `2`. -- `s3_upload_part_size_multiply_parts_count_threshold` - Each time this number of parts was uploaded to S3 `s3_min_upload_part_size multiplied` by `s3_upload_part_size_multiply_factor`. Default value us `500`. +- `s3_upload_part_size_multiply_parts_count_threshold` - Each time this number of parts was uploaded to S3, `s3_min_upload_part_size` is multiplied by `s3_upload_part_size_multiply_factor`. Default value is `500`. - `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurrently for one object. Its number should be limited. The value `0` means unlimited. Default value is `20`. Each in-flight part has a buffer with size `s3_min_upload_part_size` for the first `s3_upload_part_size_multiply_factor` parts and more when file is big enough, see `upload_part_size_multiply_factor`. With default settings one uploaded file consumes not more than `320Mb` for a file which is less than `8G`. The consumption is greater for a larger file. Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration. From 9279d1e33133d83e6d62fbc4ea167db1703d2fe9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Elena=20Torr=C3=B3?= Date: Fri, 4 Aug 2023 11:26:13 +0200 Subject: [PATCH 1571/2047] Update setting's description --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 13fde626f16..ae1b063c3c3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -78,7 +78,7 @@ class IColumn; M(UInt64, s3_min_upload_part_size, 16*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_upload_part_size_multiply_factor, 2, "Multiply s3_min_upload_part_size by this factor each time s3_multiply_parts_count_threshold parts were uploaded from a single write to S3.", 0) \ - M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3 s3_min_upload_part_size multiplied by s3_upload_part_size_multiply_factor.", 0) \ + M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3, s3_min_upload_part_size is multiplied by s3_upload_part_size_multiply_factor.", 0) \ M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited. You ", 0) \ M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \ From 3cea474ccfb64e05b37dc9243d771bd75c2a635d Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Wed, 26 Jul 2023 15:05:41 +0200 Subject: [PATCH 1572/2047] MaterializedMySQL: Fix the infinite loop in ReadBuffer::read This fixes https://github.com/ClickHouse/ClickHouse/issues/33549 The infinite loop caused because sometimes the pos of ReadBuffer overflows the end. It happens f.e. when `MySQLPacketPayloadReadBuffer::nextImpl` might return empty buffer, because `in.nextIfAtEnd();` could not read more bytes and thus no bytes available (pos == end). It might happen when a network error or the connection to MySQL was closed or killed. This leads to empty `working_buffer` but successful returning from the func. And check `if (in.eof())` from `MySQLBinlogEventReadBuffer::nextImpl()` fails and also causes empty its `working_buffer` and successul return. At the end `payload.ignore(1)` and `payload.read(c)` produces the infinite loop since it is not eof() and ++pos overflows the end of the buffer. Should be tested by `test_mysql_kill*` from test.py --- src/IO/MySQLPacketPayloadReadBuffer.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/IO/MySQLPacketPayloadReadBuffer.cpp b/src/IO/MySQLPacketPayloadReadBuffer.cpp index ab58624d0fa..2c5167ed038 100644 --- a/src/IO/MySQLPacketPayloadReadBuffer.cpp +++ b/src/IO/MySQLPacketPayloadReadBuffer.cpp @@ -45,6 +45,9 @@ bool MySQLPacketPayloadReadBuffer::nextImpl() } in.nextIfAtEnd(); + /// Don't return a buffer when no bytes available + if (!in.hasPendingData()) + return false; working_buffer = ReadBuffer::Buffer(in.position(), in.buffer().end()); size_t count = std::min(in.available(), payload_length - offset); working_buffer.resize(count); From 60103812d6c8793a4bd5d00d827cea9fed079358 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 4 Aug 2023 09:43:06 +0000 Subject: [PATCH 1573/2047] tests update --- .../0_stateless/02833_concurrrent_sessions.sh | 21 ++++++++----------- .../02835_drop_user_during_session.sh | 3 +++ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh index fd8731cd1af..ffb112763c1 100755 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel -# If tests run in parallel, results can become flaky. -# Because each test starts many processes and waits for the query to run. +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -10,10 +8,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) readonly PID=$$ # Each user uses a separate thread. -readonly TCP_USERS=( "TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users -readonly HTTP_USERS=( "HTTP_USER_${PID}" ) -readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "HTTP_WITH_SESSION_ID_USER_${PID}" ) -readonly MYSQL_USERS=( "MYSQL_USER_${PID}") +readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users +readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) +readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) +readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" @@ -34,14 +32,13 @@ done # These functions try to create a session with successful login and logout. # Sleep a small, random amount of time to make concurrency more intense. # and try to login with an invalid password. -# test is actually not timing dependent. it should work even without sleep at all. function tcp_session() { local user=$1 local i=0 while (( (i++) < 10 )); do # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.02${RANDOM})" --user="${user}" --password="pass" + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" # login failure ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' done @@ -53,7 +50,7 @@ function http_session() local i=0 while (( (i++) < 10 )); do # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.02${RANDOM})" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" # login failure ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" @@ -66,7 +63,7 @@ function http_with_session_id_session() local i=0 while (( (i++) < 10 )); do # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.02${RANDOM}" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" # login failure ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" @@ -79,7 +76,7 @@ function mysql_session() local i=0 while (( (i++) < 10 )); do # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.02${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" # login failure ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh index cd31caf870e..e848e1fe90d 100755 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -51,6 +51,7 @@ ${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USE # DROP USE CASE ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" export -f tcp_session; export -f http_session; @@ -69,6 +70,7 @@ wait # DROP ROLE CASE ${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" ${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & @@ -85,6 +87,7 @@ wait # DROP PROFILE CASE ${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" ${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & From 0032eb9ab7c0d2815bb6e5fc9e3f31ad14a2346a Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 13 Apr 2023 15:03:27 +0000 Subject: [PATCH 1574/2047] Fix IN with decimal in analyzer --- src/Analyzer/SetUtils.cpp | 11 +++++++-- src/Interpreters/ActionsVisitor.cpp | 32 ------------------------- src/Interpreters/convertFieldToType.cpp | 30 +++++++++++++++++++++++ src/Interpreters/convertFieldToType.h | 4 ++++ 4 files changed, 43 insertions(+), 34 deletions(-) diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 15eec16e899..ffcaadbe074 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -70,7 +70,11 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & { if (columns_size == 1) { - auto field = convertFieldToType(value, *block_types[0]); + Field field; + bool is_conversion_ok = convertFieldToTypeStrict(value, *block_types[0], field); + if (!is_conversion_ok) + continue; + bool need_insert_null = transform_null_in && block_types[0]->isNullable(); if (!field.isNull() || need_insert_null) columns[0]->insert(std::move(field)); @@ -98,7 +102,10 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & size_t i = 0; for (; i < tuple_size; ++i) { - tuple_values[i] = convertFieldToType(tuple[i], *block_types[i]); + bool is_conversion_ok = convertFieldToTypeStrict(tuple[i], *block_types[i], tuple_values[i]); + if (!is_conversion_ok) + break; + bool need_insert_null = transform_null_in && block_types[i]->isNullable(); if (tuple_values[i].isNull() && !need_insert_null) break; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index f8d5d91f555..95ba8b2b94a 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -94,38 +94,6 @@ static size_t getTypeDepth(const DataTypePtr & type) return 0; } -template -static bool decimalEqualsFloat(Field field, Float64 float_value) -{ - auto decimal_field = field.get>(); - auto decimal_to_float = DecimalUtils::convertTo(decimal_field.getValue(), decimal_field.getScale()); - return decimal_to_float == float_value; -} - -/// Applies stricter rules than convertFieldToType: -/// Doesn't allow : -/// - loss of precision converting to Decimal -static bool convertFieldToTypeStrict(const Field & from_value, const IDataType & to_type, Field & result_value) -{ - result_value = convertFieldToType(from_value, to_type); - if (Field::isDecimal(from_value.getType()) && Field::isDecimal(result_value.getType())) - return applyVisitor(FieldVisitorAccurateEquals{}, from_value, result_value); - if (from_value.getType() == Field::Types::Float64 && Field::isDecimal(result_value.getType())) - { - /// Convert back to Float64 and compare - if (result_value.getType() == Field::Types::Decimal32) - return decimalEqualsFloat(result_value, from_value.get()); - if (result_value.getType() == Field::Types::Decimal64) - return decimalEqualsFloat(result_value, from_value.get()); - if (result_value.getType() == Field::Types::Decimal128) - return decimalEqualsFloat(result_value, from_value.get()); - if (result_value.getType() == Field::Types::Decimal256) - return decimalEqualsFloat(result_value, from_value.get()); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown decimal type {}", result_value.getTypeName()); - } - return true; -} - /// The `convertFieldToTypeStrict` is used to prevent unexpected results in case of conversion with loss of precision. /// Example: `SELECT 33.3 :: Decimal(9, 1) AS a WHERE a IN (33.33 :: Decimal(9, 2))` /// 33.33 in the set is converted to 33.3, but it is not equal to 33.3 in the column, so the result should still be empty. diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 89c73117960..a6a7466c699 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -21,6 +21,7 @@ #include #include +#include #include #include #include @@ -565,4 +566,33 @@ Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_t return converted; } +template +static bool decimalEqualsFloat(Field field, Float64 float_value) +{ + auto decimal_field = field.get>(); + auto decimal_to_float = DecimalUtils::convertTo(decimal_field.getValue(), decimal_field.getScale()); + return decimal_to_float == float_value; +} + +bool convertFieldToTypeStrict(const Field & from_value, const IDataType & to_type, Field & result_value) +{ + result_value = convertFieldToType(from_value, to_type); + if (Field::isDecimal(from_value.getType()) && Field::isDecimal(result_value.getType())) + return applyVisitor(FieldVisitorAccurateEquals{}, from_value, result_value); + if (from_value.getType() == Field::Types::Float64 && Field::isDecimal(result_value.getType())) + { + /// Convert back to Float64 and compare + if (result_value.getType() == Field::Types::Decimal32) + return decimalEqualsFloat(result_value, from_value.get()); + if (result_value.getType() == Field::Types::Decimal64) + return decimalEqualsFloat(result_value, from_value.get()); + if (result_value.getType() == Field::Types::Decimal128) + return decimalEqualsFloat(result_value, from_value.get()); + if (result_value.getType() == Field::Types::Decimal256) + return decimalEqualsFloat(result_value, from_value.get()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown decimal type {}", result_value.getTypeName()); + } + return true; +} + } diff --git a/src/Interpreters/convertFieldToType.h b/src/Interpreters/convertFieldToType.h index 91b631d0e12..2574dae1429 100644 --- a/src/Interpreters/convertFieldToType.h +++ b/src/Interpreters/convertFieldToType.h @@ -20,4 +20,8 @@ Field convertFieldToType(const Field & from_value, const IDataType & to_type, co /// Does the same, but throws ARGUMENT_OUT_OF_BOUND if value does not fall into the range. Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_type, const IDataType * from_type_hint = nullptr); +/// Applies stricter rules than convertFieldToType, doesn't allow loss of precision converting to Decimal +/// Returns true if the conversion was successful and the result is equal to the original value +bool convertFieldToTypeStrict(const Field & from_value, const IDataType & to_type, Field & result_value); + } From 079c6783a21e714682ab207e1763e35f228f2077 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 13 Apr 2023 15:05:42 +0000 Subject: [PATCH 1575/2047] Update decimal_in tests for analyzer --- src/Interpreters/convertFieldToType.cpp | 1 + ...decimal_in_precision_issue_41125.reference | 13 +++++++ ...02421_decimal_in_precision_issue_41125.sql | 22 ++++++++++- ...ecimal_in_floating_point_literal.reference | 21 +++++++++++ ...2428_decimal_in_floating_point_literal.sql | 37 +++++++++++++++++-- 5 files changed, 90 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index a6a7466c699..fc887cdc1b2 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -33,6 +33,7 @@ namespace DB namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; extern const int TYPE_MISMATCH; extern const int UNEXPECTED_DATA_AFTER_PARSED_VALUE; } diff --git a/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.reference b/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.reference index 71c9a23879f..c13142dd26a 100644 --- a/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.reference +++ b/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.reference @@ -11,3 +11,16 @@ 1 1 1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.sql b/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.sql index f5d182be3e3..fde893626c1 100644 --- a/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.sql +++ b/tests/queries/0_stateless/02421_decimal_in_precision_issue_41125.sql @@ -1,10 +1,30 @@ DROP TABLE IF EXISTS dtest; -SELECT count() == 0 FROM (SELECT '33.3' :: Decimal(9, 1) AS a WHERE a IN ('33.33' :: Decimal(9, 2))); CREATE TABLE dtest ( `a` Decimal(18, 0), `b` Decimal(18, 1), `c` Decimal(36, 0) ) ENGINE = Memory; INSERT INTO dtest VALUES ('33', '44.4', '35'); +SELECT count() == 0 FROM (SELECT '33.3' :: Decimal(9, 1) AS a WHERE a IN ('33.33' :: Decimal(9, 2))); + +SELECT count() == 0 FROM dtest WHERE a IN toDecimal32('33.3000', 4); +SELECT count() == 0 FROM dtest WHERE a IN toDecimal64('33.3000', 4); +SELECT count() == 0 FROM dtest WHERE a IN toDecimal128('33.3000', 4); +SELECT count() == 0 FROM dtest WHERE a IN toDecimal256('33.3000', 4); + +SELECT count() == 0 FROM dtest WHERE b IN toDecimal32('44.4000', 0); +SELECT count() == 0 FROM dtest WHERE b IN toDecimal64('44.4000', 0); +SELECT count() == 0 FROM dtest WHERE b IN toDecimal128('44.4000', 0); +SELECT count() == 0 FROM dtest WHERE b IN toDecimal256('44.4000', 0); + +SELECT count() == 1 FROM dtest WHERE b IN toDecimal32('44.4000', 4); +SELECT count() == 1 FROM dtest WHERE b IN toDecimal64('44.4000', 4); +SELECT count() == 1 FROM dtest WHERE b IN toDecimal128('44.4000', 4); +SELECT count() == 1 FROM dtest WHERE b IN toDecimal256('44.4000', 4); + +SET allow_experimental_analyzer = 1; + +SELECT count() == 0 FROM (SELECT '33.3' :: Decimal(9, 1) AS a WHERE a IN ('33.33' :: Decimal(9, 2))); + SELECT count() == 0 FROM dtest WHERE a IN toDecimal32('33.3000', 4); SELECT count() == 0 FROM dtest WHERE a IN toDecimal64('33.3000', 4); SELECT count() == 0 FROM dtest WHERE a IN toDecimal128('33.3000', 4); diff --git a/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.reference b/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.reference index 378b7d8cec4..66c3d28dfc0 100644 --- a/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.reference +++ b/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.reference @@ -19,3 +19,24 @@ 1 1 1 +0 +1 +1 +0 +0 +1 +1 +0 +0 +1 +1 +0 +0 +1 +1 +0 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.sql b/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.sql index 579f468ee54..a84cb5572ba 100644 --- a/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.sql +++ b/tests/queries/0_stateless/02428_decimal_in_floating_point_literal.sql @@ -1,3 +1,8 @@ +DROP TABLE IF EXISTS decimal_in_float_test; + +CREATE TABLE decimal_in_float_test ( `a` Decimal(18, 0), `b` Decimal(36, 2) ) ENGINE = Memory; +INSERT INTO decimal_in_float_test VALUES ('33', '44.44'); + SELECT toDecimal32(1.555,3) IN (1.5551); SELECT toDecimal32(1.555,3) IN (1.5551,1.555); SELECT toDecimal32(1.555,3) IN (1.5551,1.555000); @@ -18,10 +23,36 @@ SELECT toDecimal256(1.555,3) IN (1.5551,1.555); SELECT toDecimal256(1.555,3) IN (1.5551,1.555000); SELECT toDecimal256(1.555,3) IN (1.550,1.5); -DROP TABLE IF EXISTS decimal_in_float_test; -CREATE TABLE decimal_in_float_test ( `a` Decimal(18, 0), `b` Decimal(36, 2) ) ENGINE = Memory; -INSERT INTO decimal_in_float_test VALUES ('33', '44.44'); +SELECT count() == 1 FROM decimal_in_float_test WHERE a IN (33); +SELECT count() == 1 FROM decimal_in_float_test WHERE a IN (33.0); +SELECT count() == 1 FROM decimal_in_float_test WHERE a NOT IN (33.333); +SELECT count() == 1 FROM decimal_in_float_test WHERE b IN (44.44); +SELECT count() == 1 FROM decimal_in_float_test WHERE b NOT IN (44.4,44.444); + +SET allow_experimental_analyzer = 1; + + +SELECT toDecimal32(1.555,3) IN (1.5551); +SELECT toDecimal32(1.555,3) IN (1.5551,1.555); +SELECT toDecimal32(1.555,3) IN (1.5551,1.555000); +SELECT toDecimal32(1.555,3) IN (1.550,1.5); + +SELECT toDecimal64(1.555,3) IN (1.5551); +SELECT toDecimal64(1.555,3) IN (1.5551,1.555); +SELECT toDecimal64(1.555,3) IN (1.5551,1.555000); +SELECT toDecimal64(1.555,3) IN (1.550,1.5); + +SELECT toDecimal128(1.555,3) IN (1.5551); +SELECT toDecimal128(1.555,3) IN (1.5551,1.555); +SELECT toDecimal128(1.555,3) IN (1.5551,1.555000); +SELECT toDecimal128(1.555,3) IN (1.550,1.5); + +SELECT toDecimal256(1.555,3) IN (1.5551); +SELECT toDecimal256(1.555,3) IN (1.5551,1.555); +SELECT toDecimal256(1.555,3) IN (1.5551,1.555000); +SELECT toDecimal256(1.555,3) IN (1.550,1.5); + SELECT count() == 1 FROM decimal_in_float_test WHERE a IN (33); SELECT count() == 1 FROM decimal_in_float_test WHERE a IN (33.0); From caa9dc5d82bef2b5dd8cb01299c733297f4cfe5e Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 17 Apr 2023 15:19:17 +0000 Subject: [PATCH 1576/2047] Fix hashing for set literals in analyzer --- src/Planner/CollectSets.cpp | 5 +++++ src/Planner/PlannerActionsVisitor.cpp | 7 +++---- src/Planner/PlannerContext.cpp | 18 +++++++++++++++++- src/Planner/PlannerContext.h | 2 +- .../0_stateless/00700_decimal_compare.sql | 1 + 5 files changed, 27 insertions(+), 6 deletions(-) diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 8dd7c6637bf..09e7ddb4124 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -47,6 +47,11 @@ public: const auto & settings = planner_context.getQueryContext()->getSettingsRef(); auto & sets = planner_context.getPreparedSets(); + String set_key = planner_context.createSetKey(in_first_argument->getResultType(), in_second_argument); + + if (planner_context.hasSet(set_key)) + return; + /// Tables and table functions are replaced with subquery at Analysis stage, except special Set table. auto * second_argument_table = in_second_argument->as(); StorageSet * storage_set = second_argument_table != nullptr ? dynamic_cast(second_argument_table->getStorage().get()) : nullptr; diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 7575828e64d..7508856d99b 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -100,8 +100,9 @@ public: if (isNameOfInFunction(function_node.getFunctionName())) { + const auto & in_first_argument_node = function_node.getArguments().getNodes().at(0); const auto & in_second_argument_node = function_node.getArguments().getNodes().at(1); - in_function_second_argument_node_name = planner_context.createSetKey(in_second_argument_node); + in_function_second_argument_node_name = planner_context.createSetKey(in_first_argument_node->getResultType(), in_second_argument_node); } WriteBufferFromOwnString buffer; @@ -628,8 +629,6 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma auto in_first_argument = function_node.getArguments().getNodes().at(0); auto in_second_argument = function_node.getArguments().getNodes().at(1); - //auto set_key = planner_context->createSetKey(in_second_argument); - DataTypes set_element_types; auto in_second_argument_node_type = in_second_argument->getNodeType(); @@ -665,7 +664,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma PreparedSets::toString(set_key, set_element_types)); ColumnWithTypeAndName column; - column.name = planner_context->createSetKey(in_second_argument); + column.name = planner_context->createSetKey(in_first_argument->getResultType(), in_second_argument); column.type = std::make_shared(); bool set_is_created = set->get() != nullptr; diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 3c75d4fbea8..4cc2c4eeeca 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -112,9 +113,24 @@ const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const Que return table_expression_data->getColumnIdentifierOrNull(column_name); } -PlannerContext::SetKey PlannerContext::createSetKey(const QueryTreeNodePtr & set_source_node) +PlannerContext::SetKey PlannerContext::createSetKey(const DataTypePtr & left_operand_type, const QueryTreeNodePtr & set_source_node) { auto set_source_hash = set_source_node->getTreeHash(); + + if (set_source_node->as()) + { + /* We need to hash the type of the left operand because we can build different sets for different types. + * (It's done for performance reasons. It's cheaper to convert a small set of values from literal to the type of the left operand.) + * + * For example in expression `(a :: Decimal(9, 1) IN (1.0, 2.5)) AND (b :: Decimal(9, 0) IN (1, 2.5))` + * we need to build two different sets: + * - `{1, 2.5} :: Set(Decimal(9, 1))` for a + * - `{1} :: Set(Decimal(9, 0))` for b (2.5 omitted because bercause it's not representable as Decimal(9, 0)). + */ + return "__set_" + left_operand_type->getName() + '_' + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); + } + + /// For other cases we will cast left operand to the type of the set source, so no difference in types. return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index aceb313d4b5..44b24d09c5d 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -132,7 +132,7 @@ public: using SetKey = std::string; /// Create set key for set source node - static SetKey createSetKey(const QueryTreeNodePtr & set_source_node); + static SetKey createSetKey(const DataTypePtr & left_operand_type, const QueryTreeNodePtr & set_source_node); PreparedSets & getPreparedSets() { return prepared_sets; } diff --git a/tests/queries/0_stateless/00700_decimal_compare.sql b/tests/queries/0_stateless/00700_decimal_compare.sql index 41ff8b38102..7740c75f859 100644 --- a/tests/queries/0_stateless/00700_decimal_compare.sql +++ b/tests/queries/0_stateless/00700_decimal_compare.sql @@ -27,6 +27,7 @@ SELECT a > 0, b > 0, g > 0 FROM decimal ORDER BY a DESC; SELECT a, g > toInt8(0), g > toInt16(0), g > toInt32(0), g > toInt64(0) FROM decimal ORDER BY a; SELECT a, g > toUInt8(0), g > toUInt16(0), g > toUInt32(0), g > toUInt64(0) FROM decimal ORDER BY a; SELECT a, b, g FROM decimal WHERE a IN(42) AND b IN(42) AND g IN(42); +SELECT a, b, g FROM decimal WHERE a IN(42) AND b IN(42) AND g IN(42) SETTINGS allow_experimental_analyzer = 1; SELECT a, b, g FROM decimal WHERE a > 0 AND a <= 42 AND b <= 42 AND g <= 42; SELECT d, e, f from decimal WHERE d > 0 AND d < 1 AND e > 0 AND e < 1 AND f > 0 AND f < 1; From 8050edaff6ea00ae1e1cb30685a75f457ae94a05 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 4 May 2023 14:23:24 +0000 Subject: [PATCH 1577/2047] style convertFieldToTypeStrict --- src/Analyzer/SetUtils.cpp | 14 +++++++------- src/Interpreters/ActionsVisitor.cpp | 12 ++++++------ src/Interpreters/convertFieldToType.cpp | 22 ++++++++++++++-------- src/Interpreters/convertFieldToType.h | 6 +++--- 4 files changed, 30 insertions(+), 24 deletions(-) diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index ffcaadbe074..ceda264b5a6 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -70,14 +70,13 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & { if (columns_size == 1) { - Field field; - bool is_conversion_ok = convertFieldToTypeStrict(value, *block_types[0], field); - if (!is_conversion_ok) + auto field = convertFieldToTypeStrict(value, *block_types[0]); + if (!field) continue; bool need_insert_null = transform_null_in && block_types[0]->isNullable(); - if (!field.isNull() || need_insert_null) - columns[0]->insert(std::move(field)); + if (!field->isNull() || need_insert_null) + columns[0]->insert(*field); continue; } @@ -102,9 +101,10 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & size_t i = 0; for (; i < tuple_size; ++i) { - bool is_conversion_ok = convertFieldToTypeStrict(tuple[i], *block_types[i], tuple_values[i]); - if (!is_conversion_ok) + auto converted_field = convertFieldToTypeStrict(tuple[i], *block_types[i]); + if (!converted_field) break; + tuple_values[i] = std::move(*converted_field); bool need_insert_null = transform_null_in && block_types[i]->isNullable(); if (tuple_values[i].isNull() && !need_insert_null) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 95ba8b2b94a..bdb843e9118 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -114,11 +114,10 @@ static Block createBlockFromCollection(const Collection & collection, const Data { if (columns_num == 1) { - Field field; - bool is_conversion_ok = convertFieldToTypeStrict(value, *types[0], field); + auto field = convertFieldToTypeStrict(value, *types[0]); bool need_insert_null = transform_null_in && types[0]->isNullable(); - if (is_conversion_ok && (!field.isNull() || need_insert_null)) - columns[0]->insert(field); + if (field && (!field->isNull() || need_insert_null)) + columns[0]->insert(*field); } else { @@ -139,9 +138,10 @@ static Block createBlockFromCollection(const Collection & collection, const Data size_t i = 0; for (; i < tuple_size; ++i) { - bool is_conversion_ok = convertFieldToTypeStrict(tuple[i], *types[i], tuple_values[i]); - if (!is_conversion_ok) + auto converted_field = convertFieldToTypeStrict(tuple[i], *types[i]); + if (!converted_field) break; + tuple_values[i] = std::move(*converted_field); bool need_insert_null = transform_null_in && types[i]->isNullable(); if (tuple_values[i].isNull() && !need_insert_null) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index fc887cdc1b2..0e810748ab1 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -575,25 +575,31 @@ static bool decimalEqualsFloat(Field field, Float64 float_value) return decimal_to_float == float_value; } -bool convertFieldToTypeStrict(const Field & from_value, const IDataType & to_type, Field & result_value) +std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType & to_type) { - result_value = convertFieldToType(from_value, to_type); + Field result_value = convertFieldToType(from_value, to_type); + if (Field::isDecimal(from_value.getType()) && Field::isDecimal(result_value.getType())) - return applyVisitor(FieldVisitorAccurateEquals{}, from_value, result_value); + { + bool is_equal = applyVisitor(FieldVisitorAccurateEquals{}, from_value, result_value); + return is_equal ? result_value : std::optional{}; + } + if (from_value.getType() == Field::Types::Float64 && Field::isDecimal(result_value.getType())) { /// Convert back to Float64 and compare if (result_value.getType() == Field::Types::Decimal32) - return decimalEqualsFloat(result_value, from_value.get()); + return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal64) - return decimalEqualsFloat(result_value, from_value.get()); + return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal128) - return decimalEqualsFloat(result_value, from_value.get()); + return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; if (result_value.getType() == Field::Types::Decimal256) - return decimalEqualsFloat(result_value, from_value.get()); + return decimalEqualsFloat(result_value, from_value.get()) ? result_value : std::optional{}; throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown decimal type {}", result_value.getTypeName()); } - return true; + + return result_value; } } diff --git a/src/Interpreters/convertFieldToType.h b/src/Interpreters/convertFieldToType.h index 2574dae1429..7f49ea5479d 100644 --- a/src/Interpreters/convertFieldToType.h +++ b/src/Interpreters/convertFieldToType.h @@ -20,8 +20,8 @@ Field convertFieldToType(const Field & from_value, const IDataType & to_type, co /// Does the same, but throws ARGUMENT_OUT_OF_BOUND if value does not fall into the range. Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_type, const IDataType * from_type_hint = nullptr); -/// Applies stricter rules than convertFieldToType, doesn't allow loss of precision converting to Decimal -/// Returns true if the conversion was successful and the result is equal to the original value -bool convertFieldToTypeStrict(const Field & from_value, const IDataType & to_type, Field & result_value); +/// Applies stricter rules than convertFieldToType, doesn't allow loss of precision converting to Decimal. +/// Returns `Field` if the conversion was successful and the result is equal to the original value, otherwise returns nullopt. +std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType & to_type); } From 3d7bf0626c2b2d7106f43b71c386e6d75081f163 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Fri, 4 Aug 2023 10:36:33 +0200 Subject: [PATCH 1578/2047] MaterializedMySQL: Add tests to alter named collections --- .../materialized_with_ddl.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 604f7a62428..36be2aa1672 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -2592,6 +2592,20 @@ def named_collections(clickhouse_node, mysql_node, service_name): f"/* expect: (1, 'a', 1), (2, 'b', 2) */ SELECT * FROM {db}.t1", "1\ta\t1\n2\tb\t2\n", ) + clickhouse_node.query(f"ALTER NAMED COLLECTION {db} SET port=9999") + clickhouse_node.query(f"DETACH DATABASE {db}") + mysql_node.query(f"INSERT INTO {db}.t1 VALUES (3, 'c', 3)") + assert "ConnectionFailed:" in clickhouse_node.query_and_get_error( + f"ATTACH DATABASE {db}" + ) + clickhouse_node.query(f"ALTER NAMED COLLECTION {db} SET port=3306") + clickhouse_node.query(f"ATTACH DATABASE {db}") + check_query( + clickhouse_node, + f"/* expect: (1, 'a', 1), (2, 'b', 2), (3, 'c', 3) */ SELECT * FROM {db}.t1", + "1\ta\t1\n2\tb\t2\n3\tc\t3\n", + ) + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") mysql_node.query(f"DROP DATABASE IF EXISTS {db}") From a16c764b26b4c9aa2a146a15e3d87e76d5b959b2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Aug 2023 10:07:43 +0000 Subject: [PATCH 1579/2047] fix build in Planner/CollectSets.cpp --- src/Planner/CollectSets.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 09e7ddb4124..8dd7c6637bf 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -47,11 +47,6 @@ public: const auto & settings = planner_context.getQueryContext()->getSettingsRef(); auto & sets = planner_context.getPreparedSets(); - String set_key = planner_context.createSetKey(in_first_argument->getResultType(), in_second_argument); - - if (planner_context.hasSet(set_key)) - return; - /// Tables and table functions are replaced with subquery at Analysis stage, except special Set table. auto * second_argument_table = in_second_argument->as(); StorageSet * storage_set = second_argument_table != nullptr ? dynamic_cast(second_argument_table->getStorage().get()) : nullptr; From 1142180302b8c0215d2d55aa73b6687bf2fc6d84 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Aug 2023 10:09:36 +0000 Subject: [PATCH 1580/2047] upd tests/analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index c8f2bb9f43d..db21b86753a 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -2,7 +2,6 @@ 00562_in_subquery_merge_tree 00593_union_all_assert_columns_removed 00673_subquery_prepared_set_performance -00700_decimal_compare 00717_merge_and_distributed 00725_memory_tracking 00754_distributed_optimize_skip_select_on_unused_shards @@ -89,9 +88,7 @@ 02382_join_and_filtering_set 02402_merge_engine_with_view 02404_memory_bound_merging -02421_decimal_in_precision_issue_41125 02426_orc_bug -02428_decimal_in_floating_point_literal 02428_parameterized_view 02458_use_structure_from_insertion_table 02479_race_condition_between_insert_and_droppin_mv From 0b312f541acfdde4f37d7364e52da9af215e0a63 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 4 Aug 2023 13:29:28 +0300 Subject: [PATCH 1581/2047] Revert "Remove try/catch from DatabaseFilesystem" --- .../Serializations/SerializationWrapper.h | 1 + src/Databases/DatabaseFilesystem.cpp | 38 +++++++++++-------- src/Databases/DatabaseFilesystem.h | 2 +- src/Interpreters/DatabaseCatalog.cpp | 21 ++-------- .../0_stateless/02722_database_filesystem.sh | 2 +- .../0_stateless/02724_database_s3.reference | 6 +-- .../queries/0_stateless/02724_database_s3.sh | 4 +- .../0_stateless/02725_database_hdfs.reference | 11 +++--- .../0_stateless/02725_database_hdfs.sh | 14 +++---- 9 files changed, 45 insertions(+), 54 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index 31900f93148..bf922888af9 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -77,6 +77,7 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; + void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; }; diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 59f9ee67d7b..7eaf474eea0 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -11,11 +11,9 @@ #include #include #include -#include #include - namespace fs = std::filesystem; namespace DB @@ -77,8 +75,10 @@ bool DatabaseFilesystem::checkTableFilePath(const std::string & table_path, Cont /// Check access for file before checking its existence. if (check_path && !fileOrSymlinkPathStartsWith(table_path, user_files_path)) { - /// Access denied is thrown regardless of 'throw_on_error' - throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File is not inside {}", user_files_path); + if (throw_on_error) + throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File is not inside {}", user_files_path); + else + return false; } /// Check if the corresponding file exists. @@ -128,25 +128,20 @@ bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr context_) if (tryGetTableFromCache(name)) return true; - return checkTableFilePath(getTablePath(name), context_, /* throw_on_error */ false); + return checkTableFilePath(getTablePath(name), context_, /* throw_on_error */false); } -StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_, bool throw_on_error) const +StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_) const { /// Check if table exists in loaded tables map. if (auto table = tryGetTableFromCache(name)) return table; auto table_path = getTablePath(name); - if (!checkTableFilePath(table_path, context_, throw_on_error)) - return {}; - - String format = FormatFactory::instance().getFormatFromFileName(table_path, throw_on_error); - if (format.empty()) - return {}; + checkTableFilePath(table_path, context_, /* throw_on_error */true); /// If the file exists, create a new table using TableFunctionFile and return it. - auto args = makeASTFunction("file", std::make_shared(table_path), std::make_shared(format)); + auto args = makeASTFunction("file", std::make_shared(table_path)); auto table_function = TableFunctionFactory::instance().get(args, context_); if (!table_function) @@ -163,7 +158,7 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_) const { /// getTableImpl can throw exceptions, do not catch them to show correct error to user. - if (auto storage = getTableImpl(name, context_, true)) + if (auto storage = getTableImpl(name, context_)) return storage; throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", @@ -172,7 +167,20 @@ StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_ StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr context_) const { - return getTableImpl(name, context_, false); + try + { + return getTableImpl(name, context_); + } + catch (const Exception & e) + { + /// Ignore exceptions thrown by TableFunctionFile, which indicate that there is no table + /// see tests/02722_database_filesystem.sh for more details. + if (e.code() == ErrorCodes::FILE_DOESNT_EXIST) + { + return nullptr; + } + throw; + } } bool DatabaseFilesystem::empty() const diff --git a/src/Databases/DatabaseFilesystem.h b/src/Databases/DatabaseFilesystem.h index b72891b9a5c..7fe620401dc 100644 --- a/src/Databases/DatabaseFilesystem.h +++ b/src/Databases/DatabaseFilesystem.h @@ -48,7 +48,7 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; protected: - StoragePtr getTableImpl(const String & name, ContextPtr context, bool throw_on_error) const; + StoragePtr getTableImpl(const String & name, ContextPtr context) const; StoragePtr tryGetTableFromCache(const std::string & name) const; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f8481e3f1d8..13cac5afb1b 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -336,6 +336,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( return db_and_table; } + if (table_id.database_name == TEMPORARY_DATABASE) { /// For temporary tables UUIDs are set in Context::resolveStorageID(...). @@ -368,24 +369,8 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( database = it->second; } - StoragePtr table; - if (exception) - { - try - { - table = database->getTable(table_id.table_name, context_); - } - catch (const Exception & e) - { - exception->emplace(e); - } - } - else - { - table = database->tryGetTable(table_id.table_name, context_); - } - - if (!table && exception && !exception->has_value()) + auto table = database->tryGetTable(table_id.table_name, context_); + if (!table && exception) exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs())); if (!table) diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index 7b2e1bf1a66..3b7a41bb39e 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -61,7 +61,7 @@ CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); """ 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||: # FILE_DOESNT_EXIST: unknown file -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 107" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 60" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: Cannot determine the file format by it's extension ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.myext\`;" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||: diff --git a/tests/queries/0_stateless/02724_database_s3.reference b/tests/queries/0_stateless/02724_database_s3.reference index 437549a973a..425cca6a077 100644 --- a/tests/queries/0_stateless/02724_database_s3.reference +++ b/tests/queries/0_stateless/02724_database_s3.reference @@ -17,7 +17,5 @@ test1 16 17 18 0 0 0 Test 2: check exceptions -BAD_ARGUMENTS -BAD_ARGUMENTS -BAD_ARGUMENTS -BAD_ARGUMENTS +OK +OK diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index 15e93ff117f..bb8f1f5f7ee 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -46,12 +46,12 @@ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = S3; USE test3; SELECT * FROM \"http://localhost:11111/test/a.myext\" -""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ USE test3; SELECT * FROM \"abacaba\" -""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK" # Cleanup ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ diff --git a/tests/queries/0_stateless/02725_database_hdfs.reference b/tests/queries/0_stateless/02725_database_hdfs.reference index 932a0786921..ef8adae2bbc 100644 --- a/tests/queries/0_stateless/02725_database_hdfs.reference +++ b/tests/queries/0_stateless/02725_database_hdfs.reference @@ -4,8 +4,9 @@ test1 1 2 3 test2 Test 2: check exceptions -BAD_ARGUMENTS -BAD_ARGUMENTS -BAD_ARGUMENTS -CANNOT_EXTRACT_TABLE_STRUCTURE -BAD_ARGUMENTS +OK0 +OK1 +OK2 +OK3 +OK4 +OK5 diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index c258042a917..89ff7421a6f 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -1,8 +1,6 @@ #!/usr/bin/env bash # Tags: no-fasttest, use-hdfs, no-parallel -CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none - CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh @@ -38,19 +36,19 @@ echo "Test 2: check exceptions" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = HDFS('abacaba'); -""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK0" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test4; CREATE DATABASE test4 ENGINE = HDFS; USE test4; SELECT * FROM \"abacaba/file.tsv\" -""" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" +""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK1" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK2" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK3" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK4" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK5" # Cleanup From aa9ab58994f3c75d02b12eee25e9b00537aede1b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 4 Aug 2023 13:31:15 +0300 Subject: [PATCH 1582/2047] Update 01114_database_atomic.sh (#53043) --- tests/queries/0_stateless/01114_database_atomic.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 1b295e5f36f..3e1f9eb1f43 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -13,7 +13,7 @@ DROP DATABASE IF EXISTS test_01114_2; DROP DATABASE IF EXISTS test_01114_3; " -$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Ordinary" 2>&1| grep -Fac "UNKNOWN_DATABASE_ENGINE" +$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=0 -q "CREATE DATABASE test_01114_1 ENGINE=Ordinary" 2>&1| grep -Fac "UNKNOWN_DATABASE_ENGINE" $CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" $CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_2" From 569adafd9a9e4d0c6ad52e7970321e94f230da0c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Aug 2023 10:33:25 +0000 Subject: [PATCH 1583/2047] Split tests based on type --- .../0_stateless/02661_read_from_archive.lib | 35 +++ .../02661_read_from_archive.reference | 56 ---- .../0_stateless/02661_read_from_archive.sh | 37 --- .../02661_read_from_archive_7z.reference | 116 ++++++++ .../0_stateless/02661_read_from_archive_7z.sh | 11 + .../02661_read_from_archive_tar.reference | 116 ++++++++ .../02661_read_from_archive_tar.sh | 11 + .../02661_read_from_archive_targz.reference | 116 ++++++++ .../02661_read_from_archive_targz.sh | 11 + ...661_read_from_archive_with_globs.reference | 264 ------------------ .../02661_read_from_archive_with_globs.sh | 47 ---- .../02661_read_from_archive_zip.reference | 116 ++++++++ .../02661_read_from_archive_zip.sh | 11 + 13 files changed, 543 insertions(+), 404 deletions(-) delete mode 100644 tests/queries/0_stateless/02661_read_from_archive.reference delete mode 100755 tests/queries/0_stateless/02661_read_from_archive.sh create mode 100644 tests/queries/0_stateless/02661_read_from_archive_7z.reference create mode 100755 tests/queries/0_stateless/02661_read_from_archive_7z.sh create mode 100644 tests/queries/0_stateless/02661_read_from_archive_tar.reference create mode 100755 tests/queries/0_stateless/02661_read_from_archive_tar.sh create mode 100644 tests/queries/0_stateless/02661_read_from_archive_targz.reference create mode 100755 tests/queries/0_stateless/02661_read_from_archive_targz.sh delete mode 100644 tests/queries/0_stateless/02661_read_from_archive_with_globs.reference delete mode 100755 tests/queries/0_stateless/02661_read_from_archive_with_globs.sh create mode 100644 tests/queries/0_stateless/02661_read_from_archive_zip.reference create mode 100755 tests/queries/0_stateless/02661_read_from_archive_zip.sh diff --git a/tests/queries/0_stateless/02661_read_from_archive.lib b/tests/queries/0_stateless/02661_read_from_archive.lib index fc76c19e8de..0a015306282 100644 --- a/tests/queries/0_stateless/02661_read_from_archive.lib +++ b/tests/queries/0_stateless/02661_read_from_archive.lib @@ -7,7 +7,42 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function read_archive_file() { $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${user_files_path}/$1') ORDER BY 1, 2" + $CLICKHOUSE_CLIENT --query "SELECT * FROM file('${user_files_path}/$1') ORDER BY 1, 2" $CLICKHOUSE_CLIENT --query "CREATE TABLE 02661_archive_table Engine=File('CSV', '${user_files_path}/$1')" $CLICKHOUSE_CLIENT --query "SELECT * FROM 02661_archive_table ORDER BY 1, 2" $CLICKHOUSE_CLIENT --query "DROP TABLE 02661_archive_table" +} + +function run_archive_test() { + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS 02661_archive_table" + + user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + + echo -e "1,2\n3,4" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data1.csv + echo -e "5,6\n7,8" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data2.csv + echo -e "9,10\n11,12" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data3.csv + + eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data1.csv ${CLICKHOUSE_TEST_UNIQUE_NAME}_data2.csv > /dev/null" + eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data1.csv ${CLICKHOUSE_TEST_UNIQUE_NAME}_data3.csv > /dev/null" + eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive3.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data2.csv ${CLICKHOUSE_TEST_UNIQUE_NAME}_data3.csv > /dev/null" + + echo "archive1 data1.csv" + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data1.csv" + echo "archive{1..2} data1.csv" + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1..2}.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data1.csv" + echo "archive{1,2} data{1,3}.csv" + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1,2}.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data{1,3}.csv" + echo "archive3 data*.csv" + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive3.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data*.csv" + echo "archive* *.csv" + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive*.$1 :: *.csv" + echo "archive* {2..3}.csv" + read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive*.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data{2..3}.csv" + + $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1::nonexistent.csv')" 2>&1 | grep -q "CANNOT_UNPACK_ARCHIVE" && echo "OK" || echo "FAIL" + $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive3.$1::{2..3}.csv')" 2>&1 | grep -q "CANNOT_UNPACK_ARCHIVE" && echo "OK" || echo "FAIL" + + rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1..3}.$1 + + rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_data{1..3}.csv } \ No newline at end of file diff --git a/tests/queries/0_stateless/02661_read_from_archive.reference b/tests/queries/0_stateless/02661_read_from_archive.reference deleted file mode 100644 index eeeb4edb6ba..00000000000 --- a/tests/queries/0_stateless/02661_read_from_archive.reference +++ /dev/null @@ -1,56 +0,0 @@ -Running for zip files -1 2 -3 4 -1 2 -3 4 -1 2 -1 2 -3 4 -3 4 -1 2 -1 2 -3 4 -3 4 -OK -Running for tar.gz files -1 2 -3 4 -1 2 -3 4 -1 2 -1 2 -3 4 -3 4 -1 2 -1 2 -3 4 -3 4 -OK -Running for tar files -1 2 -3 4 -1 2 -3 4 -1 2 -1 2 -3 4 -3 4 -1 2 -1 2 -3 4 -3 4 -OK -Running for 7z files -1 2 -3 4 -1 2 -3 4 -1 2 -1 2 -3 4 -3 4 -1 2 -1 2 -3 4 -3 4 -OK diff --git a/tests/queries/0_stateless/02661_read_from_archive.sh b/tests/queries/0_stateless/02661_read_from_archive.sh deleted file mode 100755 index 6d69c9e80c7..00000000000 --- a/tests/queries/0_stateless/02661_read_from_archive.sh +++ /dev/null @@ -1,37 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -# shellcheck source=./02661_read_from_archive.lib -. "$CUR_DIR"/02661_read_from_archive.lib - -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS 02661_archive_table" - -user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -echo -e "1,2\n3,4" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv - -function run_archive_test() { - echo "Running for $1 files" - - eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null" - eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv > /dev/null" - - read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" - read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_archive{1..2}.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv" - - $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1::nonexistent.csv')" 2>&1 | grep -q "CANNOT_UNPACK_ARCHIVE" && echo "OK" || echo "FAIL" - - rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive1.$1 - rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_archive2.$1 -} - -run_archive_test "zip" "zip" -run_archive_test "tar.gz" "tar -cvzf" -run_archive_test "tar" "tar -cvf" -run_archive_test "7z" "7z a" - -rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_data.csv diff --git a/tests/queries/0_stateless/02661_read_from_archive_7z.reference b/tests/queries/0_stateless/02661_read_from_archive_7z.reference new file mode 100644 index 00000000000..27edb5536ad --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive_7z.reference @@ -0,0 +1,116 @@ +archive1 data1.csv +1 2 +3 4 +1 2 +3 4 +1 2 +3 4 +archive{1..2} data1.csv +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +archive{1,2} data{1,3}.csv +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +archive3 data*.csv +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +archive* *.csv +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +archive* {2..3}.csv +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +OK +OK diff --git a/tests/queries/0_stateless/02661_read_from_archive_7z.sh b/tests/queries/0_stateless/02661_read_from_archive_7z.sh new file mode 100755 index 00000000000..62137c4bf77 --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive_7z.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# shellcheck source=./02661_read_from_archive.lib +. "$CUR_DIR"/02661_read_from_archive.lib + +run_archive_test "7z" "7z a" \ No newline at end of file diff --git a/tests/queries/0_stateless/02661_read_from_archive_tar.reference b/tests/queries/0_stateless/02661_read_from_archive_tar.reference new file mode 100644 index 00000000000..27edb5536ad --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive_tar.reference @@ -0,0 +1,116 @@ +archive1 data1.csv +1 2 +3 4 +1 2 +3 4 +1 2 +3 4 +archive{1..2} data1.csv +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +archive{1,2} data{1,3}.csv +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +archive3 data*.csv +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +archive* *.csv +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +archive* {2..3}.csv +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +OK +OK diff --git a/tests/queries/0_stateless/02661_read_from_archive_tar.sh b/tests/queries/0_stateless/02661_read_from_archive_tar.sh new file mode 100755 index 00000000000..7e7db389235 --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive_tar.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# shellcheck source=./02661_read_from_archive.lib +. "$CUR_DIR"/02661_read_from_archive.lib + +run_archive_test "tar" "tar -cvf" \ No newline at end of file diff --git a/tests/queries/0_stateless/02661_read_from_archive_targz.reference b/tests/queries/0_stateless/02661_read_from_archive_targz.reference new file mode 100644 index 00000000000..27edb5536ad --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive_targz.reference @@ -0,0 +1,116 @@ +archive1 data1.csv +1 2 +3 4 +1 2 +3 4 +1 2 +3 4 +archive{1..2} data1.csv +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +archive{1,2} data{1,3}.csv +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +archive3 data*.csv +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +archive* *.csv +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +archive* {2..3}.csv +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +OK +OK diff --git a/tests/queries/0_stateless/02661_read_from_archive_targz.sh b/tests/queries/0_stateless/02661_read_from_archive_targz.sh new file mode 100755 index 00000000000..4505a150237 --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive_targz.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# shellcheck source=./02661_read_from_archive.lib +. "$CUR_DIR"/02661_read_from_archive.lib + +run_archive_test "tar.gz" "tar -cvzf" \ No newline at end of file diff --git a/tests/queries/0_stateless/02661_read_from_archive_with_globs.reference b/tests/queries/0_stateless/02661_read_from_archive_with_globs.reference deleted file mode 100644 index 00063fecc54..00000000000 --- a/tests/queries/0_stateless/02661_read_from_archive_with_globs.reference +++ /dev/null @@ -1,264 +0,0 @@ -Running for zip files -archive{1,2} data{1,3}.csv -1 2 -1 2 -3 4 -3 4 -9 10 -11 12 -1 2 -1 2 -3 4 -3 4 -9 10 -11 12 -archive3 data*.csv -5 6 -7 8 -9 10 -11 12 -5 6 -7 8 -9 10 -11 12 -archive* *.csv -1 2 -1 2 -3 4 -3 4 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -1 2 -1 2 -3 4 -3 4 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -archive* {2..3}.csv -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -OK -Running for tar.gz files -archive{1,2} data{1,3}.csv -1 2 -1 2 -3 4 -3 4 -9 10 -11 12 -1 2 -1 2 -3 4 -3 4 -9 10 -11 12 -archive3 data*.csv -5 6 -7 8 -9 10 -11 12 -5 6 -7 8 -9 10 -11 12 -archive* *.csv -1 2 -1 2 -3 4 -3 4 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -1 2 -1 2 -3 4 -3 4 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -archive* {2..3}.csv -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -OK -Running for tar files -archive{1,2} data{1,3}.csv -1 2 -1 2 -3 4 -3 4 -9 10 -11 12 -1 2 -1 2 -3 4 -3 4 -9 10 -11 12 -archive3 data*.csv -5 6 -7 8 -9 10 -11 12 -5 6 -7 8 -9 10 -11 12 -archive* *.csv -1 2 -1 2 -3 4 -3 4 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -1 2 -1 2 -3 4 -3 4 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -archive* {2..3}.csv -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -OK -Running for 7z files -archive{1,2} data{1,3}.csv -1 2 -1 2 -3 4 -3 4 -9 10 -11 12 -1 2 -1 2 -3 4 -3 4 -9 10 -11 12 -archive3 data*.csv -5 6 -7 8 -9 10 -11 12 -5 6 -7 8 -9 10 -11 12 -archive* *.csv -1 2 -1 2 -3 4 -3 4 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -1 2 -1 2 -3 4 -3 4 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -archive* {2..3}.csv -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -5 6 -5 6 -7 8 -7 8 -9 10 -9 10 -11 12 -11 12 -OK diff --git a/tests/queries/0_stateless/02661_read_from_archive_with_globs.sh b/tests/queries/0_stateless/02661_read_from_archive_with_globs.sh deleted file mode 100755 index 12d07112a63..00000000000 --- a/tests/queries/0_stateless/02661_read_from_archive_with_globs.sh +++ /dev/null @@ -1,47 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -# shellcheck source=./02661_read_from_archive.lib -. "$CUR_DIR"/02661_read_from_archive.lib - -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS 02661_archive_table" - -user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') - -echo -e "1,2\n3,4" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data1.csv -echo -e "5,6\n7,8" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data2.csv -echo -e "9,10\n11,12" > ${CLICKHOUSE_TEST_UNIQUE_NAME}_data3.csv - -function run_archive_test() { - echo "Running for $1 files" - - eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive1.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data1.csv ${CLICKHOUSE_TEST_UNIQUE_NAME}_data2.csv > /dev/null" - eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive2.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data1.csv ${CLICKHOUSE_TEST_UNIQUE_NAME}_data3.csv > /dev/null" - eval "$2 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive3.$1 ${CLICKHOUSE_TEST_UNIQUE_NAME}_data2.csv ${CLICKHOUSE_TEST_UNIQUE_NAME}_data3.csv > /dev/null" - - echo "archive{1,2} data{1,3}.csv" - read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive{1,2}.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data{1,3}.csv" - echo "archive3 data*.csv" - read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive3.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data*.csv" - echo "archive* *.csv" - read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive*.$1 :: *.csv" - echo "archive* {2..3}.csv" - read_archive_file "${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive*.$1 :: ${CLICKHOUSE_TEST_UNIQUE_NAME}_data{2..3}.csv" - - $CLICKHOUSE_LOCAL --query "SELECT * FROM file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive3.$1::{2..3}.csv')" 2>&1 | grep -q "CANNOT_UNPACK_ARCHIVE" && echo "OK" || echo "FAIL" - - rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive1.$1 - rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive2.$1 - rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}_with_globs_archive3.$1 -} - -run_archive_test "zip" "zip" -run_archive_test "tar.gz" "tar -cvzf" -run_archive_test "tar" "tar -cvf" -run_archive_test "7z" "7z a" - -rm ${CLICKHOUSE_TEST_UNIQUE_NAME}_data{1..3}.csv \ No newline at end of file diff --git a/tests/queries/0_stateless/02661_read_from_archive_zip.reference b/tests/queries/0_stateless/02661_read_from_archive_zip.reference new file mode 100644 index 00000000000..27edb5536ad --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive_zip.reference @@ -0,0 +1,116 @@ +archive1 data1.csv +1 2 +3 4 +1 2 +3 4 +1 2 +3 4 +archive{1..2} data1.csv +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +1 2 +1 2 +3 4 +3 4 +archive{1,2} data{1,3}.csv +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +1 2 +1 2 +3 4 +3 4 +9 10 +11 12 +archive3 data*.csv +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +5 6 +7 8 +9 10 +11 12 +archive* *.csv +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +1 2 +1 2 +3 4 +3 4 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +archive* {2..3}.csv +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +5 6 +5 6 +7 8 +7 8 +9 10 +9 10 +11 12 +11 12 +OK +OK diff --git a/tests/queries/0_stateless/02661_read_from_archive_zip.sh b/tests/queries/0_stateless/02661_read_from_archive_zip.sh new file mode 100755 index 00000000000..1234464f0a6 --- /dev/null +++ b/tests/queries/0_stateless/02661_read_from_archive_zip.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# shellcheck source=./02661_read_from_archive.lib +. "$CUR_DIR"/02661_read_from_archive.lib + +run_archive_test "zip" "zip" \ No newline at end of file From 4b8b78f220ef9927dfd5a05315d36f55cdee687d Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 4 Aug 2023 12:41:55 +0200 Subject: [PATCH 1584/2047] Revert "Merge pull request #53044 from ClickHouse/revert-52155-database-filesystem-remove-catch" This reverts commit e71f20891f37e0462f6832c440a63ff70495bbd3, reversing changes made to aa9ab58994f3c75d02b12eee25e9b00537aede1b. --- .../Serializations/SerializationWrapper.h | 1 - src/Databases/DatabaseFilesystem.cpp | 38 ++++++++----------- src/Databases/DatabaseFilesystem.h | 2 +- src/Interpreters/DatabaseCatalog.cpp | 21 ++++++++-- .../0_stateless/02722_database_filesystem.sh | 2 +- .../0_stateless/02724_database_s3.reference | 6 ++- .../queries/0_stateless/02724_database_s3.sh | 4 +- .../0_stateless/02725_database_hdfs.reference | 11 +++--- .../0_stateless/02725_database_hdfs.sh | 14 ++++--- 9 files changed, 54 insertions(+), 45 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index bf922888af9..31900f93148 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -77,7 +77,6 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextJSONPretty(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings, size_t indent) const override; - void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; }; diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 7eaf474eea0..59f9ee67d7b 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -11,9 +11,11 @@ #include #include #include +#include #include + namespace fs = std::filesystem; namespace DB @@ -75,10 +77,8 @@ bool DatabaseFilesystem::checkTableFilePath(const std::string & table_path, Cont /// Check access for file before checking its existence. if (check_path && !fileOrSymlinkPathStartsWith(table_path, user_files_path)) { - if (throw_on_error) - throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File is not inside {}", user_files_path); - else - return false; + /// Access denied is thrown regardless of 'throw_on_error' + throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File is not inside {}", user_files_path); } /// Check if the corresponding file exists. @@ -128,20 +128,25 @@ bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr context_) if (tryGetTableFromCache(name)) return true; - return checkTableFilePath(getTablePath(name), context_, /* throw_on_error */false); + return checkTableFilePath(getTablePath(name), context_, /* throw_on_error */ false); } -StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_) const +StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_, bool throw_on_error) const { /// Check if table exists in loaded tables map. if (auto table = tryGetTableFromCache(name)) return table; auto table_path = getTablePath(name); - checkTableFilePath(table_path, context_, /* throw_on_error */true); + if (!checkTableFilePath(table_path, context_, throw_on_error)) + return {}; + + String format = FormatFactory::instance().getFormatFromFileName(table_path, throw_on_error); + if (format.empty()) + return {}; /// If the file exists, create a new table using TableFunctionFile and return it. - auto args = makeASTFunction("file", std::make_shared(table_path)); + auto args = makeASTFunction("file", std::make_shared(table_path), std::make_shared(format)); auto table_function = TableFunctionFactory::instance().get(args, context_); if (!table_function) @@ -158,7 +163,7 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_) const { /// getTableImpl can throw exceptions, do not catch them to show correct error to user. - if (auto storage = getTableImpl(name, context_)) + if (auto storage = getTableImpl(name, context_, true)) return storage; throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", @@ -167,20 +172,7 @@ StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_ StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr context_) const { - try - { - return getTableImpl(name, context_); - } - catch (const Exception & e) - { - /// Ignore exceptions thrown by TableFunctionFile, which indicate that there is no table - /// see tests/02722_database_filesystem.sh for more details. - if (e.code() == ErrorCodes::FILE_DOESNT_EXIST) - { - return nullptr; - } - throw; - } + return getTableImpl(name, context_, false); } bool DatabaseFilesystem::empty() const diff --git a/src/Databases/DatabaseFilesystem.h b/src/Databases/DatabaseFilesystem.h index 7fe620401dc..b72891b9a5c 100644 --- a/src/Databases/DatabaseFilesystem.h +++ b/src/Databases/DatabaseFilesystem.h @@ -48,7 +48,7 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override; protected: - StoragePtr getTableImpl(const String & name, ContextPtr context) const; + StoragePtr getTableImpl(const String & name, ContextPtr context, bool throw_on_error) const; StoragePtr tryGetTableFromCache(const std::string & name) const; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 13cac5afb1b..f8481e3f1d8 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -336,7 +336,6 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( return db_and_table; } - if (table_id.database_name == TEMPORARY_DATABASE) { /// For temporary tables UUIDs are set in Context::resolveStorageID(...). @@ -369,8 +368,24 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( database = it->second; } - auto table = database->tryGetTable(table_id.table_name, context_); - if (!table && exception) + StoragePtr table; + if (exception) + { + try + { + table = database->getTable(table_id.table_name, context_); + } + catch (const Exception & e) + { + exception->emplace(e); + } + } + else + { + table = database->tryGetTable(table_id.table_name, context_); + } + + if (!table && exception && !exception->has_value()) exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs())); if (!table) diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index 3b7a41bb39e..7b2e1bf1a66 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -61,7 +61,7 @@ CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); """ 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||: # FILE_DOESNT_EXIST: unknown file -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 60" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 107" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: Cannot determine the file format by it's extension ${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.myext\`;" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||: diff --git a/tests/queries/0_stateless/02724_database_s3.reference b/tests/queries/0_stateless/02724_database_s3.reference index 425cca6a077..437549a973a 100644 --- a/tests/queries/0_stateless/02724_database_s3.reference +++ b/tests/queries/0_stateless/02724_database_s3.reference @@ -17,5 +17,7 @@ test1 16 17 18 0 0 0 Test 2: check exceptions -OK -OK +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index bb8f1f5f7ee..15e93ff117f 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -46,12 +46,12 @@ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = S3; USE test3; SELECT * FROM \"http://localhost:11111/test/a.myext\" -""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK" +""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ USE test3; SELECT * FROM \"abacaba\" -""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK" +""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" # Cleanup ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ diff --git a/tests/queries/0_stateless/02725_database_hdfs.reference b/tests/queries/0_stateless/02725_database_hdfs.reference index ef8adae2bbc..932a0786921 100644 --- a/tests/queries/0_stateless/02725_database_hdfs.reference +++ b/tests/queries/0_stateless/02725_database_hdfs.reference @@ -4,9 +4,8 @@ test1 1 2 3 test2 Test 2: check exceptions -OK0 -OK1 -OK2 -OK3 -OK4 -OK5 +BAD_ARGUMENTS +BAD_ARGUMENTS +BAD_ARGUMENTS +CANNOT_EXTRACT_TABLE_STRUCTURE +BAD_ARGUMENTS diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index 89ff7421a6f..c258042a917 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -1,6 +1,8 @@ #!/usr/bin/env bash # Tags: no-fasttest, use-hdfs, no-parallel +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh @@ -36,19 +38,19 @@ echo "Test 2: check exceptions" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = HDFS('abacaba'); -""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK0" +""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test4; CREATE DATABASE test4 ENGINE = HDFS; USE test4; SELECT * FROM \"abacaba/file.tsv\" -""" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK1" +""" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK2" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK3" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK4" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1| grep -F "UNKNOWN_TABLE" > /dev/null && echo "OK5" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" # Cleanup From 8aaa46ca9055a52f2541107af6171d67fe90170c Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 4 Aug 2023 11:07:33 +0000 Subject: [PATCH 1585/2047] StorageJoin: do not create clone hash join with all columns --- src/Interpreters/TableJoin.h | 3 ++- src/Storages/StorageJoin.cpp | 9 +++++++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 5d14a57759f..16e641b2718 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -189,7 +189,6 @@ private: template void inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right, bool strict); - NamesAndTypesList correctedColumnsAddedByJoin() const; void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix); @@ -371,6 +370,8 @@ public: bool isSpecialStorage() const { return !right_storage_name.empty() || right_storage_join || right_kv_storage; } std::shared_ptr getStorageKeyValue() { return right_kv_storage; } + + NamesAndTypesList correctedColumnsAddedByJoin() const; }; } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 640706aae17..cc64a1a15fc 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -237,8 +238,12 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, /// Qualifies will be added by join implementation (TableJoin contains a rename mapping). analyzed_join->setRightKeys(key_names); analyzed_join->setLeftKeys(left_key_names_resorted); - - HashJoinPtr join_clone = std::make_shared(analyzed_join, getRightSampleBlock()); + Block right_sample_block; + for (const auto & name : getKeyNames()) + right_sample_block.insert(getRightSampleBlock().getByName(name)); + for (const auto & name_and_type : analyzed_join->correctedColumnsAddedByJoin()) + right_sample_block.insert(ColumnWithTypeAndName(name_and_type.type->createColumn(), name_and_type.type, name_and_type.name)); + HashJoinPtr join_clone = std::make_shared(analyzed_join, right_sample_block); RWLockImpl::LockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, context); join_clone->setLock(holder); From 12ed37e99a0c17b65002c14cac5347862e22f256 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 4 Aug 2023 13:08:01 +0200 Subject: [PATCH 1586/2047] Fix --- src/Interpreters/Cache/FileSegment.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 6afccafe94a..afe47408a5c 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -186,9 +186,7 @@ bool FileSegment::isDownloaded() const String FileSegment::getCallerId() { - if (!CurrentThread::isInitialized() - || !CurrentThread::get().getQueryContext() - || CurrentThread::getQueryId().empty()) + if (!CurrentThread::isInitialized() || CurrentThread::getQueryId().empty()) return "None:" + toString(getThreadId()); return std::string(CurrentThread::getQueryId()) + ":" + toString(getThreadId()); From 0874b507df1b2012cb10c211dc830f564e1ee068 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 4 Aug 2023 14:09:59 +0300 Subject: [PATCH 1587/2047] remove obsolete no-upgrade-check tag (#52915) --- tests/clickhouse-test | 7 ------- tests/queries/0_stateless/00061_merge_tree_alter.sql | 1 - .../00626_replace_partition_from_table_zookeeper.sh | 2 +- ...orum_insert_lost_part_and_alive_part_zookeeper_long.sql | 2 +- tests/queries/0_stateless/00942_dataparts_500.sh | 1 - .../00993_system_parts_race_condition_drop_zookeeper.sh | 2 +- .../01079_parallel_alter_detach_table_zookeeper.sh | 2 +- .../0_stateless/01111_create_drop_replicated_db_stress.sh | 2 +- tests/queries/0_stateless/01191_rename_dictionary.sql | 2 +- .../01318_long_unsuccessful_mutation_zookeeper.sh | 2 +- .../0_stateless/01378_alter_rename_with_ttl_zookeeper.sql | 2 +- tests/queries/0_stateless/01391_join_on_dict_crash.sql | 2 +- .../0_stateless/01555_system_distribution_queue_mask.sql | 1 - .../0_stateless/01576_alter_low_cardinality_and_select.sh | 1 - .../01650_fetch_patition_with_macro_in_zk_path_long.sql | 2 +- tests/queries/0_stateless/01780_column_sparse_alter.sql | 1 - .../queries/0_stateless/02022_storage_filelog_one_file.sh | 1 - .../0_stateless/02025_storage_filelog_virtual_col.sh | 1 - tests/queries/0_stateless/02067_lost_part_s3.sql | 2 +- .../02222_create_table_without_columns_metadata.sh | 2 +- tests/queries/0_stateless/02242_delete_user_race.sh | 2 +- tests/queries/0_stateless/02243_drop_user_grant_race.sh | 2 +- tests/queries/0_stateless/02273_full_sort_join.sql.j2 | 4 +--- .../0_stateless/02302_join_auto_lc_nullable_bug.sql | 1 - .../0_stateless/02306_window_move_row_number_fix.sql | 1 - .../queries/0_stateless/02313_cross_join_dup_col_names.sql | 1 - tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql | 1 - tests/queries/0_stateless/02316_const_string_intersact.sql | 1 - .../0_stateless/02320_mapped_array_witn_const_nullable.sql | 1 - .../0_stateless/02332_dist_insert_send_logs_level.sh | 1 - .../02345_partial_sort_transform_optimization.sql | 1 - tests/queries/0_stateless/02354_annoy_index.sql | 2 +- tests/queries/0_stateless/02363_mapupdate_improve.sql | 1 - .../0_stateless/02366_direct_dictionary_dict_has.sql | 1 - tests/queries/0_stateless/02366_with_fill_date.sql | 1 - .../0_stateless/02381_compress_marks_and_primary_key.sql | 2 +- .../02397_system_parts_race_condition_drop_rm.sh | 2 +- tests/queries/0_stateless/02429_low_cardinality_trash.sh | 2 +- .../0_stateless/02450_kill_distributed_query_deadlock.sh | 2 +- tests/queries/0_stateless/02725_start_stop_fetches.sh | 2 +- 40 files changed, 21 insertions(+), 48 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 8c4e4dfec47..fc175f2a05a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -505,7 +505,6 @@ class FailureReason(enum.Enum): REPLICATED_DB = "replicated-database" S3_STORAGE = "s3-storage" BUILD = "not running for current build" - NO_UPGRADE_CHECK = "not running for upgrade check" NO_PARALLEL_REPLICAS = "smth in not supported with parallel replicas" # UNKNOWN reasons @@ -949,12 +948,6 @@ class TestCase: elif tags and ("no-replicated-database" in tags) and args.replicated_database: return FailureReason.REPLICATED_DB - # TODO: remove checking "no-upgrade-check" after 23.1 - elif args.upgrade_check and ( - "no-upgrade-check" in tags or "no-upgrade-check" in tags - ): - return FailureReason.NO_UPGRADE_CHECK - elif tags and ("no-s3-storage" in tags) and args.s3_storage: return FailureReason.S3_STORAGE elif ( diff --git a/tests/queries/0_stateless/00061_merge_tree_alter.sql b/tests/queries/0_stateless/00061_merge_tree_alter.sql index 2e46b1e16d6..f2a36d6e5a3 100644 --- a/tests/queries/0_stateless/00061_merge_tree_alter.sql +++ b/tests/queries/0_stateless/00061_merge_tree_alter.sql @@ -1,4 +1,3 @@ --- Tags: no-upgrade-check DROP TABLE IF EXISTS alter_00061; set allow_deprecated_syntax_for_merge_tree=1; diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index 399511db701..c32b6d04a42 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-s3-storage, no-upgrade-check +# Tags: zookeeper, no-s3-storage # Because REPLACE PARTITION does not forces immediate removal of replaced data parts from local filesystem # (it tries to do it as quick as possible, but it still performed in separate thread asynchronously) diff --git a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql index 9c02ac795ed..a1859220c6c 100644 --- a/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql +++ b/tests/queries/0_stateless/00732_quorum_insert_lost_part_and_alive_part_zookeeper_long.sql @@ -1,4 +1,4 @@ --- Tags: long, zookeeper, no-replicated-database, no-upgrade-check +-- Tags: long, zookeeper, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards SET send_logs_level = 'fatal'; diff --git a/tests/queries/0_stateless/00942_dataparts_500.sh b/tests/queries/0_stateless/00942_dataparts_500.sh index a6c3fcd4303..91c95816590 100755 --- a/tests/queries/0_stateless/00942_dataparts_500.sh +++ b/tests/queries/0_stateless/00942_dataparts_500.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-upgrade-check # Test fix for issue #5066 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh index 811681794a5..4205f231698 100755 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, zookeeper, no-parallel, no-upgrade-check, no-replicated-database +# Tags: race, zookeeper, no-parallel, no-replicated-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh index e508b77a0c2..8133f866c58 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-parallel, no-fasttest, no-upgrade-check +# Tags: zookeeper, no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh index 59899e1c14a..f61a60a0bda 100755 --- a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh +++ b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, zookeeper, no-upgrade-check +# Tags: race, zookeeper CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01191_rename_dictionary.sql b/tests/queries/0_stateless/01191_rename_dictionary.sql index 8074e84f0ed..e9fed1dd6b2 100644 --- a/tests/queries/0_stateless/01191_rename_dictionary.sql +++ b/tests/queries/0_stateless/01191_rename_dictionary.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-upgrade-check +-- Tags: no-parallel DROP DATABASE IF EXISTS test_01191; CREATE DATABASE test_01191 ENGINE=Atomic; diff --git a/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh b/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh index f9a2ec8a34c..42941b486d6 100755 --- a/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh +++ b/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-parallel, no-upgrade-check +# Tags: long, zookeeper, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql index 43c9fa43104..b6982910ace 100644 --- a/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql @@ -1,4 +1,4 @@ --- Tags: zookeeper, no-upgrade-check +-- Tags: zookeeper DROP TABLE IF EXISTS table_rename_with_ttl; diff --git a/tests/queries/0_stateless/01391_join_on_dict_crash.sql b/tests/queries/0_stateless/01391_join_on_dict_crash.sql index 5321e03767f..854da04b334 100644 --- a/tests/queries/0_stateless/01391_join_on_dict_crash.sql +++ b/tests/queries/0_stateless/01391_join_on_dict_crash.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-upgrade-check +-- Tags: no-parallel DROP DATABASE IF EXISTS db_01391; CREATE DATABASE db_01391; diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql index 61083c3ae14..472e042a18b 100644 --- a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql @@ -1,4 +1,3 @@ --- Tags: no-upgrade-check -- force data path with the user/pass in it set use_compact_format_in_distributed_parts_names=0; diff --git a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh index 4a9b4beee5b..fcea7f57cd3 100755 --- a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh +++ b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-upgrade-check CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql index b45a1974611..f4afcb8d55e 100644 --- a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql +++ b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql @@ -1,4 +1,4 @@ --- Tags: long, no-upgrade-check +-- Tags: long DROP TABLE IF EXISTS test_01640; DROP TABLE IF EXISTS restore_01640; diff --git a/tests/queries/0_stateless/01780_column_sparse_alter.sql b/tests/queries/0_stateless/01780_column_sparse_alter.sql index bc2f6f7c91f..f33573e503a 100644 --- a/tests/queries/0_stateless/01780_column_sparse_alter.sql +++ b/tests/queries/0_stateless/01780_column_sparse_alter.sql @@ -1,4 +1,3 @@ --- Tags: no-upgrade-check SET mutations_sync = 2; diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index 3abf5c52031..32ce1643d4e 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-upgrade-check set -eu diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index e0f0114d030..0219a0421cb 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-upgrade-check set -eu diff --git a/tests/queries/0_stateless/02067_lost_part_s3.sql b/tests/queries/0_stateless/02067_lost_part_s3.sql index 7df15ab33c4..bfdf9250036 100644 --- a/tests/queries/0_stateless/02067_lost_part_s3.sql +++ b/tests/queries/0_stateless/02067_lost_part_s3.sql @@ -1,4 +1,4 @@ --- Tags: no-upgrade-check, no-fasttest +-- Tags: no-fasttest DROP TABLE IF EXISTS partslost_0; DROP TABLE IF EXISTS partslost_1; diff --git a/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh index 26646bd91a0..9cb4fb939e7 100755 --- a/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh +++ b/tests/queries/0_stateless/02222_create_table_without_columns_metadata.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-upgrade-check +# Tags: no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02242_delete_user_race.sh b/tests/queries/0_stateless/02242_delete_user_race.sh index 8f387333c33..2af54276469 100755 --- a/tests/queries/0_stateless/02242_delete_user_race.sh +++ b/tests/queries/0_stateless/02242_delete_user_race.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, no-fasttest, no-parallel, no-upgrade-check +# Tags: race, no-fasttest, no-parallel # Test tries to reproduce a race between threads: # - deletes user diff --git a/tests/queries/0_stateless/02243_drop_user_grant_race.sh b/tests/queries/0_stateless/02243_drop_user_grant_race.sh index 46ad776006e..4dce8e8124c 100755 --- a/tests/queries/0_stateless/02243_drop_user_grant_race.sh +++ b/tests/queries/0_stateless/02243_drop_user_grant_race.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, no-fasttest, no-parallel, no-upgrade-check +# Tags: race, no-fasttest, no-parallel set -e diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 6b6aa53836e..6500306356c 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -1,6 +1,4 @@ --- Tags: long, no-upgrade-check - --- TODO(@vdimir): remove no-upgrade-check tag after https://github.com/ClickHouse/ClickHouse/pull/51737 is released +-- Tags: long DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/02302_join_auto_lc_nullable_bug.sql b/tests/queries/0_stateless/02302_join_auto_lc_nullable_bug.sql index 8e0fb4a55a0..7f7285d5472 100644 --- a/tests/queries/0_stateless/02302_join_auto_lc_nullable_bug.sql +++ b/tests/queries/0_stateless/02302_join_auto_lc_nullable_bug.sql @@ -1,4 +1,3 @@ --- Tags: no-upgrade-check SET max_bytes_in_join = '100', join_algorithm = 'auto'; diff --git a/tests/queries/0_stateless/02306_window_move_row_number_fix.sql b/tests/queries/0_stateless/02306_window_move_row_number_fix.sql index f73525f92be..96dd8f6176b 100644 --- a/tests/queries/0_stateless/02306_window_move_row_number_fix.sql +++ b/tests/queries/0_stateless/02306_window_move_row_number_fix.sql @@ -1,2 +1 @@ --- Tags: no-upgrade-check SELECT nth_value(NULL, 1048577) OVER (Rows BETWEEN 1023 FOLLOWING AND UNBOUNDED FOLLOWING) diff --git a/tests/queries/0_stateless/02313_cross_join_dup_col_names.sql b/tests/queries/0_stateless/02313_cross_join_dup_col_names.sql index 08e8843f763..45390c0e8ef 100644 --- a/tests/queries/0_stateless/02313_cross_join_dup_col_names.sql +++ b/tests/queries/0_stateless/02313_cross_join_dup_col_names.sql @@ -1,4 +1,3 @@ --- Tags: no-upgrade-check -- https://github.com/ClickHouse/ClickHouse/issues/37561 diff --git a/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql b/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql index df20e5c42d4..cbf71f1d555 100644 --- a/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql +++ b/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql @@ -1,4 +1,3 @@ --- Tags: no-upgrade-check SET join_algorithm = 'partial_merge'; diff --git a/tests/queries/0_stateless/02316_const_string_intersact.sql b/tests/queries/0_stateless/02316_const_string_intersact.sql index 148d048952b..33629d2a56a 100644 --- a/tests/queries/0_stateless/02316_const_string_intersact.sql +++ b/tests/queries/0_stateless/02316_const_string_intersact.sql @@ -1,3 +1,2 @@ --- Tags: no-upgrade-check SELECT 'Play ClickHouse' InterSect SELECT 'Play ClickHouse' diff --git a/tests/queries/0_stateless/02320_mapped_array_witn_const_nullable.sql b/tests/queries/0_stateless/02320_mapped_array_witn_const_nullable.sql index 734c597051e..1dd06a26894 100644 --- a/tests/queries/0_stateless/02320_mapped_array_witn_const_nullable.sql +++ b/tests/queries/0_stateless/02320_mapped_array_witn_const_nullable.sql @@ -1,4 +1,3 @@ --- Tags: no-upgrade-check select arrayMap(x -> toNullable(1), range(number)) from numbers(3); select arrayFilter(x -> toNullable(1), range(number)) from numbers(3); diff --git a/tests/queries/0_stateless/02332_dist_insert_send_logs_level.sh b/tests/queries/0_stateless/02332_dist_insert_send_logs_level.sh index 503b94be715..5bbe31c9237 100755 --- a/tests/queries/0_stateless/02332_dist_insert_send_logs_level.sh +++ b/tests/queries/0_stateless/02332_dist_insert_send_logs_level.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-upgrade-check CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) diff --git a/tests/queries/0_stateless/02345_partial_sort_transform_optimization.sql b/tests/queries/0_stateless/02345_partial_sort_transform_optimization.sql index 07f705acd84..35ec675b7c0 100644 --- a/tests/queries/0_stateless/02345_partial_sort_transform_optimization.sql +++ b/tests/queries/0_stateless/02345_partial_sort_transform_optimization.sql @@ -1,4 +1,3 @@ --- Tags: no-upgrade-check -- Regression for PartialSortingTransform optimization that requires at least 1500 rows. SELECT * FROM (SELECT * FROM (SELECT 0 a, toNullable(number) b, toString(number) c FROM numbers(1e6)) ORDER BY a DESC, b DESC, c LIMIT 1500) limit 10; diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index fefb51dfcc9..dbe5d95dd1f 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check +-- Tags: no-fasttest, no-ubsan, no-cpu-aarch64 SET allow_experimental_annoy_index = 1; SET allow_experimental_analyzer = 0; diff --git a/tests/queries/0_stateless/02363_mapupdate_improve.sql b/tests/queries/0_stateless/02363_mapupdate_improve.sql index b4a4b8e5d91..c3cd8fff9ee 100644 --- a/tests/queries/0_stateless/02363_mapupdate_improve.sql +++ b/tests/queries/0_stateless/02363_mapupdate_improve.sql @@ -1,4 +1,3 @@ --- Tags: no-upgrade-check DROP TABLE IF EXISTS map_test; CREATE TABLE map_test(`tags` Map(String, String)) ENGINE = MergeTree PRIMARY KEY tags ORDER BY tags SETTINGS index_granularity = 8192; INSERT INTO map_test (tags) VALUES (map('fruit','apple','color','red')); diff --git a/tests/queries/0_stateless/02366_direct_dictionary_dict_has.sql b/tests/queries/0_stateless/02366_direct_dictionary_dict_has.sql index cf9f2971cb0..41334811464 100644 --- a/tests/queries/0_stateless/02366_direct_dictionary_dict_has.sql +++ b/tests/queries/0_stateless/02366_direct_dictionary_dict_has.sql @@ -1,4 +1,3 @@ --- Tags: no-upgrade-check DROP TABLE IF EXISTS test_table; CREATE TABLE test_table diff --git a/tests/queries/0_stateless/02366_with_fill_date.sql b/tests/queries/0_stateless/02366_with_fill_date.sql index 4d41facf423..aca57b127af 100644 --- a/tests/queries/0_stateless/02366_with_fill_date.sql +++ b/tests/queries/0_stateless/02366_with_fill_date.sql @@ -1,4 +1,3 @@ --- Tags: no-upgrade-check SELECT toDate('2022-02-01') AS d1 FROM numbers(18) AS number diff --git a/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql b/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql index 2fe0943745d..0c228c13f19 100644 --- a/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql +++ b/tests/queries/0_stateless/02381_compress_marks_and_primary_key.sql @@ -1,4 +1,4 @@ --- Tags: no-upgrade-check, no-random-merge-tree-settings +-- Tags: no-random-merge-tree-settings drop table if exists test_02381; create table test_02381(a UInt64, b UInt64) ENGINE = MergeTree order by (a, b) SETTINGS compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; diff --git a/tests/queries/0_stateless/02397_system_parts_race_condition_drop_rm.sh b/tests/queries/0_stateless/02397_system_parts_race_condition_drop_rm.sh index 39e513f6be4..48efc98f7c7 100755 --- a/tests/queries/0_stateless/02397_system_parts_race_condition_drop_rm.sh +++ b/tests/queries/0_stateless/02397_system_parts_race_condition_drop_rm.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, zookeeper, no-parallel, no-upgrade-check, disabled +# Tags: race, zookeeper, no-parallel, disabled CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02429_low_cardinality_trash.sh b/tests/queries/0_stateless/02429_low_cardinality_trash.sh index 91618cb2796..e115ee0824e 100755 --- a/tests/queries/0_stateless/02429_low_cardinality_trash.sh +++ b/tests/queries/0_stateless/02429_low_cardinality_trash.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-upgrade-check +# Tags: long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh b/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh index d15acba3837..03c43843d3a 100755 --- a/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh +++ b/tests/queries/0_stateless/02450_kill_distributed_query_deadlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-upgrade-check +# Tags: long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02725_start_stop_fetches.sh b/tests/queries/0_stateless/02725_start_stop_fetches.sh index 0ca687ae951..c9922455d94 100755 --- a/tests/queries/0_stateless/02725_start_stop_fetches.sh +++ b/tests/queries/0_stateless/02725_start_stop_fetches.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, zookeeper, no-parallel, no-upgrade-check, no-replicated-database +# Tags: race, zookeeper, no-parallel, no-replicated-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 552331fd541e6d57e9ad7de7eca19cab72e95b48 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 1 Aug 2023 13:37:49 +0200 Subject: [PATCH 1588/2047] Use the same default paths for clickhouse_keeper (symlink) as for clickhouse_keeper (executable), i.e. "/var/lib/clickhouse-keeper/.." --- programs/keeper/Keeper.cpp | 5 +++-- src/Coordination/Standalone/Context.h | 8 ++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index a38467c3369..45d742173bd 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -294,7 +294,7 @@ try else if (config().has("keeper_server.snapshot_storage_path")) path = std::filesystem::path(config().getString("keeper_server.snapshot_storage_path")).parent_path(); else - path = std::filesystem::path{KEEPER_DEFAULT_PATH}; + path = config().getString("path", KEEPER_DEFAULT_PATH); std::filesystem::create_directories(path); @@ -330,6 +330,7 @@ try auto global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); + global_context->setApplicationType(Context::ApplicationType::KEEPER); global_context->setPath(path); global_context->setRemoteHostFilter(config()); @@ -365,7 +366,7 @@ try } /// Initialize keeper RAFT. Do nothing if no keeper_server in config. - global_context->initializeKeeperDispatcher(/* start_async = */ true); + global_context->initializeKeeperDispatcher(/* start_async = */ false); FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher()); auto config_getter = [&] () -> const Poco::Util::AbstractConfiguration & diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index c2eee981aaa..cf0fceab34c 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -98,6 +98,14 @@ public: std::shared_ptr getFilesystemCacheLog() const; std::shared_ptr getFilesystemReadPrefetchesLog() const; + enum class ApplicationType + { + KEEPER + }; + + void setApplicationType(ApplicationType) {} + ApplicationType getApplicationType() const { return ApplicationType::KEEPER; } + IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const; std::shared_ptr getAsyncReadCounters() const; ThreadPool & getThreadPoolWriter() const; From 087c35c23d20a9d26359fe993acf944fb773ca39 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 1 Aug 2023 12:58:33 +0000 Subject: [PATCH 1589/2047] Remove duplicated dialect setting value --- src/Core/SettingsEnums.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index f3e7b692085..ec66c5faad3 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -138,7 +138,6 @@ IMPLEMENT_SETTING_ENUM(MsgPackUUIDRepresentation, ErrorCodes::BAD_ARGUMENTS, IMPLEMENT_SETTING_ENUM(Dialect, ErrorCodes::BAD_ARGUMENTS, {{"clickhouse", Dialect::clickhouse}, - {"kusto", Dialect::kusto}, {"kusto", Dialect::kusto}, {"prql", Dialect::prql}}) // FIXME: do not add 'kusto_auto' to the list. Maybe remove it from code completely? From c874e5955b49dd818315014f5baa6608a9980fba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 4 Aug 2023 12:33:11 +0000 Subject: [PATCH 1590/2047] Remove upgrade checks with sanitizers --- tests/ci/ci_config.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 9d170fe8ed6..ed61de05231 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -372,10 +372,7 @@ REQUIRED_CHECKS = [ "Stress test (msan)", "Stress test (tsan)", "Stress test (ubsan)", - "Upgrade check (asan)", "Upgrade check (debug)", - "Upgrade check (msan)", - "Upgrade check (tsan)", "Style Check", "Unit tests (asan)", "Unit tests (msan)", From 699be8ce71da07126455acf04c9e47d88f1a8077 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 4 Aug 2023 12:43:02 +0000 Subject: [PATCH 1591/2047] changed suggestions --- src/Functions/FunctionBinaryArithmetic.h | 15 +++++++-------- .../02812_pointwise_array_operations.reference | 7 +++++++ .../02812_pointwise_array_operations.sql | 10 ++++++++++ 3 files changed, 24 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 33057a16dd0..04d3c077871 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1146,10 +1146,7 @@ class FunctionBinaryArithmetic : public IFunction const auto * return_type_array = checkAndGetDataType(result_type.get()); if (!return_type_array) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Return type for function {} must be array.", getName()); - - if constexpr (!is_plus && !is_minus) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use this operation on arrays"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Return type for function {} must be array.", getName()); auto num_args = arguments.size(); DataTypes data_types; @@ -1189,7 +1186,7 @@ class FunctionBinaryArithmetic : public IFunction if (left_offsets[offset_index] != right_offsets[offset_index]) { throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, - "Cannot apply operation for arguments of different sizes. Size of the first argument: {}, size of the second argument: {}", + "Cannot apply operation for arrays of different sizes. Size of the first argument: {}, size of the second argument: {}", *left_array_col->getOffsets().data(), *right_array_col ->getOffsets().data()); } @@ -1203,10 +1200,9 @@ class FunctionBinaryArithmetic : public IFunction result_array_type = typeid_cast(result_type.get())->getNestedType(); - const auto & offsets = typeid_cast(arguments[0].column.get())->getOffsets(); size_t rows_count = 0; - if (!offsets.empty()) - rows_count = offsets.back(); + if (!left_offsets.empty()) + rows_count = left_offsets.back(); auto res = executeImpl(new_arguments, result_array_type, rows_count); return ColumnArray::create(res, typeid_cast(arguments[0].column.get())->getOffsetsPtr()); @@ -1415,6 +1411,9 @@ public: if (isArray(arguments[0]) && isArray(arguments[1])) { + if constexpr (!is_plus && !is_minus) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use this operation on arrays"); + DataTypes new_arguments { static_cast(*arguments[0]).getNestedType(), static_cast(*arguments[1]).getNestedType(), diff --git a/tests/queries/0_stateless/02812_pointwise_array_operations.reference b/tests/queries/0_stateless/02812_pointwise_array_operations.reference index 5c9702d910a..3b1b973fd3f 100644 --- a/tests/queries/0_stateless/02812_pointwise_array_operations.reference +++ b/tests/queries/0_stateless/02812_pointwise_array_operations.reference @@ -14,3 +14,10 @@ [2,5] [2,6] [0,0,0] +[(NULL,100000000000000000000),(NULL,1048833)] +[2,2] +[2,3] +[2,4] +[2,5] +[2,6] +[11,1,-2] diff --git a/tests/queries/0_stateless/02812_pointwise_array_operations.sql b/tests/queries/0_stateless/02812_pointwise_array_operations.sql index d72ced4b6fa..9bd633be232 100644 --- a/tests/queries/0_stateless/02812_pointwise_array_operations.sql +++ b/tests/queries/0_stateless/02812_pointwise_array_operations.sql @@ -6,3 +6,13 @@ SELECT ([[1,1],[2]]+[[12,1],[1]]); SELECT ([1,2]+[1,number]) from numbers(5); SELECT ([1,2::UInt64]+[1,number]) from numbers(5); SELECT ([materialize(1),materialize(2),materialize(3)]-[1,2,3]); +SELECT [(NULL, 256), (NULL, 256)] + [(1., 100000000000000000000.), (NULL, 1048577)]; +SELECT ([1,2::UInt64]+[1,number]) from numbers(5); +CREATE TABLE my_table (values Array(Int32)) ENGINE = MergeTree() ORDER BY values; +INSERT INTO my_table (values) VALUES ([12, 3, 1]); +SELECT values - [1,2,3] FROM my_table WHERE arrayExists(x -> x > 5, values); +SELECT ([12,13] % [5,6]); -- { serverError 1 } +SELECT ([2,3,4]-[1,-2,10,29]); -- { serverError 190 } +CREATE TABLE a ( x Array(UInt64), y Array(UInt64)) ENGINE = Memory; +INSERT INTO a VALUES ([2,3],[4,5]),([1,2,3], [4,5]),([6,7],[8,9,10]); +SELECT x, y, x+y FROM a; -- { serverError 190 } From 991abde851eed8f8b8202daa493c9ae640decd7e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 4 Aug 2023 13:10:50 +0000 Subject: [PATCH 1592/2047] fix reading of empty Nested(Array(...)) --- src/Storages/MergeTree/IMergeTreeReader.cpp | 2 +- src/Storages/MergeTree/IMergeTreeReader.h | 2 +- .../MergeTree/MergeTreeReaderCompact.cpp | 56 +++++++++++++++---- .../MergeTree/MergeTreeReaderCompact.h | 7 ++- ...2835_nested_array_lowcardinality.reference | 16 +++--- 5 files changed, 62 insertions(+), 21 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 73fbe447e23..8acf6413a37 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -242,7 +242,7 @@ IMergeTreeReader::ColumnNameLevel IMergeTreeReader::findColumnForOffsets(const N /// Find column that has maximal number of matching /// offsets columns with required_column. - for (const auto & part_column : data_part_info_for_read->getColumns()) + for (const auto & part_column : Nested::convertToSubcolumns(data_part_info_for_read->getColumns())) { auto name_in_storage = Nested::extractTableName(part_column.name); if (name_in_storage != required_name_in_storage) diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 97e17d56892..f6b6feb3d42 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -105,10 +105,10 @@ protected: NameSet partially_read_columns; -private: /// Alter conversions, which must be applied on fly if required AlterConversionsPtr alter_conversions; +private: /// Columns that are requested to read. NamesAndTypesList requested_columns; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 308b33cc62d..73b579eee5b 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -149,11 +149,34 @@ void MergeTreeReaderCompact::fillColumnPositions() position.reset(); } + /// If array of Nested column is missing in part, + /// we have to read its offsets if they exist. if (!position && is_array) { - /// If array of Nested column is missing in part, - /// we have to read its offsets if they exist. - auto name_level_for_offsets = findColumnForOffsets(column_to_read); + NameAndTypePair column_to_read_with_subcolumns = column_to_read; + auto [name_in_storage, subcolumn_name] = Nested::splitName(column_to_read.name); + + /// If it is a part of Nested, we need to get the column from + /// storage metatadata which is converted to Nested type with subcolumns. + /// It is required for proper counting of shared streams. + if (!subcolumn_name.empty()) + { + /// If column is renamed get the new name from storage metadata. + if (alter_conversions->columnHasNewName(name_in_storage)) + name_in_storage = alter_conversions->getColumnNewName(name_in_storage); + + if (!storage_columns_with_collected_nested) + storage_columns_with_collected_nested = ColumnsDescription( + Nested::collect(metadata_snapshot->getColumns().getAllPhysical())); + + column_to_read_with_subcolumns = storage_columns_with_collected_nested + ->getColumnOrSubcolumn( + GetColumnsOptions::All, + Nested::concatenateName(name_in_storage, subcolumn_name)); + } + + auto name_level_for_offsets = findColumnForOffsets(column_to_read_with_subcolumns); + if (name_level_for_offsets.has_value()) { column_positions[i] = data_part_info_for_read->getColumnPosition(name_level_for_offsets->first); @@ -162,7 +185,9 @@ void MergeTreeReaderCompact::fillColumnPositions() } } else + { column_positions[i] = std::move(position); + } } } @@ -297,6 +322,8 @@ void MergeTreeReaderCompact::readData( }; ISerialization::DeserializeBinaryBulkStatePtr state; + ISerialization::DeserializeBinaryBulkStatePtr state_for_prefix; + ISerialization::DeserializeBinaryBulkSettings deserialize_settings; deserialize_settings.avg_value_size_hint = avg_value_size_hints[name]; @@ -306,14 +333,18 @@ void MergeTreeReaderCompact::readData( /// In case of reading onlys offset use the correct serialization for reading of the prefix auto serialization = getSerializationInPart(name_type_in_storage); - auto serialization_for_prefix = column_for_offsets ? getSerializationInPart(*column_for_offsets) : serialization; - ColumnPtr temp_column = name_type_in_storage.type->createColumn(*serialization); - deserialize_settings.getter = buffer_getter_for_prefix; - serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state); + if (column_for_offsets) + { + auto serialization_for_prefix = getSerializationInPart(*column_for_offsets); + + deserialize_settings.getter = buffer_getter_for_prefix; + serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state_for_prefix); + } deserialize_settings.getter = buffer_getter; + serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); serialization->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state, nullptr); auto subcolumn = name_type_in_storage.type->getSubcolumn(name_and_type.getSubcolumnName(), temp_column); @@ -328,12 +359,17 @@ void MergeTreeReaderCompact::readData( { /// In case of reading only offsets use the correct serialization for reading the prefix auto serialization = getSerializationInPart(name_and_type); - auto serialization_for_prefix = column_for_offsets ? getSerializationInPart(*column_for_offsets) : serialization; - deserialize_settings.getter = buffer_getter_for_prefix; - serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state); + if (column_for_offsets) + { + auto serialization_for_prefix = getSerializationInPart(*column_for_offsets); + + deserialize_settings.getter = buffer_getter_for_prefix; + serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state_for_prefix); + } deserialize_settings.getter = buffer_getter; + serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state, nullptr); } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index ebe76d4f5cd..796bc4684a1 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -52,6 +52,12 @@ private: MergeTreeMarksLoader marks_loader; + /// Storage columns with collected separate arrays of Nested to columns of Nested type. + /// They maybe be needed for finding offsets of missed Nested columns in parts. + /// They are rarely used and are heavy to initialized, so we create them + /// only on demand and cache in this field. + std::optional storage_columns_with_collected_nested; + /// Positions of columns in part structure. using ColumnPositions = std::vector>; ColumnPositions column_positions; @@ -85,7 +91,6 @@ private: ReadBufferFromFileBase::ProfileCallback profile_callback; clockid_t clock_type; - bool initialized = false; }; diff --git a/tests/queries/0_stateless/02835_nested_array_lowcardinality.reference b/tests/queries/0_stateless/02835_nested_array_lowcardinality.reference index 5bdb2788cc1..c2936da0b4f 100644 --- a/tests/queries/0_stateless/02835_nested_array_lowcardinality.reference +++ b/tests/queries/0_stateless/02835_nested_array_lowcardinality.reference @@ -10,14 +10,14 @@ ['0','1','2','3','4','5','6','7','8'] ['','','','','','','','',''] [] [] [[]] [[]] -[[],['0']] [[],['']] -[[],['0'],['0','1']] [[],[''],['','']] -[[],['0'],['0','1'],['0','1','2']] [[],[''],['',''],['','','']] -[[],['0'],['0','1'],['0','1','2'],[]] [[],[''],['',''],['','',''],[]] -[[],['0'],['0','1'],['0','1','2'],[],['0']] [[],[''],['',''],['','',''],[],['']] -[[],['0'],['0','1'],['0','1','2'],[],['0'],['0','1']] [[],[''],['',''],['','',''],[],[''],['','']] -[[],['0'],['0','1'],['0','1','2'],[],['0'],['0','1'],['0','1','2']] [[],[''],['',''],['','',''],[],[''],['',''],['','','']] -[[],['0'],['0','1'],['0','1','2'],[],['0'],['0','1'],['0','1','2'],[]] [[],[''],['',''],['','',''],[],[''],['',''],['','',''],[]] +[[],['0']] [[],[]] +[[],['0'],['0','1']] [[],[],[]] +[[],['0'],['0','1'],['0','1','2']] [[],[],[],[]] +[[],['0'],['0','1'],['0','1','2'],[]] [[],[],[],[],[]] +[[],['0'],['0','1'],['0','1','2'],[],['0']] [[],[],[],[],[],[]] +[[],['0'],['0','1'],['0','1','2'],[],['0'],['0','1']] [[],[],[],[],[],[],[]] +[[],['0'],['0','1'],['0','1','2'],[],['0'],['0','1'],['0','1','2']] [[],[],[],[],[],[],[],[]] +[[],['0'],['0','1'],['0','1','2'],[],['0'],['0','1'],['0','1','2'],[]] [[],[],[],[],[],[],[],[],[]] [] [] [{}] [{}] [{},{'k0':0}] [{},{}] From 777e575b3166b00c541dfa887f6a722123cbce62 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 4 Aug 2023 15:27:26 +0200 Subject: [PATCH 1593/2047] Update src/Functions/FunctionBinaryArithmetic.h MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/Functions/FunctionBinaryArithmetic.h | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 108efdd4831..1ecf6cbfe34 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1407,17 +1407,17 @@ public: return getReturnTypeImplStatic(new_arguments, context); } - if (isArray(arguments[0]) && isArray(arguments[1])) - { - if constexpr (!is_plus && !is_minus) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use this operation on arrays"); + + if constexpr (is_plus || is_minus) { + if (isArray(arguments[0]) && isArray(arguments[1])) + { + DataTypes new_arguments { + static_cast(*arguments[0]).getNestedType(), + static_cast(*arguments[1]).getNestedType(), + }; - DataTypes new_arguments { - static_cast(*arguments[0]).getNestedType(), - static_cast(*arguments[1]).getNestedType(), - }; - - return std::make_shared(getReturnTypeImplStatic(new_arguments, context)); + return std::make_shared(getReturnTypeImplStatic(new_arguments, context)); + } } /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. From e7dbe50ad7ca5290e1ef052d6ae120031ce2b21a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 4 Aug 2023 15:54:44 +0200 Subject: [PATCH 1594/2047] Update FunctionBinaryArithmetic.h --- src/Functions/FunctionBinaryArithmetic.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 1ecf6cbfe34..e22db0dbc1c 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1407,8 +1407,9 @@ public: return getReturnTypeImplStatic(new_arguments, context); } - - if constexpr (is_plus || is_minus) { + + if constexpr (is_plus || is_minus) + { if (isArray(arguments[0]) && isArray(arguments[1])) { DataTypes new_arguments { From 89c99e4a6de133568e7a40f90c5176bbb918bbeb Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 4 Aug 2023 15:52:55 +0200 Subject: [PATCH 1595/2047] Revert extracting the root keeper path from the 'path' key in the configuration ('path' must be reserved for clickhouse-server). --- programs/keeper/Keeper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 45d742173bd..2a711e975d4 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -294,7 +294,7 @@ try else if (config().has("keeper_server.snapshot_storage_path")) path = std::filesystem::path(config().getString("keeper_server.snapshot_storage_path")).parent_path(); else - path = config().getString("path", KEEPER_DEFAULT_PATH); + path = KEEPER_DEFAULT_PATH; std::filesystem::create_directories(path); From 49ae853ae67009b817cac4f2c52848d328b38f25 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Aug 2023 13:57:18 +0000 Subject: [PATCH 1596/2047] Fix sanitizer warnings --- src/IO/Archives/LibArchiveReader.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 1686b12f37a..d499211c567 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -27,13 +27,13 @@ public: : path_to_archive(path_to_archive_), lock_on_reading(lock_on_reading_) { current_archive = open(path_to_archive); - current_entry = archive_entry_new(); } Handle(const Handle &) = delete; Handle(Handle && other) noexcept : current_archive(other.current_archive) , current_entry(other.current_entry) + , lock_on_reading(other.lock_on_reading) { other.current_archive = nullptr; other.current_entry = nullptr; @@ -110,6 +110,7 @@ public: const String & getFileName() const { + chassert(current_entry); if (!file_name) file_name.emplace(archive_entry_pathname(current_entry)); @@ -118,6 +119,7 @@ public: const FileInfo & getFileInfo() const { + chassert(current_entry); if (!file_info) { file_info.emplace(); @@ -130,7 +132,7 @@ public: } struct archive * current_archive; - struct archive_entry * current_entry; + struct archive_entry * current_entry = nullptr; private: void checkError(int error) const { @@ -185,7 +187,7 @@ private: /// for some archive types when we are reading headers static variables are used /// which are not thread-safe - const bool lock_on_reading = false; + const bool lock_on_reading; static inline std::mutex read_lock; mutable std::optional file_name; From f55fe91d8b0750a601e6f788c134276f2471a098 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 4 Aug 2023 15:46:50 +0200 Subject: [PATCH 1597/2047] Throw an exception if wrong default path is going to be used. --- programs/keeper/Keeper.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 2a711e975d4..49009fffb90 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -288,13 +288,27 @@ try std::string path; if (config().has("keeper_server.storage_path")) + { path = config().getString("keeper_server.storage_path"); + } + else if (std::filesystem::is_directory(std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination")) + { + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "By default 'keeper.storage_path' could be assigned to {}, but the directory {} already exists. Please specify 'keeper.storage_path' in the keeper configuration explicitly", + KEEPER_DEFAULT_PATH, String{std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination"}); + } else if (config().has("keeper_server.log_storage_path")) + { path = std::filesystem::path(config().getString("keeper_server.log_storage_path")).parent_path(); + } else if (config().has("keeper_server.snapshot_storage_path")) + { path = std::filesystem::path(config().getString("keeper_server.snapshot_storage_path")).parent_path(); + } else + { path = KEEPER_DEFAULT_PATH; + } std::filesystem::create_directories(path); From 02ed17dfa54b46b6439401fffd1591d77b07e2b7 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 4 Aug 2023 14:16:33 +0000 Subject: [PATCH 1598/2047] Analyzer: do not enable it for old servers in tests --- tests/analyzer_integration_broken_tests.txt | 15 --------------- .../test_compression_codec_read/test.py | 1 + .../test_default_compression_codec/test.py | 1 + .../integration/test_disk_over_web_server/test.py | 1 + .../test.py | 1 + .../test.py | 1 + .../test_distributed_inter_server_secret/test.py | 1 + .../test_groupBitmapAnd_on_distributed/test.py | 1 + ...st_groupBitmapAndState_on_distributed_table.py | 1 + tests/integration/test_old_versions/test.py | 7 +++++++ tests/integration/test_polymorphic_parts/test.py | 1 + .../test.py | 2 ++ .../test_replicating_constants/test.py | 1 + tests/integration/test_ttl_replicated/test.py | 3 +++ tests/integration/test_version_update/test.py | 5 +++++ .../test_version_update_after_mutation/test.py | 3 +++ 16 files changed, 30 insertions(+), 15 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 02f70c8a6df..8075bb6d230 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -136,34 +136,19 @@ test_quota/test.py::test_simpliest_quota test_quota/test.py::test_tracking_quota test_quota/test.py::test_users_xml_is_readonly test_replicated_merge_tree_compatibility/test.py::test_replicated_merge_tree_defaults_compatibility -test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_wide-Wide] -test_old_versions/test.py::test_client_is_older_than_server -test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive -test_old_versions/test.py::test_server_is_older_than_client -test_polymorphic_parts/test.py::test_compact_parts_only -test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_compact-Compact] -test_polymorphic_parts/test.py::test_polymorphic_parts_index -test_old_versions/test.py::test_distributed_query_initiator_is_older_than_shard -test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1] -test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0] test_ttl_replicated/test.py::test_ttl_table[DELETE] test_ttl_replicated/test.py::test_ttl_columns test_ttl_replicated/test.py::test_ttl_compatibility[node_left2-node_right2-2] test_ttl_replicated/test.py::test_ttl_table[] -test_version_update/test.py::test_aggregate_function_versioning_server_upgrade -test_version_update/test.py::test_aggregate_function_versioning_fetch_data_from_old_to_new_server test_ttl_replicated/test.py::test_ttl_double_delete_rule_returns_error test_ttl_replicated/test.py::test_ttl_alter_delete[test_ttl_alter_delete] test_ttl_replicated/test.py::test_ttl_alter_delete[test_ttl_alter_delete_replicated] test_ttl_replicated/test.py::test_ttl_compatibility[node_left0-node_right0-0] -test_version_update/test.py::test_modulo_partition_key_issue_23508 test_ttl_replicated/test.py::test_ttl_many_columns test_ttl_replicated/test.py::test_modify_column_ttl test_ttl_replicated/test.py::test_merge_with_ttl_timeout test_ttl_replicated/test.py::test_ttl_empty_parts test_ttl_replicated/test.py::test_ttl_compatibility[node_left1-node_right1-1] -test_version_update/test.py::test_aggregate_function_versioning_persisting_metadata -test_version_update/test.py::test_aggregate_function_versioning_issue_16587 test_ttl_replicated/test.py::test_modify_ttl test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database test_profile_events_s3/test.py::test_profile_events diff --git a/tests/integration/test_compression_codec_read/test.py b/tests/integration/test_compression_codec_read/test.py index 38cd61e241d..b39e5147d38 100644 --- a/tests/integration/test_compression_codec_read/test.py +++ b/tests/integration/test_compression_codec_read/test.py @@ -11,6 +11,7 @@ node1 = cluster.add_instance( tag="20.8.11.17", with_installed_binary=True, stay_alive=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_default_compression_codec/test.py b/tests/integration/test_default_compression_codec/test.py index c7c30f5eea4..c8b75ea9751 100644 --- a/tests/integration/test_default_compression_codec/test.py +++ b/tests/integration/test_default_compression_codec/test.py @@ -24,6 +24,7 @@ node3 = cluster.add_instance( tag="20.3.16", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node4 = cluster.add_instance("node4") diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index d62adfb3343..bc60e5c865e 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -36,6 +36,7 @@ def cluster(): with_installed_binary=True, image="clickhouse/clickhouse-server", tag="22.8.14.53", + allow_analyzer=False, ) cluster.start() diff --git a/tests/integration/test_distributed_backward_compatability/test.py b/tests/integration/test_distributed_backward_compatability/test.py index cb51142d249..c48a7ad1fa1 100644 --- a/tests/integration/test_distributed_backward_compatability/test.py +++ b/tests/integration/test_distributed_backward_compatability/test.py @@ -11,6 +11,7 @@ node_old = cluster.add_instance( tag="20.8.9.6", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node_new = cluster.add_instance( "node2", diff --git a/tests/integration/test_distributed_insert_backward_compatibility/test.py b/tests/integration/test_distributed_insert_backward_compatibility/test.py index ad61a2ad6f5..1e566d5e2da 100644 --- a/tests/integration/test_distributed_insert_backward_compatibility/test.py +++ b/tests/integration/test_distributed_insert_backward_compatibility/test.py @@ -14,6 +14,7 @@ node_dist = cluster.add_instance( tag="21.11.9.1", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 6dd25789f36..36ac07a550a 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -33,6 +33,7 @@ backward = make_instance( # version without DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 tag="23.2.3", with_installed_binary=True, + allow_analyzer=False, ) users = pytest.mark.parametrize( diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test.py b/tests/integration/test_groupBitmapAnd_on_distributed/test.py index 4dbc81236e7..f6e54a110eb 100644 --- a/tests/integration/test_groupBitmapAnd_on_distributed/test.py +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test.py @@ -19,6 +19,7 @@ node4 = cluster.add_instance( image="yandex/clickhouse-server", tag="21.5", with_zookeeper=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test_groupBitmapAndState_on_distributed_table.py b/tests/integration/test_groupBitmapAnd_on_distributed/test_groupBitmapAndState_on_distributed_table.py index 774b7708587..3cbdf7e473a 100644 --- a/tests/integration/test_groupBitmapAnd_on_distributed/test_groupBitmapAndState_on_distributed_table.py +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test_groupBitmapAndState_on_distributed_table.py @@ -20,6 +20,7 @@ node4 = cluster.add_instance( tag="21.6", with_installed_binary=True, with_zookeeper=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_old_versions/test.py b/tests/integration/test_old_versions/test.py index beef294b792..aff07c53114 100644 --- a/tests/integration/test_old_versions/test.py +++ b/tests/integration/test_old_versions/test.py @@ -10,6 +10,7 @@ node18_14 = cluster.add_instance( tag="18.14.19", with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"], + allow_analyzer=False, ) node19_1 = cluster.add_instance( "node19_1", @@ -17,6 +18,7 @@ node19_1 = cluster.add_instance( tag="19.1.16", with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"], + allow_analyzer=False, ) node19_4 = cluster.add_instance( "node19_4", @@ -24,6 +26,7 @@ node19_4 = cluster.add_instance( tag="19.4.5.35", with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"], + allow_analyzer=False, ) node19_8 = cluster.add_instance( "node19_8", @@ -31,6 +34,7 @@ node19_8 = cluster.add_instance( tag="19.8.3.8", with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"], + allow_analyzer=False, ) node19_11 = cluster.add_instance( "node19_11", @@ -38,6 +42,7 @@ node19_11 = cluster.add_instance( tag="19.11.13.74", with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"], + allow_analyzer=False, ) node19_13 = cluster.add_instance( "node19_13", @@ -45,6 +50,7 @@ node19_13 = cluster.add_instance( tag="19.13.7.57", with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"], + allow_analyzer=False, ) node19_16 = cluster.add_instance( "node19_16", @@ -52,6 +58,7 @@ node19_16 = cluster.add_instance( tag="19.16.2.2", with_installed_binary=True, main_configs=["configs/config.d/test_cluster.xml"], + allow_analyzer=False, ) old_nodes = [node18_14, node19_1, node19_4, node19_8, node19_11, node19_13, node19_16] new_node = cluster.add_instance("node_new") diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index c5859146fe9..48bcca7d505 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -363,6 +363,7 @@ node7 = cluster.add_instance( tag="19.17.8.54", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node8 = cluster.add_instance( "node8", diff --git a/tests/integration/test_replicated_merge_tree_compatibility/test.py b/tests/integration/test_replicated_merge_tree_compatibility/test.py index 68f2776e955..c30a0d86c98 100644 --- a/tests/integration/test_replicated_merge_tree_compatibility/test.py +++ b/tests/integration/test_replicated_merge_tree_compatibility/test.py @@ -9,6 +9,7 @@ node1 = cluster.add_instance( tag="20.12.4.5", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", @@ -17,6 +18,7 @@ node2 = cluster.add_instance( tag="20.12.4.5", stay_alive=True, with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_replicating_constants/test.py b/tests/integration/test_replicating_constants/test.py index 82cc5e757f1..00781e473c7 100644 --- a/tests/integration/test_replicating_constants/test.py +++ b/tests/integration/test_replicating_constants/test.py @@ -11,6 +11,7 @@ node2 = cluster.add_instance( image="yandex/clickhouse-server", tag="19.1.14", with_installed_binary=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 3b031569b8a..117ebe37dd2 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -23,6 +23,7 @@ node4 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], + allow_analyzer=False, ) node5 = cluster.add_instance( @@ -35,6 +36,7 @@ node5 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], + allow_analyzer=False, ) node6 = cluster.add_instance( "node6", @@ -46,6 +48,7 @@ node6 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], + allow_analyzer=False, ) diff --git a/tests/integration/test_version_update/test.py b/tests/integration/test_version_update/test.py index 3332fe69e86..b8fa3e7ebb4 100644 --- a/tests/integration/test_version_update/test.py +++ b/tests/integration/test_version_update/test.py @@ -15,6 +15,7 @@ node2 = cluster.add_instance( tag="21.2", with_installed_binary=True, stay_alive=True, + allow_analyzer=False, ) # Use differents nodes because if there is node.restart_from_latest_version(), then in later tests @@ -25,6 +26,7 @@ node3 = cluster.add_instance( tag="21.5", with_installed_binary=True, stay_alive=True, + allow_analyzer=False, ) node4 = cluster.add_instance( "node4", @@ -32,6 +34,7 @@ node4 = cluster.add_instance( tag="21.5", with_installed_binary=True, stay_alive=True, + allow_analyzer=False, ) node5 = cluster.add_instance( "node5", @@ -39,6 +42,7 @@ node5 = cluster.add_instance( tag="21.5", with_installed_binary=True, stay_alive=True, + allow_analyzer=False, ) node6 = cluster.add_instance( "node6", @@ -46,6 +50,7 @@ node6 = cluster.add_instance( tag="21.5", with_installed_binary=True, stay_alive=True, + allow_analyzer=False, ) diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 416220c93c3..db517e4d494 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -16,6 +16,7 @@ node1 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], + allow_analyzer=False, ) node2 = cluster.add_instance( "node2", @@ -27,6 +28,7 @@ node2 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], + allow_analyzer=False, ) node3 = cluster.add_instance( "node3", @@ -38,6 +40,7 @@ node3 = cluster.add_instance( main_configs=[ "configs/compat.xml", ], + allow_analyzer=False, ) From 376eeeb8422c8e356c2f0d9fb894cb7c492dba99 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Aug 2023 17:18:50 +0300 Subject: [PATCH 1599/2047] Fix `test_zookeeper_config` (#52988) * Fix `test_zookeeper_config` * Update tests/integration/test_zookeeper_config/test.py Co-authored-by: Alexander Tokmakov * Update tests/integration/test_zookeeper_config/test.py * Update test.py --------- Co-authored-by: Alexander Tokmakov --- tests/integration/test_zookeeper_config/test.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 65f82c2286b..5e36a97b3ca 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -2,6 +2,7 @@ import time import pytest import logging from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster( __file__, zookeeper_config_path="configs/zookeeper_config_root_a.xml" @@ -56,10 +57,11 @@ def test_chroot_with_same_root(started_cluster): for j in range(2): # Second insert to test deduplication node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) - time.sleep(1) + # Replication might take time - assert node1.query("select count() from simple").strip() == "2" - assert node2.query("select count() from simple").strip() == "2" + assert_eq_with_retry(node1, "select count() from simple", "2\n") + + assert_eq_with_retry(node2, "select count() from simple", "2\n") def test_chroot_with_different_root(started_cluster): From 21b4f828733d9e7509dad717bdb592da9c79db53 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 4 Aug 2023 14:28:58 +0000 Subject: [PATCH 1600/2047] fixed fast and style check --- src/Functions/FunctionBinaryArithmetic.h | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index e22db0dbc1c..ee5eb18c84c 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1408,18 +1408,19 @@ public: } - if constexpr (is_plus || is_minus) + if constexpr (is_plus || is_minus) { - if (isArray(arguments[0]) && isArray(arguments[1])) - { - DataTypes new_arguments { - static_cast(*arguments[0]).getNestedType(), - static_cast(*arguments[1]).getNestedType(), - }; - - return std::make_shared(getReturnTypeImplStatic(new_arguments, context)); - } + if (isArray(arguments[0]) && isArray(arguments[1])) + { + DataTypes new_arguments { + static_cast(*arguments[0]).getNestedType(), + static_cast(*arguments[1]).getNestedType(), + }; + return std::make_shared(getReturnTypeImplStatic(new_arguments, context)); + } } + else + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use this operation on arrays"); /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1], context)) From 53d9de0997f7a71a120df1864306e8dff440a140 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 4 Aug 2023 16:33:34 +0200 Subject: [PATCH 1601/2047] Fix rare race condition with key prefix directory cleanup --- src/Interpreters/Cache/Metadata.cpp | 30 ++++++++++++++--------------- src/Interpreters/Cache/Metadata.h | 4 ++++ 2 files changed, 19 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index ae9c87f5523..42fedc8aa3d 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -62,11 +62,13 @@ KeyMetadata::KeyMetadata( CleanupQueue & cleanup_queue_, DownloadQueue & download_queue_, Poco::Logger * log_, + std::shared_mutex & key_prefix_directory_mutex_, bool created_base_directory_) : key(key_) , key_path(key_path_) , cleanup_queue(cleanup_queue_) , download_queue(download_queue_) + , key_prefix_directory_mutex(key_prefix_directory_mutex_) , created_base_directory(created_base_directory_) , log(log_) { @@ -102,16 +104,21 @@ bool KeyMetadata::createBaseDirectory() { try { + std::shared_lock lock(key_prefix_directory_mutex); fs::create_directories(key_path); } - catch (...) + catch (const fs::filesystem_error & e) { - /// Avoid errors like - /// std::__1::__fs::filesystem::filesystem_error: filesystem error: in create_directories: No space left on device - /// and mark file segment with SKIP_CACHE state - tryLogCurrentException(__PRETTY_FUNCTION__); created_base_directory = false; - return false; + + if (e.code() == std::errc::no_space_on_device) + { + LOG_TRACE(log, "Failed to create base directory for key {}, " + "because no space left on device", key); + + return false; + } + throw; } } return true; @@ -200,7 +207,7 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata( it = emplace( key, std::make_shared( - key, getPathForKey(key), *cleanup_queue, *download_queue, log, is_initial_load)).first; + key, getPathForKey(key), *cleanup_queue, *download_queue, log, key_prefix_directory_mutex, is_initial_load)).first; } key_metadata = it->second; @@ -315,17 +322,10 @@ void CacheMetadata::doCleanup() try { + std::unique_lock mutex(key_prefix_directory_mutex); if (fs::exists(key_prefix_directory) && fs::is_empty(key_prefix_directory)) fs::remove(key_prefix_directory); } - catch (const fs::filesystem_error & e) - { - /// Key prefix directory can become non-empty just now, it is expected. - if (e.code() == std::errc::directory_not_empty) - continue; - LOG_ERROR(log, "Error while removing key {}: {}", cleanup_key, getCurrentExceptionMessage(true)); - chassert(false); - } catch (...) { LOG_ERROR(log, "Error while removing key {}: {}", cleanup_key, getCurrentExceptionMessage(true)); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 42d74338e12..57187f2715b 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -50,6 +51,7 @@ struct KeyMetadata : public std::map, CleanupQueue & cleanup_queue_, DownloadQueue & download_queue_, Poco::Logger * log_, + std::shared_mutex & key_prefix_directory_mutex_, bool created_base_directory_ = false); enum class KeyState @@ -76,6 +78,7 @@ private: KeyGuard guard; CleanupQueue & cleanup_queue; DownloadQueue & download_queue; + std::shared_mutex & key_prefix_directory_mutex; std::atomic created_base_directory = false; Poco::Logger * log; }; @@ -128,6 +131,7 @@ private: mutable CacheMetadataGuard guard; const CleanupQueuePtr cleanup_queue; const DownloadQueuePtr download_queue; + std::shared_mutex key_prefix_directory_mutex; Poco::Logger * log; void downloadImpl(FileSegment & file_segment, std::optional> & memory); From 50039210d482a7a2ee9b13340831d210ac375237 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 4 Aug 2023 17:27:10 +0200 Subject: [PATCH 1602/2047] Update FunctionBinaryArithmetic.h --- src/Functions/FunctionBinaryArithmetic.h | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index ee5eb18c84c..0a57deebed6 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1408,9 +1408,9 @@ public: } - if constexpr (is_plus || is_minus) + if (isArray(arguments[0]) && isArray(arguments[1])) { - if (isArray(arguments[0]) && isArray(arguments[1])) + if constexpr (is_plus || is_minus) { DataTypes new_arguments { static_cast(*arguments[0]).getNestedType(), @@ -1418,9 +1418,10 @@ public: }; return std::make_shared(getReturnTypeImplStatic(new_arguments, context)); } + else + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use this operation on arrays"); } - else - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use this operation on arrays"); + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1], context)) From 0f427cbdeaf29fd3731e192edb50c546589fa2d3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 4 Aug 2023 13:32:59 +0200 Subject: [PATCH 1603/2047] Fix 02722_database_filesystem/s3/hdfs with analyzer --- .../0_stateless/02722_database_filesystem.sh | 17 ++++++++--------- .../0_stateless/02724_database_s3.reference | 6 ++---- tests/queries/0_stateless/02724_database_s3.sh | 4 ++-- .../0_stateless/02725_database_hdfs.reference | 8 ++++---- .../queries/0_stateless/02725_database_hdfs.sh | 9 +++++---- 5 files changed, 21 insertions(+), 23 deletions(-) diff --git a/tests/queries/0_stateless/02722_database_filesystem.sh b/tests/queries/0_stateless/02722_database_filesystem.sh index 7b2e1bf1a66..99f2191c991 100755 --- a/tests/queries/0_stateless/02722_database_filesystem.sh +++ b/tests/queries/0_stateless/02722_database_filesystem.sh @@ -40,32 +40,31 @@ ${CLICKHOUSE_LOCAL} -q "SELECT COUNT(*) FROM \"${tmp_dir}/tmp.csv\"" ################# echo "Test 2: check DatabaseFilesystem access rights and errors handling on server" # DATABASE_ACCESS_DENIED: Allows list files only inside user_files -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp.csv\`;" 2>&1| grep -F "Code: 481" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`/tmp/tmp.csv\`;" 2>&1| grep -F "Code: 481" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`/tmp/tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --multiline --multiquery --query """ USE test1; SELECT COUNT(*) FROM \"../${tmp_dir}/tmp.csv\"; -""" 2>&1| grep -F "Code: 481" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../../../../../../tmp.csv\`;" 2>&1| grep -F "Code: 481" > /dev/null && echo "OK" || echo 'FAIL' ||: +""" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../../../../../../tmp.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF "PATH_ACCESS_DENIED" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: path should be inside user_files ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test2; CREATE DATABASE test2 ENGINE = Filesystem('/tmp'); -""" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||: +""" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: .../user_files/relative_unknown_dir does not exists ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ DROP DATABASE IF EXISTS test2; CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir'); -""" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||: +""" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # FILE_DOESNT_EXIST: unknown file -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 107" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "FILE_DOESNT_EXIST" > /dev/null && echo "OK" || echo 'FAIL' ||: # BAD_ARGUMENTS: Cannot determine the file format by it's extension -${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.myext\`;" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK" || echo 'FAIL' ||: - +${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.myext\`;" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # Clean ${CLICKHOUSE_CLIENT} --query "DROP DATABASE test1;" rm -rd $tmp_dir diff --git a/tests/queries/0_stateless/02724_database_s3.reference b/tests/queries/0_stateless/02724_database_s3.reference index 437549a973a..425cca6a077 100644 --- a/tests/queries/0_stateless/02724_database_s3.reference +++ b/tests/queries/0_stateless/02724_database_s3.reference @@ -17,7 +17,5 @@ test1 16 17 18 0 0 0 Test 2: check exceptions -BAD_ARGUMENTS -BAD_ARGUMENTS -BAD_ARGUMENTS -BAD_ARGUMENTS +OK +OK diff --git a/tests/queries/0_stateless/02724_database_s3.sh b/tests/queries/0_stateless/02724_database_s3.sh index 15e93ff117f..13b627c0342 100755 --- a/tests/queries/0_stateless/02724_database_s3.sh +++ b/tests/queries/0_stateless/02724_database_s3.sh @@ -46,12 +46,12 @@ DROP DATABASE IF EXISTS test3; CREATE DATABASE test3 ENGINE = S3; USE test3; SELECT * FROM \"http://localhost:11111/test/a.myext\" -""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +""" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ USE test3; SELECT * FROM \"abacaba\" -""" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +""" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # Cleanup ${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ diff --git a/tests/queries/0_stateless/02725_database_hdfs.reference b/tests/queries/0_stateless/02725_database_hdfs.reference index 932a0786921..dfc5b63647d 100644 --- a/tests/queries/0_stateless/02725_database_hdfs.reference +++ b/tests/queries/0_stateless/02725_database_hdfs.reference @@ -5,7 +5,7 @@ test1 test2 Test 2: check exceptions BAD_ARGUMENTS -BAD_ARGUMENTS -BAD_ARGUMENTS -CANNOT_EXTRACT_TABLE_STRUCTURE -BAD_ARGUMENTS +OK +OK +OK +OK diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index c258042a917..f6089cfa18a 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -47,10 +47,11 @@ USE test4; SELECT * FROM \"abacaba/file.tsv\" """ 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' | grep -oF "BAD_ARGUMENTS" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||: + +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: # Cleanup From 1bef5ec3778d71ca8cde38bf221a58ff7b838dbb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 4 Aug 2023 16:05:43 +0000 Subject: [PATCH 1604/2047] Reduce the execution time of a single query --- .../01414_mutations_and_errors_zookeeper.sh | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh b/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh index f92ab265eb8..dc88808d73b 100755 --- a/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh +++ b/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh @@ -18,19 +18,19 @@ $CLICKHOUSE_CLIENT --query " PARTITION BY date " -$CLICKHOUSE_CLIENT --query "INSERT INTO replicated_mutation_table SELECT toDate('2019-10-02'), number, '42' FROM numbers(4)" +$CLICKHOUSE_CLIENT --query "INSERT INTO replicated_mutation_table SELECT toDate('2019-10-02'), number, '42' FROM numbers(10)" -$CLICKHOUSE_CLIENT --query "INSERT INTO replicated_mutation_table SELECT toDate('2019-10-02'), number, 'Hello' FROM numbers(4)" +$CLICKHOUSE_CLIENT --query "INSERT INTO replicated_mutation_table SELECT toDate('2019-10-02'), number, 'Hello' FROM numbers(10)" $CLICKHOUSE_CLIENT --query "ALTER TABLE replicated_mutation_table UPDATE key = key + 1 WHERE sleepEachRow(1) == 0 SETTINGS mutations_sync = 2" 2>&1 | grep -o 'Mutation 0000000000 was killed' | head -n 1 & check_query="SELECT count() FROM system.mutations WHERE table='replicated_mutation_table' and database='$CLICKHOUSE_DATABASE' and mutation_id='0000000000'" -query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) +query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") while [ "$query_result" != "1" ] do - query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) + query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") sleep 0.1 done @@ -38,7 +38,7 @@ $CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE table='replicated_mutation_table while [ "$query_result" != "0" ] do - query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) + query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") sleep 0.5 done @@ -49,11 +49,11 @@ $CLICKHOUSE_CLIENT --query "ALTER TABLE replicated_mutation_table MODIFY COLUMN check_query="SELECT type = 'UInt64' FROM system.columns WHERE table='replicated_mutation_table' and database='$CLICKHOUSE_DATABASE' and name='value'" -query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) +query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") while [ "$query_result" != "1" ] do - query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) + query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") sleep 0.5 done @@ -66,7 +66,7 @@ $CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE table='replicated_mutation_table while [ "$query_result" != "0" ] do - query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) + query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") sleep 0.5 done From 06631f6a86a2e099ef69f807592ec2eb70b5aa35 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 4 Aug 2023 09:27:15 -0700 Subject: [PATCH 1605/2047] Make hasTokenOrNull return null on empty needle --- src/Functions/HasTokenImpl.h | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Functions/HasTokenImpl.h b/src/Functions/HasTokenImpl.h index ab6b6399486..661b8ae9753 100644 --- a/src/Functions/HasTokenImpl.h +++ b/src/Functions/HasTokenImpl.h @@ -39,9 +39,6 @@ struct HasTokenImpl if (start_pos != nullptr) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' does not support start_pos argument", name); - if (pattern.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle cannot be empty, because empty string isn't a token"); - if (haystack_offsets.empty()) return; @@ -49,7 +46,7 @@ struct HasTokenImpl const UInt8 * const end = haystack_data.data() + haystack_data.size(); const UInt8 * pos = begin; - if (!std::none_of(pattern.begin(), pattern.end(), isTokenSeparator)) + if (const auto has_separator = std::any_of(pattern.cbegin(), pattern.cend(), isTokenSeparator); has_separator || pattern.empty()) { if (res_null) { @@ -57,8 +54,12 @@ struct HasTokenImpl std::ranges::fill(res_null->getData(), true); return; } - else + else if (has_separator) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters"); + else if (pattern.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle cannot be empty, because empty string isn't a token"); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected internal state"); } size_t pattern_size = pattern.size(); From cc5ec9e6345fbbf154536b3a7a95f3ea9b6452d1 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Fri, 4 Aug 2023 11:34:00 -0700 Subject: [PATCH 1606/2047] build: Upgrade PRQL version --- rust/Cargo.lock | 231 ++++++++++++++++++++++++++++++++----------- rust/prql/Cargo.toml | 4 +- 2 files changed, 174 insertions(+), 61 deletions(-) diff --git a/rust/Cargo.lock b/rust/Cargo.lock index 07bbf8ba27e..04569cd3b3a 100644 --- a/rust/Cargo.lock +++ b/rust/Cargo.lock @@ -78,6 +78,55 @@ dependencies = [ "libc", ] +[[package]] +name = "anstream" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ca84f3628370c59db74ee214b3263d58f9aadd9b4fe7e711fd87dc452b7f163" +dependencies = [ + "anstyle", + "anstyle-parse", + "anstyle-query", + "anstyle-wincon", + "colorchoice", + "is-terminal", + "utf8parse", +] + +[[package]] +name = "anstyle" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a30da5c5f2d5e72842e00bcb57657162cdabef0931f40e2deb9b4140440cecd" + +[[package]] +name = "anstyle-parse" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "938874ff5980b03a87c5524b3ae5b59cf99b1d6bc836848df7bc5ada9643c333" +dependencies = [ + "utf8parse", +] + +[[package]] +name = "anstyle-query" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ca11d4be1bab0c8bc8734a9aa7bf4ee8316d462a08c6ac5052f888fef5b494b" +dependencies = [ + "windows-sys", +] + +[[package]] +name = "anstyle-wincon" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "180abfa45703aebe0093f79badacc01b8fd4ea2e35118747e5811127f926e188" +dependencies = [ + "anstyle", + "windows-sys", +] + [[package]] name = "anyhow" version = "1.0.72" @@ -89,9 +138,9 @@ dependencies = [ [[package]] name = "ariadne" -version = "0.2.0" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "367fd0ad87307588d087544707bc5fbf4805ded96c7db922b70d368fa1cb5702" +checksum = "72fe02fc62033df9ba41cba57ee19acf5e742511a140c7dbc3a873e19a19a1bd" dependencies = [ "unicode-width", "yansi", @@ -142,6 +191,12 @@ version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" +[[package]] +name = "bitflags" +version = "2.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "630be753d4e58660abd17930c71b647fe46c27ea6b63cc59e1e3851406972e42" + [[package]] name = "blake3" version = "1.4.1" @@ -204,7 +259,7 @@ version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "23170228b96236b5a7299057ac284a321457700bc8c41a4476052f0f4ba5349d" dependencies = [ - "hashbrown 0.12.3", + "hashbrown", "stacker", ] @@ -218,6 +273,12 @@ dependencies = [ "unicode-width", ] +[[package]] +name = "colorchoice" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7" + [[package]] name = "constant_time_eq" version = "0.3.0" @@ -488,21 +549,36 @@ checksum = "a26ae43d7bcc3b814de94796a5e736d4029efb0ee900c12e2d54c993ad1a1e07" [[package]] name = "enum-as-inner" -version = "0.5.1" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9720bba047d567ffc8a3cba48bf19126600e249ab7f128e9233e6376976a116" +checksum = "5ffccbb6966c05b32ef8fbac435df276c4ae4d3dc55a8cd0eb9745e6c12f546a" dependencies = [ "heck", "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.27", ] [[package]] -name = "equivalent" -version = "1.0.1" +name = "errno" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" +checksum = "6b30f669a7961ef1631673d2766cc92f52d64f7ef354d4fe0ddfd30ed52f0f4f" +dependencies = [ + "errno-dragonfly", + "libc", + "windows-sys", +] + +[[package]] +name = "errno-dragonfly" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa68f1b12764fab894d2755d2518754e71b4fd80ecfb822714a1206c2aab39bf" +dependencies = [ + "cc", + "libc", +] [[package]] name = "fnv" @@ -555,12 +631,6 @@ dependencies = [ "ahash", ] -[[package]] -name = "hashbrown" -version = "0.14.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" - [[package]] name = "heck" version = "0.4.1" @@ -603,13 +673,14 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" [[package]] -name = "indexmap" -version = "2.0.0" +name = "is-terminal" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5477fe2230a79769d8dc68e0eabf5437907c0457a5614a9e8dddb67f65eb65d" +checksum = "cb0889898416213fab133e1d33a0e5858a48177452750691bde3666d0fdbaf8b" dependencies = [ - "equivalent", - "hashbrown 0.14.0", + "hermit-abi", + "rustix", + "windows-sys", ] [[package]] @@ -621,6 +692,15 @@ dependencies = [ "either", ] +[[package]] +name = "itertools" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" +dependencies = [ + "either", +] + [[package]] name = "itoa" version = "1.0.9" @@ -657,6 +737,12 @@ dependencies = [ "cc", ] +[[package]] +name = "linux-raw-sys" +version = "0.4.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57bcfdad1b858c2db7c38303a6d2ad4dfaf5eb53dfeb0910128b2c26d6158503" + [[package]] name = "log" version = "0.4.19" @@ -708,7 +794,7 @@ version = "0.24.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fa52e972a9a719cecb6864fb88568781eb706bac2cd1d4f04a648542dbf78069" dependencies = [ - "bitflags", + "bitflags 1.3.2", "cfg-if", "libc", ] @@ -720,7 +806,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f346ff70e7dbfd675fe90590b92d59ef2de15a8779ae305ebcbfd3f0caf59be4" dependencies = [ "autocfg", - "bitflags", + "bitflags 1.3.2", "cfg-if", "libc", "memoffset 0.6.5", @@ -787,31 +873,55 @@ dependencies = [ ] [[package]] -name = "prql-compiler" -version = "0.8.1" +name = "prql-ast" +version = "0.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c99b52154002ac7f286dd2293c2f8d4e30526c1d396b14deef5ada1deef3c9ff" +checksum = "71194e75f14dbe7debdf2b5eca0812c978021a1bd23d6fe1da98b58e407e035a" dependencies = [ + "enum-as-inner", + "semver", + "serde", + "strum", +] + +[[package]] +name = "prql-compiler" +version = "0.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ff28e838b1be4227cc567a75c11caa3be25c5015f0e5fd21279c06e944ba44f" +dependencies = [ + "anstream", "anyhow", "ariadne", - "chumsky", "csv", "enum-as-inner", - "itertools", - "lazy_static", + "itertools 0.11.0", "log", "once_cell", + "prql-ast", + "prql-parser", "regex", "semver", "serde", "serde_json", - "serde_yaml", "sqlformat", "sqlparser", "strum", "strum_macros", ] +[[package]] +name = "prql-parser" +version = "0.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3182e2ef0465a960eb02519b18768e39123d3c3a0037a2d2934055a3ef901870" +dependencies = [ + "chumsky", + "itertools 0.11.0", + "prql-ast", + "semver", +] + [[package]] name = "psm" version = "0.1.21" @@ -858,7 +968,7 @@ version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a" dependencies = [ - "bitflags", + "bitflags 1.3.2", ] [[package]] @@ -907,6 +1017,19 @@ version = "0.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" +[[package]] +name = "rustix" +version = "0.38.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ee020b1716f0a80e2ace9b03441a749e402e86712f15f16fe8a8f75afac732f" +dependencies = [ + "bitflags 2.3.3", + "errno", + "libc", + "linux-raw-sys", + "windows-sys", +] + [[package]] name = "rustversion" version = "1.0.14" @@ -971,19 +1094,6 @@ dependencies = [ "serde", ] -[[package]] -name = "serde_yaml" -version = "0.9.25" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a49e178e4452f45cb61d0cd8cebc1b0fafd3e41929e996cef79aa3aca91f574" -dependencies = [ - "indexmap", - "itoa", - "ryu", - "serde", - "unsafe-libyaml", -] - [[package]] name = "skim" version = "0.10.4" @@ -991,7 +1101,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5d28de0a6cb2cdd83a076f1de9d965b973ae08b244df1aa70b432946dda0f32" dependencies = [ "beef", - "bitflags", + "bitflags 1.3.2", "chrono", "crossbeam", "defer-drop", @@ -1015,16 +1125,16 @@ version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0c12bc9199d1db8234678b7051747c07f517cdcf019262d1847b94ec8b1aee3e" dependencies = [ - "itertools", + "itertools 0.10.5", "nom", "unicode_categories", ] [[package]] name = "sqlparser" -version = "0.33.0" +version = "0.36.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "355dc4d4b6207ca8a3434fc587db0a8016130a574dbcdbfb93d7f7b5bc5b211a" +checksum = "2eaa1e88e78d2c2460d78b7dc3f0c08dbb606ab4222f9aff36f420d36e307d87" dependencies = [ "log", "serde", @@ -1051,24 +1161,24 @@ checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" [[package]] name = "strum" -version = "0.24.1" +version = "0.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" +checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" dependencies = [ "strum_macros", ] [[package]] name = "strum_macros" -version = "0.24.3" +version = "0.25.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" +checksum = "6069ca09d878a33f883cc06aaa9718ede171841d3832450354410b718b097232" dependencies = [ "heck", "proc-macro2", "quote", "rustversion", - "syn 1.0.109", + "syn 2.0.27", ] [[package]] @@ -1191,7 +1301,7 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5e19c6ab038babee3d50c8c12ff8b910bdb2196f62278776422f50390d8e53d8" dependencies = [ - "bitflags", + "bitflags 1.3.2", "lazy_static", "log", "nix 0.24.3", @@ -1223,12 +1333,6 @@ version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39ec24b3121d976906ece63c9daad25b85969647682eee313cb5779fdd69e14e" -[[package]] -name = "unsafe-libyaml" -version = "0.2.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f28467d3e1d3c6586d8f25fa243f544f5800fec42d97032474e17222c2b75cfa" - [[package]] name = "utf8parse" version = "0.2.1" @@ -1368,6 +1472,15 @@ dependencies = [ "windows-targets", ] +[[package]] +name = "windows-sys" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9" +dependencies = [ + "windows-targets", +] + [[package]] name = "windows-targets" version = "0.48.1" diff --git a/rust/prql/Cargo.toml b/rust/prql/Cargo.toml index 314d1b52391..f29aa4aaef9 100644 --- a/rust/prql/Cargo.toml +++ b/rust/prql/Cargo.toml @@ -1,12 +1,12 @@ [package] +edition = "2021" name = "_ch_rust_prql" version = "0.1.0" -edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -prql-compiler = "0.8.1" +prql-compiler = "0.9.3" serde_json = "1.0" [lib] From 94964f1d25a1dc73f6c67b38f954446abf887e3f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Aug 2023 21:09:02 +0200 Subject: [PATCH 1607/2047] Fix the test --- .../01304_polygons_sym_difference.reference | 6 ++-- .../01304_polygons_sym_difference.sql | 36 ++----------------- 2 files changed, 6 insertions(+), 36 deletions(-) diff --git a/tests/queries/0_stateless/01304_polygons_sym_difference.reference b/tests/queries/0_stateless/01304_polygons_sym_difference.reference index f10be156d6f..828e0d5d4d4 100644 --- a/tests/queries/0_stateless/01304_polygons_sym_difference.reference +++ b/tests/queries/0_stateless/01304_polygons_sym_difference.reference @@ -1,7 +1,7 @@ [[[(1,2.9),(1,1),(2.9,1),(3,0),(0,0),(0,3),(1,2.9)]],[[(1,2.9),(1,4),(4,4),(4,1),(2.9,1),(2.6,2),(2,2.6),(1,2.9)]]] -------- MultiPolygon with Polygon -[[[(-40,-40),(-40,-10),(-20,-10.307),(-20,-20),(-10,-20.879),(-10,-40),(-40,-40)]],[[(-20,20),(10,20.879),(10,10),(20,10.307),(20,-20),(-10,-20.879),(-10,-10),(-20,-10.307),(-20,20)]],[[(10,40),(40,40),(40,10),(20,10.307),(20,20),(10,20.879),(10,40)]]] +[(-40,-40),(-40,-10),(-20,-20),(-20,-10.307),(-20,20),(-10,-40),(-10,-20.879),(-10,-10),(10,10),(10,20.879),(10,40),(20,-20),(20,10.307),(20,20),(40,10),(40,40)] -------- MultiPolygon with Polygon with Holes -[[[(-50,-50),(-50,50),(50,50),(50,-50),(-50,-50)],[(-40,-40),(-10,-40),(-10,-20.879),(20,-20),(20,10.307),(40,10),(40,40),(10,40),(10,20.879),(-20,20),(-20,-10.307),(-40,-10),(-40,-40)]],[[(-20,-20),(-20,-10.307),(-10,-10),(-10,-20.879),(-20,-20)]],[[(10,10),(10,20.879),(20,20),(20,10.307),(10,10)]]] +[(-50,-50),(-50,50),(-40,-40),(-40,-10),(-20,-20),(-20,-10.307),(-20,20),(-10,-40),(-10,-20.879),(-10,-10),(10,10),(10,20.879),(10,40),(20,-20),(20,10.307),(20,20),(40,10),(40,40),(50,-50),(50,50)] -------- Polygon with Polygon with Holes -[[[(-50,-50),(-50,50),(50,50),(50,-50),(-50,-50)],[(-40,-40),(-10,-40),(-10,-20.879),(20,-20),(20,10.307),(40,10),(40,40),(10,40),(10,20.879),(-20,20),(-20,-10.307),(-40,-10),(-40,-40)]],[[(-20,-20),(-20,-10.307),(-10,-10),(-10,-20.879),(-20,-20)]],[[(10,10),(10,20.879),(20,20),(20,10.307),(10,10)]]] +[(-50,-50),(-50,50),(-40,-40),(-40,-10),(-20,-20),(-20,-10.307),(-20,20),(-10,-40),(-10,-20.879),(-10,-10),(10,10),(10,20.879),(10,40),(20,-20),(20,10.307),(20,20),(40,10),(40,40),(50,-50),(50,50)] diff --git a/tests/queries/0_stateless/01304_polygons_sym_difference.sql b/tests/queries/0_stateless/01304_polygons_sym_difference.sql index f8d7a487980..d0e022e1439 100644 --- a/tests/queries/0_stateless/01304_polygons_sym_difference.sql +++ b/tests/queries/0_stateless/01304_polygons_sym_difference.sql @@ -1,45 +1,15 @@ --- Tags: no-parallel - -DROP FUNCTION IF EXISTS sortPolygon; -DROP FUNCTION IF EXISTS rotate; -DROP FUNCTION IF EXISTS sortRing; -DROP FUNCTION IF EXISTS sortMultiPolygon; -DROP FUNCTION IF EXISTS normalizeMultiPolygon; - --- in every polygon, the first array is the main part, the rest are - the holes: -CREATE FUNCTION sortPolygon AS arr -> arrayPushFront(arraySort(arrayPopFront(arr)), arr[1]); - --- Rotate array, putting minimum element to the front: -CREATE FUNCTION rotate AS arr -> arraySort((e, i) -> (i < arrayFirstIndex(e -> e = arrayReduce('min', arr), arr), i), arr, arrayEnumerate(arr)); - --- in every ring, the first and the last points are identical, but they are selected randomly in a ring: -CREATE FUNCTION sortRing AS arr -> arrayPushBack(rotate(arrayPopBack(arr)) AS rotated, rotated[1]); - --- normalize the whole multipolygon: -CREATE FUNCTION sortMultiPolygon AS arr -> arraySort(arrayMap(polygon -> sortPolygon(arrayMap(ring -> sortRing(ring), polygon)), arr)); - --- also, cut the floating point numbers: -CREATE FUNCTION normalizeMultiPolygon AS arr -> arrayMap(x -> arrayMap(y -> arrayMap(z -> (round(z.1, 3), round(z.2, 3)), y), x), sortMultiPolygon(arr)); - - select polygonsSymDifferenceCartesian([[[(0, 0),(0, 3),(1, 2.9),(2, 2.6),(2.6, 2),(2.9, 1),(3, 0),(0, 0)]]], [[[(1., 1.),(1., 4.),(4., 4.),(4., 1.),(1., 1.)]]]); -- Google "draw wkt online" select '-------- MultiPolygon with Polygon'; select wkt(polygonsSymDifferenceSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]])) format Null; -SELECT normalizeMultiPolygon(polygonsSymDifferenceSpherical([[[(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)]], [[(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]]], [[[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]])); +SELECT arrayDistinct(arraySort(arrayMap((x, y) -> (round(x, 3), round(y, 3)), arrayFlatten(polygonsSymDifferenceSpherical([[[(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)]], [[(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]]], [[[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]]))))); select '-------- MultiPolygon with Polygon with Holes'; select wkt(polygonsSymDifferenceSpherical([[[(33.473420586689336,58.85424941916091),(32.23422397806246,58.492830557036),(32.173775363007486,58.03176922751564),(31.508840597402823,57.499784781503735),(31.750635057622702,56.86092686957355),(31.508840597402823,55.941082594334574),(32.20399967053497,55.515591939372456),(31.84130798020516,54.998862226280465),(31.418167674820367,54.422670886434275),(32.47601843828233,53.83826377018255),(32.08310244042503,53.408048308050866),(33.171177511414484,52.82758702113742),(34.77306581037117,52.91880107773494),(34.77306581037117,53.784726518357985),(34.108131044766516,54.17574726780569),(35.07530888564602,54.59813930694554),(34.25925258240394,54.96417435716029),(35.01486027059106,55.361278263643584),(33.50364489421682,55.37845402950552),(32.7480372060297,55.90721384574556),(35.67979503619571,55.68634475630185),(32.83871012861215,56.311688992608396),(34.591719965206266,56.29492065473883),(35.7100193437232,56.311688992608396),(33.83611227701915,56.695333481003644),(32.95960735872209,56.9434497616887),(36.072711034053015,57.091531913901434),(33.171177511414484,57.33702717078384),(36.193608264162954,57.499784781503735),(33.23162612646945,57.77481561306047),(36.43540272438284,58.04776787540811),(33.62454212432676,58.27099811968307),(36.344729801800376,58.54018474404165),(33.83611227701915,58.68186423448108),(34.74284150284369,59.565911441555244),(33.473420586689336,58.85424941916091)]], [[(34.65216858026123,58.91672306881671),(37.19101041256995,58.68186423448108),(36.01226241899805,58.28688958537609),(37.16078610504247,58.04776787540811),(35.74024365125068,57.79092907387934),(37.009664567405046,57.499784781503735),(35.77046795877817,57.25537683364851),(36.979440259877556,57.07510745541089),(34.22902827487645,56.794777197297435),(36.7074214921302,56.210968525786996),(34.712617195316206,56.10998276812964),(36.55629995449277,55.63519693782703),(35.13575750070099,55.53270067649592),(36.43540272438284,55.34409504165558),(34.83351442542614,55.01619492319591),(35.61934642114075,54.49294870011772),(34.89396304048112,54.12264226523038),(35.37755196092087,53.046178687628185),(37.43280487278982,52.95523300597458),(35.92158949641559,53.80257986695776),(36.91899164482259,53.856094327816805),(36.01226241899805,54.75541714463799),(37.765272255592166,55.189110239786885),(36.828318722240134,55.44708256557195),(38.03729102333953,55.652253637168315),(36.64697287707522,55.941082594334574),(38.21863686850443,56.05939028508024),(36.37495410932787,56.64551287174558),(38.30930979108689,56.992876013526654),(37.16078610504247,57.25537683364851),(38.127963945921984,57.516020773674256),(37.43280487278982,57.710289827306724),(38.33953409861437,57.935626886818994),(37.40258056526235,58.31865112960426),(38.58132855883426,58.744648733419496),(37.31190764267989,59.02578062465136),(34.65216858026123,58.91672306881671)]], [[(38.52087994377928,59.11898412389468),(39.54850639971376,58.713270635642914),(38.369758406141855,58.28688958537609),(38.85334732658162,58.06375936407028),(38.33953409861437,57.710289827306724),(38.73245009647167,57.48354156434209),(38.21863686850443,57.271721400459285),(38.97424455669155,56.87744603722649),(37.463029180317314,56.5623320541159),(38.94402024916407,56.05939028508024),(38.18841256097694,55.856355210835915),(38.490655636251795,55.53270067649592),(37.795496563119656,55.39562234093384),(38.30930979108689,55.154587013355666),(36.7074214921302,54.65063295250911),(37.31190764267989,53.92734063371401),(36.979440259877556,53.58783775557231),(37.855945178174615,52.91880107773497),(39.57873070724124,52.69956490610895),(38.33953409861437,53.281741738901104),(40.00187101262603,53.35396273604752),(39.54850639971376,53.58783775557231),(40.24366547284591,53.58783775557231),(39.97164670509855,53.98069568468355),(40.60635716317572,54.03398248547225),(40.39478701048334,54.44025165268903),(39.54850639971376,54.56310590284329),(39.54850639971376,54.87732350170489),(40.39478701048334,54.87732350170489),(40.39478701048334,55.24083903654295),(39.82052516746112,55.2752875586599),(39.760076552406154,55.75443792473942),(40.57613285564824,55.78844000174894),(40.425011318010824,56.19415599955667),(39.82052516746112,56.07626182891758),(39.79030085993364,56.41214455508424),(40.48545993306579,56.495655446714636),(40.33433839542836,56.95993246553937),(39.79030085993364,56.992876013526654),(39.72985224487867,57.46729112028032),(40.33433839542836,57.46729112028032),(40.24366547284591,58.04776787540811),(39.63917932229622,58.04776787540811),(39.63917932229622,58.382088724871295),(40.33433839542836,58.382088724871295),(40.45523562553831,58.9011152358548),(38.52087994377928,59.11898412389468)]]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]])) format Null; -SELECT normalizeMultiPolygon(polygonsSymDifferenceSpherical([[(50.,50.),(50.,-50.),(-50.,-50.),(-50.,50.),(50.,50.)],[(10.,10.),(10.,40.),(40.,40.),(40.,10.),(10.,10.)],[(-10.,-10.),(-10.,-40.),(-40.,-40.),(-40.,-10.),(-10.,-10.)]], [[[(-20.,-20.),(-20.,20.),(20.,20.),(20.,-20.),(-20.,-20.)]]])); +SELECT arrayDistinct(arraySort(arrayMap((x, y) -> (round(x, 3), round(y, 3)), arrayFlatten(polygonsSymDifferenceSpherical([[(50.,50.),(50.,-50.),(-50.,-50.),(-50.,50.),(50.,50.)],[(10.,10.),(10.,40.),(40.,40.),(40.,10.),(10.,10.)],[(-10.,-10.),(-10.,-40.),(-40.,-40.),(-40.,-10.),(-10.,-10.)]], [[[(-20.,-20.),(-20.,20.),(20.,20.),(20.,-20.),(-20.,-20.)]]]))))); select '-------- Polygon with Polygon with Holes'; select wkt(polygonsSymDifferenceSpherical([[(29.453587685533865,59.779570356240356),(29.393139070478895,52.276266797422124),(40.636581470703206,59.38168915000267),(41.21084331372543,59.103467777099866),(29.786055068336193,52.146627480315004),(31.23682182965546,52.16517054781818),(41.69443223416517,58.85424941916091),(42.51048853740727,58.47703162291134),(32.59691566839227,52.22075341251539),(34.289476889931414,52.22075341251539),(43.02430176537451,58.07974369546071),(43.02430176537451,57.25537683364851),(35.468224883503325,52.2022335126388),(37.16078610504247,52.23926559241349),(43.02430176537451,56.26136189644947),(43.02430176537451,55.326904361850836),(38.33953409861437,52.16517054781818),(40.09254393520848,52.16517054781818),(44.4146199116388,55.3097062225408),(44.47506852669377,59.80998197603594),(39.72985224487867,59.931351417569715),(30.23941968124846,53.67744677450975),(30.20919537372098,54.63314259659509),(38.73245009647167,59.94649146557819),(37.2816833351524,59.97675082987618),(30.23941968124846,55.2752875586599),(30.33009260383092,56.19415599955667),(36.28428118674541,59.96162460231375),(34.863738732953635,59.97675082987618),(30.178971066193498,56.97640788219866),(30.178971066193498,57.91957806959033),(33.65476643185424,59.94649146557819),(32.32489690064491,59.94649146557819),(30.481214141468342,58.85424941916091),(30.571887064050795,59.99187015036608),(29.453587685533865,59.779570356240356)]], [[(24.367675781249993,61.45977057029751),(19.577636718749993,58.67693767258692),(19.577636718749993,57.492213666700735),(19.445800781249996,55.87531083569678),(19.445800781249996,54.085173420886775),(17.468261718749996,53.014783245859235),(20.017089843749993,51.563412328675895),(21.203613281249993,50.205033264943324),(26.125488281249993,50.40151532278236),(27.22412109374999,48.980216985374994),(32.80517578124999,49.525208341974405),(35.26611328124999,48.74894534343292),(36.93603515624999,49.66762782262194),(42.56103515625,48.77791275550183),(43.92333984374999,49.8096315635631),(47.17529296875,49.152969656170455),(49.28466796875,50.54136296522162),(48.05419921875,51.17934297928929),(51.39404296875,52.48278022207825),(50.64697265625,53.014783245859235),(52.88818359375,53.93021986394004),(51.65771484374999,54.29088164657006),(52.66845703125,55.825973254619015),(50.25146484375,56.145549500679095),(51.92138671875,57.914847767009206),(49.15283203125,58.17070248348605),(49.59228515625,60.086762746260064),(47.043457031249986,59.88893689676584),(43.57177734375,61.37567331572748),(42.64892578125,60.630101766266705),(36.89208984374999,62.000904713685856),(36.01318359374999,61.143235250840576),(31.398925781249993,62.02152819100766),(30.563964843749996,61.05828537037917),(26.872558593749993,61.71070595883174),(26.652832031249993,61.10078883158897),(24.367675781249993,61.45977057029751)], [(24.455566406249993,59.42272750081452),(21.203613281249993,58.49369382056807),(21.335449218749993,56.89700392127261),(21.599121093749993,55.92458580482949),(25.202636718749993,55.998380955359636),(28.850097656249993,57.06463027327854),(27.09228515625,57.844750992890994),(28.806152343749996,59.17592824927138),(26.257324218749993,59.17592824927138),(24.455566406249993,59.42272750081452)], [(35.13427734375,59.84481485969107),(31.970214843749993,58.97266715450152),(33.20068359374999,56.776808316568406),(36.67236328125,56.41390137600675),(39.08935546874999,57.25528054528888),(42.69287109374999,58.03137242177638),(40.89111328124999,59.26588062825809),(37.28759765625,58.722598828043374),(37.11181640624999,59.66774058164964),(35.13427734375,59.84481485969107)], [(29.157714843749993,55.75184939173528),(22.565917968749993,55.128649068488784),(22.565917968749993,53.54030739150019),(22.038574218749996,51.48138289610097),(26.257324218749993,51.42661449707484),(30.124511718749993,50.54136296522162),(32.18994140624999,51.17934297928929),(30.124511718749993,53.173119202640635),(35.09033203124999,53.173119202640635),(33.11279296875,54.085173420886775),(29.597167968749993,55.50374985927513),(29.157714843749993,55.75184939173528)], [(42.82470703125,56.58369172128337),(36.584472656249986,55.329144408405085),(37.99072265625,53.592504809039355),(34.95849609374999,51.48138289610097),(36.54052734374999,50.40151532278236),(39.66064453124999,50.289339253291786),(39.79248046875,52.13348804077148),(41.77001953125,50.68079714532166),(44.49462890624999,51.97134580885171),(47.30712890624999,52.509534770327264),(44.05517578125,53.54030739150019),(46.60400390625,53.696706475303245),(47.61474609375,55.40406982700608),(45.37353515625,55.40406982700608),(42.82470703125,56.58369172128337)]])) format Null; -SELECT normalizeMultiPolygon(polygonsSymDifferenceSpherical([[(50., 50.), (50., -50.), (-50., -50.), (-50., 50.), (50., 50.)], [(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)], [(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]], [[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]])); - -DROP FUNCTION sortPolygon; -DROP FUNCTION rotate; -DROP FUNCTION sortRing; -DROP FUNCTION sortMultiPolygon; -DROP FUNCTION normalizeMultiPolygon; +SELECT arrayDistinct(arraySort(arrayMap((x, y) -> (round(x, 3), round(y, 3)), arrayFlatten(polygonsSymDifferenceSpherical([[(50., 50.), (50., -50.), (-50., -50.), (-50., 50.), (50., 50.)], [(10., 10.), (10., 40.), (40., 40.), (40., 10.), (10., 10.)], [(-10., -10.), (-10., -40.), (-40., -40.), (-40., -10.), (-10., -10.)]], [[(-20., -20.), (-20., 20.), (20., 20.), (20., -20.), (-20., -20.)]]))))); From dfff4704ad28568623dc9fd050d8c2740bbd5d6f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Aug 2023 21:12:24 +0200 Subject: [PATCH 1608/2047] Fix test `02434_cancel_insert_when_client_dies` --- .../queries/0_stateless/02434_cancel_insert_when_client_dies.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh index 5e2da509314..f85aaed7716 100755 --- a/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh +++ b/tests/queries/0_stateless/02434_cancel_insert_when_client_dies.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-random-settings +# Tags: no-random-settings, no-asan, no-msan, no-tsan, no-debug # shellcheck disable=SC2009 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 054562233143594b1ebd83d248567151bfa9adb2 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 4 Aug 2023 13:03:15 -0700 Subject: [PATCH 1609/2047] Add functional tests for hasTokenOrNull --- tests/queries/0_stateless/02816_has_token_empty.reference | 4 ++++ tests/queries/0_stateless/02816_has_token_empty.sql | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/tests/queries/0_stateless/02816_has_token_empty.reference b/tests/queries/0_stateless/02816_has_token_empty.reference index aa47d0d46d4..8435d77c5fe 100644 --- a/tests/queries/0_stateless/02816_has_token_empty.reference +++ b/tests/queries/0_stateless/02816_has_token_empty.reference @@ -1,2 +1,6 @@ 0 +\N +\N 0 +\N +\N diff --git a/tests/queries/0_stateless/02816_has_token_empty.sql b/tests/queries/0_stateless/02816_has_token_empty.sql index e5d6156debd..3e00959126b 100644 --- a/tests/queries/0_stateless/02816_has_token_empty.sql +++ b/tests/queries/0_stateless/02816_has_token_empty.sql @@ -2,6 +2,10 @@ SELECT hasTokenCaseInsensitive('K(G', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('Hello', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('', 'Hello'); +SELECT hasTokenCaseInsensitiveOrNull('Hello', ''); +SELECT hasTokenCaseInsensitiveOrNull('', ''); SELECT hasToken('Hello', ''); -- { serverError BAD_ARGUMENTS } SELECT hasToken('', 'Hello'); SELECT hasToken('', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasTokenOrNull('', ''); +SELECT hasTokenOrNull('Hello', ''); From 9a128fa5544cc0cf411ee367d28a564a5f7f8b98 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 4 Aug 2023 20:42:02 +0000 Subject: [PATCH 1610/2047] Fix ZstdDeflatingWriteBuffer truncating the output sometimes --- src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 115 ++++++------------ src/IO/ZstdDeflatingAppendableWriteBuffer.h | 2 + src/IO/ZstdDeflatingWriteBuffer.cpp | 31 ++--- src/IO/ZstdDeflatingWriteBuffer.h | 2 + .../02735_parquet_encoder.reference | 1 + .../0_stateless/02735_parquet_encoder.sql | 2 + 6 files changed, 54 insertions(+), 99 deletions(-) diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index 81be8d8ce4d..5c4ee6203b3 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -42,50 +42,13 @@ void ZstdDeflatingAppendableWriteBuffer::nextImpl() if (!offset()) return; - input.src = reinterpret_cast(working_buffer.begin()); - input.size = offset(); - input.pos = 0; - if (first_write && append_to_existing_file && isNeedToAddEmptyBlock()) { addEmptyBlock(); first_write = false; } - try - { - bool ended = false; - do - { - out->nextIfAtEnd(); - - output.dst = reinterpret_cast(out->buffer().begin()); - output.size = out->buffer().size(); - output.pos = out->offset(); - - size_t compression_result = ZSTD_compressStream2(cctx, &output, &input, ZSTD_e_flush); - if (ZSTD_isError(compression_result)) - throw Exception( - ErrorCodes::ZSTD_ENCODER_FAILED, - "ZSTD stream decoding failed: error code: {}; ZSTD version: {}", - ZSTD_getErrorName(compression_result), ZSTD_VERSION_STRING); - - first_write = false; - out->position() = out->buffer().begin() + output.pos; - - bool everything_was_compressed = (input.pos == input.size); - bool everything_was_flushed = compression_result == 0; - - ended = everything_was_compressed && everything_was_flushed; - } while (!ended); - } - catch (...) - { - /// Do not try to write next time after exception. - out->position() = out->buffer().begin(); - throw; - } - + flush(ZSTD_e_flush); } ZstdDeflatingAppendableWriteBuffer::~ZstdDeflatingAppendableWriteBuffer() @@ -103,58 +66,58 @@ void ZstdDeflatingAppendableWriteBuffer::finalizeImpl() } else { - try - { - finalizeBefore(); - out->finalize(); - finalizeAfter(); - } - catch (...) - { - /// Do not try to flush next time after exception. - out->position() = out->buffer().begin(); - throw; - } + finalizeBefore(); + out->finalize(); + finalizeAfter(); } } void ZstdDeflatingAppendableWriteBuffer::finalizeBefore() { - next(); - - out->nextIfAtEnd(); - - input.src = reinterpret_cast(working_buffer.begin()); - input.size = offset(); - input.pos = 0; - - output.dst = reinterpret_cast(out->buffer().begin()); - output.size = out->buffer().size(); - output.pos = out->offset(); - /// Actually we can use ZSTD_e_flush here and add empty termination /// block on each new buffer creation for non-empty file unconditionally (without isNeedToAddEmptyBlock). /// However ZSTD_decompressStream is able to read non-terminated frame (we use it in reader buffer), /// but console zstd utility cannot. - size_t remaining = ZSTD_compressStream2(cctx, &output, &input, ZSTD_e_end); - while (remaining != 0) + flush(ZSTD_e_end); +} + +void ZstdDeflatingAppendableWriteBuffer::flush(ZSTD_EndDirective mode) +{ + input.src = reinterpret_cast(working_buffer.begin()); + input.size = offset(); + input.pos = 0; + + try { - if (ZSTD_isError(remaining)) - throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, - "ZSTD stream encoder end failed: error: '{}' ZSTD version: {}", - ZSTD_getErrorName(remaining), ZSTD_VERSION_STRING); - - remaining = ZSTD_compressStream2(cctx, &output, &input, ZSTD_e_end); - - out->position() = out->buffer().begin() + output.pos; - - if (!out->hasPendingData()) + bool ended = false; + do { - out->next(); + out->nextIfAtEnd(); + output.dst = reinterpret_cast(out->buffer().begin()); output.size = out->buffer().size(); output.pos = out->offset(); - } + + size_t compression_result = ZSTD_compressStream2(cctx, &output, &input, mode); + if (ZSTD_isError(compression_result)) + throw Exception( + ErrorCodes::ZSTD_ENCODER_FAILED, + "ZSTD stream decoding failed: error code: {}; ZSTD version: {}", + ZSTD_getErrorName(compression_result), ZSTD_VERSION_STRING); + + out->position() = out->buffer().begin() + output.pos; + + bool everything_was_compressed = (input.pos == input.size); + bool everything_was_flushed = compression_result == 0; + + ended = everything_was_compressed && everything_was_flushed; + } while (!ended); + } + catch (...) + { + /// Do not try to write next time after exception. + out->position() = out->buffer().begin(); + throw; } } diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.h b/src/IO/ZstdDeflatingAppendableWriteBuffer.h index d9c4f32d6da..d082178142b 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.h +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.h @@ -52,6 +52,8 @@ private: /// NOTE: will fill compressed data to the out.working_buffer, but will not call out.next method until the buffer is full void nextImpl() override; + void flush(ZSTD_EndDirective mode); + /// Write terminating ZSTD_e_end: empty block + frame epilogue. BTW it /// should be almost noop, because frame epilogue contains only checksums, /// and they are disabled for this buffer. diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp index 8fec5c5fadb..83d8487e3e7 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingWriteBuffer.cpp @@ -32,13 +32,8 @@ ZstdDeflatingWriteBuffer::ZstdDeflatingWriteBuffer( ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() = default; -void ZstdDeflatingWriteBuffer::nextImpl() +void ZstdDeflatingWriteBuffer::flush(ZSTD_EndDirective mode) { - if (!offset()) - return; - - ZSTD_EndDirective mode = ZSTD_e_flush; - input.src = reinterpret_cast(working_buffer.begin()); input.size = offset(); input.pos = 0; @@ -54,7 +49,6 @@ void ZstdDeflatingWriteBuffer::nextImpl() output.size = out->buffer().size(); output.pos = out->offset(); - size_t compression_result = ZSTD_compressStream2(cctx, &output, &input, mode); if (ZSTD_isError(compression_result)) throw Exception( @@ -78,24 +72,15 @@ void ZstdDeflatingWriteBuffer::nextImpl() } } +void ZstdDeflatingWriteBuffer::nextImpl() +{ + if (offset()) + flush(ZSTD_e_flush); +} + void ZstdDeflatingWriteBuffer::finalizeBefore() { - next(); - - out->nextIfAtEnd(); - - input.src = reinterpret_cast(working_buffer.begin()); - input.size = offset(); - input.pos = 0; - - output.dst = reinterpret_cast(out->buffer().begin()); - output.size = out->buffer().size(); - output.pos = out->offset(); - - size_t remaining = ZSTD_compressStream2(cctx, &output, &input, ZSTD_e_end); - if (ZSTD_isError(remaining)) - throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, "zstd stream encoder end failed: zstd version: {}", ZSTD_VERSION_STRING); - out->position() = out->buffer().begin() + output.pos; + flush(ZSTD_e_end); } void ZstdDeflatingWriteBuffer::finalizeAfter() diff --git a/src/IO/ZstdDeflatingWriteBuffer.h b/src/IO/ZstdDeflatingWriteBuffer.h index ba83c18d354..a66d6085a74 100644 --- a/src/IO/ZstdDeflatingWriteBuffer.h +++ b/src/IO/ZstdDeflatingWriteBuffer.h @@ -37,6 +37,8 @@ private: void finalizeBefore() override; void finalizeAfter() override; + void flush(ZSTD_EndDirective mode); + ZSTD_CCtx * cctx; ZSTD_inBuffer input; ZSTD_outBuffer output; diff --git a/tests/queries/0_stateless/02735_parquet_encoder.reference b/tests/queries/0_stateless/02735_parquet_encoder.reference index c7d79392d85..155699329c1 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.reference +++ b/tests/queries/0_stateless/02735_parquet_encoder.reference @@ -43,6 +43,7 @@ ipv6 Nullable(FixedString(16)) [(2,0,NULL,'','[]')] 1 1 0 1 +5090915589685802007 16159458007063698496 16159458007063698496 BYTE_ARRAY String diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index 3701c685120..c8f6d8983a5 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -147,6 +147,8 @@ insert into function file(compressed_02735.parquet) select concat('aaaaaaaaaaaaa select total_compressed_size < 10000, total_uncompressed_size > 15000 from file(compressed_02735.parquet, ParquetMetadata); insert into function file(compressed_02735.parquet) select concat('aaaaaaaaaaaaaaaa', toString(number)) as s from numbers(1000) settings output_format_parquet_row_group_size = 10000, output_format_parquet_compression_method='none'; select total_compressed_size < 10000, total_uncompressed_size > 15000 from file(compressed_02735.parquet, ParquetMetadata); +insert into function file(compressed_02735.parquet) select if(number%3==1, NULL, 42) as x from numbers(70) settings output_format_parquet_compression_method='zstd'; +select sum(cityHash64(*)) from file(compressed_02735.parquet); -- Single-threaded encoding and Arrow encoder. drop table if exists other_encoders_02735; From 06717fa3ddf7f96cb2e6c34ed8454df05bb686cb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 4 Aug 2023 22:58:22 +0200 Subject: [PATCH 1611/2047] Add `abort_on_error=1` to `TSAN_OPTIONS` for stress tests --- docker/test/base/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index c36ff253aa5..47edbbb14a5 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -19,13 +19,13 @@ RUN apt-get update \ # and MEMORY_LIMIT_EXCEEDED exceptions in Functional tests (total memory limit in Functional tests is ~55.24 GiB). # TSAN will flush shadow memory when reaching this limit. # It may cause false-negatives, but it's better than OOM. -RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1'" >> /etc/environment +RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 abort_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1'" >> /etc/environment RUN echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment RUN echo "MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1'" >> /etc/environment RUN echo "LSAN_OPTIONS='suppressions=/usr/share/clickhouse-test/config/lsan_suppressions.txt'" >> /etc/environment # Sanitizer options for current shell (not current, but the one that will be spawned on "docker run") # (but w/o verbosity for TSAN, otherwise test.reference will not match) -ENV TSAN_OPTIONS='halt_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1' +ENV TSAN_OPTIONS='halt_on_error=1 abort_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1' ENV UBSAN_OPTIONS='print_stacktrace=1' ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1' From 4ed86fea2f9d940fd03f5ce940f4fe2871559055 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 4 Aug 2023 21:01:07 +0000 Subject: [PATCH 1612/2047] Fix Parquet stats for Float32 and Float64 --- src/Processors/Formats/Impl/Parquet/Write.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/Write.cpp b/src/Processors/Formats/Impl/Parquet/Write.cpp index 5ebf2be76d2..22d256b89ed 100644 --- a/src/Processors/Formats/Impl/Parquet/Write.cpp +++ b/src/Processors/Formats/Impl/Parquet/Write.cpp @@ -37,8 +37,10 @@ namespace template struct StatisticsNumeric { - T min = std::numeric_limits::max(); - T max = std::numeric_limits::min(); + T min = std::numeric_limits::has_infinity + ? std::numeric_limits::infinity() : std::numeric_limits::max(); + T max = std::numeric_limits::has_infinity + ? -std::numeric_limits::infinity() : std::numeric_limits::lowest(); void add(SourceType x) { From 555c90a50c200a99d143485a950794238a8dd11c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 4 Aug 2023 23:26:52 +0200 Subject: [PATCH 1613/2047] add to integration tests --- docker/test/integration/runner/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 064538a0448..f67c45f7114 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -130,7 +130,7 @@ COPY misc/ /misc/ # Same options as in test/base/Dockerfile # (in case you need to override them in tests) -ENV TSAN_OPTIONS='halt_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1' +ENV TSAN_OPTIONS='halt_on_error=1 abort_on_error=1 history_size=7 memory_limit_mb=46080 second_deadlock_stack=1' ENV UBSAN_OPTIONS='print_stacktrace=1' ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1' From 63d48a917a4cef332248cec7aac6a2dd4fe0de01 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 00:49:39 +0300 Subject: [PATCH 1614/2047] Update tests/queries/0_stateless/02841_join_filter_set_sparse.sql Co-authored-by: Alexander Gololobov <440544+davenger@users.noreply.github.com> --- tests/queries/0_stateless/02841_join_filter_set_sparse.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02841_join_filter_set_sparse.sql b/tests/queries/0_stateless/02841_join_filter_set_sparse.sql index 6eab29675bb..e1a33998d4f 100644 --- a/tests/queries/0_stateless/02841_join_filter_set_sparse.sql +++ b/tests/queries/0_stateless/02841_join_filter_set_sparse.sql @@ -17,3 +17,6 @@ SELECT countIf(ignore(*) == 0) FROM t1 JOIN t2 ON t1.s = t2.s; SET join_algorithm = 'full_sorting_merge', max_rows_in_set_to_optimize_join = 100_000; SELECT countIf(ignore(*) == 0) FROM t1 JOIN t2 ON t1.s = t2.s; + +DROP TABLE t1; +DROP TABLE t2; From 86036facf276b14353e33f1437829aee74dc1b0f Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Fri, 4 Aug 2023 21:53:08 +0000 Subject: [PATCH 1615/2047] timeout increase --- .../queries/0_stateless/02833_concurrrent_sessions.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh index ffb112763c1..9a51816f0e0 100755 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -92,19 +92,19 @@ export -f http_with_session_id_session; export -f mysql_session; for user in "${TCP_USERS[@]}"; do - timeout 30s bash -c "tcp_session ${user}" >/dev/null 2>&1 & + timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & done for user in "${HTTP_USERS[@]}"; do - timeout 30s bash -c "http_session ${user}" >/dev/null 2>&1 & + timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & done for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do - timeout 30s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & + timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & done for user in "${MYSQL_USERS[@]}"; do - timeout 30s bash -c "mysql_session ${user}" >/dev/null 2>&1 & + timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & done wait From e008ffb2648b2bed0d84382c580014e66b451c68 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Fri, 4 Aug 2023 15:33:31 -0700 Subject: [PATCH 1616/2047] fix test --- tests/queries/0_stateless/02766_prql.sh | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02766_prql.sh b/tests/queries/0_stateless/02766_prql.sh index f8bbd72af4e..85b1167027c 100755 --- a/tests/queries/0_stateless/02766_prql.sh +++ b/tests/queries/0_stateless/02766_prql.sh @@ -21,18 +21,18 @@ INSERT INTO aboba (user_id, message, creation_date, metric) VALUES (101, 'Hello, SET dialect = 'prql'; from aboba -derive [ +derive { a = 2, b = s\"LEFT(message, 2)\" -] -select [ user_id, message, a, b ]; +} +select { user_id, message, a, b }; from aboba filter user_id > 101 group user_id ( - aggregate [ + aggregate { metrics = sum metric - ] + } ); SET dialect = 'clickhouse'; @@ -49,10 +49,10 @@ SELECT '---'; SET dialect = 'prql'; from aboba -select [ user_id, message, metric ] +select { user_id, message, metric } derive creation_date = s\"toTimeZone(creation_date, 'Europe/Amsterdam')\" -select [ user_id, message, creation_date, metric]; +select { user_id, message, creation_date, metric}; from s\"SELECT * FROM system.users\" | select non_existent_column; # {serverError UNKNOWN_IDENTIFIER} from non_existent_table; # {serverError UNKNOWN_TABLE} -" \ No newline at end of file +" From 264bff8c9fcf25e206b86c88e1c3a6f2a5caf8b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 01:43:54 +0200 Subject: [PATCH 1617/2047] Fix a comment --- base/poco/Data/include/Poco/Data/TypeHandler.h | 2 +- programs/format/Format.cpp | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/base/poco/Data/include/Poco/Data/TypeHandler.h b/base/poco/Data/include/Poco/Data/TypeHandler.h index 34f88e986f7..e7633de7018 100644 --- a/base/poco/Data/include/Poco/Data/TypeHandler.h +++ b/base/poco/Data/include/Poco/Data/TypeHandler.h @@ -97,7 +97,7 @@ namespace Data /// /// static void extract(std::size_t pos, Person& obj, const Person& defVal, AbstractExtractor::Ptr pExt) /// { - /// // defVal is the default person we should use if we encunter NULL entries, so we take the individual fields + /// // defVal is the default person we should use if we encounter NULL entries, so we take the individual fields /// // as defaults. You can do more complex checking, ie return defVal if only one single entry of the fields is null etc... /// poco_assert_dbg (!pExt.isNull()); /// std::string lastName; diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 43c66a32302..d7d61bbcd3b 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -163,13 +163,15 @@ int mainEntryClickHouseFormat(int argc, char ** argv) { ASTPtr res = parseQueryAndMovePosition( parser, pos, end, "query", multiple, cmd_settings.max_query_size, cmd_settings.max_parser_depth); - /// For insert query with data(INSERT INTO ... VALUES ...), will lead to format fail, - /// should throw exception early and make exception message more readable. + + /// For insert query with data(INSERT INTO ... VALUES ...), that will lead to the formatting failure, + /// we should throw an exception early, and make exception message more readable. if (const auto * insert_query = res->as(); insert_query && insert_query->data) { throw Exception(DB::ErrorCodes::INVALID_FORMAT_INSERT_QUERY_WITH_DATA, "Can't format ASTInsertQuery with data, since data will be lost"); } + if (!quiet) { if (!backslash) From 712ac83542ad65b41d8bef73a734c283539c1b68 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Sat, 5 Aug 2023 01:01:23 +0000 Subject: [PATCH 1618/2047] added missed comma --- tests/queries/0_stateless/02833_concurrrent_sessions.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh index 9a51816f0e0..d1d571c6985 100755 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest long +# Tags: no-fasttest, long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From f67b51f6703954c8e2b345192318ed651d7336d0 Mon Sep 17 00:00:00 2001 From: jiyoungyoooo <140271086+jiyoungyoooo@users.noreply.github.com> Date: Fri, 4 Aug 2023 22:46:22 -0700 Subject: [PATCH 1619/2047] Update src/TableFunctions/TableFunctionRemote.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/TableFunctions/TableFunctionRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index d88d4190191..13a24e96e4b 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -159,7 +159,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name is missing"); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", help_message); } else { From d479c125784010ed516188e948fa6554e72078c7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 5 Aug 2023 14:11:22 +0300 Subject: [PATCH 1620/2047] Fixed tests --- tests/queries/0_stateless/01655_plan_optimizations.reference | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index be42a656c66..54ca55d2068 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -168,19 +168,23 @@ Filter 3 > one condition of filter is pushed down before LEFT JOIN Join +Join Filter column: notEquals(number, 1) Join > (analyzer) one condition of filter is pushed down before LEFT JOIN Join +Join Filter column: notEquals(number_0, 1_UInt8) 0 0 3 3 > one condition of filter is pushed down before INNER JOIN Join +Join Filter column: notEquals(number, 1) Join > (analyzer) one condition of filter is pushed down before INNER JOIN Join +Join Filter column: notEquals(number_0, 1_UInt8) 3 3 > filter is pushed down before UNION From 9de95879655cb0ebbd895b0e95198e6ad759f09a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 13:54:36 +0200 Subject: [PATCH 1621/2047] Something with tests --- tests/integration/test_zookeeper_config/test.py | 7 +++---- tests/queries/0_stateless/02263_format_insert_settings.sh | 2 ++ 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 5e36a97b3ca..34ac2cfa572 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -58,9 +58,7 @@ def test_chroot_with_same_root(started_cluster): node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) # Replication might take time - assert_eq_with_retry(node1, "select count() from simple", "2\n") - assert_eq_with_retry(node2, "select count() from simple", "2\n") @@ -78,5 +76,6 @@ def test_chroot_with_different_root(started_cluster): for j in range(2): # Second insert to test deduplication node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) - assert node1.query("select count() from simple_different").strip() == "1" - assert node3.query("select count() from simple_different").strip() == "1" + # Replication might take time + assert_eq_with_retry(node1, "select count() from simple_different", "1\n") + assert_eq_with_retry(node3, "select count() from simple_different", "1\n") diff --git a/tests/queries/0_stateless/02263_format_insert_settings.sh b/tests/queries/0_stateless/02263_format_insert_settings.sh index 3d5f780a38c..efb3d39ab6c 100755 --- a/tests/queries/0_stateless/02263_format_insert_settings.sh +++ b/tests/queries/0_stateless/02263_format_insert_settings.sh @@ -23,10 +23,12 @@ function run_format_both() # NOTE: that those queries may work slow, due to stack trace obtaining run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):") + # compatibility run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.") run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") + # and via server (since this is a separate code path) $CLICKHOUSE_CLIENT -q 'drop table if exists data_02263' $CLICKHOUSE_CLIENT -q 'create table data_02263 (key Int) engine=Memory()' From 10af8fcca4fedf6347fe724ed68f1e60b53b2b1a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 14:05:06 +0200 Subject: [PATCH 1622/2047] Fix Python --- tests/integration/test_zookeeper_config/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 34ac2cfa572..3b5ae7186fb 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -58,8 +58,8 @@ def test_chroot_with_same_root(started_cluster): node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) # Replication might take time - assert_eq_with_retry(node1, "select count() from simple", "2\n") - assert_eq_with_retry(node2, "select count() from simple", "2\n") + assert_eq_with_retry(node1, "select count() from simple", "2") + assert_eq_with_retry(node2, "select count() from simple", "2") def test_chroot_with_different_root(started_cluster): @@ -77,5 +77,5 @@ def test_chroot_with_different_root(started_cluster): node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) # Replication might take time - assert_eq_with_retry(node1, "select count() from simple_different", "1\n") - assert_eq_with_retry(node3, "select count() from simple_different", "1\n") + assert_eq_with_retry(node1, "select count() from simple_different", "1") + assert_eq_with_retry(node3, "select count() from simple_different", "1") From aa57fb6602bd0a815fe512b82df8b18893d268eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 14:07:55 +0200 Subject: [PATCH 1623/2047] Maybe better, but not sure --- tests/integration/test_zookeeper_config/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 3b5ae7186fb..4008df3f84f 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -57,8 +57,9 @@ def test_chroot_with_same_root(started_cluster): for j in range(2): # Second insert to test deduplication node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) - # Replication might take time + node1.query("SYSTEM SYNC REPLICA simple") assert_eq_with_retry(node1, "select count() from simple", "2") + node2.query("SYSTEM SYNC REPLICA simple") assert_eq_with_retry(node2, "select count() from simple", "2") @@ -76,6 +77,7 @@ def test_chroot_with_different_root(started_cluster): for j in range(2): # Second insert to test deduplication node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) - # Replication might take time + node1.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node1, "select count() from simple_different", "1") + node2.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node3, "select count() from simple_different", "1") From 0a6d9881a72c9476ffeb6263f924150bf3116f29 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 16:52:44 +0200 Subject: [PATCH 1624/2047] Maybe better, but not sure --- tests/integration/test_zookeeper_config/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 4008df3f84f..0c0f77ec597 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -79,5 +79,5 @@ def test_chroot_with_different_root(started_cluster): node1.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node1, "select count() from simple_different", "1") - node2.query("SYSTEM SYNC REPLICA simple_different") + node3.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node3, "select count() from simple_different", "1") From 1d793e69f25e99ba62a5f31ebcbd90bd27b67ce5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Aug 2023 14:29:49 +0000 Subject: [PATCH 1625/2047] Fix --- src/IO/Archives/LibArchiveReader.cpp | 3 ++- src/Storages/StorageFile.cpp | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index d499211c567..c6e16b46ef7 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -91,12 +91,13 @@ public: close(archive); ); - auto * entry = archive_entry_new(); + struct archive_entry * entry = nullptr; std::vector files; int error = readNextHeader(archive, &entry); while (error == ARCHIVE_OK || error == ARCHIVE_RETRY) { + chassert(entry != nullptr); std::string name = archive_entry_pathname(entry); if (!filter || filter(name)) files.push_back(std::move(name)); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index cdf5188fd7c..5203285a3f5 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -861,7 +861,7 @@ public: { if (!storage->use_table_fd) { - if (!files_iterator->fromArchive()) + if (files_iterator->fromArchive()) { auto [archive, filename] = files_iterator->nextFileFromArchive(); if (archive.empty()) @@ -878,7 +878,7 @@ public: if (file_enumerator == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to find a file in archive {}", archive); - while (file_enumerator->getFileName() != filename) + while (file_enumerator->getFileName() != current_path) { if (!file_enumerator->nextFile()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected file {} is missing from archive {}", filename, archive); From da2cb99c126c6610150d71c736d1e7e671eb2659 Mon Sep 17 00:00:00 2001 From: Jiyoung Yoo Date: Mon, 31 Jul 2023 08:09:57 -0700 Subject: [PATCH 1626/2047] Fix for issue #40220 improve error message for table function remote --- src/TableFunctions/TableFunctionRemote.cpp | 13 +++++++++++-- .../02841_remote_parameter_parsing_error.reference | 0 .../02841_remote_parameter_parsing_error.sql | 14 ++++++++++++++ 3 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 78618d4928f..0d015196b3c 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -175,7 +175,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", help_message); } else { @@ -228,7 +228,16 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr } if (arg_num < args.size()) - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + { + if (is_cluster_function) + { + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); + } + } } if (!cluster_name.empty()) diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql new file mode 100644 index 00000000000..f2ac88b62fb --- /dev/null +++ b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql @@ -0,0 +1,14 @@ +-- Tags: shard + + +select * from remote('127.0.0.1'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', sys); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system.o); -- { serverError UNKNOWN_TABLE } +select * from remote('127.0.0.1', system.one, default); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, ''); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, default, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', '', key1); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, '', key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', pwd, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system, one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From 608b5ace3610cf970e898e34b450f54b693faca5 Mon Sep 17 00:00:00 2001 From: Jiyoung Yoo Date: Sat, 5 Aug 2023 00:26:32 -0700 Subject: [PATCH 1627/2047] Additional fix for issue#40220 --- src/TableFunctions/TableFunctionRemote.cpp | 15 +++++---------- .../02841_remote_parameter_parsing_error.sql | 2 +- 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 0d015196b3c..d00cda2462f 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -175,7 +175,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", help_message); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", static_cast(help_message)); } else { @@ -220,8 +220,10 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr ++arg_num; } - if (arg_num < args.size() && !sharding_key) + if (arg_num < args.size()) { + if (sharding_key) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); sharding_key = args[arg_num]; ++arg_num; } @@ -229,14 +231,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (arg_num < args.size()) { - if (is_cluster_function) - { - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); - } + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } } diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql index f2ac88b62fb..9e467a1f69a 100644 --- a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql +++ b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql @@ -1,7 +1,6 @@ -- Tags: shard -select * from remote('127.0.0.1'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', sys); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', system); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', system.o); -- { serverError UNKNOWN_TABLE } @@ -11,4 +10,5 @@ select * from remote('127.0.0.1', system.one, default, key1); -- { serverError B select * from remote('127.0.0.1', system.one, 'default', '', key1); -- { serverError UNKNOWN_IDENTIFIER } select * from remote('127.0.0.1', system.one, default, '', key1); -- { serverError BAD_ARGUMENTS } select * from remote('127.0.0.1', system.one, 'default', pwd, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', system, one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From f65fa627e762da10ddadb9f4755e5b7fc819d70c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 5 Aug 2023 19:26:13 +0000 Subject: [PATCH 1628/2047] Update version_date.tsv and changelogs after v23.7.3.14-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.7.3.14-stable.md | 23 +++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 5 files changed, 27 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.7.3.14-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index bee4c9d7f1e..795c32f970c 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ esac ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" -ARG VERSION="23.7.2.25" +ARG VERSION="23.7.3.14" ARG PACKAGES="clickhouse-keeper" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index efad16509ea..ff308be2b6c 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="23.7.2.25" +ARG VERSION="23.7.3.14" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 036b159dc03..6a7aa49d070 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -23,7 +23,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="23.7.2.25" +ARG VERSION="23.7.3.14" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.7.3.14-stable.md b/docs/changelogs/v23.7.3.14-stable.md new file mode 100644 index 00000000000..dbe76bd19e7 --- /dev/null +++ b/docs/changelogs/v23.7.3.14-stable.md @@ -0,0 +1,23 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.7.3.14-stable (bd9a510550c) FIXME as compared to v23.7.2.25-stable (8dd1107b032) + +#### Build/Testing/Packaging Improvement +* Backported in [#53025](https://github.com/ClickHouse/ClickHouse/issues/53025): Packing inline cache into docker images sometimes causes strange special effects. Since we don't use it at all, it's good to go. [#53008](https://github.com/ClickHouse/ClickHouse/pull/53008) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix named collections on cluster 23.7 [#52687](https://github.com/ClickHouse/ClickHouse/pull/52687) ([Al Korgun](https://github.com/alkorgun)). +* Fix password leak in show create mysql table [#52962](https://github.com/ClickHouse/ClickHouse/pull/52962) ([Duc Canh Le](https://github.com/canhld94)). +* Fix ZstdDeflatingWriteBuffer truncating the output sometimes [#53064](https://github.com/ClickHouse/ClickHouse/pull/53064) ([Michael Kolupaev](https://github.com/al13n321)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Suspicious DISTINCT crashes from sqlancer [#52636](https://github.com/ClickHouse/ClickHouse/pull/52636) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix Parquet stats for Float32 and Float64 [#53067](https://github.com/ClickHouse/ClickHouse/pull/53067) ([Michael Kolupaev](https://github.com/al13n321)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 3976c676eb5..3c146e5d1c7 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v23.7.3.14-stable 2023-08-05 v23.7.2.25-stable 2023-08-03 v23.7.1.2470-stable 2023-07-27 v23.6.2.18-stable 2023-07-09 From afb1a55bf3789196e2b9f96aea5318ea0a9f9219 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 00:56:33 +0200 Subject: [PATCH 1629/2047] Remove whitespaces --- src/Interpreters/SystemLog.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 437b1b2a6bb..c4c2b0b2499 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -131,8 +131,6 @@ protected: using Base::queue; private: - - /* Saving thread data */ const StorageID table_id; const String storage_def; From 7e90df67a746fb12a9d4ecb6944beeab198d869d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 00:58:26 +0200 Subject: [PATCH 1630/2047] Remove whitespaces --- src/Interpreters/SystemLog.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index c4c2b0b2499..cf287ad7775 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -111,7 +111,7 @@ public: * and new table get created - as if previous table was not exist. */ SystemLog(ContextPtr context_, - const SystemLogSettings& settings_, + const SystemLogSettings & settings_, std::shared_ptr> queue_ = nullptr); /** Append a record into log. From 06d45cfa81c2e21963a1a82bb7fabc1bcb9b115b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 01:29:31 +0200 Subject: [PATCH 1631/2047] Allow creating system logs at startup --- programs/server/config.xml | 3 +++ src/Interpreters/SystemLog.cpp | 6 ++++++ src/Interpreters/SystemLog.h | 3 +++ 3 files changed, 12 insertions(+) diff --git a/programs/server/config.xml b/programs/server/config.xml index 14b8954fc39..3d1c92b135c 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1037,6 +1037,9 @@ + + + false +**Example** ``` sql SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 7ed2deaeda6..88e4ac03fdb 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -559,6 +559,29 @@ Result: └────────────────────────────┘ ``` +## tupleConcat + +Combines tuples passed as arguments. + +``` sql +tupleConcat(tuples) +``` + +**Arguments** + +- `tuples` – Arbitrary number of arguments of [Tuple](../../sql-reference/data-types/tuple.md) type. + +**Example** + +``` sql +SELECT tupleConcat((1, 2), (3, 4), (true, false)) AS res +``` + +``` text +┌─res──────────────────┐ +│ (1,2,3,4,true,false) │ +└──────────────────────┘ +``` ## Distance functions diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 8288d872f18..9eb222d8c09 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -208,6 +208,10 @@ public: { return FunctionFactory::instance().getImpl("mapConcat", context)->build(arguments); } + else if (isTuple(arguments.at(0).type)) + { + return FunctionFactory::instance().getImpl("tupleConcat", context)->build(arguments); + } else return std::make_unique( FunctionConcat::create(context), collections::map(arguments, [](const auto & elem) { return elem.type; }), return_type); diff --git a/src/Functions/tupleConcat.cpp b/src/Functions/tupleConcat.cpp new file mode 100644 index 00000000000..0556f4181e6 --- /dev/null +++ b/src/Functions/tupleConcat.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; +} + +/// tupleConcat(tup1, ...) - concatenate tuples. +class FunctionTupleConcat : public IFunction +{ +public: + static constexpr auto name = "tupleConcat"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} requires at least one argument.", + getName()); + + DataTypes tuple_arg_types; + + for (const auto arg_idx : collections::range(0, arguments.size())) + { + const auto * arg = arguments[arg_idx].get(); + if (!isTuple(arg)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument {} of function {}", + arg->getName(), + arg_idx + 1, + getName()); + + const auto * type = checkAndGetDataType(arg); + for (const auto & elem : type->getElements()) + tuple_arg_types.push_back(elem); + } + + return std::make_shared(tuple_arg_types); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + { + const size_t num_arguments = arguments.size(); + Columns columns; + + for (size_t i = 0; i < num_arguments; i++) + { + const DataTypeTuple * arg_type = checkAndGetDataType(arguments[i].type.get()); + + if (!arg_type) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument {} of function {}", + arguments[i].type->getName(), + i + 1, + getName()); + + ColumnPtr arg_col = arguments[i].column->convertToFullColumnIfConst(); + const ColumnTuple * tuple_col = checkAndGetColumn(arg_col.get()); + + if (!tuple_col) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of argument of function {}", + arguments[i].column->getName(), + getName()); + + for (const auto & inner_col : tuple_col->getColumns()) + columns.push_back(inner_col); + } + + return ColumnTuple::create(columns); + } +}; + +REGISTER_FUNCTION(TupleConcat) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02833_tuple_concat.reference b/tests/queries/0_stateless/02833_tuple_concat.reference new file mode 100644 index 00000000000..2c865f13ffc --- /dev/null +++ b/tests/queries/0_stateless/02833_tuple_concat.reference @@ -0,0 +1,6 @@ +(1,'y',2,'n') +(1,'y',2,'n',3,'n') +(1,2,3,'a','b','c','2020-10-08','2020-11-08') 1 2 3 a b c 2020-10-08 2020-11-08 +(1,2,1,2) 1 2 1 2 +(1,2,3,4) 1 2 3 4 +(3,4,1,2) 3 4 1 2 diff --git a/tests/queries/0_stateless/02833_tuple_concat.sql b/tests/queries/0_stateless/02833_tuple_concat.sql new file mode 100644 index 00000000000..df43e08d595 --- /dev/null +++ b/tests/queries/0_stateless/02833_tuple_concat.sql @@ -0,0 +1,23 @@ +SELECT tupleConcat(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tupleConcat((1, 'y'), 1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT tupleConcat((1, 'y'), (2, 'n')); +SELECT tupleConcat((1, 'y'), (2, 'n'), (3, 'n')); + +WITH (1,2,3) || ('a','b','c') || ('2020-10-08'::Date, '2020-11-08'::Date) AS t +SELECT t, t.1, t.2, t.3, t.4, t.5, t.6, t.7, t.8; + +DROP TABLE IF EXISTS t_02833; +CREATE TABLE t_02833 (tup Tuple(a UInt64, b UInt64)) ENGINE=Log; +INSERT INTO t_02833 VALUES ((1, 2)); + +WITH (tup || tup) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +WITH (tup || (3, 4)) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +WITH ((3, 4) || tup) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +DROP TABLE t_02833; From 0498b48c70aa60b49ae5a2f6ee702f564921062d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 9 Aug 2023 21:01:58 +0000 Subject: [PATCH 1798/2047] different linker --- cmake/linux/toolchain-s390x.cmake | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index c1cbb849bd3..f5e1cf660c9 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -22,6 +22,8 @@ set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") +set (CMAKE_STATIC_LINKER_FLAGS "${CMAKE_STATIC_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 481792181e6ac9515db966817d123afb99d9971f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 23:12:39 +0200 Subject: [PATCH 1799/2047] Remove garbage --- .../test_cluster_copier/test_two_nodes.py | 21 - .../test_cassandra.py | 99 - .../test.py | 68 - .../test_default_reading.py | 1 - .../test_default_string.py | 1 - .../test_dict_get.py | 1 - .../test_dict_get_or_default.py | 1 - .../test_extreme_deduplication/test.py | 70 - tests/integration/test_hive_query/__init__.py | 0 .../test_hive_query/configs/config.xml | 32 - .../test_hive_query/configs/hdfs-site.xml | 6 - .../test_hive_query/data/prepare_hive_data.sh | 15 - tests/integration/test_hive_query/test.py | 498 ----- .../__init__.py | 0 .../configs/host_regexp.xml | 11 - .../configs/listen_host.xml | 5 - .../coredns_config/Corefile | 8 - .../coredns_config/example.com | 1 - .../test.py | 115 - .../test.py | 68 - .../test.py | 78 - .../test_mutations_with_merge_tree/test.py | 194 -- .../test_polymorphic_parts/test.py | 48 - .../__init__.py | 0 .../configs/config.d/storage_conf.xml | 93 - .../test.py | 322 --- .../test_reverse_dns_query/__init__.py | 0 .../test_reverse_dns_query/configs/config.xml | 3 - .../configs/listen_host.xml | 5 - .../configs/reverse_dns_function.xml | 3 - .../coredns_config/Corefile | 4 - .../test_reverse_dns_query/test.py | 56 - .../test_s3_zero_copy_replication/test.py | 90 - .../test_s3_zero_copy_ttl/__init__.py | 0 .../configs/max_delayed_streams.xml | 9 - .../test_s3_zero_copy_ttl/configs/s3.xml | 39 - .../integration/test_s3_zero_copy_ttl/test.py | 94 - .../test_ttl_move_memory_usage.py | 81 - .../test_vertical_merge_memory_usage.py | 46 - .../integration/test_storage_nats/__init__.py | 0 .../clickhouse_path/format_schemas/nats.proto | 6 - .../test_storage_nats/configs/macros.xml | 7 - .../configs/named_collection.xml | 13 - .../test_storage_nats/configs/nats.xml | 6 - .../test_storage_nats/configs/users.xml | 7 - .../test_storage_nats/nats_certs.sh | 13 - .../integration/test_storage_nats/nats_pb2.py | 37 - tests/integration/test_storage_nats/test.py | 1875 ----------------- .../test_nats_json.reference | 50 - .../__init__.py | 0 .../configs/log_conf.xml | 11 - .../test_storage_postgresql_replica/test.py | 780 ------- .../test_postgresql_replica.reference | 50 - .../integration/test_storage_rabbitmq/test.py | 63 - tests/integration/test_ttl_move/test.py | 100 - 55 files changed, 5204 deletions(-) delete mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py delete mode 100644 tests/integration/test_hive_query/__init__.py delete mode 100644 tests/integration/test_hive_query/configs/config.xml delete mode 100644 tests/integration/test_hive_query/configs/hdfs-site.xml delete mode 100755 tests/integration/test_hive_query/data/prepare_hive_data.sh delete mode 100644 tests/integration/test_hive_query/test.py delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/__init__.py delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/test.py delete mode 100644 tests/integration/test_replicated_merge_tree_hdfs_zero_copy/__init__.py delete mode 100644 tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml delete mode 100644 tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py delete mode 100644 tests/integration/test_reverse_dns_query/__init__.py delete mode 100644 tests/integration/test_reverse_dns_query/configs/config.xml delete mode 100644 tests/integration/test_reverse_dns_query/configs/listen_host.xml delete mode 100644 tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml delete mode 100644 tests/integration/test_reverse_dns_query/coredns_config/Corefile delete mode 100644 tests/integration/test_reverse_dns_query/test.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/__init__.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml delete mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/s3.xml delete mode 100644 tests/integration/test_s3_zero_copy_ttl/test.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py delete mode 100644 tests/integration/test_storage_nats/__init__.py delete mode 100755 tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto delete mode 100644 tests/integration/test_storage_nats/configs/macros.xml delete mode 100644 tests/integration/test_storage_nats/configs/named_collection.xml delete mode 100644 tests/integration/test_storage_nats/configs/nats.xml delete mode 100644 tests/integration/test_storage_nats/configs/users.xml delete mode 100755 tests/integration/test_storage_nats/nats_certs.sh delete mode 100644 tests/integration/test_storage_nats/nats_pb2.py delete mode 100644 tests/integration/test_storage_nats/test.py delete mode 100644 tests/integration/test_storage_nats/test_nats_json.reference delete mode 100644 tests/integration/test_storage_postgresql_replica/__init__.py delete mode 100644 tests/integration/test_storage_postgresql_replica/configs/log_conf.xml delete mode 100644 tests/integration/test_storage_postgresql_replica/test.py delete mode 100644 tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 2b6fcf6cac2..1bd3561f24f 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -595,24 +595,3 @@ def execute_task(started_cluster, task, cmd_options): task.check() finally: zk.delete(task.zk_task_path, recursive=True) - - -# Tests -@pytest.mark.skip(reason="Too flaky :(") -def test_different_schema(started_cluster): - execute_task(started_cluster, TaskWithDifferentSchema(started_cluster), []) - - -@pytest.mark.skip(reason="Too flaky :(") -def test_ttl_columns(started_cluster): - execute_task(started_cluster, TaskTTL(started_cluster), []) - - -@pytest.mark.skip(reason="Too flaky :(") -def test_skip_index(started_cluster): - execute_task(started_cluster, TaskSkipIndex(started_cluster), []) - - -@pytest.mark.skip(reason="Too flaky :(") -def test_ttl_move_to_volume(started_cluster): - execute_task(started_cluster, TaskTTLMoveToVolume(started_cluster), []) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py deleted file mode 100644 index 90287e19bd0..00000000000 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ /dev/null @@ -1,99 +0,0 @@ -import os -import math -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/33006 - -pytestmark = pytest.mark.skip - -from .common import * - -from helpers.cluster import ClickHouseCluster -from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout -from helpers.external_sources import SourceCassandra - -SOURCE = None -cluster = None -node = None -simple_tester = None -complex_tester = None -ranged_tester = None -test_name = "cassandra" - - -def setup_module(module): - global cluster - global node - global simple_tester - global complex_tester - global ranged_tester - - cluster = ClickHouseCluster(__file__) - - SOURCE = SourceCassandra( - "Cassandra", - None, - cluster.cassandra_port, - cluster.cassandra_host, - cluster.cassandra_port, - "", - "", - ) - - simple_tester = SimpleLayoutTester(test_name) - simple_tester.cleanup() - simple_tester.create_dictionaries(SOURCE) - - complex_tester = ComplexLayoutTester(test_name) - complex_tester.create_dictionaries(SOURCE) - - ranged_tester = RangedLayoutTester(test_name) - ranged_tester.create_dictionaries(SOURCE) - # Since that all .xml configs were created - - main_configs = [] - main_configs.append(os.path.join("configs", "disable_ssl_verification.xml")) - - dictionaries = simple_tester.list_dictionaries() - - node = cluster.add_instance( - "cass_node", - main_configs=main_configs, - dictionaries=dictionaries, - with_cassandra=True, - ) - - -def teardown_module(module): - simple_tester.cleanup() - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - simple_tester.prepare(cluster) - complex_tester.prepare(cluster) - ranged_tester.prepare(cluster) - - yield cluster - - finally: - cluster.shutdown() - - -@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_SIMPLE)) -def test_simple(started_cluster, layout_name): - simple_tester.execute(layout_name, node) - - -@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_COMPLEX)) -def test_complex(started_cluster, layout_name): - complex_tester.execute(layout_name, node) - - -@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_RANGED)) -def test_ranged(started_cluster, layout_name): - ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/test.py b/tests/integration/test_dictionaries_complex_key_cache_string/test.py index ae9cd4e7c91..bca7122a2a8 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/test.py +++ b/tests/integration/test_dictionaries_complex_key_cache_string/test.py @@ -27,71 +27,3 @@ def started_cluster(): yield cluster finally: cluster.shutdown() - - -@pytest.mark.skip(reason="SSD cache test can run on disk only") -@pytest.mark.parametrize("type", ["memory", "ssd"]) -def test_memory_consumption(started_cluster, type): - node = started_cluster.instances[f"node_{type}"] - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "w" * 8 - ) - ) - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "x" * 16 - ) - ) - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "y" * 32 - ) - ) - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "z" * 64 - ) - ) - - # Fill dictionary - node.query( - "select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)" - ) - - allocated_first = int( - node.query( - "select bytes_allocated from system.dictionaries where name = 'radars'" - ).strip() - ) - - alloc_array = [] - for i in range(5): - node.query( - "select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)" - ) - - allocated = int( - node.query( - "select bytes_allocated from system.dictionaries where name = 'radars'" - ).strip() - ) - alloc_array.append(allocated) - - # size doesn't grow - assert all(allocated_first >= a for a in alloc_array) - - for i in range(5): - node.query( - "select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)" - ) - - allocated = int( - node.query( - "select bytes_allocated from system.dictionaries where name = 'radars'" - ).strip() - ) - alloc_array.append(allocated) - - # size doesn't grow - assert all(allocated_first >= a for a in alloc_array) diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py index 85c45d5df3c..921fb4e4154 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py @@ -38,7 +38,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_default_reading(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py index 92d681698bc..6eedf63f95c 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -53,7 +53,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_return_real_values(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py index 05f638ec337..8e45af44640 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py @@ -38,7 +38,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_simple_dict_get(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py index 1da8fd3325a..8d6242f4711 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py @@ -38,7 +38,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_simple_dict_get_or_default(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_extreme_deduplication/test.py b/tests/integration/test_extreme_deduplication/test.py index 03451933171..3632369154a 100644 --- a/tests/integration/test_extreme_deduplication/test.py +++ b/tests/integration/test_extreme_deduplication/test.py @@ -71,73 +71,3 @@ def test_deduplication_window_in_seconds(started_cluster): assert TSV.toMat(node.query("SELECT count() FROM simple"))[0][0] == "3" node1.query("""DROP TABLE simple ON CLUSTER test_cluster""") - - -# Currently this test just reproduce incorrect behavior that sould be fixed -@pytest.mark.skip(reason="Flapping test") -def test_deduplication_works_in_case_of_intensive_inserts(started_cluster): - inserters = [] - fetchers = [] - - node1.query( - """ - CREATE TABLE simple ON CLUSTER test_cluster (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id""" - ) - - node1.query("INSERT INTO simple VALUES (0, 0)") - - for node in nodes: - host = node.ip_address - - inserters.append( - CommandRequest( - ["/bin/bash"], - timeout=10, - stdin=""" -set -e -for i in `seq 1000`; do - {} --host {} -q "INSERT INTO simple VALUES (0, 0)" -done -""".format( - cluster.get_client_cmd(), host - ), - ) - ) - - fetchers.append( - CommandRequest( - ["/bin/bash"], - timeout=10, - stdin=""" -set -e -for i in `seq 1000`; do - res=`{} --host {} -q "SELECT count() FROM simple"` - if [[ $? -ne 0 || $res -ne 1 ]]; then - echo "Selected $res elements! Host: {}" 1>&2 - exit -1 - fi; -done -""".format( - cluster.get_client_cmd(), host, node.name - ), - ) - ) - - # There were not errors during INSERTs - for inserter in inserters: - try: - inserter.get_answer() - except QueryTimeoutExceedException: - # Only timeout is accepted - pass - - # There were not errors during SELECTs - for fetcher in fetchers: - try: - fetcher.get_answer() - except QueryTimeoutExceedException: - # Only timeout is accepted - pass - - node1.query("""DROP TABLE simple ON CLUSTER test_cluster""") diff --git a/tests/integration/test_hive_query/__init__.py b/tests/integration/test_hive_query/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_hive_query/configs/config.xml b/tests/integration/test_hive_query/configs/config.xml deleted file mode 100644 index 7de1391e56c..00000000000 --- a/tests/integration/test_hive_query/configs/config.xml +++ /dev/null @@ -1,32 +0,0 @@ - - - - - - - localhost - 9000 - - - - - - - true - /tmp/clickhouse_local_cache,/tmp/clickhouse_local_cache1 - 207374182400 - 1048576 - - - - /etc/clickhouse-server/extra_conf.d/hdfs-site.xml - - - - system - metric_log
- 1000 - 1000 -
- -
diff --git a/tests/integration/test_hive_query/configs/hdfs-site.xml b/tests/integration/test_hive_query/configs/hdfs-site.xml deleted file mode 100644 index 82c525ea414..00000000000 --- a/tests/integration/test_hive_query/configs/hdfs-site.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - dfs.replication - 1 - - diff --git a/tests/integration/test_hive_query/data/prepare_hive_data.sh b/tests/integration/test_hive_query/data/prepare_hive_data.sh deleted file mode 100755 index 495ea201870..00000000000 --- a/tests/integration/test_hive_query/data/prepare_hive_data.sh +++ /dev/null @@ -1,15 +0,0 @@ -#!/bin/bash -hive -e "create database test" - -hive -e "drop table if exists test.demo; create table test.demo(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'; create table test.demo_orc(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'; " -hive -e "drop table if exists test.parquet_demo; create table test.parquet_demo(id string, score int) PARTITIONED BY(day string, hour string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'" -hive -e "drop table if exists test.demo_text; create table test.demo_text(id string, score int, day string)row format delimited fields terminated by ','; load data local inpath '/demo_data.txt' into table test.demo_text " -hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text" - -hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '00' as hour from test.demo;" -hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '01' as hour from test.demo;" - -hive -e "drop table if exists test.test_hive_types; CREATE TABLE test.test_hive_types( f_tinyint tinyint, f_smallint smallint, f_int int, f_integer int, f_bigint bigint, f_float float, f_double double, f_decimal decimal(10,0), f_timestamp timestamp, f_date date, f_string string, f_varchar varchar(100), f_char char(100), f_bool boolean, f_array_int array, f_array_string array, f_array_float array, f_map_int map, f_map_string map, f_map_float map, f_struct struct>) PARTITIONED BY( day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat';" - -hive -e "insert into test.test_hive_types partition(day='2022-02-20') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-20 14:47:04', '2022-02-20', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 10, 'y', 'xyz')); insert into test.test_hive_types partition(day='2022-02-19') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-19 14:47:04', '2022-02-19', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 11, 'y', 'abc'));" - diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py deleted file mode 100644 index 791ae03f9f6..00000000000 --- a/tests/integration/test_hive_query/test.py +++ /dev/null @@ -1,498 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/43541 - -pytestmark = pytest.mark.skip - -import logging -import os - -import time -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV - -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "h0_0_0", - main_configs=["configs/config.xml"], - extra_configs=["configs/hdfs-site.xml", "data/prepare_hive_data.sh"], - with_hive=True, - ) - - logging.info("Starting cluster ...") - cluster.start() - cluster.copy_file_to_container( - "roottesthivequery_hdfs1_1", - "/ClickHouse/tests/integration/test_hive_query/data/prepare_hive_data.sh", - "/prepare_hive_data.sh", - ) - cluster.exec_in_container( - "roottesthivequery_hdfs1_1", ["bash", "-c", "bash /prepare_hive_data.sh"] - ) - yield cluster - finally: - cluster.shutdown() - - -def test_create_parquet_table(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - test_passed = False - for i in range(10): - node.query("set input_format_parquet_allow_missing_columns = true") - result = node.query( - """ -DROP TABLE IF EXISTS default.demo_parquet; -CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - logging.info("create result {}".format(result)) - if result.strip() == "": - test_passed = True - break - time.sleep(60) - assert test_passed - - -def test_create_parquet_table_1(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - for i in range(10): - node.query("set input_format_parquet_allow_missing_columns = true") - result = node.query( - """ -DROP TABLE IF EXISTS default.demo_parquet_parts; -CREATE TABLE default.demo_parquet_parts (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String), `hour` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'parquet_demo') PARTITION BY(day, hour); - """ - ) - logging.info("create result {}".format(result)) - if result.strip() == "": - test_passed = True - break - time.sleep(60) - assert test_passed - - -def test_create_orc_table(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - test_passed = False - for i in range(10): - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_orc; - CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day) - """ - ) - logging.info("create result {}".format(result)) - if result.strip() == "": - test_passed = True - break - time.sleep(60) - - assert test_passed - - -def test_create_text_table(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_text; - CREATE TABLE default.demo_text (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_text') PARTITION BY (tuple()) - """ - ) - logging.info("create result {}".format(result)) - - assert result.strip() == "" - - -def test_parquet_groupby(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT day, count(*) FROM default.demo_parquet group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_parquet_in_filter(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT count(*) FROM default.demo_parquet_parts where day = '2021-11-05' and hour in ('00') - """ - ) - expected_result = """2 -""" - logging.info("query result:{}".format(result)) - assert result == expected_result - - -def test_orc_groupby(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_orc; - CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day); - SELECT day, count(*) FROM default.demo_orc group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -@pytest.mark.parametrize( - "table,use_local_cache_for_remote_storage,enable_orc_file_minmax_index,enable_orc_stripe_minmax_index", - [ - pytest.param( - "demo_orc_no_cache_no_index", - "false", - "false", - "false", - id="demo_orc_no_cache_no_index", - ), - pytest.param( - "demo_orc_with_cache_no_index", - "true", - "false", - "false", - id="demo_orc_with_cache_no_index", - ), - pytest.param( - "demo_orc_no_cache_file_index", - "false", - "true", - "false", - id="demo_orc_no_cache_file_index", - ), - pytest.param( - "demo_orc_with_cache_file_index", - "true", - "true", - "false", - id="demo_orc_with_cache_file_index", - ), - pytest.param( - "demo_orc_no_cache_stripe_index", - "false", - "true", - "true", - id="demo_orc_no_cache_stripe_index", - ), - pytest.param( - "demo_orc_with_cache_stripe_index", - "true", - "true", - "true", - id="demo_orc_with_cache_stripe_index", - ), - ], -) -def test_orc_minmax_index( - started_cluster, - table, - use_local_cache_for_remote_storage, - enable_orc_file_minmax_index, - enable_orc_stripe_minmax_index, -): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.{table}; - CREATE TABLE default.{table} (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day) - SETTINGS enable_orc_file_minmax_index = {enable_orc_file_minmax_index}, enable_orc_stripe_minmax_index = {enable_orc_stripe_minmax_index}; - """.format( - table=table, - enable_orc_file_minmax_index=enable_orc_file_minmax_index, - enable_orc_stripe_minmax_index=enable_orc_stripe_minmax_index, - ) - ) - assert result.strip() == "" - - for i in range(2): - result = node.query( - """ - SELECT day, id, score FROM default.{table} where day >= '2021-11-05' and day <= '2021-11-16' and score >= 15 and score <= 30 order by day, id - SETTINGS use_local_cache_for_remote_storage = {use_local_cache_for_remote_storage} - """.format( - table=table, - use_local_cache_for_remote_storage=use_local_cache_for_remote_storage, - ) - ) - - assert ( - result - == """2021-11-05 abd 15 -2021-11-16 aaa 22 -""" - ) - - -@pytest.mark.parametrize( - "table,use_local_cache_for_remote_storage,enable_parquet_rowgroup_minmax_index", - [ - pytest.param( - "demo_parquet_no_cache_no_index", - "false", - "false", - id="demo_parquet_no_cache_no_index", - ), - pytest.param( - "demo_parquet_with_cache_no_index", - "true", - "false", - id="demo_parquet_with_cache_no_index", - ), - pytest.param( - "demo_parquet_no_cache_rowgroup_index", - "false", - "true", - id="demo_parquet_no_cache_rowgroup_index", - ), - pytest.param( - "demo_parquet_with_cache_rowgroup_index", - "true", - "true", - id="demo_parquet_with_cache_rowgroup_index", - ), - ], -) -def test_parquet_minmax_index( - started_cluster, - table, - use_local_cache_for_remote_storage, - enable_parquet_rowgroup_minmax_index, -): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.{table}; - CREATE TABLE default.{table} (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - SETTINGS enable_parquet_rowgroup_minmax_index = {enable_parquet_rowgroup_minmax_index} - """.format( - table=table, - enable_parquet_rowgroup_minmax_index=enable_parquet_rowgroup_minmax_index, - ) - ) - assert result.strip() == "" - - for i in range(2): - result = node.query( - """ - SELECT day, id, score FROM default.{table} where day >= '2021-11-05' and day <= '2021-11-16' and score >= 15 and score <= 30 order by day, id - SETTINGS use_local_cache_for_remote_storage = {use_local_cache_for_remote_storage} - """.format( - table=table, - use_local_cache_for_remote_storage=use_local_cache_for_remote_storage, - ) - ) - - assert ( - result - == """2021-11-05 abd 15 -2021-11-16 aaa 22 -""" - ) - - -def test_hive_columns_prunning(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT count(*) FROM default.demo_parquet_parts where day = '2021-11-05' - """ - ) - expected_result = """4 -""" - logging.info("query result:{}".format(result)) - assert result == expected_result - - -def test_text_count(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_orc; - CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day); - SELECT day, count(*) FROM default.demo_orc group by day order by day SETTINGS format_csv_delimiter = '\x01' - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_parquet_groupby_with_cache(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT day, count(*) FROM default.demo_parquet group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_parquet_groupby_by_hive_function(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT day, count(*) FROM hive('thrift://hivetest:9083', 'test', 'demo', '`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)', 'day') group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_cache_read_bytes(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - test_passed = False - for i in range(10): - result = node.query( - """ - SELECT * FROM default.demo_parquet_1 settings input_format_parquet_allow_missing_columns = true - """ - ) - node.query("system flush logs") - result = node.query( - "select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0" - ) - if result.strip() == "0": - logging.info("ProfileEvent_ExternalDataSourceLocalCacheReadBytes == 0") - time.sleep(10) - continue - test_passed = True - break - assert test_passed - - -def test_cache_dir_use(started_cluster): - node = started_cluster.instances["h0_0_0"] - result0 = node.exec_in_container( - ["bash", "-c", "ls /tmp/clickhouse_local_cache | wc -l"] - ) - result1 = node.exec_in_container( - ["bash", "-c", "ls /tmp/clickhouse_local_cache1 | wc -l"] - ) - assert result0 != "0" and result1 != "0" - - -def test_hive_struct_type(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - CREATE TABLE IF NOT EXISTS default.test_hive_types (`f_tinyint` Int8, `f_smallint` Int16, `f_int` Int32, `f_integer` Int32, `f_bigint` Int64, `f_float` Float32, `f_double` Float64, `f_decimal` Float64, `f_timestamp` DateTime, `f_date` Date, `f_string` String, `f_varchar` String, `f_char` String, `f_bool` Boolean, `f_array_int` Array(Int32), `f_array_string` Array(String), `f_array_float` Array(Float32), `f_map_int` Map(String, Int32), `f_map_string` Map(String, String), `f_map_float` Map(String, Float32), `f_struct` Tuple(a String, b Int32, c Float32, d Tuple(x Int32, y String)), `day` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'test_hive_types') PARTITION BY (day) - """ - ) - result = node.query( - """ - SELECT * FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 - """ - ) - expected_result = """1 2 3 4 5 6.11 7.22 8 2022-02-20 14:47:04 2022-02-20 hello world hello world hello world true [1,2,3] ['hello world','hello world'] [1.1,1.2] {'a':100,'b':200,'c':300} {'a':'aa','b':'bb','c':'cc'} {'a':111.1,'b':222.2,'c':333.3} ('aaa',200,333.3,(10,'xyz')) 2022-02-20""" - assert result.strip() == expected_result - - result = node.query( - """ - SELECT day, f_struct.a, f_struct.d.x FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 - """ - ) - expected_result = """2022-02-20 aaa 10""" - - -def test_table_alter_add(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") - result = node.query( - """ -CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - result = node.query( - """ -ALTER TABLE default.demo_parquet_1 ADD COLUMN id Nullable(String) FIRST - """ - ) - result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") - - expected_result = "id\tNullable(String)\t\t\t\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)" - assert result.strip() == expected_result - - -def test_table_alter_drop(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") - result = node.query( - """ -CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - result = node.query( - """ -ALTER TABLE default.demo_parquet_1 DROP COLUMN id - """ - ) - - result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") - expected_result = """score\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)""" - assert result.strip() == expected_result - - -def test_table_alter_comment(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") - result = node.query( - """ -CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - - result = node.query( - """ALTER TABLE default.demo_parquet_1 COMMENT COLUMN id 'Text comment'""" - ) - result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") - expected_result = """id\tNullable(String)\t\t\tText comment\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)""" - assert result.strip() == expected_result diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/__init__.py b/tests/integration/test_host_regexp_multiple_ptr_records/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml deleted file mode 100644 index 7a2141e6c7e..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - - - test1\.example\.com$ - - default - - - \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml deleted file mode 100644 index 58ef55cd3f3..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml +++ /dev/null @@ -1,5 +0,0 @@ - - :: - 0.0.0.0 - 1 - diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile deleted file mode 100644 index 3edf37dafa5..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile +++ /dev/null @@ -1,8 +0,0 @@ -. { - hosts /example.com { - reload "20ms" - fallthrough - } - forward . 127.0.0.11 - log -} diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com deleted file mode 100644 index 9beb415c290..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com +++ /dev/null @@ -1 +0,0 @@ -filled in runtime, but needs to exist in order to be volume mapped in docker \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/test.py b/tests/integration/test_host_regexp_multiple_ptr_records/test.py deleted file mode 100644 index 36f76140a2c..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/test.py +++ /dev/null @@ -1,115 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/51471 - -pytestmark = pytest.mark.skip - -import socket -from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check -from time import sleep -import os - -DOCKER_COMPOSE_PATH = get_docker_compose_path() -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - -cluster = ClickHouseCluster(__file__) - -ch_server = cluster.add_instance( - "clickhouse-server", - with_coredns=True, - main_configs=["configs/listen_host.xml"], - user_configs=["configs/host_regexp.xml"], - ipv6_address="2001:3984:3989::1:1111", -) - -client = cluster.add_instance( - "clickhouse-client", - ipv6_address="2001:3984:3989::1:1112", -) - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def check_ptr_record(ip, hostname): - try: - host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip) - if hostname.lower() == host.lower(): - return True - except socket.herror: - pass - return False - - -def setup_dns_server(ip): - domains_string = "test3.example.com test2.example.com test1.example.com" - example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' - run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) - - # DNS server takes time to reload the configuration. - for try_num in range(10): - if all(check_ptr_record(ip, host) for host in domains_string.split()): - break - sleep(1) - - -def setup_ch_server(dns_server_ip): - ch_server.exec_in_container( - (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) - ) - ch_server.exec_in_container( - (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) - ) - ch_server.query("SYSTEM DROP DNS CACHE") - - -def build_endpoint_v4(ip): - return f"'http://{ip}:8123/?query=SELECT+1&user=test_dns'" - - -def build_endpoint_v6(ip): - return build_endpoint_v4(f"[{ip}]") - - -def test_host_regexp_multiple_ptr_v4_fails_with_wrong_resolution(started_cluster): - server_ip = cluster.get_instance_ip("clickhouse-server") - random_ip = "9.9.9.9" - dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) - - setup_dns_server(random_ip) - setup_ch_server(dns_server_ip) - - endpoint = build_endpoint_v4(server_ip) - - assert "1\n" != client.exec_in_container(["bash", "-c", f"curl {endpoint}"]) - - -def test_host_regexp_multiple_ptr_v4(started_cluster): - server_ip = cluster.get_instance_ip("clickhouse-server") - client_ip = cluster.get_instance_ip("clickhouse-client") - dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) - - setup_dns_server(client_ip) - setup_ch_server(dns_server_ip) - - endpoint = build_endpoint_v4(server_ip) - - assert "1\n" == client.exec_in_container(["bash", "-c", f"curl {endpoint}"]) - - -def test_host_regexp_multiple_ptr_v6(started_cluster): - setup_dns_server(client.ipv6_address) - setup_ch_server(cluster.get_instance_global_ipv6(cluster.coredns_host)) - - endpoint = build_endpoint_v6(ch_server.ipv6_address) - - assert "1\n" == client.exec_in_container(["bash", "-c", f"curl -6 {endpoint}"]) diff --git a/tests/integration/test_insert_into_distributed_through_materialized_view/test.py b/tests/integration/test_insert_into_distributed_through_materialized_view/test.py index a5f92002450..b1eb0df2d43 100644 --- a/tests/integration/test_insert_into_distributed_through_materialized_view/test.py +++ b/tests/integration/test_insert_into_distributed_through_materialized_view/test.py @@ -108,74 +108,6 @@ def test_reconnect(started_cluster): assert remote.query("SELECT count(*) FROM local1").strip() == "3" -@pytest.mark.skip(reason="Flapping test") -def test_inserts_batching(started_cluster): - instance = instance_test_inserts_batching - - with PartitionManager() as pm: - pm.partition_instances(instance, remote) - - instance.query("INSERT INTO local2_source(d, x) VALUES ('2000-01-01', 1)") - # Sleep a bit so that this INSERT forms a batch of its own. - time.sleep(0.2) - - instance.query("INSERT INTO local2_source(x, d) VALUES (2, '2000-01-01')") - - for i in range(3, 7): - instance.query( - "INSERT INTO local2_source(d, x) VALUES ('2000-01-01', {})".format(i) - ) - - for i in range(7, 9): - instance.query( - "INSERT INTO local2_source(x, d) VALUES ({}, '2000-01-01')".format(i) - ) - - instance.query("INSERT INTO local2_source(d, x) VALUES ('2000-01-01', 9)") - - # After ALTER the structure of the saved blocks will be different - instance.query("DROP TABLE local2_view") - instance.query("ALTER TABLE distributed ADD COLUMN s String") - - # Memory Engine doesn't support ALTER so we just DROP/CREATE everything - instance.query("DROP TABLE local2_source") - instance.query( - "CREATE TABLE local2_source (d Date, x UInt32, s String) ENGINE = Memory" - ) - instance.query( - "CREATE MATERIALIZED VIEW local2_view to distributed AS SELECT d,x,s FROM local2_source" - ) - - for i in range(10, 13): - instance.query( - "INSERT INTO local2_source(d, x) VALUES ('2000-01-01', {})".format(i) - ) - - time.sleep(1.0) - - result = remote.query( - "SELECT _part, groupArray(x) FROM local2 GROUP BY _part ORDER BY _part" - ) - - # Explanation: as merges are turned off on remote instance, active parts in local2 table correspond 1-to-1 - # to inserted blocks. - # Batches of max 3 rows are formed as min_insert_block_size_rows = 3. - # Blocks: - # 1. Failed batch that is retried with the same contents. - # 2. Full batch of inserts regardless of the order of columns thanks to the view. - # 3. Full batch of inserts regardless order of columns thanks to the view. - # 4. Full batch of inserts after ALTER (that have different block structure). - # 5. What was left to insert before ALTER. - expected = """\ -20000101_20000101_1_1_0 [1] -20000101_20000101_2_2_0 [2,3,4] -20000101_20000101_3_3_0 [5,6,7] -20000101_20000101_4_4_0 [10,11,12] -20000101_20000101_5_5_0 [8,9] -""" - assert TSV(result) == TSV(expected) - - def test_inserts_local(started_cluster): instance = instance_test_inserts_local_cluster instance.query("INSERT INTO local_source VALUES ('2000-01-01', 1)") diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py index 1ec44d8a002..b099d0513e1 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -41,81 +41,3 @@ def started_cluster(): def get_fake_zk(node): return ku.get_fake_zk(cluster, node) - - -@pytest.mark.skip(reason="test is flaky because changes are not properly waited for") -def test_reconfig_replace_leader_in_one_command(started_cluster): - """ - Remove leader from a cluster of 3 and add a new node to this cluster in a single command - """ - - zk1 = get_fake_zk(node1) - config = ku.get_config_str(zk1) - - assert len(config.split("\n")) == 3 - assert "node1" in config - assert "node2" in config - assert "node3" in config - assert "node4" not in config - - for i in range(100): - zk1.create(f"/test_four_{i}", b"somedata") - - zk2 = get_fake_zk(node2) - zk2.sync("/test_four_0") - ku.wait_configs_equal(config, zk2) - - zk3 = get_fake_zk(node3) - zk3.sync("/test_four_0") - ku.wait_configs_equal(config, zk3) - - for i in range(100): - assert zk2.exists(f"/test_four_{i}") is not None - assert zk3.exists(f"/test_four_{i}") is not None - - assert ku.is_leader(cluster, node1) - node4.start_clickhouse() - config, _ = zk2.reconfig( - joining="server.4=node4:9234", leaving="1", new_members=None - ) - config = config.decode("utf-8") - - print("After removing 1 and adding 4", config) - assert len(config.split("\n")) == 3 - assert "node1" not in config - assert "node2" in config - assert "node3" in config - assert "node4" in config - - ku.wait_until_connected(cluster, node4) - time.sleep(1) - - zk4 = get_fake_zk(node4) - zk4.sync("/test_four_0") - ku.wait_configs_equal(config, zk4) - - for i in range(100): - assert zk4.exists(f"test_four_{i}") is not None - zk4.create(f"/test_four_{100 + i}", b"somedata") - - with pytest.raises(Exception): - zk1.stop() - zk1.close() - zk1 = get_fake_zk(node1) - zk1.sync("/test_four_0") - - zk2.stop() - zk2.close() - zk2 = get_fake_zk(node2) - zk2.sync("/test_four_0") - ku.wait_configs_equal(config, zk2) - - zk3.stop() - zk3.close() - zk3 = get_fake_zk(node3) - zk3.sync("/test_four_0") - ku.wait_configs_equal(config, zk3) - - for i in range(200): - assert zk2.exists(f"test_four_{i}") is not None - assert zk3.exists(f"test_four_{i}") is not None diff --git a/tests/integration/test_mutations_with_merge_tree/test.py b/tests/integration/test_mutations_with_merge_tree/test.py index e7d7abc8c7c..0b4a750b38e 100644 --- a/tests/integration/test_mutations_with_merge_tree/test.py +++ b/tests/integration/test_mutations_with_merge_tree/test.py @@ -27,111 +27,6 @@ def started_cluster(): cluster.shutdown() -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -def test_mutations_in_partition_background(started_cluster): - try: - numbers = 100 - - name = "test_mutations_in_partition" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i} WHERE a = {i}""" - ) - - for i in range(1, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} UPDATE b = 'changed' IN PARTITION {i} WHERE a = {i} """ - ) - - def count_and_changed(): - return instance_test_mutations.query( - f"SELECT count(), countIf(b == 'changed') FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - all_done = False - for wait_times_for_mutation in range( - 100 - ): # wait for replication 80 seconds max - time.sleep(0.8) - - if count_and_changed() == ["66,33"]: - all_done = True - break - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert (count_and_changed(), all_done) == (["66,33"], True) - assert instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() == ["67,67"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") - - -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -@pytest.mark.parametrize("sync", [("last",), ("all",)]) -def test_mutations_in_partition_sync(started_cluster, sync): - try: - numbers = 10 - - name = "test_mutations_in_partition_sync" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i} WHERE a = {i}""" - + (" SETTINGS mutations_sync = 1" if sync == "all" else "") - ) - - for reverse_index, i in reversed( - list(enumerate(reversed(range(1, numbers, 3)))) - ): - instance_test_mutations.query( - f"""ALTER TABLE {name} UPDATE b = 'changed' IN PARTITION {i} WHERE a = {i}""" - + ( - " SETTINGS mutations_sync = 1" - if not reverse_index or sync == "all" - else "" - ) - ) - - def count_and_changed(): - return instance_test_mutations.query( - f"SELECT count(), countIf(b == 'changed') FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert count_and_changed() == ["6,3"] - assert instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() == ["7,7"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") - - def test_mutations_with_merge_background_task(started_cluster): instance_test_mutations.query( """SYSTEM STOP MERGES test_mutations_with_ast_elements""" @@ -190,92 +85,3 @@ def test_mutations_with_truncate_table(started_cluster): ).rstrip() == "0" ) - - -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -def test_mutations_will_not_hang_for_non_existing_parts_sync(started_cluster): - try: - numbers = 100 - - name = "test_mutations_will_not_hang_for_non_existing_parts_sync" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i+1000} WHERE a = {i} SETTINGS mutations_sync = 1""" - ) - - def count(): - return instance_test_mutations.query( - f"SELECT count() FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert count() == [f"{numbers}"] - assert instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() == [f"34,34"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") - - -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -def test_mutations_will_not_hang_for_non_existing_parts_async(started_cluster): - try: - numbers = 100 - - name = "test_mutations_will_not_hang_for_non_existing_parts_async" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i+1000} WHERE a = {i}""" - ) - - def count(): - return instance_test_mutations.query( - f"SELECT count() FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - def count_and_sum_is_done(): - return instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - all_done = False - for wait_times_for_mutation in range( - 100 - ): # wait for replication 80 seconds max - time.sleep(0.8) - - if count_and_sum_is_done() == ["34,34"]: - all_done = True - break - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert count() == [f"{numbers}"] - assert count_and_sum_is_done() == ["34,34"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 48bcca7d505..debb509de90 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -414,54 +414,6 @@ def start_cluster_diff_versions(): cluster.shutdown() -@pytest.mark.skip(reason="compatability is temporary broken") -def test_polymorphic_parts_diff_versions(start_cluster_diff_versions): - # Check that replication with Wide parts works between different versions. - - node_old = node7 - node_new = node8 - - insert_random_data("polymorphic_table", node7, 100) - node8.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20) - - assert node8.query("SELECT count() FROM polymorphic_table") == "100\n" - assert ( - node8.query( - "SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table' and active" - ) - == "Wide\n" - ) - - -@pytest.mark.skip(reason="compatability is temporary broken") -def test_polymorphic_parts_diff_versions_2(start_cluster_diff_versions): - # Replication doesn't work on old version if part is created in compact format, because - # this version doesn't know anything about it. It's considered to be ok. - - node_old = node7 - node_new = node8 - - insert_random_data("polymorphic_table_2", node_new, 100) - - assert node_new.query("SELECT count() FROM polymorphic_table_2") == "100\n" - assert node_old.query("SELECT count() FROM polymorphic_table_2") == "0\n" - with pytest.raises(Exception): - node_old.query("SYSTEM SYNC REPLICA polymorphic_table_2", timeout=3) - - node_old.restart_with_latest_version(fix_metadata=True) - - node_old.query("SYSTEM SYNC REPLICA polymorphic_table_2", timeout=20) - - # Works after update - assert node_old.query("SELECT count() FROM polymorphic_table_2") == "100\n" - assert ( - node_old.query( - "SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table_2' and active" - ) - == "Compact\n" - ) - - def test_polymorphic_parts_non_adaptive(start_cluster): node1.query("SYSTEM STOP MERGES") node2.query("SYSTEM STOP MERGES") diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/__init__.py b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml deleted file mode 100644 index bb4aba94e0b..00000000000 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml +++ /dev/null @@ -1,93 +0,0 @@ - - - - - hdfs - hdfs://hdfs1:9000/clickhouse1/ - - true - - - hdfs - hdfs://hdfs1:9000/clickhouse1/ - - true - - - hdfs - hdfs://hdfs1:9000/clickhouse2/ - - true - - - - - -
- hdfs1 -
-
-
- - -
- default -
- - hdfs1 - -
- 0.0 -
- - -
- hdfs2 -
- - hdfs1 - -
-
- - -
- hdfs1_again -
- - hdfs1 - -
-
-
-
- - - 1024000 - 1 - true - - - - - - - node1 - 9000 - - - - - node2 - 9000 - - - - - - - test_cluster - 1 - - true -
diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py deleted file mode 100644 index eb3d62eb718..00000000000 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py +++ /dev/null @@ -1,322 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/42561 - -pytestmark = pytest.mark.skip - -import logging -from string import Template -import time - -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry - -from pyhdfs import HdfsClient - -SHARDS = 2 -FILES_OVERHEAD_PER_TABLE = 1 # format_version.txt -FILES_OVERHEAD_PER_PART_COMPACT = 7 - - -def wait_for_hdfs_objects(cluster, fp, expected, num_tries=30): - fs = HdfsClient(hosts=cluster.hdfs_ip) - while num_tries > 0: - num_hdfs_objects = len(fs.listdir(fp)) - if num_hdfs_objects == expected: - break - num_tries -= 1 - time.sleep(1) - assert len(fs.listdir(fp)) == expected - - -@pytest.fixture(scope="module") -def cluster(): - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "node1", - main_configs=["configs/config.d/storage_conf.xml"], - macros={"replica": "node1"}, - with_zookeeper=True, - with_hdfs=True, - ) - cluster.add_instance( - "node2", - main_configs=["configs/config.d/storage_conf.xml"], - macros={"replica": "node2"}, - with_zookeeper=True, - with_hdfs=True, - ) - logging.info("Starting cluster...") - cluster.start() - if cluster.instances["node1"].is_debug_build(): - # https://github.com/ClickHouse/ClickHouse/issues/27814 - pytest.skip( - "libhdfs3 calls rand function which does not pass harmful check in debug build" - ) - logging.info("Cluster started") - - fs = HdfsClient(hosts=cluster.hdfs_ip) - fs.mkdirs("/clickhouse1") - fs.mkdirs("/clickhouse2") - logging.info("Created HDFS directory") - - yield cluster - finally: - cluster.shutdown() - - -def test_hdfs_zero_copy_replication_insert(cluster): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - """ - CREATE TABLE hdfs_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/hdfs_test', '{replica}') - ORDER BY (dt, id) - SETTINGS storage_policy='hdfs_only' - """ - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", SHARDS * FILES_OVERHEAD_PER_TABLE - ) - - node1.query("INSERT INTO hdfs_test VALUES (now() - INTERVAL 3 DAY, 10)") - node2.query("SYSTEM SYNC REPLICA hdfs_test", timeout=30) - assert node1.query("SELECT count() FROM hdfs_test FORMAT Values") == "(1)" - assert node2.query("SELECT count() FROM hdfs_test FORMAT Values") == "(1)" - assert ( - node1.query("SELECT id FROM hdfs_test ORDER BY dt FORMAT Values") == "(10)" - ) - assert ( - node2.query("SELECT id FROM hdfs_test ORDER BY dt FORMAT Values") == "(10)" - ) - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hdfs_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hdfs_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - wait_for_hdfs_objects( - cluster, - "/clickhouse1", - SHARDS * FILES_OVERHEAD_PER_TABLE + FILES_OVERHEAD_PER_PART_COMPACT, - ) - finally: - node1.query("DROP TABLE IF EXISTS hdfs_test SYNC") - node2.query("DROP TABLE IF EXISTS hdfs_test SYNC") - - -@pytest.mark.parametrize( - ("storage_policy", "init_objects"), - [("hybrid", 0), ("tiered", 0), ("tiered_copy", FILES_OVERHEAD_PER_TABLE)], -) -def test_hdfs_zero_copy_replication_single_move(cluster, storage_policy, init_objects): - node1 = cluster.instances["node1"] - try: - node1.query( - Template( - """ - CREATE TABLE single_node_move_test (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/single_node_move_test', '{replica}') - ORDER BY (dt, id) - SETTINGS storage_policy='$policy',temporary_directories_lifetime=1 - """ - ).substitute(policy=storage_policy) - ) - wait_for_hdfs_objects(cluster, "/clickhouse1", init_objects) - - node1.query( - "INSERT INTO single_node_move_test VALUES (now() - INTERVAL 3 DAY, 10), (now() - INTERVAL 1 DAY, 11)" - ) - assert ( - node1.query( - "SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values" - ) - == "(10),(11)" - ) - - node1.query( - "ALTER TABLE single_node_move_test MOVE PARTITION ID 'all' TO VOLUME 'external'" - ) - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='single_node_move_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node1.query( - "SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values" - ) - == "(10),(11)" - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT - ) - - node1.query( - "ALTER TABLE single_node_move_test MOVE PARTITION ID 'all' TO VOLUME 'main'" - ) - assert ( - node1.query( - "SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values" - ) - == "(10),(11)" - ) - finally: - node1.query("DROP TABLE IF EXISTS single_node_move_test SYNC") - - -@pytest.mark.parametrize( - ("storage_policy", "init_objects"), - [("hybrid", 0), ("tiered", 0), ("tiered_copy", SHARDS * FILES_OVERHEAD_PER_TABLE)], -) -def test_hdfs_zero_copy_replication_move(cluster, storage_policy, init_objects): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - Template( - """ - CREATE TABLE move_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/move_test', '{replica}') - ORDER BY (dt, id) - SETTINGS storage_policy='$policy' - """ - ).substitute(policy=storage_policy) - ) - wait_for_hdfs_objects(cluster, "/clickhouse1", init_objects) - - node1.query( - "INSERT INTO move_test VALUES (now() - INTERVAL 3 DAY, 10), (now() - INTERVAL 1 DAY, 11)" - ) - node2.query("SYSTEM SYNC REPLICA move_test", timeout=30) - - assert ( - node1.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - assert ( - node2.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - - node1.query( - "ALTER TABLE move_test MOVE PARTITION ID 'all' TO VOLUME 'external'" - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT - ) - - node2.query( - "ALTER TABLE move_test MOVE PARTITION ID 'all' TO VOLUME 'external'" - ) - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='move_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='move_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node1.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - assert ( - node2.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT - ) - finally: - node1.query("DROP TABLE IF EXISTS move_test SYNC") - node2.query("DROP TABLE IF EXISTS move_test SYNC") - - -@pytest.mark.parametrize(("storage_policy"), ["hybrid", "tiered", "tiered_copy"]) -def test_hdfs_zero_copy_with_ttl_move(cluster, storage_policy): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - Template( - """ - CREATE TABLE ttl_move_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/ttl_move_test', '{replica}') - ORDER BY (dt, id) - TTL dt + INTERVAL 2 DAY TO VOLUME 'external' - SETTINGS storage_policy='$policy' - """ - ).substitute(policy=storage_policy) - ) - - node1.query("INSERT INTO ttl_move_test VALUES (now() - INTERVAL 3 DAY, 10)") - node1.query("INSERT INTO ttl_move_test VALUES (now() - INTERVAL 1 DAY, 11)") - - node1.query("OPTIMIZE TABLE ttl_move_test FINAL") - node2.query("SYSTEM SYNC REPLICA ttl_move_test", timeout=30) - - assert_eq_with_retry(node1, "SELECT count() FROM ttl_move_test", "2") - assert_eq_with_retry(node2, "SELECT count() FROM ttl_move_test", "2") - assert ( - node1.query("SELECT id FROM ttl_move_test ORDER BY id FORMAT Values") - == "(10),(11)" - ) - assert ( - node2.query("SELECT id FROM ttl_move_test ORDER BY id FORMAT Values") - == "(10),(11)" - ) - finally: - node1.query("DROP TABLE IF EXISTS ttl_move_test SYNC") - node2.query("DROP TABLE IF EXISTS ttl_move_test SYNC") - - -def test_hdfs_zero_copy_with_ttl_delete(cluster): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - """ - CREATE TABLE ttl_delete_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/ttl_delete_test', '{replica}') - ORDER BY (dt, id) - TTL dt + INTERVAL 2 DAY - SETTINGS storage_policy='tiered' - """ - ) - - node1.query("INSERT INTO ttl_delete_test VALUES (now() - INTERVAL 3 DAY, 10)") - node1.query("INSERT INTO ttl_delete_test VALUES (now() - INTERVAL 1 DAY, 11)") - - node1.query("OPTIMIZE TABLE ttl_delete_test FINAL") - node2.query("SYSTEM SYNC REPLICA ttl_delete_test", timeout=30) - - assert_eq_with_retry(node1, "SELECT count() FROM ttl_delete_test", "1") - assert_eq_with_retry(node2, "SELECT count() FROM ttl_delete_test", "1") - - assert ( - node1.query("SELECT id FROM ttl_delete_test ORDER BY id FORMAT Values") - == "(11)" - ) - assert ( - node2.query("SELECT id FROM ttl_delete_test ORDER BY id FORMAT Values") - == "(11)" - ) - finally: - node1.query("DROP TABLE IF EXISTS ttl_delete_test SYNC") - node2.query("DROP TABLE IF EXISTS ttl_delete_test SYNC") diff --git a/tests/integration/test_reverse_dns_query/__init__.py b/tests/integration/test_reverse_dns_query/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_reverse_dns_query/configs/config.xml b/tests/integration/test_reverse_dns_query/configs/config.xml deleted file mode 100644 index 5ce55afa2a7..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/config.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_reverse_dns_query/configs/listen_host.xml b/tests/integration/test_reverse_dns_query/configs/listen_host.xml deleted file mode 100644 index 58ef55cd3f3..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/listen_host.xml +++ /dev/null @@ -1,5 +0,0 @@ - - :: - 0.0.0.0 - 1 - diff --git a/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml b/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml deleted file mode 100644 index 35d0a07c6a6..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_reverse_dns_query/coredns_config/Corefile b/tests/integration/test_reverse_dns_query/coredns_config/Corefile deleted file mode 100644 index 84d297f7cdf..00000000000 --- a/tests/integration/test_reverse_dns_query/coredns_config/Corefile +++ /dev/null @@ -1,4 +0,0 @@ -. { - forward . 127.0.0.11 - log -} diff --git a/tests/integration/test_reverse_dns_query/test.py b/tests/integration/test_reverse_dns_query/test.py deleted file mode 100644 index 6a39bd7b586..00000000000 --- a/tests/integration/test_reverse_dns_query/test.py +++ /dev/null @@ -1,56 +0,0 @@ -import pytest - -# FIXME This test is broken -# https://github.com/ClickHouse/ClickHouse/issues/53194 - -pytestmark = pytest.mark.skip - -from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check -from time import sleep -import os - -DOCKER_COMPOSE_PATH = get_docker_compose_path() -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - -cluster = ClickHouseCluster(__file__) - -ch_server = cluster.add_instance( - "clickhouse-server", - with_coredns=True, - main_configs=[ - "configs/config.xml", - "configs/reverse_dns_function.xml", - "configs/listen_host.xml", - ], -) - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def setup_ch_server(dns_server_ip): - ch_server.exec_in_container( - (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) - ) - ch_server.exec_in_container( - (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) - ) - ch_server.query("SYSTEM DROP DNS CACHE") - - -def test_reverse_dns_query(started_cluster): - dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) - - setup_ch_server(dns_server_ip) - - for _ in range(0, 200): - response = ch_server.query("select reverseDNSQuery('2001:4860:4860::8888')") - assert response == "['dns.google']\n" diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 2a4e0eece08..0ca81a27802 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -153,96 +153,6 @@ def test_s3_zero_copy_replication(started_cluster, policy): node2.query("DROP TABLE IF EXISTS s3_test SYNC") -@pytest.mark.skip(reason="Test is flaky (and never was stable)") -def test_s3_zero_copy_on_hybrid_storage(started_cluster): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - - node1.query( - """ - CREATE TABLE hybrid_test ON CLUSTER test_cluster (id UInt32, value String) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/hybrid_test', '{}') - ORDER BY id - SETTINGS storage_policy='hybrid',temporary_directories_lifetime=1 - """.format( - "{replica}" - ) - ) - - node1.query("INSERT INTO hybrid_test VALUES (0,'data'),(1,'data')") - node2.query("SYSTEM SYNC REPLICA hybrid_test", timeout=30) - - assert ( - node1.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - assert ( - node2.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','default')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','default')" - ) - - node1.query("ALTER TABLE hybrid_test MOVE PARTITION ID 'all' TO DISK 's31'") - - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','s31')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','default')" - ) - - # Total objects in S3 - s3_objects = get_large_objects_count(cluster, size=0) - - node2.query("ALTER TABLE hybrid_test MOVE PARTITION ID 'all' TO DISK 's31'") - - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','s31')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','s31')" - ) - - # Check that after moving partition on node2 no new obects on s3 - wait_for_large_objects_count(cluster, s3_objects, size=0) - - assert ( - node1.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - assert ( - node2.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - - node1.query("DROP TABLE IF EXISTS hybrid_test SYNC") - node2.query("DROP TABLE IF EXISTS hybrid_test SYNC") - - def insert_data_time(node, table, number_of_mb, time, start=0): values = ",".join( f"({x},{time})" diff --git a/tests/integration/test_s3_zero_copy_ttl/__init__.py b/tests/integration/test_s3_zero_copy_ttl/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml b/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml deleted file mode 100644 index 54f7152690b..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - - 10 - - - diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml deleted file mode 100644 index 7bb7fa875e4..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml +++ /dev/null @@ -1,39 +0,0 @@ - - - - - s3 - http://minio1:9001/root/data/ - minio - minio123 - - - - - - -
- default -
- - s3_disk - -
-
- - -
- s3_disk -
-
-
-
-
- - - true - 1.0 - - - true -
diff --git a/tests/integration/test_s3_zero_copy_ttl/test.py b/tests/integration/test_s3_zero_copy_ttl/test.py deleted file mode 100644 index 04bff4a44fb..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/test.py +++ /dev/null @@ -1,94 +0,0 @@ -#!/usr/bin/env python3 -import time - -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True -) -node2 = cluster.add_instance( - "node2", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True -) -node3 = cluster.add_instance( - "node3", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - finally: - cluster.shutdown() - - -def test_ttl_move_and_s3(started_cluster): - for i, node in enumerate([node1, node2, node3]): - node.query( - """ - CREATE TABLE s3_test_with_ttl (date DateTime, id UInt32, value String) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/s3_test', '{}') - ORDER BY id - PARTITION BY id - TTL date TO DISK 's3_disk' - SETTINGS storage_policy='s3_and_default', temporary_directories_lifetime=1 - """.format( - i - ) - ) - - node1.query("SYSTEM STOP MOVES s3_test_with_ttl") - - node2.query("SYSTEM STOP MOVES s3_test_with_ttl") - - for i in range(30): - if i % 2 == 0: - node = node1 - else: - node = node2 - - node.query( - f"INSERT INTO s3_test_with_ttl SELECT now() + 5, {i}, randomPrintableASCII(1048570)" - ) - - node1.query("SYSTEM SYNC REPLICA s3_test_with_ttl") - node2.query("SYSTEM SYNC REPLICA s3_test_with_ttl") - node3.query("SYSTEM SYNC REPLICA s3_test_with_ttl") - - assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - - node1.query("SYSTEM START MOVES s3_test_with_ttl") - node2.query("SYSTEM START MOVES s3_test_with_ttl") - - assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - - for attempt in reversed(range(5)): - time.sleep(5) - - print( - node1.query( - "SELECT * FROM system.parts WHERE table = 's3_test_with_ttl' FORMAT Vertical" - ) - ) - - minio = cluster.minio_client - objects = minio.list_objects(cluster.minio_bucket, "data/", recursive=True) - counter = 0 - for obj in objects: - print(f"Objectname: {obj.object_name}, metadata: {obj.metadata}") - counter += 1 - - print(f"Total objects: {counter}") - - if counter == 330: - break - - print(f"Attempts remaining: {attempt}") - - assert counter == 330 diff --git a/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py deleted file mode 100644 index 5fbe426074f..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py +++ /dev/null @@ -1,81 +0,0 @@ -#!/usr/bin/env python3 -import time - -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/45887 - -pytestmark = pytest.mark.skip - -from helpers.cluster import ClickHouseCluster - - -single_node_cluster = ClickHouseCluster(__file__) -small_node = single_node_cluster.add_instance( - "small_node", main_configs=["configs/s3.xml"], with_minio=True -) - - -@pytest.fixture(scope="module") -def started_single_node_cluster(): - try: - single_node_cluster.start() - - yield single_node_cluster - finally: - single_node_cluster.shutdown() - - -def test_move_and_s3_memory_usage(started_single_node_cluster): - if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): - pytest.skip("Disabled for debug and sanitizers. Too slow.") - - small_node.query( - "CREATE TABLE s3_test_with_ttl (x UInt32, a String codec(NONE), b String codec(NONE), c String codec(NONE), d String codec(NONE), e String codec(NONE)) engine = MergeTree order by x partition by x SETTINGS storage_policy='s3_and_default'" - ) - - for _ in range(10): - small_node.query( - "insert into s3_test_with_ttl select 0, repeat('a', 100), repeat('b', 100), repeat('c', 100), repeat('d', 100), repeat('e', 100) from zeros(400000) settings max_block_size = 8192, max_insert_block_size=10000000, min_insert_block_size_rows=10000000" - ) - - # After this, we should have 5 columns per 10 * 100 * 400000 ~ 400 MB; total ~2G data in partition - small_node.query( - "optimize table s3_test_with_ttl final", - settings={ - "send_logs_level": "error", - "allow_prefetched_read_pool_for_remote_filesystem": 0, - }, - ) - - small_node.query("system flush logs") - # Will take memory usage from metric_log. - # It is easier then specifying total memory limit (insert queries can hit this limit). - small_node.query("truncate table system.metric_log") - - small_node.query( - "alter table s3_test_with_ttl move partition 0 to volume 'external'", - settings={ - "send_logs_level": "error", - "allow_prefetched_read_pool_for_remote_filesystem": 0, - }, - ) - small_node.query("system flush logs") - max_usage = small_node.query( - """ - select max(m.val - am.val * 4096) from - (select toStartOfMinute(event_time) as time, max(CurrentMetric_MemoryTracking) as val from system.metric_log group by time) as m join - (select toStartOfMinute(event_time) as time, min(value) as val from system.asynchronous_metric_log where metric='jemalloc.arenas.all.pdirty' group by time) as am using time;""" - ) - # 3G limit is a big one. However, we can hit it anyway with parallel s3 writes enabled. - # Also actual value can be bigger because of memory drift. - # Increase it a little bit if test fails. - assert int(max_usage) < 3e9 - res = small_node.query( - "select * from system.errors where last_error_message like '%Memory limit%' limit 1", - settings={ - "allow_prefetched_read_pool_for_remote_filesystem": 0, - }, - ) - assert res == "" diff --git a/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py deleted file mode 100644 index fb9f3eb67b9..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py +++ /dev/null @@ -1,46 +0,0 @@ -#!/usr/bin/env python3 -import time - -import pytest -from helpers.cluster import ClickHouseCluster - - -single_node_cluster = ClickHouseCluster(__file__) -small_node = single_node_cluster.add_instance( - "small_node", - main_configs=["configs/s3.xml"], - user_configs=["configs/max_delayed_streams.xml"], - with_minio=True, -) - - -@pytest.fixture(scope="module") -def started_single_node_cluster(): - try: - single_node_cluster.start() - - yield single_node_cluster - finally: - single_node_cluster.shutdown() - - -def test_vertical_merge_memory_usage(started_single_node_cluster): - if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): - pytest.skip("Disabled for debug and sanitizers. Too slow.") - - small_node.query( - "create table tvm2 (c0 UInt64, c1 UInt64, c2 UInt64, c3 UInt64, c4 UInt64, c5 UInt64, c6 UInt64, c7 UInt64, c8 UInt64, c9 UInt64, c10 UInt64, c11 UInt64, c12 UInt64, c13 UInt64, c14 UInt64, c15 UInt64, c16 UInt64, c17 UInt64, c18 UInt64, c19 UInt64, c20 UInt64, c21 UInt64, c22 UInt64, c23 UInt64, c24 UInt64, c25 UInt64, c26 UInt64, c27 UInt64, c28 UInt64, c29 UInt64, c30 UInt64, c31 UInt64, c32 UInt64, c33 UInt64, c34 UInt64, c35 UInt64, c36 UInt64, c37 UInt64, c38 UInt64, c39 UInt64, c40 UInt64, c41 UInt64, c42 UInt64, c43 UInt64, c44 UInt64, c45 UInt64, c46 UInt64, c47 UInt64, c48 UInt64, c49 UInt64, c50 UInt64, c51 UInt64, c52 UInt64, c53 UInt64, c54 UInt64, c55 UInt64, c56 UInt64, c57 UInt64, c58 UInt64, c59 UInt64, c60 UInt64, c61 UInt64, c62 UInt64, c63 UInt64, c64 UInt64, c65 UInt64, c66 UInt64, c67 UInt64, c68 UInt64, c69 UInt64, c70 UInt64, c71 UInt64, c72 UInt64, c73 UInt64, c74 UInt64, c75 UInt64, c76 UInt64, c77 UInt64, c78 UInt64, c79 UInt64, c80 UInt64, c81 UInt64, c82 UInt64, c83 UInt64, c84 UInt64, c85 UInt64, c86 UInt64, c87 UInt64, c88 UInt64, c89 UInt64, c90 UInt64, c91 UInt64, c92 UInt64, c93 UInt64, c94 UInt64, c95 UInt64, c96 UInt64, c97 UInt64, c98 UInt64, c99 UInt64, c100 UInt64, c101 UInt64, c102 UInt64, c103 UInt64, c104 UInt64, c105 UInt64, c106 UInt64, c107 UInt64, c108 UInt64, c109 UInt64, c110 UInt64, c111 UInt64, c112 UInt64, c113 UInt64, c114 UInt64, c115 UInt64, c116 UInt64, c117 UInt64, c118 UInt64, c119 UInt64, c120 UInt64, c121 UInt64, c122 UInt64, c123 UInt64, c124 UInt64, c125 UInt64, c126 UInt64, c127 UInt64, c128 UInt64, c129 UInt64, c130 UInt64, c131 UInt64, c132 UInt64, c133 UInt64, c134 UInt64, c135 UInt64, c136 UInt64, c137 UInt64, c138 UInt64, c139 UInt64, c140 UInt64, c141 UInt64, c142 UInt64, c143 UInt64, c144 UInt64, c145 UInt64, c146 UInt64, c147 UInt64, c148 UInt64, c149 UInt64, c150 UInt64, c151 UInt64, c152 UInt64, c153 UInt64, c154 UInt64, c155 UInt64, c156 UInt64, c157 UInt64, c158 UInt64, c159 UInt64, c160 UInt64, c161 UInt64, c162 UInt64, c163 UInt64, c164 UInt64, c165 UInt64, c166 UInt64, c167 UInt64, c168 UInt64, c169 UInt64, c170 UInt64, c171 UInt64, c172 UInt64, c173 UInt64, c174 UInt64, c175 UInt64, c176 UInt64, c177 UInt64, c178 UInt64, c179 UInt64, c180 UInt64, c181 UInt64, c182 UInt64, c183 UInt64, c184 UInt64, c185 UInt64, c186 UInt64, c187 UInt64, c188 UInt64, c189 UInt64, c190 UInt64, c191 UInt64, c192 UInt64, c193 UInt64, c194 UInt64, c195 UInt64, c196 UInt64, c197 UInt64, c198 UInt64, c199 UInt64, c200 UInt64, c201 UInt64, c202 UInt64, c203 UInt64, c204 UInt64, c205 UInt64, c206 UInt64, c207 UInt64, c208 UInt64, c209 UInt64, c210 UInt64, c211 UInt64, c212 UInt64, c213 UInt64, c214 UInt64, c215 UInt64, c216 UInt64, c217 UInt64, c218 UInt64, c219 UInt64, c220 UInt64, c221 UInt64, c222 UInt64, c223 UInt64, c224 UInt64, c225 UInt64, c226 UInt64, c227 UInt64, c228 UInt64, c229 UInt64, c230 UInt64, c231 UInt64, c232 UInt64, c233 UInt64, c234 UInt64, c235 UInt64, c236 UInt64, c237 UInt64, c238 UInt64, c239 UInt64, c240 UInt64, c241 UInt64, c242 UInt64, c243 UInt64, c244 UInt64, c245 UInt64, c246 UInt64, c247 UInt64, c248 UInt64, c249 UInt64, c250 UInt64, c251 UInt64, c252 UInt64, c253 UInt64, c254 UInt64, c255 UInt64, c256 UInt64, c257 UInt64, c258 UInt64, c259 UInt64, c260 UInt64, c261 UInt64, c262 UInt64, c263 UInt64, c264 UInt64, c265 UInt64, c266 UInt64, c267 UInt64, c268 UInt64, c269 UInt64, c270 UInt64, c271 UInt64, c272 UInt64, c273 UInt64, c274 UInt64, c275 UInt64, c276 UInt64, c277 UInt64, c278 UInt64, c279 UInt64, c280 UInt64, c281 UInt64, c282 UInt64, c283 UInt64, c284 UInt64, c285 UInt64, c286 UInt64, c287 UInt64, c288 UInt64, c289 UInt64, c290 UInt64, c291 UInt64, c292 UInt64, c293 UInt64, c294 UInt64, c295 UInt64, c296 UInt64, c297 UInt64, c298 UInt64, c299 UInt64) engine = MergeTree order by tuple() settings min_rows_for_wide_part = 10, min_bytes_for_wide_part=0, storage_policy = 's3', vertical_merge_algorithm_min_rows_to_activate=1" - ) - - small_node.query( - "insert into tvm2 select number + 0, number + 1, number + 2, number + 3, number + 4, number + 5, number + 6, number + 7, number + 8, number + 9, number + 10, number + 11, number + 12, number + 13, number + 14, number + 15, number + 16, number + 17, number + 18, number + 19, number + 20, number + 21, number + 22, number + 23, number + 24, number + 25, number + 26, number + 27, number + 28, number + 29, number + 30, number + 31, number + 32, number + 33, number + 34, number + 35, number + 36, number + 37, number + 38, number + 39, number + 40, number + 41, number + 42, number + 43, number + 44, number + 45, number + 46, number + 47, number + 48, number + 49, number + 50, number + 51, number + 52, number + 53, number + 54, number + 55, number + 56, number + 57, number + 58, number + 59, number + 60, number + 61, number + 62, number + 63, number + 64, number + 65, number + 66, number + 67, number + 68, number + 69, number + 70, number + 71, number + 72, number + 73, number + 74, number + 75, number + 76, number + 77, number + 78, number + 79, number + 80, number + 81, number + 82, number + 83, number + 84, number + 85, number + 86, number + 87, number + 88, number + 89, number + 90, number + 91, number + 92, number + 93, number + 94, number + 95, number + 96, number + 97, number + 98, number + 99, number + 100, number + 101, number + 102, number + 103, number + 104, number + 105, number + 106, number + 107, number + 108, number + 109, number + 110, number + 111, number + 112, number + 113, number + 114, number + 115, number + 116, number + 117, number + 118, number + 119, number + 120, number + 121, number + 122, number + 123, number + 124, number + 125, number + 126, number + 127, number + 128, number + 129, number + 130, number + 131, number + 132, number + 133, number + 134, number + 135, number + 136, number + 137, number + 138, number + 139, number + 140, number + 141, number + 142, number + 143, number + 144, number + 145, number + 146, number + 147, number + 148, number + 149, number + 150, number + 151, number + 152, number + 153, number + 154, number + 155, number + 156, number + 157, number + 158, number + 159, number + 160, number + 161, number + 162, number + 163, number + 164, number + 165, number + 166, number + 167, number + 168, number + 169, number + 170, number + 171, number + 172, number + 173, number + 174, number + 175, number + 176, number + 177, number + 178, number + 179, number + 180, number + 181, number + 182, number + 183, number + 184, number + 185, number + 186, number + 187, number + 188, number + 189, number + 190, number + 191, number + 192, number + 193, number + 194, number + 195, number + 196, number + 197, number + 198, number + 199, number + 200, number + 201, number + 202, number + 203, number + 204, number + 205, number + 206, number + 207, number + 208, number + 209, number + 210, number + 211, number + 212, number + 213, number + 214, number + 215, number + 216, number + 217, number + 218, number + 219, number + 220, number + 221, number + 222, number + 223, number + 224, number + 225, number + 226, number + 227, number + 228, number + 229, number + 230, number + 231, number + 232, number + 233, number + 234, number + 235, number + 236, number + 237, number + 238, number + 239, number + 240, number + 241, number + 242, number + 243, number + 244, number + 245, number + 246, number + 247, number + 248, number + 249, number + 250, number + 251, number + 252, number + 253, number + 254, number + 255, number + 256, number + 257, number + 258, number + 259, number + 260, number + 261, number + 262, number + 263, number + 264, number + 265, number + 266, number + 267, number + 268, number + 269, number + 270, number + 271, number + 272, number + 273, number + 274, number + 275, number + 276, number + 277, number + 278, number + 279, number + 280, number + 281, number + 282, number + 283, number + 284, number + 285, number + 286, number + 287, number + 288, number + 289, number + 290, number + 291, number + 292, number + 293, number + 294, number + 295, number + 296, number + 297, number + 298, number + 299 from numbers(20)" - ) - small_node.query("optimize table tvm2 final") - small_node.query("system flush logs") - - # Should be about 25M - res = small_node.query( - "select formatReadableSize(peak_memory_usage), * from system.part_log where table = 'tvm2' and database = currentDatabase() and event_date >= today() - 1 and event_type = 'MergeParts' and peak_memory_usage > 100000000 format Vertical" - ) - - assert res == "" diff --git a/tests/integration/test_storage_nats/__init__.py b/tests/integration/test_storage_nats/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto b/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto deleted file mode 100755 index 090ed917cdd..00000000000 --- a/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto +++ /dev/null @@ -1,6 +0,0 @@ -syntax = "proto3"; - - message ProtoKeyValue { - uint64 key = 1; - string value = 2; - } diff --git a/tests/integration/test_storage_nats/configs/macros.xml b/tests/integration/test_storage_nats/configs/macros.xml deleted file mode 100644 index 4aa547e049e..00000000000 --- a/tests/integration/test_storage_nats/configs/macros.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - nats1:4444 - macro - JSONEachRow - - diff --git a/tests/integration/test_storage_nats/configs/named_collection.xml b/tests/integration/test_storage_nats/configs/named_collection.xml deleted file mode 100644 index 15817f321f0..00000000000 --- a/tests/integration/test_storage_nats/configs/named_collection.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - nats1:4444 - named - JSONEachRow - 111 - 12 - click - house - - - diff --git a/tests/integration/test_storage_nats/configs/nats.xml b/tests/integration/test_storage_nats/configs/nats.xml deleted file mode 100644 index 0a8be9122ad..00000000000 --- a/tests/integration/test_storage_nats/configs/nats.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - click - house - - diff --git a/tests/integration/test_storage_nats/configs/users.xml b/tests/integration/test_storage_nats/configs/users.xml deleted file mode 100644 index 2cef0a6de3c..00000000000 --- a/tests/integration/test_storage_nats/configs/users.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - - 1 - - - diff --git a/tests/integration/test_storage_nats/nats_certs.sh b/tests/integration/test_storage_nats/nats_certs.sh deleted file mode 100755 index 689221c39e4..00000000000 --- a/tests/integration/test_storage_nats/nats_certs.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/bin/bash -set -euxo pipefail - -mkdir -p "${NATS_CERT_DIR}/ca" -mkdir -p "${NATS_CERT_DIR}/nats" -openssl req -newkey rsa:4096 -x509 -days 365 -nodes -batch -keyout "${NATS_CERT_DIR}/ca/ca-key.pem" -out "${NATS_CERT_DIR}/ca/ca-cert.pem" -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=ca" -openssl req -newkey rsa:4096 -nodes -batch -keyout "${NATS_CERT_DIR}/nats/server-key.pem" -out "${NATS_CERT_DIR}/nats/server-req.pem" -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=server" -openssl x509 -req -days 365 -in "${NATS_CERT_DIR}/nats/server-req.pem" -CA "${NATS_CERT_DIR}/ca/ca-cert.pem" -CAkey "${NATS_CERT_DIR}/ca/ca-key.pem" -CAcreateserial -out "${NATS_CERT_DIR}/nats/server-cert.pem" -extfile <( -cat <<-EOF -subjectAltName = DNS:localhost, DNS:nats1 -EOF -) -rm -f "${NATS_CERT_DIR}/nats/server-req.pem" diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py deleted file mode 100644 index e9e5cb72363..00000000000 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ /dev/null @@ -1,37 +0,0 @@ -# -*- coding: utf-8 -*- -# Generated by the protocol buffer compiler. DO NOT EDIT! -# source: clickhouse_path/format_schemas/nats.proto -"""Generated protocol buffer code.""" -from google.protobuf import descriptor as _descriptor -from google.protobuf import descriptor_pool as _descriptor_pool -from google.protobuf import message as _message -from google.protobuf import reflection as _reflection -from google.protobuf import symbol_database as _symbol_database - -# @@protoc_insertion_point(imports) - -_sym_db = _symbol_database.Default() - - -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n)clickhouse_path/format_schemas/nats.proto"+\n\rProtoKeyValue\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3' -) - - -_PROTOKEYVALUE = DESCRIPTOR.message_types_by_name["ProtoKeyValue"] -ProtoKeyValue = _reflection.GeneratedProtocolMessageType( - "ProtoKeyValue", - (_message.Message,), - { - "DESCRIPTOR": _PROTOKEYVALUE, - "__module__": "clickhouse_path.format_schemas.nats_pb2" - # @@protoc_insertion_point(class_scope:ProtoKeyValue) - }, -) -_sym_db.RegisterMessage(ProtoKeyValue) - -if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _PROTOKEYVALUE._serialized_start = 45 - _PROTOKEYVALUE._serialized_end = 88 -# @@protoc_insertion_point(module_scope) diff --git a/tests/integration/test_storage_nats/test.py b/tests/integration/test_storage_nats/test.py deleted file mode 100644 index 4d7e4cf813d..00000000000 --- a/tests/integration/test_storage_nats/test.py +++ /dev/null @@ -1,1875 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/39185 - -pytestmark = pytest.mark.skip - -import json -import os.path as p -import random -import subprocess -import threading -import logging -import time -from random import randrange -import math - -import asyncio -from google.protobuf.internal.encoder import _VarintBytes -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster, check_nats_is_available, nats_connect_ssl -from helpers.test_tools import TSV - -from . import nats_pb2 - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "instance", - main_configs=[ - "configs/nats.xml", - "configs/macros.xml", - "configs/named_collection.xml", - ], - user_configs=["configs/users.xml"], - with_nats=True, - clickhouse_path_dir="clickhouse_path", -) - - -# Helpers - - -def wait_nats_to_start(nats_port, ssl_ctx=None, timeout=180): - start = time.time() - while time.time() - start < timeout: - try: - if asyncio.run(check_nats_is_available(nats_port, ssl_ctx=ssl_ctx)): - logging.debug("NATS is available") - return - time.sleep(0.5) - except Exception as ex: - logging.debug("Can't connect to NATS " + str(ex)) - time.sleep(0.5) - - -def nats_check_result(result, check=False, ref_file="test_nats_json.reference"): - fpath = p.join(p.dirname(__file__), ref_file) - with open(fpath) as reference: - if check: - assert TSV(result) == TSV(reference) - else: - return TSV(result) == TSV(reference) - - -def kill_nats(nats_id): - p = subprocess.Popen(("docker", "stop", nats_id), stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def revive_nats(nats_id, nats_port): - p = subprocess.Popen(("docker", "start", nats_id), stdout=subprocess.PIPE) - p.communicate() - wait_nats_to_start(nats_port) - - -# Fixtures - - -@pytest.fixture(scope="module") -def nats_cluster(): - try: - cluster.start() - logging.debug("nats_id is {}".format(instance.cluster.nats_docker_id)) - instance.query("CREATE DATABASE test") - - yield cluster - - finally: - cluster.shutdown() - - -@pytest.fixture(autouse=True) -def nats_setup_teardown(): - print("NATS is available - running test") - yield # run test - instance.query("DROP DATABASE test SYNC") - instance.query("CREATE DATABASE test") - - -# Tests - - -async def nats_produce_messages(cluster_inst, subject, messages=(), bytes=None): - nc = await nats_connect_ssl( - cluster_inst.nats_port, - user="click", - password="house", - ssl_ctx=cluster_inst.nats_ssl_context, - ) - logging.debug("NATS connection status: " + str(nc.is_connected)) - - for message in messages: - await nc.publish(subject, message.encode()) - if bytes is not None: - await nc.publish(subject, bytes) - logging.debug("Finished publising to " + subject) - - await nc.close() - return messages - - -def check_table_is_ready(instance, table_name): - try: - instance.query("SELECT * FROM {}".format(table_name)) - return True - except Exception: - return False - - -def test_nats_select(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'select', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "select", messages)) - - # The order of messages in select * from test.nats is not guaranteed, so sleep to collect everything in one select - time.sleep(1) - - result = "" - while True: - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_select_empty(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'empty', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - - assert int(instance.query("SELECT count() FROM test.nats")) == 0 - - -def test_nats_json_without_delimiter(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'json', - nats_format = 'JSONEachRow'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = "" - for i in range(25): - messages += json.dumps({"key": i, "value": i}) + "\n" - - all_messages = [messages] - asyncio.run(nats_produce_messages(nats_cluster, "json", all_messages)) - - messages = "" - for i in range(25, 50): - messages += json.dumps({"key": i, "value": i}) + "\n" - all_messages = [messages] - asyncio.run(nats_produce_messages(nats_cluster, "json", all_messages)) - - time.sleep(1) - - result = "" - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_csv_with_delimiter(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'csv', - nats_format = 'CSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append("{i}, {i}".format(i=i)) - - asyncio.run(nats_produce_messages(nats_cluster, "csv", messages)) - - time.sleep(1) - - result = "" - for _ in range(60): - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_tsv_with_delimiter(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'tsv', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append("{i}\t{i}".format(i=i)) - - asyncio.run(nats_produce_messages(nats_cluster, "tsv", messages)) - - result = "" - for _ in range(60): - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -# - - -def test_nats_macros(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = '{nats_url}', - nats_subjects = '{nats_subjects}', - nats_format = '{nats_format}' - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - message = "" - for i in range(50): - message += json.dumps({"key": i, "value": i}) + "\n" - asyncio.run(nats_produce_messages(nats_cluster, "macro", [message])) - - time.sleep(1) - - result = "" - for _ in range(60): - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_materialized_view(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mv', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - - CREATE TABLE test.view2 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS - SELECT * FROM test.nats group by (key, value); - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view2 ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_materialized_view_with_subquery(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mvsq', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM (SELECT * FROM test.nats); - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mvsq", messages)) - - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_many_materialized_views(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view1; - DROP TABLE IF EXISTS test.view2; - DROP TABLE IF EXISTS test.consumer1; - DROP TABLE IF EXISTS test.consumer2; - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mmv', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view1 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE TABLE test.view2 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer1 TO test.view1 AS - SELECT * FROM test.nats; - CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mmv", messages)) - - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result1 = instance.query("SELECT * FROM test.view1 ORDER BY key") - result2 = instance.query("SELECT * FROM test.view2 ORDER BY key") - if nats_check_result(result1) and nats_check_result(result2): - break - - instance.query( - """ - DROP TABLE test.consumer1; - DROP TABLE test.consumer2; - DROP TABLE test.view1; - DROP TABLE test.view2; - """ - ) - - nats_check_result(result1, True) - nats_check_result(result2, True) - - -def test_nats_protobuf(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value String) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'pb', - nats_format = 'Protobuf', - nats_schema = 'nats.proto:ProtoKeyValue'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - data = b"" - for i in range(0, 20): - msg = nats_pb2.ProtoKeyValue() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster, "pb", bytes=data)) - data = b"" - for i in range(20, 21): - msg = nats_pb2.ProtoKeyValue() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster, "pb", bytes=data)) - data = b"" - for i in range(21, 50): - msg = nats_pb2.ProtoKeyValue() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster, "pb", bytes=data)) - - result = "" - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_big_message(nats_cluster): - # Create batchs of messages of size ~100Kb - nats_messages = 1000 - batch_messages = 1000 - messages = [ - json.dumps({"key": i, "value": "x" * 100}) * batch_messages - for i in range(nats_messages) - ] - - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value String) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'big', - nats_format = 'JSONEachRow'; - CREATE TABLE test.view (key UInt64, value String) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - asyncio.run(nats_produce_messages(nats_cluster, "big", messages)) - - while True: - result = instance.query("SELECT count() FROM test.view") - if int(result) == batch_messages * nats_messages: - break - - assert ( - int(result) == nats_messages * batch_messages - ), "ClickHouse lost some messages: {}".format(result) - - -def test_nats_mv_combo(nats_cluster): - NUM_MV = 5 - NUM_CONSUMERS = 4 - - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'combo', - nats_num_consumers = {}, - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - """.format( - NUM_CONSUMERS - ) - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - for mv_id in range(NUM_MV): - instance.query( - """ - DROP TABLE IF EXISTS test.combo_{0}; - DROP TABLE IF EXISTS test.combo_{0}_mv; - CREATE TABLE test.combo_{0} (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.combo_{0}_mv TO test.combo_{0} AS - SELECT * FROM test.nats; - """.format( - mv_id - ) - ) - - time.sleep(2) - - i = [0] - messages_num = 10000 - - def produce(): - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({"key": i[0], "value": i[0]})) - i[0] += 1 - asyncio.run(nats_produce_messages(nats_cluster, "combo", messages)) - - threads = [] - threads_num = 20 - - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - while True: - result = 0 - for mv_id in range(NUM_MV): - result += int( - instance.query("SELECT count() FROM test.combo_{0}".format(mv_id)) - ) - if int(result) == messages_num * threads_num * NUM_MV: - break - time.sleep(1) - - for thread in threads: - thread.join() - - for mv_id in range(NUM_MV): - instance.query( - """ - DROP TABLE test.combo_{0}_mv; - DROP TABLE test.combo_{0}; - """.format( - mv_id - ) - ) - - assert ( - int(result) == messages_num * threads_num * NUM_MV - ), "ClickHouse lost some messages: {}".format(result) - - -def test_nats_insert(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'insert', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - values = [] - for i in range(50): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("insert") - await sub.unsubscribe(50) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - while True: - try: - instance.query("INSERT INTO test.nats VALUES {}".format(values)) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - thread.join() - - result = "\n".join(insert_messages) - nats_check_result(result, True) - - -def test_nats_many_subjects_insert_wrong(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'insert1,insert2.>,insert3.*.foo', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - values = [] - for i in range(50): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - # no subject specified - instance.query_and_get_error("INSERT INTO test.nats VALUES {}".format(values)) - - # can't insert into wildcard subjects - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert2.>' VALUES {}".format( - values - ) - ) - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.*.foo' VALUES {}".format( - values - ) - ) - - # specified subject is not among engine's subjects - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert4' VALUES {}".format( - values - ) - ) - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.foo.baz' VALUES {}".format( - values - ) - ) - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='foo.insert2' VALUES {}".format( - values - ) - ) - - -def test_nats_many_subjects_insert_right(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'right_insert1,right_insert2', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - values = [] - for i in range(50): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("right_insert1") - await sub.unsubscribe(50) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - while True: - try: - instance.query( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='right_insert1' VALUES {}".format( - values - ) - ) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - thread.join() - - result = "\n".join(insert_messages) - nats_check_result(result, True) - - -def test_nats_many_inserts(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.nats_many; - DROP TABLE IF EXISTS test.nats_consume; - DROP TABLE IF EXISTS test.view_many; - DROP TABLE IF EXISTS test.consumer_many; - CREATE TABLE test.nats_many (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'many_inserts', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.nats_consume (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'many_inserts', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view_many (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer_many TO test.view_many AS - SELECT * FROM test.nats_consume; - """ - ) - while not check_table_is_ready(instance, "test.nats_consume"): - logging.debug("Table test.nats_consume is not yet ready") - time.sleep(0.5) - - messages_num = 10000 - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - def insert(): - while True: - try: - instance.query("INSERT INTO test.nats_many VALUES {}".format(values)) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - - threads = [] - threads_num = 10 - for _ in range(threads_num): - threads.append(threading.Thread(target=insert)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - for thread in threads: - thread.join() - - time_limit_sec = 300 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT count() FROM test.view_many") - print(result, messages_num * threads_num) - if int(result) >= messages_num * threads_num: - break - time.sleep(1) - - instance.query( - """ - DROP TABLE test.nats_consume; - DROP TABLE test.nats_many; - DROP TABLE test.consumer_many; - DROP TABLE test.view_many; - """ - ) - - assert ( - int(result) == messages_num * threads_num - ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format( - result - ) - - -def test_nats_overloaded_insert(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view_overload; - DROP TABLE IF EXISTS test.consumer_overload; - DROP TABLE IF EXISTS test.nats_consume; - CREATE TABLE test.nats_consume (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'over', - nats_num_consumers = 5, - nats_max_block_size = 10000, - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.nats_overload (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'over', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view_overload (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key - SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3, - cleanup_thread_preferred_points_per_iteration=0; - CREATE MATERIALIZED VIEW test.consumer_overload TO test.view_overload AS - SELECT * FROM test.nats_consume; - """ - ) - while not check_table_is_ready(instance, "test.nats_consume"): - logging.debug("Table test.nats_consume is not yet ready") - time.sleep(0.5) - - messages_num = 100000 - - def insert(): - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - while True: - try: - instance.query( - "INSERT INTO test.nats_overload VALUES {}".format(values) - ) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - - threads = [] - threads_num = 5 - for _ in range(threads_num): - threads.append(threading.Thread(target=insert)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - time_limit_sec = 300 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT count() FROM test.view_overload") - time.sleep(1) - if int(result) >= messages_num * threads_num: - break - - instance.query( - """ - DROP TABLE test.consumer_overload; - DROP TABLE test.view_overload; - DROP TABLE test.nats_consume; - DROP TABLE test.nats_overload; - """ - ) - - for thread in threads: - thread.join() - - assert ( - int(result) == messages_num * threads_num - ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format( - result - ) - - -def test_nats_virtual_column(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats_virtuals (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'virtuals', - nats_format = 'JSONEachRow'; - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT value, key, _subject FROM test.nats_virtuals; - """ - ) - while not check_table_is_ready(instance, "test.nats_virtuals"): - logging.debug("Table test.nats_virtuals is not yet ready") - time.sleep(0.5) - - message_num = 10 - i = 0 - messages = [] - for _ in range(message_num): - messages.append(json.dumps({"key": i, "value": i})) - i += 1 - - asyncio.run(nats_produce_messages(nats_cluster, "virtuals", messages)) - - while True: - result = instance.query("SELECT count() FROM test.view") - time.sleep(1) - if int(result) == message_num: - break - - result = instance.query( - """ - SELECT key, value, _subject - FROM test.view ORDER BY key - """ - ) - - expected = """\ -0 0 virtuals -1 1 virtuals -2 2 virtuals -3 3 virtuals -4 4 virtuals -5 5 virtuals -6 6 virtuals -7 7 virtuals -8 8 virtuals -9 9 virtuals -""" - - instance.query( - """ - DROP TABLE test.nats_virtuals; - DROP TABLE test.view; - """ - ) - - assert TSV(result) == TSV(expected) - - -def test_nats_virtual_column_with_materialized_view(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats_virtuals_mv (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'virtuals_mv', - nats_format = 'JSONEachRow'; - CREATE TABLE test.view (key UInt64, value UInt64, subject String) ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _subject as subject - FROM test.nats_virtuals_mv; - """ - ) - while not check_table_is_ready(instance, "test.nats_virtuals_mv"): - logging.debug("Table test.nats_virtuals_mv is not yet ready") - time.sleep(0.5) - - message_num = 10 - i = 0 - messages = [] - for _ in range(message_num): - messages.append(json.dumps({"key": i, "value": i})) - i += 1 - - asyncio.run(nats_produce_messages(nats_cluster, "virtuals_mv", messages)) - - while True: - result = instance.query("SELECT count() FROM test.view") - time.sleep(1) - if int(result) == message_num: - break - - result = instance.query("SELECT key, value, subject FROM test.view ORDER BY key") - expected = """\ -0 0 virtuals_mv -1 1 virtuals_mv -2 2 virtuals_mv -3 3 virtuals_mv -4 4 virtuals_mv -5 5 virtuals_mv -6 6 virtuals_mv -7 7 virtuals_mv -8 8 virtuals_mv -9 9 virtuals_mv -""" - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - DROP TABLE test.nats_virtuals_mv - """ - ) - - assert TSV(result) == TSV(expected) - - -def test_nats_many_consumers_to_each_queue(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - """ - ) - - num_tables = 4 - for table_id in range(num_tables): - print(("Setting up table {}".format(table_id))) - instance.query( - """ - DROP TABLE IF EXISTS test.many_consumers_{0}; - DROP TABLE IF EXISTS test.many_consumers_{0}_mv; - CREATE TABLE test.many_consumers_{0} (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'many_consumers', - nats_num_consumers = 2, - nats_queue_group = 'many_consumers', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS - SELECT key, value FROM test.many_consumers_{0}; - """.format( - table_id - ) - ) - while not check_table_is_ready( - instance, "test.many_consumers_{}".format(table_id) - ): - logging.debug( - "Table test.many_consumers_{} is not yet ready".format(table_id) - ) - time.sleep(0.5) - - i = [0] - messages_num = 1000 - - def produce(): - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({"key": i[0], "value": i[0]})) - i[0] += 1 - asyncio.run(nats_produce_messages(nats_cluster, "many_consumers", messages)) - - threads = [] - threads_num = 20 - - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - result1 = "" - while True: - result1 = instance.query("SELECT count() FROM test.destination") - time.sleep(1) - if int(result1) == messages_num * threads_num: - break - - for thread in threads: - thread.join() - - for consumer_id in range(num_tables): - instance.query( - """ - DROP TABLE test.many_consumers_{0}; - DROP TABLE test.many_consumers_{0}_mv; - """.format( - consumer_id - ) - ) - - instance.query( - """ - DROP TABLE test.destination; - """ - ) - - assert ( - int(result1) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result1) - - -def test_nats_restore_failed_connection_without_losses_on_write(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.consume; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE TABLE test.consume (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'producer_reconnect', - nats_format = 'JSONEachRow', - nats_num_consumers = 2, - nats_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.consume; - DROP TABLE IF EXISTS test.producer_reconnect; - CREATE TABLE test.producer_reconnect (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'producer_reconnect', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.consume"): - logging.debug("Table test.consume is not yet ready") - time.sleep(0.5) - - messages_num = 100000 - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - while True: - try: - instance.query( - "INSERT INTO test.producer_reconnect VALUES {}".format(values) - ) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - - while int(instance.query("SELECT count() FROM test.view")) == 0: - time.sleep(0.1) - - kill_nats(nats_cluster.nats_docker_id) - time.sleep(4) - revive_nats(nats_cluster.nats_docker_id, nats_cluster.nats_port) - - while True: - result = instance.query("SELECT count(DISTINCT key) FROM test.view") - time.sleep(1) - if int(result) == messages_num: - break - - instance.query( - """ - DROP TABLE test.consume; - DROP TABLE test.producer_reconnect; - """ - ) - - assert int(result) == messages_num, "ClickHouse lost some messages: {}".format( - result - ) - - -def test_nats_no_connection_at_startup_1(nats_cluster): - # no connection when table is initialized - nats_cluster.pause_container("nats1") - instance.query_and_get_error( - """ - CREATE TABLE test.cs (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'cs', - nats_format = 'JSONEachRow', - nats_num_consumers = '5', - nats_row_delimiter = '\\n'; - """ - ) - nats_cluster.unpause_container("nats1") - - -def test_nats_no_connection_at_startup_2(nats_cluster): - instance.query( - """ - CREATE TABLE test.cs (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'cs', - nats_format = 'JSONEachRow', - nats_num_consumers = '5', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.cs; - """ - ) - - instance.query("DETACH TABLE test.cs") - nats_cluster.pause_container("nats1") - instance.query("ATTACH TABLE test.cs") - nats_cluster.unpause_container("nats1") - while not check_table_is_ready(instance, "test.cs"): - logging.debug("Table test.cs is not yet ready") - time.sleep(0.5) - - messages_num = 1000 - messages = [] - for i in range(messages_num): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "cs", messages)) - - for _ in range(20): - result = instance.query("SELECT count() FROM test.view") - time.sleep(1) - if int(result) == messages_num: - break - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.cs; - """ - ) - - assert int(result) == messages_num, "ClickHouse lost some messages: {}".format( - result - ) - - -def test_nats_format_factory_settings(nats_cluster): - instance.query( - """ - CREATE TABLE test.format_settings ( - id String, date DateTime - ) ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'format_settings', - nats_format = 'JSONEachRow', - date_time_input_format = 'best_effort'; - """ - ) - while not check_table_is_ready(instance, "test.format_settings"): - logging.debug("Table test.format_settings is not yet ready") - time.sleep(0.5) - - message = json.dumps( - {"id": "format_settings_test", "date": "2021-01-19T14:42:33.1829214Z"} - ) - expected = instance.query( - """SELECT parseDateTimeBestEffort(CAST('2021-01-19T14:42:33.1829214Z', 'String'))""" - ) - - asyncio.run(nats_produce_messages(nats_cluster, "format_settings", [message])) - - while True: - result = instance.query("SELECT date FROM test.format_settings") - if result == expected: - break - - instance.query( - """ - CREATE TABLE test.view ( - id String, date DateTime - ) ENGINE = MergeTree ORDER BY id; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.format_settings; - """ - ) - - asyncio.run(nats_produce_messages(nats_cluster, "format_settings", [message])) - while True: - result = instance.query("SELECT date FROM test.view") - if result == expected: - break - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.format_settings; - """ - ) - - assert result == expected - - -def test_nats_bad_args(nats_cluster): - instance.query_and_get_error( - """ - CREATE TABLE test.drop (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_secure = true, - nats_format = 'JSONEachRow'; - """ - ) - - -def test_nats_drop_mv(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mv', - nats_format = 'JSONEachRow'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(20): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - instance.query("DROP VIEW test.consumer") - messages = [] - for i in range(20, 40): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - instance.query( - """ - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - messages = [] - for i in range(40, 50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - while True: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - instance.query("DROP VIEW test.consumer") - messages = [] - for i in range(50, 60): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - count = 0 - while True: - count = int(instance.query("SELECT count() FROM test.nats")) - if count: - break - - assert count > 0 - - -def test_nats_predefined_configuration(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS(nats1) """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - asyncio.run( - nats_produce_messages( - nats_cluster, "named", [json.dumps({"key": 1, "value": 2})] - ) - ) - while True: - result = instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if result == "1\t2\n": - break - - -def test_format_with_prefix_and_suffix(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'custom', - nats_format = 'CustomSeparated'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("custom") - await sub.unsubscribe(2) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - "INSERT INTO test.nats select number*10 as key, number*100 as value from numbers(2) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" - ) - - thread.join() - - assert ( - "".join(insert_messages) - == "\n0\t0\n\n\n10\t100\n\n" - ) - - -def test_max_rows_per_message(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'custom1', - nats_format = 'CustomSeparated', - nats_max_rows_per_message = 3, - format_custom_result_before_delimiter = '\n', - format_custom_result_after_delimiter = '\n'; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - num_rows = 5 - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("custom1") - await sub.unsubscribe(2) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - f"INSERT INTO test.nats select number*10 as key, number*100 as value from numbers({num_rows}) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" - ) - - thread.join() - - assert ( - "".join(insert_messages) - == "\n0\t0\n10\t100\n20\t200\n\n\n30\t300\n40\t400\n\n" - ) - - attempt = 0 - rows = 0 - while attempt < 100: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view") - assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" - - -def test_row_based_formats(nats_cluster): - num_rows = 10 - - for format_name in [ - "TSV", - "TSVWithNamesAndTypes", - "TSKV", - "CSV", - "CSVWithNamesAndTypes", - "CustomSeparatedWithNamesAndTypes", - "Values", - "JSON", - "JSONEachRow", - "JSONCompactEachRow", - "JSONCompactEachRowWithNamesAndTypes", - "JSONObjectEachRow", - "Avro", - "RowBinary", - "RowBinaryWithNamesAndTypes", - "MsgPack", - ]: - print(format_name) - - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = '{format_name}', - nats_format = '{format_name}'; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.nats; - """ - ) - - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - insert_messages = 0 - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe(format_name) - await sub.unsubscribe(2) - async for msg in sub.messages: - nonlocal insert_messages - insert_messages += 1 - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - f"INSERT INTO test.nats select number*10 as key, number*100 as value from numbers({num_rows})" - ) - - thread.join() - - assert insert_messages == 2 - - attempt = 0 - rows = 0 - while attempt < 100: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - expected = "" - for i in range(num_rows): - expected += str(i * 10) + "\t" + str(i * 100) + "\n" - - result = instance.query("SELECT * FROM test.view") - assert result == expected - - -def test_block_based_formats_1(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'PrettySpace', - nats_format = 'PrettySpace'; - """ - ) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("PrettySpace") - await sub.unsubscribe(3) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - attempt = 0 - while attempt < 100: - try: - instance.query( - "INSERT INTO test.nats SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0;" - ) - break - except Exception: - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - attempt += 1 - thread.join() - - data = [] - for message in insert_messages: - splitted = message.split("\n") - assert splitted[0] == " \x1b[1mkey\x1b[0m \x1b[1mvalue\x1b[0m" - assert splitted[1] == "" - assert splitted[-1] == "" - data += [line.split() for line in splitted[2:-1]] - - assert data == [ - ["0", "0"], - ["10", "100"], - ["20", "200"], - ["30", "300"], - ["40", "400"], - ] - - -def test_block_based_formats_2(nats_cluster): - num_rows = 100 - - for format_name in [ - "JSONColumns", - "Native", - "Arrow", - "Parquet", - "ORC", - "JSONCompactColumns", - ]: - print(format_name) - - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = '{format_name}', - nats_format = '{format_name}'; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.nats; - """ - ) - - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - insert_messages = 0 - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe(format_name) - await sub.unsubscribe(9) - async for msg in sub.messages: - nonlocal insert_messages - insert_messages += 1 - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - f"INSERT INTO test.nats SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0;" - ) - - thread.join() - - assert insert_messages == 9 - - attempt = 0 - rows = 0 - while attempt < 100: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view ORDER by key") - expected = "" - for i in range(num_rows): - expected += str(i * 10) + "\t" + str(i * 100) + "\n" - assert result == expected - - -if __name__ == "__main__": - cluster.start() - input("Cluster created, press any key to destroy...") - cluster.shutdown() diff --git a/tests/integration/test_storage_nats/test_nats_json.reference b/tests/integration/test_storage_nats/test_nats_json.reference deleted file mode 100644 index 959bb2aad74..00000000000 --- a/tests/integration/test_storage_nats/test_nats_json.reference +++ /dev/null @@ -1,50 +0,0 @@ -0 0 -1 1 -2 2 -3 3 -4 4 -5 5 -6 6 -7 7 -8 8 -9 9 -10 10 -11 11 -12 12 -13 13 -14 14 -15 15 -16 16 -17 17 -18 18 -19 19 -20 20 -21 21 -22 22 -23 23 -24 24 -25 25 -26 26 -27 27 -28 28 -29 29 -30 30 -31 31 -32 32 -33 33 -34 34 -35 35 -36 36 -37 37 -38 38 -39 39 -40 40 -41 41 -42 42 -43 43 -44 44 -45 45 -46 46 -47 47 -48 48 -49 49 diff --git a/tests/integration/test_storage_postgresql_replica/__init__.py b/tests/integration/test_storage_postgresql_replica/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml b/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml deleted file mode 100644 index 27c7107ce5e..00000000000 --- a/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - trace - /var/log/clickhouse-server/log.log - /var/log/clickhouse-server/log.err.log - 1000M - 10 - /var/log/clickhouse-server/stderr.log - /var/log/clickhouse-server/stdout.log - - diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py deleted file mode 100644 index 66495700102..00000000000 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ /dev/null @@ -1,780 +0,0 @@ -import pytest - -# FIXME Tests with MaterializedPostgresSQL are temporarily disabled -# https://github.com/ClickHouse/ClickHouse/issues/36898 -# https://github.com/ClickHouse/ClickHouse/issues/38677 -# https://github.com/ClickHouse/ClickHouse/pull/39272#issuecomment-1190087190 - -pytestmark = pytest.mark.skip - -import time -import psycopg2 -import os.path as p - -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry -from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT -from helpers.test_tools import TSV - -import threading - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "instance", - main_configs=["configs/log_conf.xml"], - with_postgres=True, - stay_alive=True, -) - -postgres_table_template = """ - CREATE TABLE IF NOT EXISTS {} ( - key Integer NOT NULL, value Integer, PRIMARY KEY(key)) - """ - -queries = [ - "INSERT INTO {} select i, i from generate_series(0, 10000) as t(i);", - "DELETE FROM {} WHERE (value*value) % 3 = 0;", - "UPDATE {} SET value = value + 125 WHERE key % 2 = 0;", - "UPDATE {} SET key=key+20000 WHERE key%2=0", - "INSERT INTO {} select i, i from generate_series(40000, 50000) as t(i);", - "DELETE FROM {} WHERE key % 10 = 0;", - "UPDATE {} SET value = value + 101 WHERE key % 2 = 1;", - "UPDATE {} SET key=key+80000 WHERE key%2=1", - "DELETE FROM {} WHERE value % 2 = 0;", - "UPDATE {} SET value = value + 2000 WHERE key % 5 = 0;", - "INSERT INTO {} select i, i from generate_series(200000, 250000) as t(i);", - "DELETE FROM {} WHERE value % 3 = 0;", - "UPDATE {} SET value = value * 2 WHERE key % 3 = 0;", - "UPDATE {} SET key=key+500000 WHERE key%2=1", - "INSERT INTO {} select i, i from generate_series(1000000, 1050000) as t(i);", - "DELETE FROM {} WHERE value % 9 = 2;", - "UPDATE {} SET key=key+10000000", - "UPDATE {} SET value = value + 2 WHERE key % 3 = 1;", - "DELETE FROM {} WHERE value%5 = 0;", -] - - -@pytest.mark.timeout(30) -def check_tables_are_synchronized( - table_name, order_by="key", postgres_database="postgres_database" -): - while True: - expected = instance.query( - "select * from {}.{} order by {};".format( - postgres_database, table_name, order_by - ) - ) - result = instance.query( - "select * from test.{} order by {};".format(table_name, order_by) - ) - if result == expected: - break - - assert result == expected - - -def get_postgres_conn( - ip, port, database=False, auto_commit=True, database_name="postgres_database" -): - if database == True: - conn_string = "host={} port={} dbname='{}' user='postgres' password='mysecretpassword'".format( - ip, port, database_name - ) - else: - conn_string = ( - "host={} port={} user='postgres' password='mysecretpassword'".format( - ip, port - ) - ) - - conn = psycopg2.connect(conn_string) - if auto_commit: - conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) - conn.autocommit = True - return conn - - -def create_postgres_db(cursor, name): - cursor.execute("CREATE DATABASE {}".format(name)) - - -def create_clickhouse_postgres_db(ip, port, name="postgres_database"): - instance.query( - """ - CREATE DATABASE {} - ENGINE = PostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')""".format( - name, ip, port, name - ) - ) - - -def create_materialized_table(ip, port, table_name="postgresql_replica"): - instance.query( - f""" - CREATE TABLE test.{table_name} (key Int64, value Int64) - ENGINE = MaterializedPostgreSQL( - '{ip}:{port}', 'postgres_database', '{table_name}', 'postgres', 'mysecretpassword') - PRIMARY KEY key; """ - ) - - -def create_postgres_table(cursor, table_name, replica_identity_full=False): - cursor.execute("DROP TABLE IF EXISTS {}".format(table_name)) - cursor.execute(postgres_table_template.format(table_name)) - if replica_identity_full: - cursor.execute("ALTER TABLE {} REPLICA IDENTITY FULL;".format(table_name)) - - -def postgresql_replica_check_result( - result, check=False, ref_file="test_postgresql_replica.reference" -): - fpath = p.join(p.dirname(__file__), ref_file) - with open(fpath) as reference: - if check: - assert TSV(result) == TSV(reference) - else: - return TSV(result) == TSV(reference) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - conn = get_postgres_conn(ip=cluster.postgres_ip, port=cluster.postgres_port) - cursor = conn.cursor() - create_postgres_db(cursor, "postgres_database") - create_clickhouse_postgres_db( - ip=cluster.postgres_ip, port=cluster.postgres_port - ) - - instance.query("CREATE DATABASE test") - yield cluster - - finally: - cluster.shutdown() - - -@pytest.mark.timeout(320) -def test_initial_load_from_snapshot(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.2) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_no_connection_at_startup(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - time.sleep(3) - - instance.query("DETACH TABLE test.postgresql_replica") - started_cluster.pause_container("postgres1") - - instance.query("ATTACH TABLE test.postgresql_replica") - time.sleep(3) - started_cluster.unpause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) == 0: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_detach_attach_is_ok(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) == 0: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - postgresql_replica_check_result(result, True) - - instance.query("DETACH TABLE test.postgresql_replica") - instance.query("ATTACH TABLE test.postgresql_replica") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_replicating_insert_queries(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 10: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 10 + number, 10 + number from numbers(10)" - ) - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 20 + number, 20 + number from numbers(10)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 30: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 30 + number, 30 + number from numbers(10)" - ) - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 40 + number, 40 + number from numbers(10)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 50: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_replicating_delete_queries(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.2) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 100: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("DELETE FROM postgresql_replica WHERE key > 49;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - - -@pytest.mark.timeout(320) -def test_replicating_update_queries(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 50: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("UPDATE postgresql_replica SET value = value - 10;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - - -@pytest.mark.timeout(320) -def test_resume_from_written_version(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 50: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 100: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query("DETACH TABLE test.postgresql_replica") - - cursor.execute("DELETE FROM postgresql_replica WHERE key > 49;") - cursor.execute("UPDATE postgresql_replica SET value = value - 10;") - - instance.query("ATTACH TABLE test.postgresql_replica") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - - -@pytest.mark.timeout(320) -def test_many_replication_messages(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 100000: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - print("SYNC OK") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000, 100000)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 200000: - time.sleep(1) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - print("INSERT OK") - - result = instance.query("SELECT key FROM test.postgresql_replica ORDER BY key;") - expected = instance.query("SELECT number from numbers(200000)") - assert result == expected - - cursor.execute("UPDATE postgresql_replica SET value = key + 1 WHERE key < 100000;") - - result = instance.query( - "SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;" - ) - expected = instance.query("SELECT number from numbers(100000)") - - while result != expected: - time.sleep(1) - result = instance.query( - "SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;" - ) - print("UPDATE OK") - - cursor.execute("DELETE FROM postgresql_replica WHERE key % 2 = 1;") - cursor.execute("DELETE FROM postgresql_replica WHERE key != value;") - - result = instance.query( - "SELECT count() FROM (SELECT * FROM test.postgresql_replica);" - ) - while int(result) != 50000: - time.sleep(1) - result = instance.query( - "SELECT count() FROM (SELECT * FROM test.postgresql_replica);" - ) - print("DELETE OK") - - cursor.execute("DROP TABLE postgresql_replica;") - - -@pytest.mark.timeout(320) -def test_connection_loss(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - i = 50 - while i < 100000: - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format( - i - ) - ) - i += 10000 - - started_cluster.pause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - print(int(result)) - time.sleep(6) - - started_cluster.unpause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) < 100050: - time.sleep(1) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("DROP TABLE postgresql_replica;") - assert int(result) == 100050 - - -@pytest.mark.timeout(320) -def test_clickhouse_restart(started_cluster): - pytest.skip("Temporary disabled (FIXME)") - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - i = 50 - while i < 100000: - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format( - i - ) - ) - i += 10000 - - instance.restart_clickhouse() - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) < 100050: - time.sleep(1) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("DROP TABLE postgresql_replica;") - print(result) - assert int(result) == 100050 - - -def test_rename_table(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 25: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "RENAME TABLE test.postgresql_replica TO test.postgresql_replica_renamed" - ) - assert ( - int(instance.query("SELECT count() FROM test.postgresql_replica_renamed;")) - == 25 - ) - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25, 25)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica_renamed;") - while int(result) != 50: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica_renamed;") - - result = instance.query( - "SELECT * FROM test.postgresql_replica_renamed ORDER BY key;" - ) - postgresql_replica_check_result(result, True) - cursor.execute("DROP TABLE postgresql_replica;") - instance.query("DROP TABLE IF EXISTS test.postgresql_replica_renamed") - - -def test_virtual_columns(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)" - ) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 10: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - # just check that it works, no check with `expected` because _version is taken as LSN, which will be different each time. - result = instance.query( - "SELECT key, value, _sign, _version FROM test.postgresql_replica;" - ) - print(result) - cursor.execute("DROP TABLE postgresql_replica;") - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -def test_abrupt_connection_loss_while_heavy_replication(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - table_name = "postgresql_replica" - create_postgres_table(cursor, table_name) - - instance.query(f"DROP TABLE IF EXISTS test.{table_name}") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - for i in range(len(queries)): - query = queries[i].format(table_name) - cursor.execute(query) - print("query {}".format(query.format(table_name))) - - started_cluster.pause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica") - print(result) # Just debug - - started_cluster.unpause_container("postgres1") - - check_tables_are_synchronized("postgresql_replica") - - result = instance.query("SELECT count() FROM test.postgresql_replica") - print(result) # Just debug - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -def test_abrupt_server_restart_while_heavy_replication(started_cluster): - # FIXME (kssenii) temporary disabled - if instance.is_built_with_sanitizer(): - pytest.skip("Temporary disabled (FIXME)") - - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - table_name = "postgresql_replica_697" - create_postgres_table(cursor, table_name) - - instance.query(f"INSERT INTO postgres_database.{table_name} SELECT -1, 1") - instance.query(f"DROP TABLE IF EXISTS test.{table_name} SYNC") - create_materialized_table( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - table_name=table_name, - ) - - n = 1 - while int(instance.query(f"select count() from test.{table_name}")) != 1: - sleep(1) - n += 1 - if n > 10: - break - - for query in queries: - cursor.execute(query.format(table_name)) - print("query {}".format(query.format(table_name))) - - instance.restart_clickhouse() - - result = instance.query(f"SELECT count() FROM test.{table_name}") - print(result) # Just debug - - check_tables_are_synchronized(table_name) - - result = instance.query(f"SELECT count() FROM test.{table_name}") - print(result) # Just debug - instance.query(f"DROP TABLE test.{table_name} SYNC") - - -def test_drop_table_immediately(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - instance.query("DROP TABLE test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - check_tables_are_synchronized("postgresql_replica") - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -if __name__ == "__main__": - cluster.start() - input("Cluster created, press any key to destroy...") - cluster.shutdown() diff --git a/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference b/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference deleted file mode 100644 index 959bb2aad74..00000000000 --- a/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference +++ /dev/null @@ -1,50 +0,0 @@ -0 0 -1 1 -2 2 -3 3 -4 4 -5 5 -6 6 -7 7 -8 8 -9 9 -10 10 -11 11 -12 12 -13 13 -14 14 -15 15 -16 16 -17 17 -18 18 -19 19 -20 20 -21 21 -22 22 -23 23 -24 24 -25 25 -26 26 -27 27 -28 28 -29 29 -30 30 -31 31 -32 32 -33 33 -34 34 -35 35 -36 36 -37 37 -38 38 -39 39 -40 40 -41 41 -42 42 -43 43 -44 44 -45 45 -46 46 -47 47 -48 48 -49 49 diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 751279f5e5a..80d2050b394 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -511,69 +511,6 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): rabbitmq_check_result(result2, True) -@pytest.mark.skip(reason="clichouse_path with rabbitmq.proto fails to be exported") -def test_rabbitmq_protobuf(rabbitmq_cluster): - instance.query( - """ - CREATE TABLE test.rabbitmq (key UInt64, value String) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'pb', - rabbitmq_format = 'Protobuf', - rabbitmq_flush_interval_ms=1000, - rabbitmq_max_block_size=100, - rabbitmq_schema = 'rabbitmq.proto:KeyValueProto'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq; - """ - ) - - credentials = pika.PlainCredentials("root", "clickhouse") - parameters = pika.ConnectionParameters( - rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, "/", credentials - ) - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - - data = "" - for i in range(0, 20): - msg = rabbitmq_pb2.KeyValueProto() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - channel.basic_publish(exchange="pb", routing_key="", body=data) - data = "" - for i in range(20, 21): - msg = rabbitmq_pb2.KeyValueProto() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - channel.basic_publish(exchange="pb", routing_key="", body=data) - data = "" - for i in range(21, 50): - msg = rabbitmq_pb2.KeyValueProto() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - channel.basic_publish(exchange="pb", routing_key="", body=data) - - connection.close() - - result = "" - while True: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if rabbitmq_check_result(result): - break - - rabbitmq_check_result(result, True) - - def test_rabbitmq_big_message(rabbitmq_cluster): # Create batchs of messages of size ~100Kb rabbitmq_messages = 1000 diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index a2f28e21666..c1c076277bb 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1529,106 +1529,6 @@ def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): node1.query("DROP TABLE IF EXISTS {name} SYNC".format(name=name)) -@pytest.mark.skip(reason="Flacky test") -@pytest.mark.parametrize( - "name,positive", - [ - pytest.param("test_double_move_while_select_negative", 0, id="negative"), - pytest.param("test_double_move_while_select_positive", 1, id="positive"), - ], -) -def test_double_move_while_select(started_cluster, name, positive): - name = unique_table_name(name) - - try: - node1.query( - """ - CREATE TABLE {name} ( - n Int64, - s String - ) ENGINE = MergeTree - ORDER BY tuple() - PARTITION BY n - SETTINGS storage_policy='small_jbod_with_external',temporary_directories_lifetime=1 - """.format( - name=name - ) - ) - - node1.query( - "INSERT INTO {name} VALUES (1, randomPrintableASCII(10*1024*1024))".format( - name=name - ) - ) - - parts = node1.query( - "SELECT name FROM system.parts WHERE table = '{name}' AND active = 1".format( - name=name - ) - ).splitlines() - assert len(parts) == 1 - - node1.query( - "ALTER TABLE {name} MOVE PART '{part}' TO DISK 'external'".format( - name=name, part=parts[0] - ) - ) - - def long_select(): - if positive: - node1.query( - "SELECT sleep(3), sleep(2), sleep(1), n FROM {name}".format( - name=name - ) - ) - - thread = threading.Thread(target=long_select) - thread.start() - - time.sleep(1) - - node1.query( - "ALTER TABLE {name} MOVE PART '{part}' TO DISK 'jbod1'".format( - name=name, part=parts[0] - ) - ) - - # Fill jbod1 to force ClickHouse to make move of partition 1 to external. - node1.query( - "INSERT INTO {name} VALUES (2, randomPrintableASCII(9*1024*1024))".format( - name=name - ) - ) - node1.query( - "INSERT INTO {name} VALUES (3, randomPrintableASCII(9*1024*1024))".format( - name=name - ) - ) - node1.query( - "INSERT INTO {name} VALUES (4, randomPrintableASCII(9*1024*1024))".format( - name=name - ) - ) - - wait_parts_mover(node1, name, retry_count=40) - - # If SELECT locked old part on external, move shall fail. - assert node1.query( - "SELECT disk_name FROM system.parts WHERE table = '{name}' AND active = 1 AND name = '{part}'".format( - name=name, part=parts[0] - ) - ).splitlines() == ["jbod1" if positive else "external"] - - thread.join() - - assert node1.query( - "SELECT n FROM {name} ORDER BY n".format(name=name) - ).splitlines() == ["1", "2", "3", "4"] - - finally: - node1.query("DROP TABLE IF EXISTS {name} SYNC".format(name=name)) - - @pytest.mark.parametrize( "name,engine,positive", [ From f2c3000a93e70d3eb6994a79bfccc84c5ba275e4 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 9 Aug 2023 21:40:06 +0000 Subject: [PATCH 1800/2047] Improve exception in ALTER query --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++++ tests/queries/0_stateless/02834_alter_exception.reference | 0 tests/queries/0_stateless/02834_alter_exception.sql | 4 ++++ 3 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02834_alter_exception.reference create mode 100644 tests/queries/0_stateless/02834_alter_exception.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6179c70ca57..8938a3b6fbd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3166,6 +3166,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } } + if (command.type == AlterCommand::MODIFY_QUERY) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "ALTER MODIFY QUERY is not supported by MergeTree engines family"); + if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned) { throw Exception(ErrorCodes::BAD_ARGUMENTS, diff --git a/tests/queries/0_stateless/02834_alter_exception.reference b/tests/queries/0_stateless/02834_alter_exception.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02834_alter_exception.sql b/tests/queries/0_stateless/02834_alter_exception.sql new file mode 100644 index 00000000000..d42f40fcbf7 --- /dev/null +++ b/tests/queries/0_stateless/02834_alter_exception.sql @@ -0,0 +1,4 @@ +DROP TABLE IF EXISTS alter_02834; +CREATE TABLE alter_02834 (a UInt64) ENGINE=MergeTree() ORDER BY a; +ALTER TABLE alter_02834 MODIFY QUERY SELECT a FROM alter_02834; -- { serverError NOT_IMPLEMENTED } +DROP TABLE alter_02834; From ce30b93b34348700351e0d462edb5baa6ddb5685 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 9 Aug 2023 23:46:45 +0200 Subject: [PATCH 1801/2047] Move repeated code for ClickHouseHelper to _insert_post --- tests/ci/clickhouse_helper.py | 63 ++++++++--------------------------- 1 file changed, 14 insertions(+), 49 deletions(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index dbf26288469..b165cda37e8 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -47,45 +47,9 @@ class ClickHouseHelper: params[k] = v with open(file, "rb") as data_fd: - for i in range(5): - try: - response = requests.post( - url, params=params, data=data_fd, headers=auth - ) - except Exception as e: - error = f"Received exception while sending data to {url} on {i} attempt: {e}" - logging.warning(error) - continue - - logging.info("Response content '%s'", response.content) - - if response.ok: - break - - error = ( - "Cannot insert data into clickhouse at try " - + str(i) - + ": HTTP code " - + str(response.status_code) - + ": '" - + str(response.text) - + "'" - ) - - if response.status_code >= 500: - # A retriable error - time.sleep(1) - continue - - logging.info( - "Request headers '%s', body '%s'", - response.request.headers, - response.request.body, - ) - - raise InsertException(error) - else: - raise InsertException(error) + ClickHouseHelper._insert_post( + url, params=params, data=data_fd, headers=auth + ) @staticmethod def insert_json_str(url, auth, db, table, json_str): @@ -95,12 +59,18 @@ class ClickHouseHelper: "date_time_input_format": "best_effort", "send_logs_level": "warning", } + ClickHouseHelper._insert_post(url, params=params, data=json_str, headers=auth) + + @staticmethod + def _insert_post(*args, **kwargs): + url = "" + if args: + url = args[0] + url = kwargs.get("url", url) for i in range(5): try: - response = requests.post( - url, params=params, data=json_str, headers=auth - ) + response = requests.post(*args, **kwargs) except Exception as e: error = f"Received exception while sending data to {url} on {i} attempt: {e}" logging.warning(error) @@ -112,13 +82,8 @@ class ClickHouseHelper: break error = ( - "Cannot insert data into clickhouse at try " - + str(i) - + ": HTTP code " - + str(response.status_code) - + ": '" - + str(response.text) - + "'" + f"Cannot insert data into clickhouse at try {i}: HTTP code " + f"{response.status_code}: '{response.text}'" ) if response.status_code >= 500: From 8c63088a31d2b1d829a356f810392f9cf1484e93 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 9 Aug 2023 21:54:58 +0000 Subject: [PATCH 1802/2047] Fixes --- .../02415_all_new_functions_must_be_documented.reference | 1 + utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 61a2e4e9f02..231f268ba57 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -881,6 +881,7 @@ tumble tumbleEnd tumbleStart tuple +tupleConcat tupleDivide tupleDivideByNumber tupleElement diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a314815e2c4..1a7e8e16bd1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2421,6 +2421,7 @@ tsv tui tumbleEnd tumbleStart +tupleConcat tupleDivide tupleDivideByNumber tupleElement From a79f2f140d25657a248f0d96bbe0241065a4260d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 00:19:24 +0200 Subject: [PATCH 1803/2047] Fix bad test `00417_kill_query` --- .../0_stateless/00417_kill_query.reference | 4 +-- tests/queries/0_stateless/00417_kill_query.sh | 25 ++++++++++++------- 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/00417_kill_query.reference b/tests/queries/0_stateless/00417_kill_query.reference index 1a3b47964c0..0dd1d78ac82 100644 --- a/tests/queries/0_stateless/00417_kill_query.reference +++ b/tests/queries/0_stateless/00417_kill_query.reference @@ -1,2 +1,2 @@ -SELECT sleep(1) FROM system.numbers LIMIT 30 -SELECT sleep(1) FROM system.numbers LIMIT 31 +SELECT sleep(1) FROM system.numbers LIMIT 300 +SELECT sleep(1) FROM system.numbers LIMIT 301 diff --git a/tests/queries/0_stateless/00417_kill_query.sh b/tests/queries/0_stateless/00417_kill_query.sh index cd5b788a147..e64e93fd4a0 100755 --- a/tests/queries/0_stateless/00417_kill_query.sh +++ b/tests/queries/0_stateless/00417_kill_query.sh @@ -7,19 +7,26 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -QUERY_FIELND_NUM=4 +QUERY_FIELD_NUM=4 -$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 30" &>/dev/null & -sleep 1 -$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' and query LIKE 'SELECT sleep(%' AND (elapsed >= 0.) SYNC" | cut -f $QUERY_FIELND_NUM +$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 300" &>/dev/null & -# 31 is for the query to be different from the previous one -$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 31" &>/dev/null & -sleep 1 -$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' and query = 'SELECT sleep(1) FROM system.numbers LIMIT 31' ASYNC" | cut -f $QUERY_FIELND_NUM +while true +do + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' AND query LIKE 'SELECT sleep(%' AND (elapsed >= 0.) SYNC" | cut -f $QUERY_FIELD_NUM | grep '.' && break + sleep 0.1 +done + +# 31 is for the query to be different from the previous one +$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 301" &>/dev/null & + +while true +do + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' AND query = 'SELECT sleep(1) FROM system.numbers LIMIT 301' ASYNC" | cut -f $QUERY_FIELD_NUM | grep '.' && break + sleep 0.1 +done $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 ASYNC" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 FORMAT TabSeparated" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 SYNC FORMAT TabSeparated" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 1 TEST" &>/dev/null - From 2802508c07917dc0b1d9b2d9ba91752c21a59cdf Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 9 Aug 2023 22:27:49 +0000 Subject: [PATCH 1804/2047] different linker --- cmake/linux/toolchain-s390x.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index f5e1cf660c9..349d7fc83a6 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -23,7 +23,6 @@ set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") -set (CMAKE_STATIC_LINKER_FLAGS "${CMAKE_STATIC_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 087cc5bfed6736fc31f29e20a16f8161e40018c6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 9 Aug 2023 23:18:27 +0000 Subject: [PATCH 1805/2047] different linker --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 349d7fc83a6..188d9ed75b7 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=gold") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=gold") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=gold") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 65d3778b031dab996556e7e95597d7802daf3432 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 8 Aug 2023 20:15:04 +0000 Subject: [PATCH 1806/2047] Fix warning in test_replicated_database --- tests/integration/test_replicated_database/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 1235f7d34df..d0a04f40b69 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1226,7 +1226,7 @@ def test_force_synchronous_settings(started_cluster): def select_func(): dummy_node.query( - "SELECT sleepEachRow(1) FROM test_force_synchronous_settings.t" + "SELECT sleepEachRow(1) FROM test_force_synchronous_settings.t SETTINGS function_sleep_max_microseconds_per_block = 0" ) select_thread = threading.Thread(target=select_func) From eb61074e070587edb048fb02f26852dd3a2fd398 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 02:37:23 +0200 Subject: [PATCH 1807/2047] Fix test `02428_delete_with_settings` --- tests/queries/0_stateless/02428_delete_with_settings.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02428_delete_with_settings.sql b/tests/queries/0_stateless/02428_delete_with_settings.sql index 071a3f74184..618c08608fc 100644 --- a/tests/queries/0_stateless/02428_delete_with_settings.sql +++ b/tests/queries/0_stateless/02428_delete_with_settings.sql @@ -1,5 +1,5 @@ drop table if exists test; -create table test (id Int32, key String) engine=MergeTree() order by tuple(); +create table test (id Int32, key String) engine=MergeTree() order by tuple() settings index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into test select number, toString(number) from numbers(1000000); delete from test where id % 2 = 0 SETTINGS mutations_sync=0; select count() from test; From a9b760a15bf284dbe20882ddc696bdf28efed061 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 00:40:06 +0000 Subject: [PATCH 1808/2047] different linker --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 188d9ed75b7..23d14d6ed26 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=gold") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=gold") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=gold") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=bfd") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=bfd") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=bfd") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From c071f6755d70a5ce9f1924fc44a905cc3122da5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:10:03 +0300 Subject: [PATCH 1809/2047] Revert "Added new tests for session_log and fixed the inconsistency between login and logout." --- src/Interpreters/Session.cpp | 44 +-- src/Interpreters/Session.h | 2 - src/Interpreters/SessionLog.cpp | 14 +- src/Interpreters/SessionLog.h | 9 +- src/Server/HTTPHandler.cpp | 3 +- .../test.py | 5 +- tests/integration/test_session_log/.gitignore | 1 - .../integration/test_session_log/__init__.py | 0 .../test_session_log/configs/log.xml | 9 - .../test_session_log/configs/ports.xml | 9 - .../test_session_log/configs/session_log.xml | 9 - .../test_session_log/configs/users.xml | 23 -- .../protos/clickhouse_grpc.proto | 1 - tests/integration/test_session_log/test.py | 289 ------------------ .../02833_concurrrent_sessions.reference | 34 --- .../0_stateless/02833_concurrrent_sessions.sh | 138 --------- .../02834_remote_session_log.reference | 13 - .../0_stateless/02834_remote_session_log.sh | 56 ---- .../02835_drop_user_during_session.reference | 8 - .../02835_drop_user_during_session.sh | 113 ------- 20 files changed, 27 insertions(+), 753 deletions(-) delete mode 100644 tests/integration/test_session_log/.gitignore delete mode 100644 tests/integration/test_session_log/__init__.py delete mode 100644 tests/integration/test_session_log/configs/log.xml delete mode 100644 tests/integration/test_session_log/configs/ports.xml delete mode 100644 tests/integration/test_session_log/configs/session_log.xml delete mode 100644 tests/integration/test_session_log/configs/users.xml delete mode 120000 tests/integration/test_session_log/protos/clickhouse_grpc.proto delete mode 100644 tests/integration/test_session_log/test.py delete mode 100644 tests/queries/0_stateless/02833_concurrrent_sessions.reference delete mode 100755 tests/queries/0_stateless/02833_concurrrent_sessions.sh delete mode 100644 tests/queries/0_stateless/02834_remote_session_log.reference delete mode 100755 tests/queries/0_stateless/02834_remote_session_log.sh delete mode 100644 tests/queries/0_stateless/02835_drop_user_during_session.reference delete mode 100755 tests/queries/0_stateless/02835_drop_user_during_session.sh diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 0a6435cff75..de2a779b740 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -520,8 +520,6 @@ ContextMutablePtr Session::makeSessionContext() {}, session_context->getSettingsRef().max_sessions_for_user); - recordLoginSucess(session_context); - return session_context; } @@ -584,8 +582,6 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: { session_name_ }, max_sessions_for_user); - recordLoginSucess(session_context); - return session_context; } @@ -659,38 +655,24 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (user_id) user = query_context->getUser(); - /// Interserver does not create session context - recordLoginSucess(query_context); + if (!notified_session_log_about_login) + { + if (auto session_log = getSessionLog()) + { + session_log->addLoginSuccess( + auth_id, + named_session ? std::optional(named_session->key.second) : std::nullopt, + *query_context, + user); + + notified_session_log_about_login = true; + } + } return query_context; } -void Session::recordLoginSucess(ContextPtr login_context) const -{ - if (notified_session_log_about_login) - return; - - if (!login_context) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Session or query context must be created"); - - if (auto session_log = getSessionLog()) - { - const auto & settings = login_context->getSettingsRef(); - const auto access = login_context->getAccess(); - - session_log->addLoginSuccess(auth_id, - named_session ? named_session->key.second : "", - settings, - access, - getClientInfo(), - user); - - notified_session_log_about_login = true; - } -} - - void Session::releaseSessionID() { if (!named_session) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 81ef987b428..51c0e3c71fa 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -97,8 +97,6 @@ public: private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; - void recordLoginSucess(ContextPtr login_context) const; - mutable bool notified_session_log_about_login = false; const UUID auth_id; diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index b847eaf9824..0a8a7fc18c5 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -199,13 +199,12 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(auth_failure_reason.data(), auth_failure_reason.length()); } -void SessionLog::addLoginSuccess(const UUID & auth_id, - const String & session_id, - const Settings & settings, - const ContextAccessPtr & access, - const ClientInfo & client_info, - const UserPtr & login_user) +void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user) { + const auto access = login_context.getAccess(); + const auto & settings = login_context.getSettingsRef(); + const auto & client_info = login_context.getClientInfo(); + DB::SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS); log_entry.client_info = client_info; @@ -216,7 +215,8 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, } log_entry.external_auth_server = login_user ? login_user->auth_data.getLDAPServerName() : ""; - log_entry.session_id = session_id; + if (session_id) + log_entry.session_id = *session_id; if (const auto roles_info = access->getRolesInfo()) log_entry.roles = roles_info->getCurrentRolesNames(); diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index 8757bc12270..1282ac09c4d 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -20,7 +20,6 @@ enum SessionLogElementType : int8_t class ContextAccess; struct User; using UserPtr = std::shared_ptr; -using ContextAccessPtr = std::shared_ptr; /** A struct which will be inserted as row into session_log table. * @@ -73,13 +72,7 @@ class SessionLog : public SystemLog using SystemLog::SystemLog; public: - void addLoginSuccess(const UUID & auth_id, - const String & session_id, - const Settings & settings, - const ContextAccessPtr & access, - const ClientInfo & client_info, - const UserPtr & login_user); - + void addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user); void addLoginFailure(const UUID & auth_id, const ClientInfo & info, const std::optional & user, const Exception & reason); void addLogOut(const UUID & auth_id, const UserPtr & login_user, const ClientInfo & client_info); }; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 10d5e7a0242..ff5690a3b07 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -561,7 +561,8 @@ void HTTPHandler::processQuery( session->makeSessionContext(); } - auto context = session->makeQueryContext(); + auto client_info = session->getClientInfo(); + auto context = session->makeQueryContext(std::move(client_info)); /// This parameter is used to tune the behavior of output formats (such as Native) for compatibility. if (params.has("client_protocol_version")) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 65587933fed..2930262f63e 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -27,7 +27,10 @@ proto_dir = os.path.join(SCRIPT_DIR, "./protos") gen_dir = os.path.join(SCRIPT_DIR, "./_gen") os.makedirs(gen_dir, exist_ok=True) run_and_check( - f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", + "python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \ + {proto_dir}/clickhouse_grpc.proto".format( + proto_dir=proto_dir, gen_dir=gen_dir + ), shell=True, ) diff --git a/tests/integration/test_session_log/.gitignore b/tests/integration/test_session_log/.gitignore deleted file mode 100644 index edf565ec632..00000000000 --- a/tests/integration/test_session_log/.gitignore +++ /dev/null @@ -1 +0,0 @@ -_gen diff --git a/tests/integration/test_session_log/__init__.py b/tests/integration/test_session_log/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_session_log/configs/log.xml b/tests/integration/test_session_log/configs/log.xml deleted file mode 100644 index 7a079b81e69..00000000000 --- a/tests/integration/test_session_log/configs/log.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log - 1000M - 10 - - \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/ports.xml b/tests/integration/test_session_log/configs/ports.xml deleted file mode 100644 index fbaefc16b3a..00000000000 --- a/tests/integration/test_session_log/configs/ports.xml +++ /dev/null @@ -1,9 +0,0 @@ - - 5433 - 9001 - 9100 - - - false - - \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml deleted file mode 100644 index a0e4e3e2216..00000000000 --- a/tests/integration/test_session_log/configs/session_log.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - system - session_log
- - toYYYYMM(event_date) - 7500 -
-
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml deleted file mode 100644 index 0416dfadc8a..00000000000 --- a/tests/integration/test_session_log/configs/users.xml +++ /dev/null @@ -1,23 +0,0 @@ - - - - 0 - - - - - - - pass - - - pass - - - pass - - - pass - - - \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto deleted file mode 120000 index 25d15f11e3b..00000000000 --- a/tests/integration/test_session_log/protos/clickhouse_grpc.proto +++ /dev/null @@ -1 +0,0 @@ -../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py deleted file mode 100644 index dbb39993ce3..00000000000 --- a/tests/integration/test_session_log/test.py +++ /dev/null @@ -1,289 +0,0 @@ -import os - -import grpc -import pymysql.connections -import psycopg2 as py_psql -import pytest -import random -import sys -import threading - -from helpers.cluster import ClickHouseCluster, run_and_check - -POSTGRES_SERVER_PORT = 5433 -MYSQL_SERVER_PORT = 9001 -GRPC_PORT = 9100 -SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -DEFAULT_ENCODING = "utf-8" - -# Use grpcio-tools to generate *pb2.py files from *.proto. -proto_dir = os.path.join(SCRIPT_DIR, "./protos") -gen_dir = os.path.join(SCRIPT_DIR, "./_gen") -os.makedirs(gen_dir, exist_ok=True) -run_and_check( - f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", - shell=True, -) - -sys.path.append(gen_dir) - -import clickhouse_grpc_pb2 -import clickhouse_grpc_pb2_grpc - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "node", - main_configs=[ - "configs/ports.xml", - "configs/log.xml", - "configs/session_log.xml", - ], - user_configs=["configs/users.xml"], - # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 - env_variables={ - "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") - }, -) - - -def grpc_get_url(): - return f"{instance.ip_address}:{GRPC_PORT}" - - -def grpc_create_insecure_channel(): - channel = grpc.insecure_channel(grpc_get_url()) - grpc.channel_ready_future(channel).result(timeout=2) - return channel - - -session_id_counter = 0 - - -def next_session_id(): - global session_id_counter - session_id = session_id_counter - session_id_counter += 1 - return str(session_id) - - -def grpc_query(query, user_, pass_, raise_exception): - try: - query_info = clickhouse_grpc_pb2.QueryInfo( - query=query, - session_id=next_session_id(), - user_name=user_, - password=pass_, - ) - channel = grpc_create_insecure_channel() - stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) - result = stub.ExecuteQuery(query_info) - if result and result.HasField("exception"): - raise Exception(result.exception.display_text) - - return result.output.decode(DEFAULT_ENCODING) - except Exception: - assert raise_exception - - -def postgres_query(query, user_, pass_, raise_exception): - try: - client = py_psql.connect( - host=instance.ip_address, - port=POSTGRES_SERVER_PORT, - user=user_, - password=pass_, - database="default", - ) - cursor = client.cursor() - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -def mysql_query(query, user_, pass_, raise_exception): - try: - client = pymysql.connections.Connection( - host=instance.ip_address, - user=user_, - password=pass_, - database="default", - port=MYSQL_SERVER_PORT, - ) - cursor = client.cursor(pymysql.cursors.DictCursor) - if raise_exception: - with pytest.raises(Exception): - cursor.execute(query) - else: - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_grpc_session(started_cluster): - grpc_query("SELECT 1", "grpc_user", "pass", False) - grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) - grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "grpc_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" - ) - assert logout_records == "grpc_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "grpc_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_mysql_session(started_cluster): - mysql_query("SELECT 1", "mysql_user", "pass", False) - mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) - mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "mysql_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" - ) - assert logout_records == "mysql_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "mysql_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_postgres_session(started_cluster): - postgres_query("SELECT 1", "postgres_user", "pass", False) - postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) - postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "postgres_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" - ) - assert logout_records == "postgres_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "postgres_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_parallel_sessions(started_cluster): - thread_list = [] - for _ in range(10): - # Sleep time does not significantly matter here, - # test should pass even without sleeping. - for function in [postgres_query, grpc_query, mysql_query]: - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "pass", - False, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "wrong_pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "wrong_parallel_user", - "pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - - for thread in thread_list: - thread.join() - - instance.query("SYSTEM FLUSH LOGS") - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" - ) - assert port_0_sessions == "90\n" - - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" - ) - assert port_0_sessions == "0\n" - - address_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" - ) - assert address_0_sessions == "0\n" - - grpc_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" - ) - assert grpc_sessions == "30\n" - - mysql_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" - ) - assert mysql_sessions == "30\n" - - postgres_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" - ) - assert postgres_sessions == "30\n" - - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "30\n" - - logout_failure_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" - ) - assert logout_failure_sessions == "30\n" diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference deleted file mode 100644 index bfe507e8eac..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.reference +++ /dev/null @@ -1,34 +0,0 @@ -sessions: -150 -port_0_sessions: -0 -address_0_sessions: -0 -tcp_sessions -60 -http_sessions -30 -http_with_session_id_sessions -30 -my_sql_sessions -30 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh deleted file mode 100755 index d1d571c6985..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ /dev/null @@ -1,138 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, long - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -# Each user uses a separate thread. -readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users -readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) -readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) -readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") -readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) - -readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" - -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; -done - -# All _session functions execute in separate threads. -# These functions try to create a session with successful login and logout. -# Sleep a small, random amount of time to make concurrency more intense. -# and try to login with an invalid password. -function tcp_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' - done -} - -function http_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" - done -} - -function http_with_session_id_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" - done -} - -function mysql_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" - - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; -export -f mysql_session; - -for user in "${TCP_USERS[@]}"; do - timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_USERS[@]}"; do - timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do - timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & -done - -for user in "${MYSQL_USERS[@]}"; do - timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & -done - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" - -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" - -echo "tcp_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" -echo "http_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "http_with_session_id_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "my_sql_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" - echo "Corresponding LoginSuccess/Logout" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" - echo "LoginFailure" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" - done diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference deleted file mode 100644 index e2680982ab0..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.reference +++ /dev/null @@ -1,13 +0,0 @@ -0 -0 -0 -0 -client_port 0 connections: -0 -client_address '::' connections: -0 -login failures: -0 -TCP Login and logout count is equal -HTTP Login and logout count is equal -MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh deleted file mode 100755 index 64f20a70cb1..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.sh +++ /dev/null @@ -1,56 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ -readonly TEST_USER=$"02834_USER_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "client_port 0 connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" - -echo "client_address '::' connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" - -echo "login failures:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" - -# remote(...) function sometimes reuses old cached sessions for query execution. -# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. - -for interface in 'TCP' 'HTTP' 'MySQL' -do - LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` - CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` - - if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then - echo "${interface} Login and logout count is equal" - else - TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` - echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" - fi -done - -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference deleted file mode 100644 index 7252faab8c6..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.reference +++ /dev/null @@ -1,8 +0,0 @@ -port_0_sessions: -0 -address_0_sessions: -0 -Corresponding LoginSuccess/Logout -9 -LoginFailure -0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh deleted file mode 100755 index e848e1fe90d..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ /dev/null @@ -1,113 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -readonly TEST_USER="02835_USER_${PID}" -readonly TEST_ROLE="02835_ROLE_${PID}" -readonly TEST_PROFILE="02835_PROFILE_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -function tcp_session() -{ - local user=$1 - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" -} - -function http_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -function http_with_session_id_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. -function wait_for_queries_start() -{ - local user=$1 - local queries_count=$2 - # 10 seconds waiting - counter=0 retries=100 - while [[ $counter -lt $retries ]]; do - result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") - if [[ $result == "${queries_count}" ]]; then - break; - fi - sleep 0.1 - ((++counter)) - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -# DROP USE CASE -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP ROLE CASE -${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP PROFILE CASE -${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" -echo "Corresponding LoginSuccess/Logout" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" -echo "LoginFailure" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From 6edf4a777f33a84db9c18b08741207251fa5fc3d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 03:26:11 +0200 Subject: [PATCH 1810/2047] Remove unrecognizable garbage from the performance test --- docker/test/performance-comparison/compare.sh | 64 ++++++------------- 1 file changed, 21 insertions(+), 43 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 798d2a40b12..c0422816787 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -666,8 +666,8 @@ create view partial_query_times as select * from -- queries with new functions added in the tested PR). create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv') settings output_format_decimal_trailing_zeros = 1 - as select toDecimal64(time_median, 3) time, - toDecimal64(time_stddev / time_median, 3) relative_time_stddev, + as select round(time_median, 3) time, + round(time_stddev / time_median, 3) relative_time_stddev, test, query_index, query_display_name from partial_query_times join query_display_names using (test, query_index) @@ -746,21 +746,21 @@ create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') -- of NaN to decimal. (left > right ? left / right : right / left) as times_change_float, isFinite(times_change_float) as times_change_finite, - toDecimal64(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, + round(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, times_change_finite ? (left > right ? '-' : '+') || toString(times_change_decimal) || 'x' : '--' as times_change_str select - toDecimal64(left, 3), toDecimal64(right, 3), times_change_str, - toDecimal64(diff, 3), toDecimal64(stat_threshold, 3), + round(left, 3), round(right, 3), times_change_str, + round(diff, 3), round(stat_threshold, 3), changed_fail, test, query_index, query_display_name from queries where changed_show order by abs(diff) desc; create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') settings output_format_decimal_trailing_zeros = 1 as select - toDecimal64(left, 3), toDecimal64(right, 3), toDecimal64(diff, 3), - toDecimal64(stat_threshold, 3), unstable_fail, test, query_index, query_display_name + round(left, 3), round(right, 3), round(diff, 3), + round(stat_threshold, 3), unstable_fail, test, query_index, query_display_name from queries where unstable_show order by stat_threshold desc; @@ -792,8 +792,8 @@ create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes settings output_format_decimal_trailing_zeros = 1 as with (times_speedup >= 1 - ? '-' || toString(toDecimal64(times_speedup, 3)) || 'x' - : '+' || toString(toDecimal64(1 / times_speedup, 3)) || 'x') + ? '-' || toString(round(times_speedup, 3)) || 'x' + : '+' || toString(round(1 / times_speedup, 3)) || 'x') as times_speedup_str select test, times_speedup_str, queries, bad, changed, unstable -- Not sure what's the precedence of UNION ALL vs WHERE & ORDER BY, hence all @@ -818,10 +818,10 @@ create view total_client_time_per_query as select * create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') settings output_format_decimal_trailing_zeros = 1 - as select client, server, toDecimal64(client/server, 3) p, + as select client, server, round(client/server, 3) p, test, query_display_name from total_client_time_per_query left join query_display_names using (test, query_index) - where p > toDecimal64(1.02, 3) order by p desc; + where p > round(1.02, 3) order by p desc; create table wall_clock_time_per_test engine Memory as select * from file('wall-clock-times.tsv', TSV, 'test text, real float, user float, system float'); @@ -902,12 +902,12 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') settings output_format_decimal_trailing_zeros = 1 as select test, - toDecimal64(real, 3), - toDecimal64(total_client_time, 3), + round(real, 3), + round(total_client_time, 3), queries, - toDecimal64(query_max, 3), - toDecimal64(avg_real_per_query, 3), - toDecimal64(query_min, 3), + round(query_max, 3), + round(avg_real_per_query, 3), + round(query_min, 3), runs from ( select * from test_times_view @@ -926,14 +926,14 @@ create table all_tests_report engine File(TSV, 'report/all-queries.tsv') -- of NaN to decimal. (left > right ? left / right : right / left) as times_change_float, isFinite(times_change_float) as times_change_finite, - toDecimal64(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, + round(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, times_change_finite ? (left > right ? '-' : '+') || toString(times_change_decimal) || 'x' : '--' as times_change_str select changed_fail, unstable_fail, - toDecimal64(left, 3), toDecimal64(right, 3), times_change_str, - toDecimal64(isFinite(diff) ? diff : 0, 3), - toDecimal64(isFinite(stat_threshold) ? stat_threshold : 0, 3), + round(left, 3), round(right, 3), times_change_str, + round(isFinite(diff) ? diff : 0, 3), + round(isFinite(stat_threshold) ? stat_threshold : 0, 3), test, query_index, query_display_name from queries order by test, query_index; @@ -1044,27 +1044,6 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes, order by count() desc ; -create table metric_devation engine File(TSVWithNamesAndTypes, - 'report/metric-deviation.$version.tsv') - settings output_format_decimal_trailing_zeros = 1 - -- first goes the key used to split the file with grep - as select test, query_index, query_display_name, - toDecimal64(d, 3) d, q, metric - from ( - select - test, query_index, - (q[3] - q[1])/q[2] d, - quantilesExact(0, 0.5, 1)(value) q, metric - from (select * from unstable_run_metrics - union all select * from unstable_run_traces - union all select * from unstable_run_metrics_2) mm - group by test, query_index, metric - having isFinite(d) and d > 0.5 and q[3] > 5 - ) metrics - left join query_display_names using (test, query_index) - order by test, query_index, d desc - ; - create table stacks engine File(TSV, 'report/stacks.$version.tsv') as select -- first goes the key used to split the file with grep @@ -1175,7 +1154,7 @@ create table metrics engine File(TSV, 'metrics/metrics.tsv') as create table changes engine File(TSV, 'metrics/changes.tsv') settings output_format_decimal_trailing_zeros = 1 as select metric, left, right, - toDecimal64(diff, 3), toDecimal64(times_diff, 3) + round(diff, 3), round(times_diff, 3) from ( select metric, median(left) as left, median(right) as right, (right - left) / left diff, @@ -1226,7 +1205,6 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') '$SHA_TO_TEST' :: LowCardinality(String) AS commit_sha, '${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME:-Performance}' :: LowCardinality(String) AS check_name, '$(sed -n 's/.*/\1/p' report.html)' :: LowCardinality(String) AS check_status, - -- TODO toDateTime() can't parse output of 'date', so no time for now. (($(date +%s) - $CHPC_CHECK_START_TIMESTAMP) * 1000) :: UInt64 AS check_duration_ms, fromUnixTimestamp($CHPC_CHECK_START_TIMESTAMP) check_start_time, test_name :: LowCardinality(String) AS test_name , From f2621e70aed39f2bf1a061b4820714f84bf7324a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 03:27:07 +0200 Subject: [PATCH 1811/2047] Remove unrecognizable garbage from the performance test --- docker/test/performance-comparison/compare.sh | 8 -------- 1 file changed, 8 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index c0422816787..f949e66ab17 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -665,7 +665,6 @@ create view partial_query_times as select * from -- Report for backward-incompatible ('partial') queries that we could only run on the new server (e.g. -- queries with new functions added in the tested PR). create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv') - settings output_format_decimal_trailing_zeros = 1 as select round(time_median, 3) time, round(time_stddev / time_median, 3) relative_time_stddev, test, query_index, query_display_name @@ -739,7 +738,6 @@ create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv') ; create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') - settings output_format_decimal_trailing_zeros = 1 as with -- server_time is sometimes reported as zero (if it's less than 1 ms), -- so we have to work around this to not get an error about conversion @@ -757,7 +755,6 @@ create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') from queries where changed_show order by abs(diff) desc; create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') - settings output_format_decimal_trailing_zeros = 1 as select round(left, 3), round(right, 3), round(diff, 3), round(stat_threshold, 3), unstable_fail, test, query_index, query_display_name @@ -789,7 +786,6 @@ create view total_speedup as ; create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes.tsv') - settings output_format_decimal_trailing_zeros = 1 as with (times_speedup >= 1 ? '-' || toString(round(times_speedup, 3)) || 'x' @@ -817,7 +813,6 @@ create view total_client_time_per_query as select * 'test text, query_index int, client float, server float'); create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') - settings output_format_decimal_trailing_zeros = 1 as select client, server, round(client/server, 3) p, test, query_display_name from total_client_time_per_query left join query_display_names using (test, query_index) @@ -899,7 +894,6 @@ create view test_times_view_total as ; create table test_times_report engine File(TSV, 'report/test-times.tsv') - settings output_format_decimal_trailing_zeros = 1 as select test, round(real, 3), @@ -919,7 +913,6 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') -- report for all queries page, only main metric create table all_tests_report engine File(TSV, 'report/all-queries.tsv') - settings output_format_decimal_trailing_zeros = 1 as with -- server_time is sometimes reported as zero (if it's less than 1 ms), -- so we have to work around this to not get an error about conversion @@ -1152,7 +1145,6 @@ create table metrics engine File(TSV, 'metrics/metrics.tsv') as -- Show metrics that have changed create table changes engine File(TSV, 'metrics/changes.tsv') - settings output_format_decimal_trailing_zeros = 1 as select metric, left, right, round(diff, 3), round(times_diff, 3) from ( From 3c6d140c179c29ce890f190529e29d1e9ec3db8b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 10 Aug 2023 01:31:59 +0000 Subject: [PATCH 1812/2047] Fix sigle quote escaping in PostgreSQL engine --- src/Storages/StoragePostgreSQL.cpp | 5 +++++ tests/integration/test_storage_postgresql/test.py | 14 ++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 3551ee36819..11558b39ad3 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -45,6 +45,7 @@ #include +#include namespace DB { @@ -123,6 +124,10 @@ Pipe StoragePostgreSQL::read( column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::DoubleQuotes, remote_table_schema, remote_table_name, context_); + + /// Single quotes in PostgreSQL are escaped through repetition + boost::replace_all(query, "\\'", "''"); + LOG_TRACE(log, "Query: {}", query); Block sample_block; diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 686eb1ea751..3a36d050f17 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -726,6 +726,20 @@ def test_auto_close_connection(started_cluster): assert count == 2 +def test_single_quotes(started_cluster): + cursor = started_cluster.postgres_conn.cursor() + cursor.execute(f"DROP TABLE IF EXISTS single_quote_fails") + cursor.execute(f"CREATE TABLE single_quote_fails(text varchar(255))") + node1.query( + "CREATE TABLE default.single_quote_fails (text String) ENGINE = PostgreSQL('postgres1:5432', 'postgres', 'single_quote_fails', 'postgres', 'mysecretpassword')" + ) + node1.query("SELECT * FROM single_quote_fails WHERE text = ''''") + node1.query("SELECT * FROM single_quote_fails WHERE text = '\\''") + node1.query("SELECT * FROM single_quote_fails WHERE text like '%a''a%'") + node1.query("SELECT * FROM single_quote_fails WHERE text like '%a\\'a%'") + cursor.execute(f"DROP TABLE single_quote_fails") + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 4ba5273939804ed2f639cc4748eaaa7f3b9d4af2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 03:56:27 +0200 Subject: [PATCH 1813/2047] Disable randomization in `02273_full_sort_join` --- tests/queries/0_stateless/02273_full_sort_join.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 6500306356c..2a13e11e116 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-random-settings DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From 0b138fdf16eeeebef564a20928e83e6a7b1d7122 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:15:19 +0200 Subject: [PATCH 1814/2047] Remove outdated Dockerfile --- docker/client/Dockerfile | 34 ---------------------------------- docker/client/README.md | 7 ------- 2 files changed, 41 deletions(-) delete mode 100644 docker/client/Dockerfile delete mode 100644 docker/client/README.md diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile deleted file mode 100644 index 1c185daec75..00000000000 --- a/docker/client/Dockerfile +++ /dev/null @@ -1,34 +0,0 @@ -FROM ubuntu:18.04 - -# ARG for quick switch to a given ubuntu mirror -ARG apt_archive="http://archive.ubuntu.com" -RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list - -ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/" -ARG version=22.1.1.* - -RUN apt-get update \ - && apt-get install --yes --no-install-recommends \ - apt-transport-https \ - ca-certificates \ - dirmngr \ - gnupg \ - && mkdir -p /etc/apt/sources.list.d \ - && apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 \ - && echo $repository > /etc/apt/sources.list.d/clickhouse.list \ - && apt-get update \ - && env DEBIAN_FRONTEND=noninteractive \ - apt-get install --allow-unauthenticated --yes --no-install-recommends \ - clickhouse-client=$version \ - clickhouse-common-static=$version \ - locales \ - tzdata \ - && rm -rf /var/lib/apt/lists/* /var/cache/debconf \ - && apt-get clean - -RUN locale-gen en_US.UTF-8 -ENV LANG en_US.UTF-8 -ENV LANGUAGE en_US:en -ENV LC_ALL en_US.UTF-8 - -ENTRYPOINT ["/usr/bin/clickhouse-client"] diff --git a/docker/client/README.md b/docker/client/README.md deleted file mode 100644 index bbcc7d60794..00000000000 --- a/docker/client/README.md +++ /dev/null @@ -1,7 +0,0 @@ -# ClickHouse Client Docker Image - -For more information see [ClickHouse Server Docker Image](https://hub.docker.com/r/clickhouse/clickhouse-server/). - -## License - -View [license information](https://github.com/ClickHouse/ClickHouse/blob/master/LICENSE) for the software contained in this image. From fc3970e3ff46d6795ac811d51edb29c5b6128a2d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:30:03 +0200 Subject: [PATCH 1815/2047] Update readme --- docker/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/README.md b/docker/README.md index ec52ddd143e..bae02201add 100644 --- a/docker/README.md +++ b/docker/README.md @@ -1,5 +1,5 @@ ## ClickHouse Dockerfiles -This directory contain Dockerfiles for `clickhouse-client` and `clickhouse-server`. They are updated in each release. +This directory contain Dockerfiles for `clickhouse-server`. They are updated in each release. Also there is bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. From 2a89587300a408610591cf251546a6c59affc034 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:32:39 +0200 Subject: [PATCH 1816/2047] Address review comments --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 290222c9d26..0ef25741b18 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -109,8 +109,8 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] fi -# Wait for the server to start -while true +# Wait for the server to start, but not for too long. +for _ in {1..100} do clickhouse-client --query "SELECT 1" && break sleep 1 From 8fba4d2c4a0bf6b9f9061bf19980f98158ef6846 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:41:19 +0200 Subject: [PATCH 1817/2047] Protection if the repository is old --- docker/test/stateless/run.sh | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 0ef25741b18..6c930fa583b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -117,7 +117,7 @@ do done # Initialize export of system logs to ClickHouse Cloud -if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" -a -f /repository/utils/export-logs-in-ci/setup.sh ] then export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type" # TODO: Check if the password will appear in the logs. @@ -131,9 +131,6 @@ then export CLICKHOUSE_CI_LOGS_PASSWORD='' fi -# TODO what is this? Remove it. -sleep 5 - attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 function fn_exists() { From 0ba3fea7fefa1237434e1cca36b8d8ac11fb73e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:44:20 +0200 Subject: [PATCH 1818/2047] Apply recommendations --- docker/test/stateless/Dockerfile | 2 ++ docker/test/stateless/run.sh | 4 ++-- docker/test/stateless/setup_export_logs.sh | 1 + 3 files changed, 5 insertions(+), 2 deletions(-) create mode 120000 docker/test/stateless/setup_export_logs.sh diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 86ca7924f74..48d26c84ed8 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -87,4 +87,6 @@ RUN npm install -g azurite \ COPY run.sh / COPY setup_minio.sh / COPY setup_hdfs_minicluster.sh / +COPY setup_export_logs.sh / + CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 6c930fa583b..015ba7d7423 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -117,13 +117,13 @@ do done # Initialize export of system logs to ClickHouse Cloud -if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" -a -f /repository/utils/export-logs-in-ci/setup.sh ] +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] then export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type" # TODO: Check if the password will appear in the logs. export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}" - /repository/utils/export-logs-in-ci/setup.sh + ./setup_export_logs.sh # Unset variables after use export CONNECTION_PARAMETERS='' diff --git a/docker/test/stateless/setup_export_logs.sh b/docker/test/stateless/setup_export_logs.sh new file mode 120000 index 00000000000..b6e2f981cd4 --- /dev/null +++ b/docker/test/stateless/setup_export_logs.sh @@ -0,0 +1 @@ +../../../utils/export-logs-in-ci/setup.sh \ No newline at end of file From e4fbca2e2e7e21229e2a68254570c7effeaddbf9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:47:44 +0200 Subject: [PATCH 1819/2047] Apply review recommendations --- tests/ci/functional_test_check.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index fdea5be9a6c..c84884bcfaf 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -130,7 +130,6 @@ def get_run_command( return ( f"docker run --volume={builds_path}:/package_folder " f"--volume={repo_path}/tests:/usr/share/clickhouse-test " - f"--volume={repo_path}:/repository " f"{volume_with_broken_test} " f"--volume={result_path}:/test_output --volume={server_log_path}:/var/log/clickhouse-server " f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}" From 931d75ec571882c86374b5a2cdfa7c4b1fe07012 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 05:05:17 +0200 Subject: [PATCH 1820/2047] Maybe better --- src/Core/Field.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 89faaed8a72..de5f7ba9c92 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include namespace DB From 7a6d438b88c03004c105c614749fc7cbab69da42 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 05:11:37 +0200 Subject: [PATCH 1821/2047] Remove magic_enum --- src/Core/Field.cpp | 64 ++++++++++++++++++++++++---------------------- src/Core/Field.h | 2 +- 2 files changed, 34 insertions(+), 32 deletions(-) diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index de5f7ba9c92..1fcf663a744 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -9,9 +9,10 @@ #include #include #include -#include +using namespace std::literals; + namespace DB { @@ -21,12 +22,6 @@ namespace ErrorCodes extern const int DECIMAL_OVERFLOW; } -/// Keep in mind, that "magic_enum" is very expensive for compiler. -std::string_view Field::getTypeName() const -{ - return magic_enum::enum_name(which); -} - inline Field getBinaryValue(UInt8 type, ReadBuffer & buf) { switch (type) @@ -590,34 +585,41 @@ String toString(const Field & x) x); } -String fieldTypeToString(Field::Types::Which type) +std::string_view fieldTypeToString(Field::Types::Which type) { switch (type) { - case Field::Types::Which::Null: return "Null"; - case Field::Types::Which::Array: return "Array"; - case Field::Types::Which::Tuple: return "Tuple"; - case Field::Types::Which::Map: return "Map"; - case Field::Types::Which::Object: return "Object"; - case Field::Types::Which::AggregateFunctionState: return "AggregateFunctionState"; - case Field::Types::Which::Bool: return "Bool"; - case Field::Types::Which::String: return "String"; - case Field::Types::Which::Decimal32: return "Decimal32"; - case Field::Types::Which::Decimal64: return "Decimal64"; - case Field::Types::Which::Decimal128: return "Decimal128"; - case Field::Types::Which::Decimal256: return "Decimal256"; - case Field::Types::Which::Float64: return "Float64"; - case Field::Types::Which::Int64: return "Int64"; - case Field::Types::Which::Int128: return "Int128"; - case Field::Types::Which::Int256: return "Int256"; - case Field::Types::Which::UInt64: return "UInt64"; - case Field::Types::Which::UInt128: return "UInt128"; - case Field::Types::Which::UInt256: return "UInt256"; - case Field::Types::Which::UUID: return "UUID"; - case Field::Types::Which::IPv4: return "IPv4"; - case Field::Types::Which::IPv6: return "IPv6"; - case Field::Types::Which::CustomType: return "CustomType"; + case Field::Types::Which::Null: return "Null"sv; + case Field::Types::Which::Array: return "Array"sv; + case Field::Types::Which::Tuple: return "Tuple"sv; + case Field::Types::Which::Map: return "Map"sv; + case Field::Types::Which::Object: return "Object"sv; + case Field::Types::Which::AggregateFunctionState: return "AggregateFunctionState"sv; + case Field::Types::Which::Bool: return "Bool"sv; + case Field::Types::Which::String: return "String"sv; + case Field::Types::Which::Decimal32: return "Decimal32"sv; + case Field::Types::Which::Decimal64: return "Decimal64"sv; + case Field::Types::Which::Decimal128: return "Decimal128"sv; + case Field::Types::Which::Decimal256: return "Decimal256"sv; + case Field::Types::Which::Float64: return "Float64"sv; + case Field::Types::Which::Int64: return "Int64"sv; + case Field::Types::Which::Int128: return "Int128"sv; + case Field::Types::Which::Int256: return "Int256"sv; + case Field::Types::Which::UInt64: return "UInt64"sv; + case Field::Types::Which::UInt128: return "UInt128"sv; + case Field::Types::Which::UInt256: return "UInt256"sv; + case Field::Types::Which::UUID: return "UUID"sv; + case Field::Types::Which::IPv4: return "IPv4"sv; + case Field::Types::Which::IPv6: return "IPv6"sv; + case Field::Types::Which::CustomType: return "CustomType"sv; } } +/// Keep in mind, that "magic_enum" is very expensive for compiler, that's why we don't use it. +std::string_view Field::getTypeName() const +{ + return fieldTypeToString(which); +} + + } diff --git a/src/Core/Field.h b/src/Core/Field.h index 6666e66e8d5..12542ca0bf1 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -1004,7 +1004,7 @@ void writeFieldText(const Field & x, WriteBuffer & buf); String toString(const Field & x); -String fieldTypeToString(Field::Types::Which type); +std::string_view fieldTypeToString(Field::Types::Which type); } From b8be7eef044052a24d0c00873d150d8de3a5ea4f Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 03:45:59 +0000 Subject: [PATCH 1822/2047] Added new tests for session_log and bugfix --- src/Interpreters/Session.cpp | 44 ++- src/Interpreters/Session.h | 2 + src/Interpreters/SessionLog.cpp | 14 +- src/Interpreters/SessionLog.h | 9 +- src/Server/HTTPHandler.cpp | 3 +- tests/integration/test_session_log/.gitignore | 1 + .../integration/test_session_log/__init__.py | 0 .../test_session_log/configs/log.xml | 9 + .../test_session_log/configs/ports.xml | 9 + .../test_session_log/configs/session_log.xml | 9 + .../test_session_log/configs/users.xml | 23 ++ .../protos/clickhouse_grpc.proto | 1 + tests/integration/test_session_log/test.py | 289 ++++++++++++++++++ .../02833_concurrrent_sessions.reference | 34 +++ .../0_stateless/02833_concurrrent_sessions.sh | 138 +++++++++ .../02834_remote_session_log.reference | 13 + .../0_stateless/02834_remote_session_log.sh | 56 ++++ .../02835_drop_user_during_session.reference | 8 + .../02835_drop_user_during_session.sh | 114 +++++++ 19 files changed, 753 insertions(+), 23 deletions(-) create mode 100644 tests/integration/test_session_log/.gitignore create mode 100644 tests/integration/test_session_log/__init__.py create mode 100644 tests/integration/test_session_log/configs/log.xml create mode 100644 tests/integration/test_session_log/configs/ports.xml create mode 100644 tests/integration/test_session_log/configs/session_log.xml create mode 100644 tests/integration/test_session_log/configs/users.xml create mode 120000 tests/integration/test_session_log/protos/clickhouse_grpc.proto create mode 100644 tests/integration/test_session_log/test.py create mode 100644 tests/queries/0_stateless/02833_concurrrent_sessions.reference create mode 100755 tests/queries/0_stateless/02833_concurrrent_sessions.sh create mode 100644 tests/queries/0_stateless/02834_remote_session_log.reference create mode 100755 tests/queries/0_stateless/02834_remote_session_log.sh create mode 100644 tests/queries/0_stateless/02835_drop_user_during_session.reference create mode 100755 tests/queries/0_stateless/02835_drop_user_during_session.sh diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index de2a779b740..0a6435cff75 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -520,6 +520,8 @@ ContextMutablePtr Session::makeSessionContext() {}, session_context->getSettingsRef().max_sessions_for_user); + recordLoginSucess(session_context); + return session_context; } @@ -582,6 +584,8 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: { session_name_ }, max_sessions_for_user); + recordLoginSucess(session_context); + return session_context; } @@ -655,24 +659,38 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (user_id) user = query_context->getUser(); - if (!notified_session_log_about_login) - { - if (auto session_log = getSessionLog()) - { - session_log->addLoginSuccess( - auth_id, - named_session ? std::optional(named_session->key.second) : std::nullopt, - *query_context, - user); - - notified_session_log_about_login = true; - } - } + /// Interserver does not create session context + recordLoginSucess(query_context); return query_context; } +void Session::recordLoginSucess(ContextPtr login_context) const +{ + if (notified_session_log_about_login) + return; + + if (!login_context) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session or query context must be created"); + + if (auto session_log = getSessionLog()) + { + const auto & settings = login_context->getSettingsRef(); + const auto access = login_context->getAccess(); + + session_log->addLoginSuccess(auth_id, + named_session ? named_session->key.second : "", + settings, + access, + getClientInfo(), + user); + + notified_session_log_about_login = true; + } +} + + void Session::releaseSessionID() { if (!named_session) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 51c0e3c71fa..81ef987b428 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -97,6 +97,8 @@ public: private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; + void recordLoginSucess(ContextPtr login_context) const; + mutable bool notified_session_log_about_login = false; const UUID auth_id; diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 0a8a7fc18c5..b847eaf9824 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -199,12 +199,13 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(auth_failure_reason.data(), auth_failure_reason.length()); } -void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user) +void SessionLog::addLoginSuccess(const UUID & auth_id, + const String & session_id, + const Settings & settings, + const ContextAccessPtr & access, + const ClientInfo & client_info, + const UserPtr & login_user) { - const auto access = login_context.getAccess(); - const auto & settings = login_context.getSettingsRef(); - const auto & client_info = login_context.getClientInfo(); - DB::SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS); log_entry.client_info = client_info; @@ -215,8 +216,7 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional ses } log_entry.external_auth_server = login_user ? login_user->auth_data.getLDAPServerName() : ""; - if (session_id) - log_entry.session_id = *session_id; + log_entry.session_id = session_id; if (const auto roles_info = access->getRolesInfo()) log_entry.roles = roles_info->getCurrentRolesNames(); diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index 1282ac09c4d..8757bc12270 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -20,6 +20,7 @@ enum SessionLogElementType : int8_t class ContextAccess; struct User; using UserPtr = std::shared_ptr; +using ContextAccessPtr = std::shared_ptr; /** A struct which will be inserted as row into session_log table. * @@ -72,7 +73,13 @@ class SessionLog : public SystemLog using SystemLog::SystemLog; public: - void addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user); + void addLoginSuccess(const UUID & auth_id, + const String & session_id, + const Settings & settings, + const ContextAccessPtr & access, + const ClientInfo & client_info, + const UserPtr & login_user); + void addLoginFailure(const UUID & auth_id, const ClientInfo & info, const std::optional & user, const Exception & reason); void addLogOut(const UUID & auth_id, const UserPtr & login_user, const ClientInfo & client_info); }; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index ff5690a3b07..10d5e7a0242 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -561,8 +561,7 @@ void HTTPHandler::processQuery( session->makeSessionContext(); } - auto client_info = session->getClientInfo(); - auto context = session->makeQueryContext(std::move(client_info)); + auto context = session->makeQueryContext(); /// This parameter is used to tune the behavior of output formats (such as Native) for compatibility. if (params.has("client_protocol_version")) diff --git a/tests/integration/test_session_log/.gitignore b/tests/integration/test_session_log/.gitignore new file mode 100644 index 00000000000..edf565ec632 --- /dev/null +++ b/tests/integration/test_session_log/.gitignore @@ -0,0 +1 @@ +_gen diff --git a/tests/integration/test_session_log/__init__.py b/tests/integration/test_session_log/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_session_log/configs/log.xml b/tests/integration/test_session_log/configs/log.xml new file mode 100644 index 00000000000..7a079b81e69 --- /dev/null +++ b/tests/integration/test_session_log/configs/log.xml @@ -0,0 +1,9 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/ports.xml b/tests/integration/test_session_log/configs/ports.xml new file mode 100644 index 00000000000..fbaefc16b3a --- /dev/null +++ b/tests/integration/test_session_log/configs/ports.xml @@ -0,0 +1,9 @@ + + 5433 + 9001 + 9100 + + + false + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml new file mode 100644 index 00000000000..a0e4e3e2216 --- /dev/null +++ b/tests/integration/test_session_log/configs/session_log.xml @@ -0,0 +1,9 @@ + + + system + session_log
+ + toYYYYMM(event_date) + 7500 +
+
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml new file mode 100644 index 00000000000..0416dfadc8a --- /dev/null +++ b/tests/integration/test_session_log/configs/users.xml @@ -0,0 +1,23 @@ + + + + 0 + + + + + + + pass + + + pass + + + pass + + + pass + + + \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto new file mode 120000 index 00000000000..25d15f11e3b --- /dev/null +++ b/tests/integration/test_session_log/protos/clickhouse_grpc.proto @@ -0,0 +1 @@ +../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py new file mode 100644 index 00000000000..dbb39993ce3 --- /dev/null +++ b/tests/integration/test_session_log/test.py @@ -0,0 +1,289 @@ +import os + +import grpc +import pymysql.connections +import psycopg2 as py_psql +import pytest +import random +import sys +import threading + +from helpers.cluster import ClickHouseCluster, run_and_check + +POSTGRES_SERVER_PORT = 5433 +MYSQL_SERVER_PORT = 9001 +GRPC_PORT = 9100 +SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_ENCODING = "utf-8" + +# Use grpcio-tools to generate *pb2.py files from *.proto. +proto_dir = os.path.join(SCRIPT_DIR, "./protos") +gen_dir = os.path.join(SCRIPT_DIR, "./_gen") +os.makedirs(gen_dir, exist_ok=True) +run_and_check( + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", + shell=True, +) + +sys.path.append(gen_dir) + +import clickhouse_grpc_pb2 +import clickhouse_grpc_pb2_grpc + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "node", + main_configs=[ + "configs/ports.xml", + "configs/log.xml", + "configs/session_log.xml", + ], + user_configs=["configs/users.xml"], + # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 + env_variables={ + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") + }, +) + + +def grpc_get_url(): + return f"{instance.ip_address}:{GRPC_PORT}" + + +def grpc_create_insecure_channel(): + channel = grpc.insecure_channel(grpc_get_url()) + grpc.channel_ready_future(channel).result(timeout=2) + return channel + + +session_id_counter = 0 + + +def next_session_id(): + global session_id_counter + session_id = session_id_counter + session_id_counter += 1 + return str(session_id) + + +def grpc_query(query, user_, pass_, raise_exception): + try: + query_info = clickhouse_grpc_pb2.QueryInfo( + query=query, + session_id=next_session_id(), + user_name=user_, + password=pass_, + ) + channel = grpc_create_insecure_channel() + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) + result = stub.ExecuteQuery(query_info) + if result and result.HasField("exception"): + raise Exception(result.exception.display_text) + + return result.output.decode(DEFAULT_ENCODING) + except Exception: + assert raise_exception + + +def postgres_query(query, user_, pass_, raise_exception): + try: + client = py_psql.connect( + host=instance.ip_address, + port=POSTGRES_SERVER_PORT, + user=user_, + password=pass_, + database="default", + ) + cursor = client.cursor() + cursor.execute(query) + cursor.fetchall() + except Exception: + assert raise_exception + + +def mysql_query(query, user_, pass_, raise_exception): + try: + client = pymysql.connections.Connection( + host=instance.ip_address, + user=user_, + password=pass_, + database="default", + port=MYSQL_SERVER_PORT, + ) + cursor = client.cursor(pymysql.cursors.DictCursor) + if raise_exception: + with pytest.raises(Exception): + cursor.execute(query) + else: + cursor.execute(query) + cursor.fetchall() + except Exception: + assert raise_exception + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_grpc_session(started_cluster): + grpc_query("SELECT 1", "grpc_user", "pass", False) + grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) + grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "grpc_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" + ) + assert logout_records == "grpc_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "grpc_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_mysql_session(started_cluster): + mysql_query("SELECT 1", "mysql_user", "pass", False) + mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) + mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "mysql_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" + ) + assert logout_records == "mysql_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "mysql_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_postgres_session(started_cluster): + postgres_query("SELECT 1", "postgres_user", "pass", False) + postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) + postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "postgres_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" + ) + assert logout_records == "postgres_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "postgres_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_parallel_sessions(started_cluster): + thread_list = [] + for _ in range(10): + # Sleep time does not significantly matter here, + # test should pass even without sleeping. + for function in [postgres_query, grpc_query, mysql_query]: + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "pass", + False, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "wrong_pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "wrong_parallel_user", + "pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + + for thread in thread_list: + thread.join() + + instance.query("SYSTEM FLUSH LOGS") + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" + ) + assert port_0_sessions == "90\n" + + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" + ) + assert port_0_sessions == "0\n" + + address_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" + ) + assert address_0_sessions == "0\n" + + grpc_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" + ) + assert grpc_sessions == "30\n" + + mysql_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" + ) + assert mysql_sessions == "30\n" + + postgres_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" + ) + assert postgres_sessions == "30\n" + + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "30\n" + + logout_failure_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" + ) + assert logout_failure_sessions == "30\n" diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference new file mode 100644 index 00000000000..bfe507e8eac --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.reference @@ -0,0 +1,34 @@ +sessions: +150 +port_0_sessions: +0 +address_0_sessions: +0 +tcp_sessions +60 +http_sessions +30 +http_with_session_id_sessions +30 +my_sql_sessions +30 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh new file mode 100755 index 00000000000..26b48462a76 --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -0,0 +1,138 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +# Each user uses a separate thread. +readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users +readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) +readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) +readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") +readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) + +readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" + +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; +done + +# All _session functions execute in separate threads. +# These functions try to create a session with successful login and logout. +# Sleep a small, random amount of time to make concurrency more intense. +# and try to login with an invalid password. +function tcp_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' + done +} + +function http_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" + done +} + +function http_with_session_id_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" + done +} + +function mysql_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" + + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; +export -f mysql_session; + +for user in "${TCP_USERS[@]}"; do + timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_USERS[@]}"; do + timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do + timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & +done + +for user in "${MYSQL_USERS[@]}"; do + timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & +done + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" + +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" + +echo "tcp_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" +echo "http_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "http_with_session_id_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "my_sql_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" + echo "Corresponding LoginSuccess/Logout" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" + echo "LoginFailure" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" + done diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference new file mode 100644 index 00000000000..e2680982ab0 --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.reference @@ -0,0 +1,13 @@ +0 +0 +0 +0 +client_port 0 connections: +0 +client_address '::' connections: +0 +login failures: +0 +TCP Login and logout count is equal +HTTP Login and logout count is equal +MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh new file mode 100755 index 00000000000..3bedfb6c9ee --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ +readonly TEST_USER=$"02834_USER_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "client_port 0 connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" + +echo "client_address '::' connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" + +echo "login failures:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" + +# remote(...) function sometimes reuses old cached sessions for query execution. +# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. + +for interface in 'TCP' 'HTTP' 'MySQL' +do + LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` + CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` + + if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then + echo "${interface} Login and logout count is equal" + else + TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` + echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" + fi +done + +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference new file mode 100644 index 00000000000..7252faab8c6 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.reference @@ -0,0 +1,8 @@ +port_0_sessions: +0 +address_0_sessions: +0 +Corresponding LoginSuccess/Logout +9 +LoginFailure +0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh new file mode 100755 index 00000000000..347ebd22f96 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -0,0 +1,114 @@ +#!/usr/bin/env bash +# Tags: no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +readonly TEST_USER="02835_USER_${PID}" +readonly TEST_ROLE="02835_ROLE_${PID}" +readonly TEST_PROFILE="02835_PROFILE_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +function tcp_session() +{ + local user=$1 + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" +} + +function http_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" +} + +function http_with_session_id_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" +} + +# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. +function wait_for_queries_start() +{ + local user=$1 + local queries_count=$2 + # 10 seconds waiting + counter=0 retries=100 + while [[ $counter -lt $retries ]]; do + result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") + if [[ $result == "${queries_count}" ]]; then + break; + fi + sleep 0.1 + ((++counter)) + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +# DROP USE CASE +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP ROLE CASE +${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP PROFILE CASE +${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" +echo "Corresponding LoginSuccess/Logout" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" +echo "LoginFailure" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From 26aded5062f73e14f428af0dc2f4280fae813964 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 04:11:07 +0000 Subject: [PATCH 1823/2047] Used main connections for suggestions --- src/Client/ClientBase.cpp | 8 ++++ src/Client/Suggest.cpp | 41 ++++++++++++++----- src/Client/Suggest.h | 9 ++++ tests/integration/parallel_skip.json | 3 +- .../test.py | 18 ++++++++ 5 files changed, 68 insertions(+), 11 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a72de2645d4..9e4d79cd323 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -105,6 +105,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int CANNOT_OPEN_FILE; extern const int FILE_ALREADY_EXISTS; + extern const int USER_SESSION_LIMIT_EXCEEDED; } } @@ -2408,6 +2409,13 @@ void ClientBase::runInteractive() } } + if (suggest && suggest->getLastError() == ErrorCodes::USER_SESSION_LIMIT_EXCEEDED) + { + // If a separate connection loading suggestions failed to open a new session, + // use the main session to receive them. + suggest->load(*connection, connection_parameters.timeouts, config().getInt("suggestion_limit")); + } + try { if (!processQueryText(input)) diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 00e0ebd8b91..c854d471fae 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -22,9 +22,11 @@ namespace DB { namespace ErrorCodes { + extern const int OK; extern const int LOGICAL_ERROR; extern const int UNKNOWN_PACKET_FROM_SERVER; extern const int DEADLOCK_AVOIDED; + extern const int USER_SESSION_LIMIT_EXCEEDED; } Suggest::Suggest() @@ -121,21 +123,24 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p } catch (const Exception & e) { + last_error = e.code(); if (e.code() == ErrorCodes::DEADLOCK_AVOIDED) continue; - - /// Client can successfully connect to the server and - /// get ErrorCodes::USER_SESSION_LIMIT_EXCEEDED for suggestion connection. - - /// We should not use std::cerr here, because this method works concurrently with the main thread. - /// WriteBufferFromFileDescriptor will write directly to the file descriptor, avoiding data race on std::cerr. - - WriteBufferFromFileDescriptor out(STDERR_FILENO, 4096); - out << "Cannot load data for command line suggestions: " << getCurrentExceptionMessage(false, true) << "\n"; - out.next(); + else if (e.code() != ErrorCodes::USER_SESSION_LIMIT_EXCEEDED) + { + /// We should not use std::cerr here, because this method works concurrently with the main thread. + /// WriteBufferFromFileDescriptor will write directly to the file descriptor, avoiding data race on std::cerr. + /// + /// USER_SESSION_LIMIT_EXCEEDED is ignored here. The client will try to receive + /// suggestions using the main connection later. + WriteBufferFromFileDescriptor out(STDERR_FILENO, 4096); + out << "Cannot load data for command line suggestions: " << getCurrentExceptionMessage(false, true) << "\n"; + out.next(); + } } catch (...) { + last_error = getCurrentExceptionCode(); WriteBufferFromFileDescriptor out(STDERR_FILENO, 4096); out << "Cannot load data for command line suggestions: " << getCurrentExceptionMessage(false, true) << "\n"; out.next(); @@ -148,6 +153,21 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p }); } +void Suggest::load(IServerConnection & connection, + const ConnectionTimeouts & timeouts, + Int32 suggestion_limit) +{ + try + { + fetch(connection, timeouts, getLoadSuggestionQuery(suggestion_limit, true)); + } + catch (...) + { + std::cerr << "Suggestions loading exception: " << getCurrentExceptionMessage(false, true) << std::endl; + last_error = getCurrentExceptionCode(); + } +} + void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query) { connection.sendQuery( @@ -176,6 +196,7 @@ void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & t return; case Protocol::Server::EndOfStream: + last_error = ErrorCodes::OK; return; default: diff --git a/src/Client/Suggest.h b/src/Client/Suggest.h index cfe9315879c..5cecdc4501b 100644 --- a/src/Client/Suggest.h +++ b/src/Client/Suggest.h @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -28,9 +29,15 @@ public: template void load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit); + void load(IServerConnection & connection, + const ConnectionTimeouts & timeouts, + Int32 suggestion_limit); + /// Older server versions cannot execute the query loading suggestions. static constexpr int MIN_SERVER_REVISION = DBMS_MIN_PROTOCOL_VERSION_WITH_VIEW_IF_PERMITTED; + int getLastError() const { return last_error.load(); } + private: void fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query); @@ -38,6 +45,8 @@ private: /// Words are fetched asynchronously. std::thread loading_thread; + + std::atomic last_error { -1 }; }; } diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index dec51396c51..d056225fee4 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -91,5 +91,6 @@ "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_http_named_session", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_grpc", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp_and_others", - "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_setting_in_query" + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_setting_in_query", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_client_suggestions_load" ] diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 2930262f63e..925fa05881d 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -10,6 +10,7 @@ import threading from helpers.cluster import ClickHouseCluster, run_and_check from helpers.test_tools import assert_logs_contain_with_retry +from helpers.uclient import client, prompt MAX_SESSIONS_FOR_USER = 2 POSTGRES_SERVER_PORT = 5433 @@ -209,3 +210,20 @@ def test_profile_max_sessions_for_user_tcp_and_others(started_cluster): def test_profile_max_sessions_for_user_setting_in_query(started_cluster): instance.query_and_get_error("SET max_sessions_for_user = 10") + + +def test_profile_max_sessions_for_user_client_suggestions_connection(started_cluster): + command_text = f"{started_cluster.get_client_cmd()} --host {instance.ip_address} --port 9000 -u {TEST_USER} --password {TEST_PASSWORD}" + with client(name="client1>", log=None, command=command_text) as client1: + client1.expect(prompt) + with client(name="client2>", log=None, command=command_text) as client2: + client2.expect(prompt) + with client(name="client3>", log=None, command=command_text) as client3: + client3.expect("USER_SESSION_LIMIT_EXCEEDED") + + client1.send("SELECT 'CLIENT_1_SELECT' FORMAT CSV") + client1.expect("CLIENT_1_SELECT") + client1.expect(prompt) + client2.send("SELECT 'CLIENT_2_SELECT' FORMAT CSV") + client2.expect("CLIENT_2_SELECT") + client2.expect(prompt) From 7b0036dee3ef050c060ee2aee0ab9e6faf927e41 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 04:23:40 +0000 Subject: [PATCH 1824/2047] moved notified_session_log_about_login=true line --- src/Interpreters/Session.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 0a6435cff75..f8bd70afdb6 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -685,9 +685,9 @@ void Session::recordLoginSucess(ContextPtr login_context) const access, getClientInfo(), user); - - notified_session_log_about_login = true; } + + notified_session_log_about_login = true; } From d499f8030a34d5076c7a9fc97d94a5f04b2898d6 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 05:20:00 +0000 Subject: [PATCH 1825/2047] reverted change in test --- tests/integration/test_profile_max_sessions_for_user/test.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 2930262f63e..65587933fed 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -27,10 +27,7 @@ proto_dir = os.path.join(SCRIPT_DIR, "./protos") gen_dir = os.path.join(SCRIPT_DIR, "./_gen") os.makedirs(gen_dir, exist_ok=True) run_and_check( - "python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \ - {proto_dir}/clickhouse_grpc.proto".format( - proto_dir=proto_dir, gen_dir=gen_dir - ), + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", shell=True, ) From 7321f5e543387aa65e20fe12421ee692eb05aa64 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 10 Aug 2023 06:32:28 +0000 Subject: [PATCH 1826/2047] Better --- src/IO/WriteHelpers.h | 15 ++++++++--- src/Parsers/ASTLiteral.cpp | 26 +++++++++++++++++-- src/Parsers/IAST.h | 7 ++++- src/Parsers/LiteralEscapingStyle.h | 14 ++++++++++ src/Storages/StorageMySQL.cpp | 1 + src/Storages/StoragePostgreSQL.cpp | 6 +---- src/Storages/StorageSQLite.cpp | 1 + src/Storages/StorageXDBC.cpp | 1 + .../transformQueryForExternalDatabase.cpp | 7 ++++- .../transformQueryForExternalDatabase.h | 1 + .../test_storage_postgresql/test.py | 20 +++++++------- 11 files changed, 78 insertions(+), 21 deletions(-) create mode 100644 src/Parsers/LiteralEscapingStyle.h diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index aa4c9b17e48..d092c7b8ea5 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -304,9 +304,10 @@ inline void writeJSONString(const char * begin, const char * end, WriteBuffer & /** Will escape quote_character and a list of special characters('\b', '\f', '\n', '\r', '\t', '\0', '\\'). * - when escape_quote_with_quote is true, use backslash to escape list of special characters, * and use quote_character to escape quote_character. such as: 'hello''world' - * - otherwise use backslash to escape list of special characters and quote_character + * otherwise use backslash to escape list of special characters and quote_character + * - when escape_backslash_with_backslash is true, backslash is escaped with another backslash */ -template +template void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & buf) { const char * pos = begin; @@ -360,7 +361,8 @@ void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & b writeChar('0', buf); break; case '\\': - writeChar('\\', buf); + if constexpr (escape_backslash_with_backslash) + writeChar('\\', buf); writeChar('\\', buf); break; default: @@ -466,6 +468,13 @@ inline void writeQuotedString(std::string_view ref, WriteBuffer & buf) writeAnyQuotedString<'\''>(ref.data(), ref.data() + ref.size(), buf); } +inline void writeQuotedStringPostgreSQL(std::string_view ref, WriteBuffer & buf) +{ + writeChar('\'', buf); + writeAnyEscapedString<'\'', true, false>(ref.data(), ref.data() + ref.size(), buf); + writeChar('\'', buf); +} + inline void writeDoubleQuotedString(const String & s, WriteBuffer & buf) { writeAnyQuotedString<'"'>(s, buf); diff --git a/src/Parsers/ASTLiteral.cpp b/src/Parsers/ASTLiteral.cpp index 5c76f6f33bf..4a9a3d8df5b 100644 --- a/src/Parsers/ASTLiteral.cpp +++ b/src/Parsers/ASTLiteral.cpp @@ -93,7 +93,7 @@ void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const void ASTLiteral::appendColumnNameImplLegacy(WriteBuffer & ostr) const { - /// 100 - just arbitrary value. + /// 100 - just arbitrary value. constexpr auto min_elements_for_hashing = 100; /// Special case for very large arrays. Instead of listing all elements, will use hash of them. @@ -118,9 +118,31 @@ void ASTLiteral::appendColumnNameImplLegacy(WriteBuffer & ostr) const } } +/// Use different rules for escaping backslashes and quotes +class FieldVisitorToStringPostgreSQL : public StaticVisitor +{ +public: + template + String operator() (const T & x) const { return visitor(x); } + +private: + FieldVisitorToString visitor; +}; + +template<> +String FieldVisitorToStringPostgreSQL::operator() (const String & x) const +{ + WriteBufferFromOwnString wb; + writeQuotedStringPostgreSQL(x, wb); + return wb.str(); +} + void ASTLiteral::formatImplWithoutAlias(const FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const { - settings.ostr << applyVisitor(FieldVisitorToString(), value); + if (settings.literal_escaping_style == LiteralEscapingStyle::Regular) + settings.ostr << applyVisitor(FieldVisitorToString(), value); + else + settings.ostr << applyVisitor(FieldVisitorToStringPostgreSQL(), value); } } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index d217876459f..58bc9702142 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -197,6 +198,7 @@ public: IdentifierQuotingStyle identifier_quoting_style; bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys). char nl_or_ws; /// Newline or whitespace. + LiteralEscapingStyle literal_escaping_style; explicit FormatSettings( WriteBuffer & ostr_, @@ -204,7 +206,8 @@ public: bool hilite_ = false, bool always_quote_identifiers_ = false, IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks, - bool show_secrets_ = true) + bool show_secrets_ = true, + LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular) : ostr(ostr_) , one_line(one_line_) , hilite(hilite_) @@ -212,6 +215,7 @@ public: , identifier_quoting_style(identifier_quoting_style_) , show_secrets(show_secrets_) , nl_or_ws(one_line ? ' ' : '\n') + , literal_escaping_style(literal_escaping_style_) { } @@ -223,6 +227,7 @@ public: , identifier_quoting_style(other.identifier_quoting_style) , show_secrets(other.show_secrets) , nl_or_ws(other.nl_or_ws) + , literal_escaping_style(other.literal_escaping_style) { } diff --git a/src/Parsers/LiteralEscapingStyle.h b/src/Parsers/LiteralEscapingStyle.h new file mode 100644 index 00000000000..10d4d84a85d --- /dev/null +++ b/src/Parsers/LiteralEscapingStyle.h @@ -0,0 +1,14 @@ +#pragma once + + +namespace DB +{ + +/// Method to escape single quotes. +enum class LiteralEscapingStyle +{ + Regular, /// Escape backslashes with backslash (\\) and quotes with backslash (\') + PostgreSQL, /// Do not escape backslashes (\), escape quotes with quote ('') +}; + +} diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index b0a220eb1d2..76a439eabaf 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -104,6 +104,7 @@ Pipe StorageMySQL::read( column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::BackticksMySQL, + LiteralEscapingStyle::Regular, remote_database_name, remote_table_name, context_); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 11558b39ad3..f233d4ff213 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -123,11 +123,7 @@ Pipe StoragePostgreSQL::read( query_info_, column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), - IdentifierQuotingStyle::DoubleQuotes, remote_table_schema, remote_table_name, context_); - - /// Single quotes in PostgreSQL are escaped through repetition - boost::replace_all(query, "\\'", "''"); - + IdentifierQuotingStyle::DoubleQuotes, LiteralEscapingStyle::PostgreSQL, remote_table_schema, remote_table_name, context_); LOG_TRACE(log, "Query: {}", query); Block sample_block; diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index d5ae6f2383f..d5db5763da9 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -91,6 +91,7 @@ Pipe StorageSQLite::read( column_names, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "", remote_table_name, context_); diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index b532d1c91f0..1715cde9d1e 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -79,6 +79,7 @@ std::function StorageXDBC::getReadPOSTDataCallback( column_names, columns_description.getOrdinary(), bridge_helper->getIdentifierQuotingStyle(), + LiteralEscapingStyle::Regular, remote_database_name, remote_table_name, local_context); diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 375510e62bf..84a696a1e9c 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -258,6 +258,7 @@ String transformQueryForExternalDatabaseImpl( Names used_columns, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context) @@ -337,7 +338,8 @@ String transformQueryForExternalDatabaseImpl( IAST::FormatSettings settings( out, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ identifier_quoting_style != IdentifierQuotingStyle::None, - /*identifier_quoting_style*/ identifier_quoting_style); + /*identifier_quoting_style*/ identifier_quoting_style, /*show_secrets_*/ true, + /*literal_escaping_style*/ literal_escaping_style); select->format(settings); @@ -351,6 +353,7 @@ String transformQueryForExternalDatabase( const Names & column_names, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context) @@ -375,6 +378,7 @@ String transformQueryForExternalDatabase( column_names, available_columns, identifier_quoting_style, + literal_escaping_style, database, table, context); @@ -386,6 +390,7 @@ String transformQueryForExternalDatabase( query_info.syntax_analyzer_result->requiredSourceColumns(), available_columns, identifier_quoting_style, + literal_escaping_style, database, table, context); diff --git a/src/Storages/transformQueryForExternalDatabase.h b/src/Storages/transformQueryForExternalDatabase.h index 0f2b0a5822f..fb6af21907e 100644 --- a/src/Storages/transformQueryForExternalDatabase.h +++ b/src/Storages/transformQueryForExternalDatabase.h @@ -31,6 +31,7 @@ String transformQueryForExternalDatabase( const Names & column_names, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context); diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 3a36d050f17..d4f8fab3a82 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -726,18 +726,20 @@ def test_auto_close_connection(started_cluster): assert count == 2 -def test_single_quotes(started_cluster): +def test_literal_escaping(started_cluster): cursor = started_cluster.postgres_conn.cursor() - cursor.execute(f"DROP TABLE IF EXISTS single_quote_fails") - cursor.execute(f"CREATE TABLE single_quote_fails(text varchar(255))") + cursor.execute(f"DROP TABLE IF EXISTS escaping") + cursor.execute(f"CREATE TABLE escaping(text varchar(255))") node1.query( - "CREATE TABLE default.single_quote_fails (text String) ENGINE = PostgreSQL('postgres1:5432', 'postgres', 'single_quote_fails', 'postgres', 'mysecretpassword')" + "CREATE TABLE default.escaping (text String) ENGINE = PostgreSQL('postgres1:5432', 'postgres', 'escaping', 'postgres', 'mysecretpassword')" ) - node1.query("SELECT * FROM single_quote_fails WHERE text = ''''") - node1.query("SELECT * FROM single_quote_fails WHERE text = '\\''") - node1.query("SELECT * FROM single_quote_fails WHERE text like '%a''a%'") - node1.query("SELECT * FROM single_quote_fails WHERE text like '%a\\'a%'") - cursor.execute(f"DROP TABLE single_quote_fails") + node1.query("SELECT * FROM escaping WHERE text = ''''") # ' -> '' + node1.query("SELECT * FROM escaping WHERE text = '\\''") # ' -> '' + node1.query("SELECT * FROM escaping WHERE text = '\\\\\\''") # \' -> \'' + node1.query("SELECT * FROM escaping WHERE text = '\\\\\\''") # \' -> \'' + node1.query("SELECT * FROM escaping WHERE text like '%a''a%'") # %a'a% -> %a''a% + node1.query("SELECT * FROM escaping WHERE text like '%a\\'a%'") # %a'a% -> %a''a% + cursor.execute(f"DROP TABLE escaping") if __name__ == "__main__": From 5cdeacf4cf61c0ac228eb63b7178392a6436d41c Mon Sep 17 00:00:00 2001 From: Ruslan Mardugalliamov Date: Sun, 6 Aug 2023 15:33:36 -0400 Subject: [PATCH 1827/2047] Add hints for HTTP handlers Add hints to HTTP handlers to help users avoid misspellings. For example, if a user mistakenly writes `/dashboad` instead of `/dashboard`, they will now get a hint that /dashboard is the correct handler. This change will improve the user experience by making it easier for users to find the correct handlers. #47662 --- programs/keeper/CMakeLists.txt | 1 + src/Server/HTTPHandlerFactory.cpp | 4 ++++ src/Server/HTTPPathHints.cpp | 16 ++++++++++++++ src/Server/HTTPPathHints.h | 22 +++++++++++++++++++ src/Server/HTTPRequestHandlerFactoryMain.cpp | 2 +- src/Server/HTTPRequestHandlerFactoryMain.h | 4 ++++ src/Server/NotFoundHandler.cpp | 3 ++- src/Server/NotFoundHandler.h | 3 +++ ...ggest_http_page_in_error_message.reference | 4 ++++ ...2842_suggest_http_page_in_error_message.sh | 12 ++++++++++ 10 files changed, 69 insertions(+), 2 deletions(-) create mode 100644 src/Server/HTTPPathHints.cpp create mode 100644 src/Server/HTTPPathHints.h create mode 100644 tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference create mode 100755 tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 43a8d84b513..a43a312ba54 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -57,6 +57,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/IO/ReadBuffer.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTPPathHints.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperTCPHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/TCPServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/NotFoundHandler.cpp diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 78e374ee9e0..1c911034da1 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -132,21 +132,25 @@ void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IS auto ping_handler = std::make_shared>(server, ping_response_expression); ping_handler->attachStrictPath("/ping"); ping_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/ping"); factory.addHandler(ping_handler); auto replicas_status_handler = std::make_shared>(server); replicas_status_handler->attachNonStrictPath("/replicas_status"); replicas_status_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/replicas_status"); factory.addHandler(replicas_status_handler); auto play_handler = std::make_shared>(server); play_handler->attachNonStrictPath("/play"); play_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/play"); factory.addHandler(play_handler); auto dashboard_handler = std::make_shared>(server); dashboard_handler->attachNonStrictPath("/dashboard"); dashboard_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/dashboard"); factory.addHandler(dashboard_handler); auto js_handler = std::make_shared>(server); diff --git a/src/Server/HTTPPathHints.cpp b/src/Server/HTTPPathHints.cpp new file mode 100644 index 00000000000..51ef3eabffe --- /dev/null +++ b/src/Server/HTTPPathHints.cpp @@ -0,0 +1,16 @@ +#include + +namespace DB +{ + +void HTTPPathHints::add(const String & http_path) +{ + http_paths.push_back(http_path); +} + +std::vector HTTPPathHints::getAllRegisteredNames() const +{ + return http_paths; +} + +} diff --git a/src/Server/HTTPPathHints.h b/src/Server/HTTPPathHints.h new file mode 100644 index 00000000000..708816ebf07 --- /dev/null +++ b/src/Server/HTTPPathHints.h @@ -0,0 +1,22 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +class HTTPPathHints : public IHints<1, HTTPPathHints> +{ +public: + std::vector getAllRegisteredNames() const override; + void add(const String & http_path); + +private: + std::vector http_paths; +}; + +using HTTPPathHintsPtr = std::shared_ptr; + +} diff --git a/src/Server/HTTPRequestHandlerFactoryMain.cpp b/src/Server/HTTPRequestHandlerFactoryMain.cpp index 61a2909d30f..5481bcd5083 100644 --- a/src/Server/HTTPRequestHandlerFactoryMain.cpp +++ b/src/Server/HTTPRequestHandlerFactoryMain.cpp @@ -29,7 +29,7 @@ std::unique_ptr HTTPRequestHandlerFactoryMain::createRequest || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) { - return std::unique_ptr(new NotFoundHandler); + return std::unique_ptr(new NotFoundHandler(hints.getHints(request.getURI()))); } return nullptr; diff --git a/src/Server/HTTPRequestHandlerFactoryMain.h b/src/Server/HTTPRequestHandlerFactoryMain.h index b0e57bd6b3b..07b278d831c 100644 --- a/src/Server/HTTPRequestHandlerFactoryMain.h +++ b/src/Server/HTTPRequestHandlerFactoryMain.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -15,11 +16,14 @@ public: void addHandler(HTTPRequestHandlerFactoryPtr child_factory) { child_factories.emplace_back(child_factory); } + void addPathToHints(const std::string & http_path) { hints.add(http_path); } + std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; private: Poco::Logger * log; std::string name; + HTTPPathHints hints; std::vector child_factories; }; diff --git a/src/Server/NotFoundHandler.cpp b/src/Server/NotFoundHandler.cpp index 3181708b9b7..5b1db508551 100644 --- a/src/Server/NotFoundHandler.cpp +++ b/src/Server/NotFoundHandler.cpp @@ -10,7 +10,8 @@ void NotFoundHandler::handleRequest(HTTPServerRequest & request, HTTPServerRespo try { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_NOT_FOUND); - *response.send() << "There is no handle " << request.getURI() << "\n\n" + *response.send() << "There is no handle " << request.getURI() + << (!hints.empty() ? fmt::format(". Maybe you meant {}.", hints.front()) : "") << "\n\n" << "Use / or /ping for health checks.\n" << "Or /replicas_status for more sophisticated health checks.\n\n" << "Send queries from your program with POST method or GET /?query=...\n\n" diff --git a/src/Server/NotFoundHandler.h b/src/Server/NotFoundHandler.h index 749ac388c4d..1cbfcd57f8f 100644 --- a/src/Server/NotFoundHandler.h +++ b/src/Server/NotFoundHandler.h @@ -9,7 +9,10 @@ namespace DB class NotFoundHandler : public HTTPRequestHandler { public: + NotFoundHandler(std::vector hints_) : hints(std::move(hints_)) {} void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override; +private: + std::vector hints; }; } diff --git a/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference new file mode 100644 index 00000000000..0025187be30 --- /dev/null +++ b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference @@ -0,0 +1,4 @@ +There is no handle /sashboards. Maybe you meant /dashboard +There is no handle /sashboard. Maybe you meant /dashboard +There is no handle /sashboarb. Maybe you meant /dashboard +There is no handle /sashboaxb. Maybe you meant /dashboard diff --git a/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh new file mode 100755 index 00000000000..cf69c742777 --- /dev/null +++ b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +export CLICKHOUSE_URL="${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/" + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboards" | grep -o ".* Maybe you meant /dashboard" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboard" | grep -o ".* Maybe you meant /dashboard" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboarb" | grep -o ".* Maybe you meant /dashboard" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboaxb" | grep -o ".* Maybe you meant /dashboard" From 0ff5d12788f1656f61c5b8df2a716675aef02f88 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 11:14:55 +0000 Subject: [PATCH 1828/2047] Added decription to the test + race condition fix --- .../test_profile_max_sessions_for_user/test.py | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 925fa05881d..78e201f88b9 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -214,7 +214,21 @@ def test_profile_max_sessions_for_user_setting_in_query(started_cluster): def test_profile_max_sessions_for_user_client_suggestions_connection(started_cluster): command_text = f"{started_cluster.get_client_cmd()} --host {instance.ip_address} --port 9000 -u {TEST_USER} --password {TEST_PASSWORD}" - with client(name="client1>", log=None, command=command_text) as client1: + command_text_without_suggestions = command_text + " --disable_suggestion" + + # Launch client1 without suggestions to avoid a race condition: + # Client1 opens a session. + # Client1 opens a session for suggestion connection. + # Client2 fails to open a session and gets the USER_SESSION_LIMIT_EXCEEDED error. + # + # Expected order: + # Client1 opens a session. + # Client2 opens a session. + # Client2 fails to open a session for suggestions and with USER_SESSION_LIMIT_EXCEEDED (No error printed). + # Client3 fails to open a session. + # Client1 executes the query. + # Client2 loads suggestions from the server using the main connection and executes a query. + with client(name="client1>", log=None, command=command_text_without_suggestions) as client1: client1.expect(prompt) with client(name="client2>", log=None, command=command_text) as client2: client2.expect(prompt) From 7ed7707ab7e6ccd6b2f26675f3349b29e703b442 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 11:19:16 +0000 Subject: [PATCH 1829/2047] black run --- tests/integration/test_profile_max_sessions_for_user/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 78e201f88b9..c5c33b1cddb 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -228,7 +228,9 @@ def test_profile_max_sessions_for_user_client_suggestions_connection(started_clu # Client3 fails to open a session. # Client1 executes the query. # Client2 loads suggestions from the server using the main connection and executes a query. - with client(name="client1>", log=None, command=command_text_without_suggestions) as client1: + with client( + name="client1>", log=None, command=command_text_without_suggestions + ) as client1: client1.expect(prompt) with client(name="client2>", log=None, command=command_text) as client2: client2.expect(prompt) From 4b30900fed3dcafa015a7d6dc4d6d91ea362a966 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Aug 2023 14:44:16 +0300 Subject: [PATCH 1830/2047] Revert "Improve CHECK TABLE system query" --- .../sql-reference/statements/check-table.md | 107 +++------- src/Common/FileChecker.cpp | 42 ++-- src/Common/FileChecker.h | 37 +--- src/Interpreters/InterpreterCheckQuery.cpp | 201 ++++-------------- src/Storages/CheckResults.h | 2 + src/Storages/IStorage.cpp | 11 - src/Storages/IStorage.h | 41 +--- src/Storages/StorageLog.cpp | 9 +- src/Storages/StorageLog.h | 16 +- src/Storages/StorageMergeTree.cpp | 26 +-- src/Storages/StorageMergeTree.h | 29 +-- src/Storages/StorageProxy.h | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 37 ++-- src/Storages/StorageReplicatedMergeTree.h | 31 +-- src/Storages/StorageStripeLog.cpp | 10 +- src/Storages/StorageStripeLog.h | 17 +- .../0_stateless/00063_check_query.reference | 1 - .../queries/0_stateless/00063_check_query.sql | 3 - .../queries/0_stateless/00961_check_table.sql | 12 +- ...1042_check_query_and_last_granule_size.sql | 12 +- ...02235_check_table_sparse_serialization.sql | 3 +- .../02841_check_table_progress.reference | 2 - .../0_stateless/02841_check_table_progress.sh | 29 --- 23 files changed, 138 insertions(+), 544 deletions(-) delete mode 100644 tests/queries/0_stateless/02841_check_table_progress.reference delete mode 100755 tests/queries/0_stateless/02841_check_table_progress.sh diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index db8c32249ef..0209d59b018 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -5,38 +5,19 @@ sidebar_label: CHECK TABLE title: "CHECK TABLE Statement" --- -The `CHECK TABLE` query in ClickHouse is used to perform a validation check on a specific table or its partitions. It ensures the integrity of the data by verifying the checksums and other internal data structures. +Checks if the data in the table is corrupted. -Particularly it compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. - -:::note -The `CHECK TABLE`` query may read all the data in the table and hold some resources, making it resource-intensive. -Consider the potential impact on performance and resource utilization before executing this query. -::: - -## Syntax - -The basic syntax of the query is as follows: - -```sql -CHECK TABLE table_name [PARTITION partition_expression] [FORMAT format] [SETTINGS check_query_single_value_result = (0|1) [, other_settings] ] +``` sql +CHECK TABLE [db.]name [PARTITION partition_expr] ``` -- `table_name`: Specifies the name of the table that you want to check. -- `partition_expression`: (Optional) If you want to check a specific partition of the table, you can use this expression to specify the partition. -- `FORMAT format`: (Optional) Allows you to specify the output format of the result. -- `SETTINGS`: (Optional) Allows additional settings. - - **`check_query_single_value_result`**: (Optional) This setting allows you to toggle between a detailed result (`0`) or a summarized result (`1`). - - Other settings (e.g. `max_threads` can be applied as well). +The `CHECK TABLE` query compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. +The query response contains the `result` column with a single row. The row has a value of +[Boolean](../../sql-reference/data-types/boolean.md) type: -The query response depends on the value of contains `check_query_single_value_result` setting. -In case of `check_query_single_value_result = 1` only `result` column with a single row is returned. Value inside this row is `1` if the integrity check is passed and `0` if data is corrupted. - -With `check_query_single_value_result = 0` the query returns the following columns: - - `part_path`: Indicates the path to the data part or file name. - - `is_passed`: Returns 1 if the check for this part is successful, 0 otherwise. - - `message`: Any additional messages related to the check, such as errors or success messages. +- 0 - The data in the table is corrupted. +- 1 - The data maintains integrity. The `CHECK TABLE` query supports the following table engines: @@ -45,15 +26,30 @@ The `CHECK TABLE` query supports the following table engines: - [StripeLog](../../engines/table-engines/log-family/stripelog.md) - [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md) -Performed over the tables with another table engines causes an `NOT_IMPLEMETED` exception. +Performed over the tables with another table engines causes an exception. Engines from the `*Log` family do not provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner. -## Examples +## Checking the MergeTree Family Tables -By default `CHECK TABLE` query shows the general table check status: +For `MergeTree` family engines, if [check_query_single_value_result](../../operations/settings/settings.md#check_query_single_value_result) = 0, the `CHECK TABLE` query shows a check status for every individual data part of a table on the local server. ```sql +SET check_query_single_value_result = 0; +CHECK TABLE test_table; +``` + +```text +┌─part_path─┬─is_passed─┬─message─┐ +│ all_1_4_1 │ 1 │ │ +│ all_1_4_2 │ 1 │ │ +└───────────┴───────────┴─────────┘ +``` + +If `check_query_single_value_result` = 1, the `CHECK TABLE` query shows the general table check status. + +```sql +SET check_query_single_value_result = 1; CHECK TABLE test_table; ``` @@ -63,60 +59,11 @@ CHECK TABLE test_table; └────────┘ ``` -If you want to see the check status for every individual data part you may use `check_query_single_value_result` setting. - -Also, to check a specific partition of the table, you can use the `PARTITION` keyword. - -```sql -CHECK TABLE t0 PARTITION ID '201003' -FORMAT PrettyCompactMonoBlock -SETTINGS check_query_single_value_result = 0 -``` - -Output: - -```text -┌─part_path────┬─is_passed─┬─message─┐ -│ 201003_7_7_0 │ 1 │ │ -│ 201003_3_3_0 │ 1 │ │ -└──────────────┴───────────┴─────────┘ -``` - -### Receiving a 'Corrupted' Result - -:::warning -Disclaimer: The procedure described here, including the manual manipulating or removing files directly from the data directory, is for experimental or development environments only. Do **not** attempt this on a production server, as it may lead to data loss or other unintended consequences. -::: - -Remove the existing checksum file: - -```bash -rm /var/lib/clickhouse-server/data/default/t0/201003_3_3_0/checksums.txt -``` - -```sql -CHECK TABLE t0 PARTITION ID '201003' -FORMAT PrettyCompactMonoBlock -SETTINGS check_query_single_value_result = 0 - - -Output: - -```text -┌─part_path────┬─is_passed─┬─message──────────────────────────────────┐ -│ 201003_7_7_0 │ 1 │ │ -│ 201003_3_3_0 │ 1 │ Checksums recounted and written to disk. │ -└──────────────┴───────────┴──────────────────────────────────────────┘ -``` - -If the checksums.txt file is missing, it can be restored. It will be recalculated and rewritten during the execution of the CHECK TABLE command for the specific partition, and the status will still be reported as 'success.'" - - ## If the Data Is Corrupted If the table is corrupted, you can copy the non-corrupted data to another table. To do this: 1. Create a new table with the same structure as damaged table. To do this execute the query `CREATE TABLE AS `. -2. Set the `max_threads` value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. +2. Set the [max_threads](../../operations/settings/settings.md#settings-max_threads) value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. 3. Execute the query `INSERT INTO SELECT * FROM `. This request copies the non-corrupted data from the damaged table to another table. Only the data before the corrupted part will be copied. 4. Restart the `clickhouse-client` to reset the `max_threads` value. diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 122ea83835d..876bc4e641c 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -82,35 +82,33 @@ size_t FileChecker::getTotalSize() const } -FileChecker::DataValidationTasksPtr FileChecker::getDataValidationTasks() +CheckResults FileChecker::check() const { - return std::make_unique(map); -} - -CheckResult FileChecker::checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const -{ - String name; - size_t expected_size; - bool is_finished = check_data_tasks->next(name, expected_size); - if (is_finished) - { - has_nothing_to_do = true; + if (map.empty()) return {}; - } - String path = parentPath(files_info_path) + name; - bool exists = fileReallyExists(path); - auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files. + CheckResults results; - if (real_size != expected_size) + for (const auto & name_size : map) { - String failure_message = exists - ? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(expected_size)) - : ("File " + path + " doesn't exist"); - return CheckResult(name, false, failure_message); + const String & name = name_size.first; + String path = parentPath(files_info_path) + name; + bool exists = fileReallyExists(path); + auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files. + + if (real_size != name_size.second) + { + String failure_message = exists + ? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(name_size.second)) + : ("File " + path + " doesn't exist"); + results.emplace_back(name, false, failure_message); + break; + } + + results.emplace_back(name, true, ""); } - return CheckResult(name, true, ""); + return results; } void FileChecker::repair() diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 8ffc310b84d..bb0383e4b56 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace Poco { class Logger; } @@ -29,11 +28,7 @@ public: bool empty() const { return map.empty(); } /// Check the files whose parameters are specified in sizes.json - /// See comment in IStorage::checkDataNext - struct DataValidationTasks; - using DataValidationTasksPtr = std::unique_ptr; - DataValidationTasksPtr getDataValidationTasks(); - CheckResult checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const; + CheckResults check() const; /// Truncate files that have excessive size to the expected size. /// Throw exception if the file size is less than expected. @@ -46,36 +41,6 @@ public: /// Returns total size of all files. size_t getTotalSize() const; - struct DataValidationTasks - { - DataValidationTasks(const std::map & map_) - : map(map_), it(map.begin()) - {} - - bool next(String & out_name, size_t & out_size) - { - std::lock_guard lock(mutex); - if (it == map.end()) - return true; - out_name = it->first; - out_size = it->second; - ++it; - return false; - } - - size_t size() const - { - std::lock_guard lock(mutex); - return std::distance(it, map.end()); - } - - const std::map & map; - - mutable std::mutex mutex; - using Iterator = std::map::const_iterator; - Iterator it; - }; - private: void load(); diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index bd530654dd2..333aed84873 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -8,201 +8,72 @@ #include #include #include -#include #include -#include - -#include -#include - namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace { -Block getSingleValueBlock(UInt8 value) +NamesAndTypes getBlockStructure() { - return Block{{ColumnUInt8::create(1, value), std::make_shared(), "result"}}; -} - -Block getHeaderForCheckResult() -{ - auto names_and_types = NamesAndTypes{ + return { {"part_path", std::make_shared()}, {"is_passed", std::make_shared()}, {"message", std::make_shared()}, }; - - return Block({ - {names_and_types[0].type->createColumn(), names_and_types[0].type, names_and_types[0].name}, - {names_and_types[1].type->createColumn(), names_and_types[1].type, names_and_types[1].name}, - {names_and_types[2].type->createColumn(), names_and_types[2].type, names_and_types[2].name}, - }); } -Chunk getChunkFromCheckResult(const CheckResult & check_result) -{ - MutableColumns columns = getHeaderForCheckResult().cloneEmptyColumns(); - columns[0]->insert(check_result.fs_path); - columns[1]->insert(static_cast(check_result.success)); - columns[2]->insert(check_result.failure_message); - return Chunk(std::move(columns), 1); -} - -class TableCheckWorkerProcessor : public ISource -{ - -public: - TableCheckWorkerProcessor(IStorage::DataValidationTasksPtr check_data_tasks_, StoragePtr table_) - : ISource(getHeaderForCheckResult()) - , table(table_) - , check_data_tasks(check_data_tasks_) - { - } - - String getName() const override { return "TableCheckWorkerProcessor"; } - -protected: - - std::optional tryGenerate() override - { - bool has_nothing_to_do = false; - auto check_result = table->checkDataNext(check_data_tasks, has_nothing_to_do); - if (has_nothing_to_do) - return {}; - - /// We can omit manual `progess` call, ISource will may count it automatically by returned chunk - /// However, we want to report only rows in progress - progress(1, 0); - - if (!check_result.success) - { - LOG_WARNING(&Poco::Logger::get("InterpreterCheckQuery"), - "Check query for table {} failed, path {}, reason: {}", - table->getStorageID().getNameForLogs(), - check_result.fs_path, - check_result.failure_message); - } - - return getChunkFromCheckResult(check_result); - } - -private: - StoragePtr table; - IStorage::DataValidationTasksPtr check_data_tasks; -}; - -class TableCheckResultEmitter : public IAccumulatingTransform -{ -public: - TableCheckResultEmitter() : IAccumulatingTransform(getHeaderForCheckResult(), getSingleValueBlock(1).cloneEmpty()) {} - - String getName() const override { return "TableCheckResultEmitter"; } - - void consume(Chunk chunk) override - { - if (result_value == 0) - return; - - auto columns = chunk.getColumns(); - if (columns.size() != 3) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong number of columns: {}", columns.size()); - - const auto * col = checkAndGetColumn(columns[1].get()); - for (size_t i = 0; i < col->size(); ++i) - { - if (col->getElement(i) == 0) - { - result_value = 0; - return; - } - } - } - - Chunk generate() override - { - if (is_value_emitted.exchange(true)) - return {}; - auto block = getSingleValueBlock(result_value); - return Chunk(block.getColumns(), block.rows()); - } - -private: - std::atomic result_value{1}; - std::atomic_bool is_value_emitted{false}; -}; - } -InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) - : WithContext(context_) - , query_ptr(query_ptr_) + +InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) { } + BlockIO InterpreterCheckQuery::execute() { const auto & check = query_ptr->as(); - const auto & context = getContext(); - auto table_id = context->resolveStorageID(check, Context::ResolveOrdinary); + auto table_id = getContext()->resolveStorageID(check, Context::ResolveOrdinary); - context->checkAccess(AccessType::SHOW_TABLES, table_id); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); + getContext()->checkAccess(AccessType::SHOW_TABLES, table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); + auto check_results = table->checkData(query_ptr, getContext()); - auto check_data_tasks = table->getCheckTaskList(query_ptr, context); + Block block; + if (getContext()->getSettingsRef().check_query_single_value_result) + { + bool result = std::all_of(check_results.begin(), check_results.end(), [] (const CheckResult & res) { return res.success; }); + auto column = ColumnUInt8::create(); + column->insertValue(static_cast(result)); + block = Block{{std::move(column), std::make_shared(), "result"}}; + } + else + { + auto block_structure = getBlockStructure(); + auto path_column = block_structure[0].type->createColumn(); + auto is_passed_column = block_structure[1].type->createColumn(); + auto message_column = block_structure[2].type->createColumn(); - const auto & settings = context->getSettingsRef(); + for (const auto & check_result : check_results) + { + path_column->insert(check_result.fs_path); + is_passed_column->insert(static_cast(check_result.success)); + message_column->insert(check_result.failure_message); + } + + block = Block({ + {std::move(path_column), block_structure[0].type, block_structure[0].name}, + {std::move(is_passed_column), block_structure[1].type, block_structure[1].name}, + {std::move(message_column), block_structure[2].type, block_structure[2].name}}); + } BlockIO res; - { - auto processors = std::make_shared(); + res.pipeline = QueryPipeline(std::make_shared(std::move(block))); - std::vector worker_ports; - - size_t num_streams = std::max(1, settings.max_threads); - - for (size_t i = 0; i < num_streams; ++i) - { - auto worker_processor = std::make_shared(check_data_tasks, table); - if (i == 0) - worker_processor->addTotalRowsApprox(check_data_tasks->size()); - worker_ports.emplace_back(&worker_processor->getPort()); - processors->emplace_back(worker_processor); - } - - OutputPort * resize_outport; - { - auto resize_processor = std::make_shared(getHeaderForCheckResult(), worker_ports.size(), 1); - - auto & resize_inputs = resize_processor->getInputs(); - auto resize_inport_it = resize_inputs.begin(); - for (size_t i = 0; i < worker_ports.size(); ++i, ++resize_inport_it) - connect(*worker_ports[i], *resize_inport_it); - - resize_outport = &resize_processor->getOutputs().front(); - processors->emplace_back(resize_processor); - } - - if (settings.check_query_single_value_result) - { - auto emitter_processor = std::make_shared(); - auto * input_port = &emitter_processor->getInputPort(); - processors->emplace_back(emitter_processor); - - connect(*resize_outport, *input_port); - } - - res.pipeline = QueryPipeline(Pipe(std::move(processors))); - res.pipeline.setNumThreads(num_streams); - } return res; } diff --git a/src/Storages/CheckResults.h b/src/Storages/CheckResults.h index 2e4652fea29..b342b014fa4 100644 --- a/src/Storages/CheckResults.h +++ b/src/Storages/CheckResults.h @@ -22,4 +22,6 @@ struct CheckResult {} }; +using CheckResults = std::vector; + } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 85299f63165..ae7659e074f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -273,17 +273,6 @@ bool IStorage::isStaticStorage() const return false; } -IStorage::DataValidationTasksPtr IStorage::getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); -} - -CheckResult IStorage::checkDataNext(DataValidationTasksPtr & /* check_task_list */, bool & has_nothing_to_do) -{ - has_nothing_to_do = true; - return {}; -} - void IStorage::adjustCreateQueryForBackup(ASTPtr &) const { } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index b1e20c55782..ec92f57aeda 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -595,45 +595,8 @@ public: /// Provides a hint that the storage engine may evaluate the IN-condition by using an index. virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */, ContextPtr /* query_context */, const StorageMetadataPtr & /* metadata_snapshot */) const { return false; } - - /** A list of tasks to check a validity of data. - * Each IStorage implementation may interpret this task in its own way. - * E.g. for some storages it to check data it need to check a list of files in filesystem, for others it can be a list of parts. - * Also it may hold resources (e.g. locks) required during check. - */ - struct DataValidationTasksBase - { - /// Number of entries left to check. - /// It decreases after each call to checkDataNext(). - virtual size_t size() const = 0; - virtual ~DataValidationTasksBase() = default; - }; - - using DataValidationTasksPtr = std::shared_ptr; - - virtual DataValidationTasksPtr getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */); - - /** Executes one task from the list. - * If no tasks left, sets has_nothing_to_do to true. - * Note: Function `checkDataNext` is accessing `check_task_list` thread-safely, - * and can be called simultaneously for the same `getCheckTaskList` result - * to process different tasks in parallel. - * Usage: - * - * auto check_task_list = storage.getCheckTaskList(query, context); - * size_t total_tasks = check_task_list->size(); - * while (true) - * { - * size_t tasks_left = check_task_list->size(); - * std::cout << "Checking data: " << (total_tasks - tasks_left) << " / " << total_tasks << " tasks done." << std::endl; - * bool has_nothing_to_do = false; - * auto result = storage.checkDataNext(check_task_list, has_nothing_to_do); - * if (has_nothing_to_do) - * break; - * doSomething(result); - * } - */ - virtual CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do); + /// Checks validity of the data + virtual CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); } /// Checks that table could be dropped right now /// Otherwise - throws an exception with detailed information. diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0e9f83e886a..87aa71f3e8d 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -866,18 +866,15 @@ SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetada return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -IStorage::DataValidationTasksPtr StorageLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context) +CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); + + return file_checker.check(); } -CheckResult StorageLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks, has_nothing_to_do); -} IStorage::ColumnSizeByName StorageLog::getColumnSizes() const { diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 95f95088aa2..f1d05ed39ac 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -59,8 +59,7 @@ public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr local_context) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; @@ -143,19 +142,6 @@ private: std::atomic total_rows = 0; std::atomic total_bytes = 0; - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - DataValidationTasks(FileChecker::DataValidationTasksPtr file_checker_tasks_, ReadLock && lock_) - : file_checker_tasks(std::move(file_checker_tasks_)), lock(std::move(lock_)) - {} - - size_t size() const override { return file_checker_tasks->size(); } - - FileChecker::DataValidationTasksPtr file_checker_tasks; - /// Lock to prevent table modification while checking - ReadLock lock; - }; - FileChecker file_checker; const size_t max_compress_block_size; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 97fc7a6731f..ad9013d9f13 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2197,8 +2197,9 @@ void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type) background_moves_assignee.trigger(); } -IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context) +CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) { + CheckResults results; DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) { @@ -2208,14 +2209,7 @@ IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr else data_parts = getVisibleDataPartsVector(local_context); - return std::make_unique(std::move(data_parts), local_context); -} - -CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - auto * data_validation_tasks = assert_cast(check_task_list.get()); - auto local_context = data_validation_tasks->context; - if (auto part = data_validation_tasks->next()) + for (auto & part : data_parts) { /// If the checksums file is not present, calculate the checksums and write them to disk. static constexpr auto checksums_path = "checksums.txt"; @@ -2230,12 +2224,12 @@ CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_ part_mutable.writeChecksums(part->checksums, local_context->getWriteSettings()); part->checkMetadata(); - return CheckResult(part->name, true, "Checksums recounted and written to disk."); + results.emplace_back(part->name, true, "Checksums recounted and written to disk."); } catch (const Exception & ex) { tryLogCurrentException(log, __PRETTY_FUNCTION__); - return CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); } } else @@ -2244,19 +2238,15 @@ CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_ { checkDataPart(part, true); part->checkMetadata(); - return CheckResult(part->name, true, ""); + results.emplace_back(part->name, true, ""); } catch (const Exception & ex) { - return CheckResult(part->name, false, ex.message()); + results.emplace_back(part->name, false, ex.message()); } } } - else - { - has_nothing_to_do = true; - return {}; - } + return results; } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index ec4796e4941..c77e5140d75 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -108,8 +108,7 @@ public: void onActionLockRemove(StorageActionBlockType action_type) override; - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr context) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; @@ -279,32 +278,6 @@ private: friend class MergePlainMergeTreeTask; friend class MutatePlainMergeTreeTask; - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - DataValidationTasks(DataPartsVector && parts_, ContextPtr context_) - : parts(std::move(parts_)), it(parts.begin()), context(std::move(context_)) - {} - - DataPartPtr next() - { - std::lock_guard lock(mutex); - if (it == parts.end()) - return nullptr; - return *(it++); - } - - size_t size() const override - { - std::lock_guard lock(mutex); - return std::distance(it, parts.end()); - } - - mutable std::mutex mutex; - DataPartsVector parts; - DataPartsVector::const_iterator it; - - ContextPtr context; - }; protected: std::map getAlterMutationCommandsForPart(const DataPartPtr & part) const override; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index ea908bea032..21ed4b91c62 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -149,9 +149,7 @@ public: return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override { return getNested()->getCheckTaskList(query, context); } - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override { return getNested()->checkDataNext(check_task_list, has_nothing_to_do); } - + CheckResults checkData(const ASTPtr & query, ContextPtr context) override { return getNested()->checkData(query, context); } void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); } bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); } Strings getDataPaths() const override { return getNested()->getDataPaths(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c08f1ebcc48..7fce373e26b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8481,8 +8481,9 @@ void StorageReplicatedMergeTree::enqueuePartForCheck(const String & part_name, t part_check_thread.enqueuePart(part_name, delay_to_check_seconds); } -IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context) +CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) { + CheckResults results; DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) { @@ -8492,30 +8493,24 @@ IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(co else data_parts = getVisibleDataPartsVector(local_context); - auto part_check_lock = part_check_thread.pausePartsCheck(); - return std::make_unique(std::move(data_parts), std::move(part_check_lock)); -} - -CheckResult StorageReplicatedMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - - if (auto part = assert_cast(check_task_list.get())->next()) { - try + auto part_check_lock = part_check_thread.pausePartsCheck(); + + for (auto & part : data_parts) { - return CheckResult(part_check_thread.checkPartAndFix(part->name)); - } - catch (const Exception & ex) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - return CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + try + { + results.push_back(part_check_thread.checkPartAndFix(part->name)); + } + catch (const Exception & ex) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + } } } - else - { - has_nothing_to_do = true; - return {}; - } + + return results; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 2bc18aa3b0a..78ef39f032f 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -230,8 +230,7 @@ public: /// Add a part to the queue of parts whose data you want to check in the background thread. void enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds = 0); - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr context) override; /// Checks ability to use granularity bool canUseAdaptiveGranularity() const override; @@ -991,34 +990,6 @@ private: bool waitZeroCopyLockToDisappear(const ZeroCopyLock & lock, size_t milliseconds_to_wait) override; void startupImpl(bool from_attach_thread); - - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - explicit DataValidationTasks(DataPartsVector && parts_, std::unique_lock && parts_check_lock_) - : parts_check_lock(std::move(parts_check_lock_)), parts(std::move(parts_)), it(parts.begin()) - {} - - DataPartPtr next() - { - std::lock_guard lock(mutex); - if (it == parts.end()) - return nullptr; - return *(it++); - } - - size_t size() const override - { - std::lock_guard lock(mutex); - return std::distance(it, parts.end()); - } - - std::unique_lock parts_check_lock; - - mutable std::mutex mutex; - DataPartsVector parts; - DataPartsVector::const_iterator it; - }; - }; String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index a3cbff96199..0bfef5ed5e5 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -403,18 +403,16 @@ SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const Storage return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -IStorage::DataValidationTasksPtr StorageStripeLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context) + +CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); + + return file_checker.check(); } -CheckResult StorageStripeLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks, has_nothing_to_do); -} void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 5d4e2fcbd3a..f889a1de71b 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -53,8 +53,7 @@ public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr ocal_context) override; bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } @@ -94,20 +93,6 @@ private: const DiskPtr disk; String table_path; - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - DataValidationTasks(FileChecker::DataValidationTasksPtr file_checker_tasks_, ReadLock && lock_) - : file_checker_tasks(std::move(file_checker_tasks_)), lock(std::move(lock_)) - {} - - size_t size() const override { return file_checker_tasks->size(); } - - FileChecker::DataValidationTasksPtr file_checker_tasks; - - /// Lock to prevent table modification while checking - ReadLock lock; - }; - String data_file_path; String index_file_path; FileChecker file_checker; diff --git a/tests/queries/0_stateless/00063_check_query.reference b/tests/queries/0_stateless/00063_check_query.reference index e8183f05f5d..6ed281c757a 100644 --- a/tests/queries/0_stateless/00063_check_query.reference +++ b/tests/queries/0_stateless/00063_check_query.reference @@ -1,3 +1,2 @@ 1 1 -1 diff --git a/tests/queries/0_stateless/00063_check_query.sql b/tests/queries/0_stateless/00063_check_query.sql index 263cf94fb4a..e7362074a05 100644 --- a/tests/queries/0_stateless/00063_check_query.sql +++ b/tests/queries/0_stateless/00063_check_query.sql @@ -8,9 +8,6 @@ INSERT INTO check_query_tiny_log VALUES (1, 'A'), (2, 'B'), (3, 'C'); CHECK TABLE check_query_tiny_log; --- Settings and FORMAT are supported -CHECK TABLE check_query_tiny_log SETTINGS max_threads = 16; -CHECK TABLE check_query_tiny_log FORMAT Null SETTINGS max_threads = 8, check_query_single_value_result = 0; DROP TABLE IF EXISTS check_query_log; diff --git a/tests/queries/0_stateless/00961_check_table.sql b/tests/queries/0_stateless/00961_check_table.sql index 079acc8cdbb..0e0b2c3b483 100644 --- a/tests/queries/0_stateless/00961_check_table.sql +++ b/tests/queries/0_stateless/00961_check_table.sql @@ -3,29 +3,29 @@ DROP TABLE IF EXISTS mt_table; CREATE TABLE mt_table (d Date, key UInt64, data String) ENGINE = MergeTree() PARTITION BY toYYYYMM(d) ORDER BY key; -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; INSERT INTO mt_table VALUES (toDate('2019-01-02'), 1, 'Hello'), (toDate('2019-01-02'), 2, 'World'); -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; INSERT INTO mt_table VALUES (toDate('2019-01-02'), 3, 'quick'), (toDate('2019-01-02'), 4, 'brown'); SELECT '========'; -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; OPTIMIZE TABLE mt_table FINAL; SELECT '========'; -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; SELECT '========'; INSERT INTO mt_table VALUES (toDate('2019-02-03'), 5, '!'), (toDate('2019-02-03'), 6, '?'); -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; SELECT '========'; @@ -33,6 +33,6 @@ INSERT INTO mt_table VALUES (toDate('2019-02-03'), 7, 'jump'), (toDate('2019-02- OPTIMIZE TABLE mt_table FINAL; -CHECK TABLE mt_table PARTITION 201902 SETTINGS max_threads = 1; +CHECK TABLE mt_table PARTITION 201902; DROP TABLE IF EXISTS mt_table; diff --git a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql index eccb2d25878..b66aff8384d 100644 --- a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql +++ b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql @@ -7,11 +7,11 @@ CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeT -- Rows in this table are short, so granularity will be 8192. INSERT INTO check_query_test SELECT number, toString(number) FROM system.numbers LIMIT 81920; -CHECK TABLE check_query_test SETTINGS max_threads = 1; +CHECK TABLE check_query_test; OPTIMIZE TABLE check_query_test; -CHECK TABLE check_query_test SETTINGS max_threads = 1; +CHECK TABLE check_query_test; DROP TABLE IF EXISTS check_query_test; @@ -21,18 +21,18 @@ CREATE TABLE check_query_test_non_adaptive (SomeKey UInt64, SomeValue String) EN INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 81920; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; OPTIMIZE TABLE check_query_test_non_adaptive; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 77; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; OPTIMIZE TABLE check_query_test_non_adaptive; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; DROP TABLE IF EXISTS check_query_test_non_adaptive; diff --git a/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql b/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql index 625be63e0c0..0ac97404c46 100644 --- a/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql +++ b/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql @@ -12,6 +12,7 @@ SELECT name, column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_02235' ORDER BY name, column; -CHECK TABLE t_sparse_02235 SETTINGS check_query_single_value_result = 0, max_threads = 1; +SET check_query_single_value_result = 0; +CHECK TABLE t_sparse_02235; DROP TABLE t_sparse_02235; diff --git a/tests/queries/0_stateless/02841_check_table_progress.reference b/tests/queries/0_stateless/02841_check_table_progress.reference deleted file mode 100644 index 541dab48def..00000000000 --- a/tests/queries/0_stateless/02841_check_table_progress.reference +++ /dev/null @@ -1,2 +0,0 @@ -Ok -Ok diff --git a/tests/queries/0_stateless/02841_check_table_progress.sh b/tests/queries/0_stateless/02841_check_table_progress.sh deleted file mode 100755 index 166386b999b..00000000000 --- a/tests/queries/0_stateless/02841_check_table_progress.sh +++ /dev/null @@ -1,29 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t0"; -${CLICKHOUSE_CLIENT} -q "CREATE TABLE t0 (x UInt64, val String) ENGINE = MergeTree ORDER BY x PARTITION BY x % 100"; -${CLICKHOUSE_CLIENT} -q "INSERT INTO t0 SELECT sipHash64(number), randomPrintableASCII(1000) FROM numbers(1000)"; - - -# Check that we have at least 3 different values for read_rows -UNIQUE_VALUES=$( - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d @- <<< "CHECK TABLE t0" -v |& { - grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: | grep -o '"read_rows"\s*:\s*"[0-9]*"' - } | uniq | wc -l -) - -[ "$UNIQUE_VALUES" -ge "3" ] && echo "Ok" || echo "Fail: got $UNIQUE_VALUES" - - -# Check that we have we have at least 100 total_rows_to_read (at least one check task per partition) -MAX_TOTAL_VALUE=$( - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d @- <<< "CHECK TABLE t0" -v |& { - grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: | grep -o '"total_rows_to_read"\s*:\s*"[0-9]*"' | grep -o '[0-9]*' - } | sort -n | tail -1 -) - -[ "$MAX_TOTAL_VALUE" -ge "100" ] && echo "Ok" || echo "Fail: got $MAX_TOTAL_VALUE" From 1377d86ed9d6aaf17878b8c7d2960a0053b1111d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 17:01:09 +0300 Subject: [PATCH 1831/2047] Update src/Functions/array/arrayAUC.cpp MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/Functions/array/arrayAUC.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index caf929ba038..b7bd7dcc0ad 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -127,8 +127,8 @@ private: } static void vector( - const IColumn & data1, - const IColumn & data2, + const IColumn & scores, + const IColumn & labels, const ColumnArray::Offsets & offsets, PaddedPODArray & result) { From 635e7e74a86b80519544167eeb2d771a612d6a34 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Aug 2023 16:52:57 +0200 Subject: [PATCH 1832/2047] add garbage --- .../test_s3_zero_copy_ttl/__init__.py | 0 .../configs/max_delayed_streams.xml | 9 ++ .../test_s3_zero_copy_ttl/configs/s3.xml | 39 ++++++++ .../integration/test_s3_zero_copy_ttl/test.py | 94 +++++++++++++++++++ .../test_vertical_merge_memory_usage.py | 46 +++++++++ 5 files changed, 188 insertions(+) create mode 100644 tests/integration/test_s3_zero_copy_ttl/__init__.py create mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml create mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/s3.xml create mode 100644 tests/integration/test_s3_zero_copy_ttl/test.py create mode 100644 tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py diff --git a/tests/integration/test_s3_zero_copy_ttl/__init__.py b/tests/integration/test_s3_zero_copy_ttl/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml b/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml new file mode 100644 index 00000000000..54f7152690b --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml @@ -0,0 +1,9 @@ + + + + + + 10 + + + diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml new file mode 100644 index 00000000000..7bb7fa875e4 --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml @@ -0,0 +1,39 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + + +
+ default +
+ + s3_disk + +
+
+ + +
+ s3_disk +
+
+
+
+
+ + + true + 1.0 + + + true +
diff --git a/tests/integration/test_s3_zero_copy_ttl/test.py b/tests/integration/test_s3_zero_copy_ttl/test.py new file mode 100644 index 00000000000..04bff4a44fb --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/test.py @@ -0,0 +1,94 @@ +#!/usr/bin/env python3 +import time + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) +node2 = cluster.add_instance( + "node2", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) +node3 = cluster.add_instance( + "node3", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_ttl_move_and_s3(started_cluster): + for i, node in enumerate([node1, node2, node3]): + node.query( + """ + CREATE TABLE s3_test_with_ttl (date DateTime, id UInt32, value String) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/s3_test', '{}') + ORDER BY id + PARTITION BY id + TTL date TO DISK 's3_disk' + SETTINGS storage_policy='s3_and_default', temporary_directories_lifetime=1 + """.format( + i + ) + ) + + node1.query("SYSTEM STOP MOVES s3_test_with_ttl") + + node2.query("SYSTEM STOP MOVES s3_test_with_ttl") + + for i in range(30): + if i % 2 == 0: + node = node1 + else: + node = node2 + + node.query( + f"INSERT INTO s3_test_with_ttl SELECT now() + 5, {i}, randomPrintableASCII(1048570)" + ) + + node1.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + node2.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + node3.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + + assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + + node1.query("SYSTEM START MOVES s3_test_with_ttl") + node2.query("SYSTEM START MOVES s3_test_with_ttl") + + assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + + for attempt in reversed(range(5)): + time.sleep(5) + + print( + node1.query( + "SELECT * FROM system.parts WHERE table = 's3_test_with_ttl' FORMAT Vertical" + ) + ) + + minio = cluster.minio_client + objects = minio.list_objects(cluster.minio_bucket, "data/", recursive=True) + counter = 0 + for obj in objects: + print(f"Objectname: {obj.object_name}, metadata: {obj.metadata}") + counter += 1 + + print(f"Total objects: {counter}") + + if counter == 330: + break + + print(f"Attempts remaining: {attempt}") + + assert counter == 330 diff --git a/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py new file mode 100644 index 00000000000..fb9f3eb67b9 --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py @@ -0,0 +1,46 @@ +#!/usr/bin/env python3 +import time + +import pytest +from helpers.cluster import ClickHouseCluster + + +single_node_cluster = ClickHouseCluster(__file__) +small_node = single_node_cluster.add_instance( + "small_node", + main_configs=["configs/s3.xml"], + user_configs=["configs/max_delayed_streams.xml"], + with_minio=True, +) + + +@pytest.fixture(scope="module") +def started_single_node_cluster(): + try: + single_node_cluster.start() + + yield single_node_cluster + finally: + single_node_cluster.shutdown() + + +def test_vertical_merge_memory_usage(started_single_node_cluster): + if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): + pytest.skip("Disabled for debug and sanitizers. Too slow.") + + small_node.query( + "create table tvm2 (c0 UInt64, c1 UInt64, c2 UInt64, c3 UInt64, c4 UInt64, c5 UInt64, c6 UInt64, c7 UInt64, c8 UInt64, c9 UInt64, c10 UInt64, c11 UInt64, c12 UInt64, c13 UInt64, c14 UInt64, c15 UInt64, c16 UInt64, c17 UInt64, c18 UInt64, c19 UInt64, c20 UInt64, c21 UInt64, c22 UInt64, c23 UInt64, c24 UInt64, c25 UInt64, c26 UInt64, c27 UInt64, c28 UInt64, c29 UInt64, c30 UInt64, c31 UInt64, c32 UInt64, c33 UInt64, c34 UInt64, c35 UInt64, c36 UInt64, c37 UInt64, c38 UInt64, c39 UInt64, c40 UInt64, c41 UInt64, c42 UInt64, c43 UInt64, c44 UInt64, c45 UInt64, c46 UInt64, c47 UInt64, c48 UInt64, c49 UInt64, c50 UInt64, c51 UInt64, c52 UInt64, c53 UInt64, c54 UInt64, c55 UInt64, c56 UInt64, c57 UInt64, c58 UInt64, c59 UInt64, c60 UInt64, c61 UInt64, c62 UInt64, c63 UInt64, c64 UInt64, c65 UInt64, c66 UInt64, c67 UInt64, c68 UInt64, c69 UInt64, c70 UInt64, c71 UInt64, c72 UInt64, c73 UInt64, c74 UInt64, c75 UInt64, c76 UInt64, c77 UInt64, c78 UInt64, c79 UInt64, c80 UInt64, c81 UInt64, c82 UInt64, c83 UInt64, c84 UInt64, c85 UInt64, c86 UInt64, c87 UInt64, c88 UInt64, c89 UInt64, c90 UInt64, c91 UInt64, c92 UInt64, c93 UInt64, c94 UInt64, c95 UInt64, c96 UInt64, c97 UInt64, c98 UInt64, c99 UInt64, c100 UInt64, c101 UInt64, c102 UInt64, c103 UInt64, c104 UInt64, c105 UInt64, c106 UInt64, c107 UInt64, c108 UInt64, c109 UInt64, c110 UInt64, c111 UInt64, c112 UInt64, c113 UInt64, c114 UInt64, c115 UInt64, c116 UInt64, c117 UInt64, c118 UInt64, c119 UInt64, c120 UInt64, c121 UInt64, c122 UInt64, c123 UInt64, c124 UInt64, c125 UInt64, c126 UInt64, c127 UInt64, c128 UInt64, c129 UInt64, c130 UInt64, c131 UInt64, c132 UInt64, c133 UInt64, c134 UInt64, c135 UInt64, c136 UInt64, c137 UInt64, c138 UInt64, c139 UInt64, c140 UInt64, c141 UInt64, c142 UInt64, c143 UInt64, c144 UInt64, c145 UInt64, c146 UInt64, c147 UInt64, c148 UInt64, c149 UInt64, c150 UInt64, c151 UInt64, c152 UInt64, c153 UInt64, c154 UInt64, c155 UInt64, c156 UInt64, c157 UInt64, c158 UInt64, c159 UInt64, c160 UInt64, c161 UInt64, c162 UInt64, c163 UInt64, c164 UInt64, c165 UInt64, c166 UInt64, c167 UInt64, c168 UInt64, c169 UInt64, c170 UInt64, c171 UInt64, c172 UInt64, c173 UInt64, c174 UInt64, c175 UInt64, c176 UInt64, c177 UInt64, c178 UInt64, c179 UInt64, c180 UInt64, c181 UInt64, c182 UInt64, c183 UInt64, c184 UInt64, c185 UInt64, c186 UInt64, c187 UInt64, c188 UInt64, c189 UInt64, c190 UInt64, c191 UInt64, c192 UInt64, c193 UInt64, c194 UInt64, c195 UInt64, c196 UInt64, c197 UInt64, c198 UInt64, c199 UInt64, c200 UInt64, c201 UInt64, c202 UInt64, c203 UInt64, c204 UInt64, c205 UInt64, c206 UInt64, c207 UInt64, c208 UInt64, c209 UInt64, c210 UInt64, c211 UInt64, c212 UInt64, c213 UInt64, c214 UInt64, c215 UInt64, c216 UInt64, c217 UInt64, c218 UInt64, c219 UInt64, c220 UInt64, c221 UInt64, c222 UInt64, c223 UInt64, c224 UInt64, c225 UInt64, c226 UInt64, c227 UInt64, c228 UInt64, c229 UInt64, c230 UInt64, c231 UInt64, c232 UInt64, c233 UInt64, c234 UInt64, c235 UInt64, c236 UInt64, c237 UInt64, c238 UInt64, c239 UInt64, c240 UInt64, c241 UInt64, c242 UInt64, c243 UInt64, c244 UInt64, c245 UInt64, c246 UInt64, c247 UInt64, c248 UInt64, c249 UInt64, c250 UInt64, c251 UInt64, c252 UInt64, c253 UInt64, c254 UInt64, c255 UInt64, c256 UInt64, c257 UInt64, c258 UInt64, c259 UInt64, c260 UInt64, c261 UInt64, c262 UInt64, c263 UInt64, c264 UInt64, c265 UInt64, c266 UInt64, c267 UInt64, c268 UInt64, c269 UInt64, c270 UInt64, c271 UInt64, c272 UInt64, c273 UInt64, c274 UInt64, c275 UInt64, c276 UInt64, c277 UInt64, c278 UInt64, c279 UInt64, c280 UInt64, c281 UInt64, c282 UInt64, c283 UInt64, c284 UInt64, c285 UInt64, c286 UInt64, c287 UInt64, c288 UInt64, c289 UInt64, c290 UInt64, c291 UInt64, c292 UInt64, c293 UInt64, c294 UInt64, c295 UInt64, c296 UInt64, c297 UInt64, c298 UInt64, c299 UInt64) engine = MergeTree order by tuple() settings min_rows_for_wide_part = 10, min_bytes_for_wide_part=0, storage_policy = 's3', vertical_merge_algorithm_min_rows_to_activate=1" + ) + + small_node.query( + "insert into tvm2 select number + 0, number + 1, number + 2, number + 3, number + 4, number + 5, number + 6, number + 7, number + 8, number + 9, number + 10, number + 11, number + 12, number + 13, number + 14, number + 15, number + 16, number + 17, number + 18, number + 19, number + 20, number + 21, number + 22, number + 23, number + 24, number + 25, number + 26, number + 27, number + 28, number + 29, number + 30, number + 31, number + 32, number + 33, number + 34, number + 35, number + 36, number + 37, number + 38, number + 39, number + 40, number + 41, number + 42, number + 43, number + 44, number + 45, number + 46, number + 47, number + 48, number + 49, number + 50, number + 51, number + 52, number + 53, number + 54, number + 55, number + 56, number + 57, number + 58, number + 59, number + 60, number + 61, number + 62, number + 63, number + 64, number + 65, number + 66, number + 67, number + 68, number + 69, number + 70, number + 71, number + 72, number + 73, number + 74, number + 75, number + 76, number + 77, number + 78, number + 79, number + 80, number + 81, number + 82, number + 83, number + 84, number + 85, number + 86, number + 87, number + 88, number + 89, number + 90, number + 91, number + 92, number + 93, number + 94, number + 95, number + 96, number + 97, number + 98, number + 99, number + 100, number + 101, number + 102, number + 103, number + 104, number + 105, number + 106, number + 107, number + 108, number + 109, number + 110, number + 111, number + 112, number + 113, number + 114, number + 115, number + 116, number + 117, number + 118, number + 119, number + 120, number + 121, number + 122, number + 123, number + 124, number + 125, number + 126, number + 127, number + 128, number + 129, number + 130, number + 131, number + 132, number + 133, number + 134, number + 135, number + 136, number + 137, number + 138, number + 139, number + 140, number + 141, number + 142, number + 143, number + 144, number + 145, number + 146, number + 147, number + 148, number + 149, number + 150, number + 151, number + 152, number + 153, number + 154, number + 155, number + 156, number + 157, number + 158, number + 159, number + 160, number + 161, number + 162, number + 163, number + 164, number + 165, number + 166, number + 167, number + 168, number + 169, number + 170, number + 171, number + 172, number + 173, number + 174, number + 175, number + 176, number + 177, number + 178, number + 179, number + 180, number + 181, number + 182, number + 183, number + 184, number + 185, number + 186, number + 187, number + 188, number + 189, number + 190, number + 191, number + 192, number + 193, number + 194, number + 195, number + 196, number + 197, number + 198, number + 199, number + 200, number + 201, number + 202, number + 203, number + 204, number + 205, number + 206, number + 207, number + 208, number + 209, number + 210, number + 211, number + 212, number + 213, number + 214, number + 215, number + 216, number + 217, number + 218, number + 219, number + 220, number + 221, number + 222, number + 223, number + 224, number + 225, number + 226, number + 227, number + 228, number + 229, number + 230, number + 231, number + 232, number + 233, number + 234, number + 235, number + 236, number + 237, number + 238, number + 239, number + 240, number + 241, number + 242, number + 243, number + 244, number + 245, number + 246, number + 247, number + 248, number + 249, number + 250, number + 251, number + 252, number + 253, number + 254, number + 255, number + 256, number + 257, number + 258, number + 259, number + 260, number + 261, number + 262, number + 263, number + 264, number + 265, number + 266, number + 267, number + 268, number + 269, number + 270, number + 271, number + 272, number + 273, number + 274, number + 275, number + 276, number + 277, number + 278, number + 279, number + 280, number + 281, number + 282, number + 283, number + 284, number + 285, number + 286, number + 287, number + 288, number + 289, number + 290, number + 291, number + 292, number + 293, number + 294, number + 295, number + 296, number + 297, number + 298, number + 299 from numbers(20)" + ) + small_node.query("optimize table tvm2 final") + small_node.query("system flush logs") + + # Should be about 25M + res = small_node.query( + "select formatReadableSize(peak_memory_usage), * from system.part_log where table = 'tvm2' and database = currentDatabase() and event_date >= today() - 1 and event_type = 'MergeParts' and peak_memory_usage > 100000000 format Vertical" + ) + + assert res == "" From 708fd914bc708345fcc46fadd19b47eceddc786f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 5 Aug 2023 13:47:57 +0200 Subject: [PATCH 1833/2047] Fix 02263_format_insert_settings flakiness I guess the problem was with the async nature of the process substitution ("2> >(cmd)"), let's avoid using this feature of bash. CI: https://s3.amazonaws.com/clickhouse-test-reports/52683/b98cb7fa145d1a92c2c78421be1eeb8fe8353d53/stateless_tests__aarch64_.html Signed-off-by: Azat Khuzhin Co-authored-by: Alexey Milovidov Update 02263_format_insert_settings.sh --- .../02263_format_insert_settings.reference | 6 +++--- .../0_stateless/02263_format_insert_settings.sh | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02263_format_insert_settings.reference b/tests/queries/0_stateless/02263_format_insert_settings.reference index 721e7960875..e2d1ec3980e 100644 --- a/tests/queries/0_stateless/02263_format_insert_settings.reference +++ b/tests/queries/0_stateless/02263_format_insert_settings.reference @@ -1,6 +1,6 @@ -insert into foo settings max_threads=1 +[multi] insert into foo settings max_threads=1 Syntax error (query): failed at position 40 (end of query): -insert into foo format tsv settings max_threads=1 +[multi] insert into foo format tsv settings max_threads=1 Can't format ASTInsertQuery with data, since data will be lost. [multi] insert into foo format tsv settings max_threads=1 INSERT INTO foo @@ -8,7 +8,7 @@ SETTINGS max_threads = 1 FORMAT tsv [oneline] insert into foo format tsv settings max_threads=1 INSERT INTO foo SETTINGS max_threads = 1 FORMAT tsv -insert into foo settings max_threads=1 format tsv settings max_threads=1 +[multi] insert into foo settings max_threads=1 format tsv settings max_threads=1 You have SETTINGS before and after FORMAT Cannot parse input: expected '\n' before: 'settings max_threads=1 1' 1 diff --git a/tests/queries/0_stateless/02263_format_insert_settings.sh b/tests/queries/0_stateless/02263_format_insert_settings.sh index efb3d39ab6c..8b156ffec83 100755 --- a/tests/queries/0_stateless/02263_format_insert_settings.sh +++ b/tests/queries/0_stateless/02263_format_insert_settings.sh @@ -8,7 +8,7 @@ function run_format() { local q="$1" && shift - echo "$q" + echo "[multi] $q" $CLICKHOUSE_FORMAT "$@" <<<"$q" } function run_format_both() @@ -22,20 +22,20 @@ function run_format_both() } # NOTE: that those queries may work slow, due to stack trace obtaining -run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):") +run_format 'insert into foo settings max_threads=1' |& grep --max-count 2 --only-matching -e "Syntax error (query): failed at position .* (end of query):" -e '^\[.*$' # compatibility -run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.") +run_format 'insert into foo format tsv settings max_threads=1' |& grep --max-count 2 --only-matching -e "Can't format ASTInsertQuery with data, since data will be lost." -e '^\[.*$' run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert -run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") +run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert |& grep --max-count 2 --only-matching -e "You have SETTINGS before and after FORMAT" -e '^\[.*$' # and via server (since this is a separate code path) $CLICKHOUSE_CLIENT -q 'drop table if exists data_02263' $CLICKHOUSE_CLIENT -q 'create table data_02263 (key Int) engine=Memory()' -$CLICKHOUSE_CLIENT -q 'insert into data_02263 format TSV settings max_threads=1 1' 2> >(grep -m1 -F -o "Cannot parse input: expected '\n' before: 'settings max_threads=1 1'") +$CLICKHOUSE_CLIENT -q 'insert into data_02263 format TSV settings max_threads=1 1' |& grep --max-count 1 -F --only-matching "Cannot parse input: expected '\n' before: 'settings max_threads=1 1'" $CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 format TSV settings max_threads=1 1' $CLICKHOUSE_CLIENT -q 'select * from data_02263' -$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 settings max_threads=1 format tsv settings max_threads=1' 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") +$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 settings max_threads=1 format tsv settings max_threads=1' |& grep --max-count 1 -F --only-matching "You have SETTINGS before and after FORMAT" $CLICKHOUSE_CLIENT -q 'drop table data_02263' run_format_both 'insert into foo values' From cda633a1f66d68f282722d9cdfa7ab445a50f9cc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 15:48:51 +0000 Subject: [PATCH 1834/2047] different linker --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 23d14d6ed26..4d4d1fab473 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=bfd") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=bfd") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=bfd") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 4f0be777c5e186f3ebeabd5c1d8fd9adc2fe761b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 10 Aug 2023 13:18:32 +0000 Subject: [PATCH 1835/2047] Fix build --- .../tests/gtest_transform_query_for_external_database.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 5c1442ece11..749a154c19d 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -127,7 +127,8 @@ static void checkOld( std::string transformed_query = transformQueryForExternalDatabase( query_info, query_info.syntax_analyzer_result->requiredSourceColumns(), - state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context); + state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "test", "table", state.context); EXPECT_EQ(transformed_query, expected) << query; } @@ -180,7 +181,8 @@ static void checkNewAnalyzer( query_info.table_expression = findTableExpression(query_node->getJoinTree(), "table"); std::string transformed_query = transformQueryForExternalDatabase( - query_info, column_names, state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context); + query_info, column_names, state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "test", "table", state.context); EXPECT_EQ(transformed_query, expected) << query; } From 3d59ebe108016a83bba161751f728b08d5f94d70 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Aug 2023 20:11:22 +0200 Subject: [PATCH 1836/2047] fix --- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataPartInMemory.h | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 5 ++- .../test.py | 45 ++++++++++--------- 5 files changed, 31 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 1df091ab1a3..195fdbc4d05 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -369,7 +369,7 @@ public: /// Makes clone of a part in detached/ directory via hard links virtual DataPartStoragePtr makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot, - const DiskTransactionPtr & disk_transaction = {}) const; + const DiskTransactionPtr & disk_transaction = {}) const; /// NOLINT /// Makes full clone of part in specified subdirectory (relative to storage data directory, e.g. "detached") on another disk MutableDataPartStoragePtr makeCloneOnDisk(const DiskPtr & disk, const String & directory_name) const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ed9127de977..395b480a84f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2636,7 +2636,7 @@ size_t MergeTreeData::clearOldBrokenPartsFromDetachedDirectory() if (!part_info.dir_name.starts_with("deleting_")) continue; - time_t startup_time = current_time + static_cast(Context::getGlobalContextInstance()->getUptimeSeconds()); + time_t startup_time = current_time - static_cast(Context::getGlobalContextInstance()->getUptimeSeconds()); time_t last_touch_time = get_last_touched_time(part_info); /// Maybe it's being deleted right now (for example, in ALTER DROP DETACHED) diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 29506a54fdc..95a17cbf589 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -43,7 +43,7 @@ public: String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; } void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) override; DataPartStoragePtr makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot, - const DiskTransactionPtr & disk_transaction = {}) const override; + const DiskTransactionPtr & disk_transaction = {}) const override; /// NOLINT std::optional getColumnModificationTime(const String & /* column_name */) const override { return {}; } MutableDataPartStoragePtr flushToDisk(const String & new_relative_path, const StorageMetadataPtr & metadata_snapshot) const; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index bf0acef89c2..fa5a40cf27a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -651,7 +651,10 @@ bool ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData::Mutabl part->version.setCreationTID(Tx::PrehistoricTID, nullptr); String block_id = deduplicate ? fmt::format("{}_{}", part->info.partition_id, part->checksums.getTotalChecksumHex()) : ""; bool deduplicated = commitPart(zookeeper, part, block_id, replicas_num, /* writing_existing_part */ true).second; - PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, watch.elapsed(), profile_events_scope.getSnapshot())); + + /// Set a special error code if the block is duplicate + int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; + PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, watch.elapsed(), profile_events_scope.getSnapshot()), ExecutionStatus(error)); return deduplicated; } catch (...) diff --git a/tests/integration/test_broken_detached_part_clean_up/test.py b/tests/integration/test_broken_detached_part_clean_up/test.py index e7341deae35..bdf993ddedf 100644 --- a/tests/integration/test_broken_detached_part_clean_up/test.py +++ b/tests/integration/test_broken_detached_part_clean_up/test.py @@ -57,7 +57,11 @@ def remove_broken_detached_part_impl(table, node, expect_broken_prefix): ] ) - for name in ['unexpected_all_42_1337_5', 'deleting_all_123_456_7', 'tmp-fetch_all_12_34_5']: + for name in [ + "unexpected_all_42_1337_5", + "deleting_all_123_456_7", + "covered-by-broken_all_12_34_5", + ]: node.exec_in_container(["mkdir", f"{path_to_detached}../{name}"]) node.exec_in_container( [ @@ -67,9 +71,7 @@ def remove_broken_detached_part_impl(table, node, expect_broken_prefix): f"{path_to_detached}../{name}", ] ) - result = node.exec_in_container( - ["stat", f"{path_to_detached}../{name}"] - ) + result = node.exec_in_container(["stat", f"{path_to_detached}../{name}"]) print(result) assert "Modify: 2013-12-03" in result node.exec_in_container( @@ -89,21 +91,19 @@ def remove_broken_detached_part_impl(table, node, expect_broken_prefix): node.query(f"ATTACH TABLE {table}") node.wait_for_log_line( - "Removing detached part deleting_all_123_456_7", timeout=90, look_behind_lines=1000000 + "Removing detached part deleting_all_123_456_7", + timeout=90, + look_behind_lines=1000000, ) - - result = node.exec_in_container(["ls", path_to_detached]) - print(result) - assert f"{expect_broken_prefix}_all_3_3_0" in result - assert "all_1_1_0" in result - assert "trash" in result - assert "broken_all_fake" in result - assert "unexpected_all_42_1337_5" in result - assert "deleting_all_123_456_7" not in result - - time.sleep(15) - assert node.contains_in_log( - "Removed broken detached part unexpected_all_42_1337_5 due to a timeout" + node.wait_for_log_line( + f"Removed broken detached part {expect_broken_prefix}_all_3_3_0 due to a timeout", + timeout=10, + look_behind_lines=1000000, + ) + node.wait_for_log_line( + "Removed broken detached part unexpected_all_42_1337_5 due to a timeout", + timeout=10, + look_behind_lines=1000000, ) result = node.exec_in_container(["ls", path_to_detached]) @@ -112,13 +112,16 @@ def remove_broken_detached_part_impl(table, node, expect_broken_prefix): assert "all_1_1_0" in result assert "trash" in result assert "broken_all_fake" in result - assert "tmp-fetch_all_12_34_5" in result + assert "covered-by-broken_all_12_34_5" in result assert "unexpected_all_42_1337_5" not in result assert "deleting_all_123_456_7" not in result - node.query(f"ALTER TABLE {table} DROP DETACHED PART 'tmp-fetch_all_12_34_5'", settings={"allow_drop_detached": 1}) + node.query( + f"ALTER TABLE {table} DROP DETACHED PART 'covered-by-broken_all_12_34_5'", + settings={"allow_drop_detached": 1}, + ) result = node.exec_in_container(["ls", path_to_detached]) - assert "tmp-fetch_all_12_34_5" not in result + assert "covered-by-broken_all_12_34_5" not in result node.query(f"DROP TABLE {table} SYNC") From 3acb10b8bc08077c9ac39149517709ddc2c3163a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 20:51:03 +0200 Subject: [PATCH 1837/2047] Inhibit randomization in `00906_low_cardinality_cache` --- tests/ci/stress.py | 2 +- tests/queries/0_stateless/00906_low_cardinality_cache.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index eb829cf519c..2c566144f2c 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -302,7 +302,7 @@ if __name__ == "__main__": have_long_running_queries = prepare_for_hung_check(args.drop_databases) except Exception as ex: have_long_running_queries = True - logging.error("Failed to prepare for hung check %s", str(ex)) + logging.error("Failed to prepare for hung check: %s", str(ex)) logging.info("Checking if some queries hung") cmd = " ".join( [ diff --git a/tests/queries/0_stateless/00906_low_cardinality_cache.sql b/tests/queries/0_stateless/00906_low_cardinality_cache.sql index cd2ceabcf6d..55eacd0db44 100644 --- a/tests/queries/0_stateless/00906_low_cardinality_cache.sql +++ b/tests/queries/0_stateless/00906_low_cardinality_cache.sql @@ -1,5 +1,5 @@ drop table if exists lc_00906; -create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b; +create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into lc_00906 select '0123456789' from numbers(100000000); select count(), b from lc_00906 group by b; drop table if exists lc_00906; From 3aca2408548bc149f933379506250e49238a24de Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 20:59:33 +0200 Subject: [PATCH 1838/2047] Change the default of max_concurrent_queries from 100 to 1000 --- programs/server/config.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 14b8954fc39..85cdda63558 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -317,7 +317,7 @@ 0 - 100 + 1000 - false - - \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml deleted file mode 100644 index a0e4e3e2216..00000000000 --- a/tests/integration/test_session_log/configs/session_log.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - system - session_log
- - toYYYYMM(event_date) - 7500 -
-
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml deleted file mode 100644 index 0416dfadc8a..00000000000 --- a/tests/integration/test_session_log/configs/users.xml +++ /dev/null @@ -1,23 +0,0 @@ - - - - 0 - - - - - - - pass - - - pass - - - pass - - - pass - - - \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto deleted file mode 120000 index 25d15f11e3b..00000000000 --- a/tests/integration/test_session_log/protos/clickhouse_grpc.proto +++ /dev/null @@ -1 +0,0 @@ -../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py deleted file mode 100644 index dbb39993ce3..00000000000 --- a/tests/integration/test_session_log/test.py +++ /dev/null @@ -1,289 +0,0 @@ -import os - -import grpc -import pymysql.connections -import psycopg2 as py_psql -import pytest -import random -import sys -import threading - -from helpers.cluster import ClickHouseCluster, run_and_check - -POSTGRES_SERVER_PORT = 5433 -MYSQL_SERVER_PORT = 9001 -GRPC_PORT = 9100 -SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -DEFAULT_ENCODING = "utf-8" - -# Use grpcio-tools to generate *pb2.py files from *.proto. -proto_dir = os.path.join(SCRIPT_DIR, "./protos") -gen_dir = os.path.join(SCRIPT_DIR, "./_gen") -os.makedirs(gen_dir, exist_ok=True) -run_and_check( - f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", - shell=True, -) - -sys.path.append(gen_dir) - -import clickhouse_grpc_pb2 -import clickhouse_grpc_pb2_grpc - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "node", - main_configs=[ - "configs/ports.xml", - "configs/log.xml", - "configs/session_log.xml", - ], - user_configs=["configs/users.xml"], - # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 - env_variables={ - "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") - }, -) - - -def grpc_get_url(): - return f"{instance.ip_address}:{GRPC_PORT}" - - -def grpc_create_insecure_channel(): - channel = grpc.insecure_channel(grpc_get_url()) - grpc.channel_ready_future(channel).result(timeout=2) - return channel - - -session_id_counter = 0 - - -def next_session_id(): - global session_id_counter - session_id = session_id_counter - session_id_counter += 1 - return str(session_id) - - -def grpc_query(query, user_, pass_, raise_exception): - try: - query_info = clickhouse_grpc_pb2.QueryInfo( - query=query, - session_id=next_session_id(), - user_name=user_, - password=pass_, - ) - channel = grpc_create_insecure_channel() - stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) - result = stub.ExecuteQuery(query_info) - if result and result.HasField("exception"): - raise Exception(result.exception.display_text) - - return result.output.decode(DEFAULT_ENCODING) - except Exception: - assert raise_exception - - -def postgres_query(query, user_, pass_, raise_exception): - try: - client = py_psql.connect( - host=instance.ip_address, - port=POSTGRES_SERVER_PORT, - user=user_, - password=pass_, - database="default", - ) - cursor = client.cursor() - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -def mysql_query(query, user_, pass_, raise_exception): - try: - client = pymysql.connections.Connection( - host=instance.ip_address, - user=user_, - password=pass_, - database="default", - port=MYSQL_SERVER_PORT, - ) - cursor = client.cursor(pymysql.cursors.DictCursor) - if raise_exception: - with pytest.raises(Exception): - cursor.execute(query) - else: - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_grpc_session(started_cluster): - grpc_query("SELECT 1", "grpc_user", "pass", False) - grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) - grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "grpc_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" - ) - assert logout_records == "grpc_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "grpc_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_mysql_session(started_cluster): - mysql_query("SELECT 1", "mysql_user", "pass", False) - mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) - mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "mysql_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" - ) - assert logout_records == "mysql_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "mysql_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_postgres_session(started_cluster): - postgres_query("SELECT 1", "postgres_user", "pass", False) - postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) - postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "postgres_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" - ) - assert logout_records == "postgres_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "postgres_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_parallel_sessions(started_cluster): - thread_list = [] - for _ in range(10): - # Sleep time does not significantly matter here, - # test should pass even without sleeping. - for function in [postgres_query, grpc_query, mysql_query]: - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "pass", - False, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "wrong_pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "wrong_parallel_user", - "pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - - for thread in thread_list: - thread.join() - - instance.query("SYSTEM FLUSH LOGS") - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" - ) - assert port_0_sessions == "90\n" - - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" - ) - assert port_0_sessions == "0\n" - - address_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" - ) - assert address_0_sessions == "0\n" - - grpc_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" - ) - assert grpc_sessions == "30\n" - - mysql_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" - ) - assert mysql_sessions == "30\n" - - postgres_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" - ) - assert postgres_sessions == "30\n" - - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "30\n" - - logout_failure_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" - ) - assert logout_failure_sessions == "30\n" diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference deleted file mode 100644 index bfe507e8eac..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.reference +++ /dev/null @@ -1,34 +0,0 @@ -sessions: -150 -port_0_sessions: -0 -address_0_sessions: -0 -tcp_sessions -60 -http_sessions -30 -http_with_session_id_sessions -30 -my_sql_sessions -30 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh deleted file mode 100755 index 26b48462a76..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ /dev/null @@ -1,138 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, no-debug - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -# Each user uses a separate thread. -readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users -readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) -readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) -readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") -readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) - -readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" - -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; -done - -# All _session functions execute in separate threads. -# These functions try to create a session with successful login and logout. -# Sleep a small, random amount of time to make concurrency more intense. -# and try to login with an invalid password. -function tcp_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' - done -} - -function http_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" - done -} - -function http_with_session_id_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" - done -} - -function mysql_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" - - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; -export -f mysql_session; - -for user in "${TCP_USERS[@]}"; do - timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_USERS[@]}"; do - timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do - timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & -done - -for user in "${MYSQL_USERS[@]}"; do - timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & -done - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" - -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" - -echo "tcp_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" -echo "http_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "http_with_session_id_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "my_sql_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" - echo "Corresponding LoginSuccess/Logout" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" - echo "LoginFailure" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" - done diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference deleted file mode 100644 index e2680982ab0..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.reference +++ /dev/null @@ -1,13 +0,0 @@ -0 -0 -0 -0 -client_port 0 connections: -0 -client_address '::' connections: -0 -login failures: -0 -TCP Login and logout count is equal -HTTP Login and logout count is equal -MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh deleted file mode 100755 index 3bedfb6c9ee..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.sh +++ /dev/null @@ -1,56 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ -readonly TEST_USER=$"02834_USER_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "client_port 0 connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" - -echo "client_address '::' connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" - -echo "login failures:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" - -# remote(...) function sometimes reuses old cached sessions for query execution. -# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. - -for interface in 'TCP' 'HTTP' 'MySQL' -do - LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` - CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` - - if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then - echo "${interface} Login and logout count is equal" - else - TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` - echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" - fi -done - -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference deleted file mode 100644 index 7252faab8c6..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.reference +++ /dev/null @@ -1,8 +0,0 @@ -port_0_sessions: -0 -address_0_sessions: -0 -Corresponding LoginSuccess/Logout -9 -LoginFailure -0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh deleted file mode 100755 index 347ebd22f96..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ /dev/null @@ -1,114 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-debug - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -readonly TEST_USER="02835_USER_${PID}" -readonly TEST_ROLE="02835_ROLE_${PID}" -readonly TEST_PROFILE="02835_PROFILE_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -function tcp_session() -{ - local user=$1 - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" -} - -function http_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -function http_with_session_id_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. -function wait_for_queries_start() -{ - local user=$1 - local queries_count=$2 - # 10 seconds waiting - counter=0 retries=100 - while [[ $counter -lt $retries ]]; do - result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") - if [[ $result == "${queries_count}" ]]; then - break; - fi - sleep 0.1 - ((++counter)) - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -# DROP USE CASE -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP ROLE CASE -${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP PROFILE CASE -${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" -echo "Corresponding LoginSuccess/Logout" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" -echo "LoginFailure" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From 0d9e3ca34501123cce18c495eb9cc5f0b9b4a9ff Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 11 Aug 2023 00:00:36 +0200 Subject: [PATCH 1854/2047] remove no-parallel tag from some tests --- .../InterpreterShowTablesQuery.cpp | 4 +-- src/Interpreters/executeQuery.cpp | 4 ++- src/Parsers/ASTShowTablesQuery.cpp | 19 +++++++++++--- src/Parsers/ASTShowTablesQuery.h | 5 +++- src/Parsers/ParserShowTablesQuery.cpp | 4 +-- src/Parsers/QueryParameterVisitor.cpp | 2 +- tests/clickhouse-test | 2 +- ...80_show_tables_and_system_tables.reference | 6 ++--- .../00080_show_tables_and_system_tables.sql | 19 +++++++------- .../00158_buffer_and_nonexistent_table.sql | 17 ++++++------- .../00508_materialized_view_to.sql | 22 ++++++---------- ...t_database_when_create_materializ_view.sql | 25 +++++++------------ .../00604_show_create_database.reference | 2 +- .../00604_show_create_database.sql | 8 +++--- ...3_max_block_size_system_tables_columns.sql | 1 - .../00740_database_in_nested_view.sql | 21 ++++++---------- .../00815_left_join_on_stepanel.sql | 8 ++---- .../queries/0_stateless/01015_attach_part.sql | 1 - .../0_stateless/01021_only_tuple_columns.sql | 1 - 19 files changed, 80 insertions(+), 91 deletions(-) diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 2f1a4a32bee..5fe0a862e05 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -116,10 +116,10 @@ String InterpreterShowTablesQuery::getRewrittenQuery() return rewritten_query.str(); } - if (query.temporary && !query.from.empty()) + if (query.temporary && !query.getFrom().empty()) throw Exception(ErrorCodes::SYNTAX_ERROR, "The `FROM` and `TEMPORARY` cannot be used together in `SHOW TABLES`"); - String database = getContext()->resolveDatabase(query.from); + String database = getContext()->resolveDatabase(query.getFrom()); DatabaseCatalog::instance().assertDatabaseExists(database); WriteBufferFromOwnString rewritten_query; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 578ca3b41f9..f66326f570a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -728,7 +728,9 @@ static std::tuple executeQueryImpl( is_create_parameterized_view = create_query->isParameterizedView(); /// Replace ASTQueryParameter with ASTLiteral for prepared statements. - if (!is_create_parameterized_view && context->hasQueryParameters()) + /// Even if we don't have parameters in query_context, check that AST doesn't have unknown parameters + bool probably_has_params = find_first_symbols<'{'>(begin, end) != end; + if (!is_create_parameterized_view && probably_has_params) { ReplaceQueryParameterVisitor visitor(context->getQueryParameters()); visitor.visit(ast); diff --git a/src/Parsers/ASTShowTablesQuery.cpp b/src/Parsers/ASTShowTablesQuery.cpp index 7b4e052bc86..03ae96fa288 100644 --- a/src/Parsers/ASTShowTablesQuery.cpp +++ b/src/Parsers/ASTShowTablesQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -10,10 +11,20 @@ ASTPtr ASTShowTablesQuery::clone() const { auto res = std::make_shared(*this); res->children.clear(); + if (from) + res->set(res->from, from->clone()); + cloneOutputOptions(*res); return res; } +String ASTShowTablesQuery::getFrom() const +{ + String name; + tryGetIdentifierNameInto(from, name); + return name; +} + void ASTShowTablesQuery::formatLike(const FormatSettings & settings) const { if (!like.empty()) @@ -72,9 +83,11 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW " << (temporary ? "TEMPORARY " : "") << (dictionaries ? "DICTIONARIES" : "TABLES") << (settings.hilite ? hilite_none : ""); - if (!from.empty()) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") - << backQuoteIfNeed(from); + if (from) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : ""); + from->formatImpl(settings, state, frame); + } formatLike(settings); diff --git a/src/Parsers/ASTShowTablesQuery.h b/src/Parsers/ASTShowTablesQuery.h index 2878df54bcc..2fd4c3dac96 100644 --- a/src/Parsers/ASTShowTablesQuery.h +++ b/src/Parsers/ASTShowTablesQuery.h @@ -24,8 +24,9 @@ public: bool caches = false; bool full = false; + IAST * from; + String cluster_str; - String from; String like; bool not_like = false; @@ -38,6 +39,8 @@ public: ASTPtr clone() const override; QueryKind getQueryKind() const override { return QueryKind::Show; } + String getFrom() const; + protected: void formatLike(const FormatSettings & settings) const; void formatLimit(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const; diff --git a/src/Parsers/ParserShowTablesQuery.cpp b/src/Parsers/ParserShowTablesQuery.cpp index 3540a6d3fc8..e3728eb2cd6 100644 --- a/src/Parsers/ParserShowTablesQuery.cpp +++ b/src/Parsers/ParserShowTablesQuery.cpp @@ -36,7 +36,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserKeyword s_where("WHERE"); ParserKeyword s_limit("LIMIT"); ParserStringLiteral like_p; - ParserIdentifier name_p; + ParserIdentifier name_p(true); ParserExpressionWithOptionalAlias exp_elem(false); ASTPtr like; @@ -174,7 +174,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return false; } - tryGetIdentifierNameInto(database, query->from); + query->set(query->from, database); if (like) query->like = like->as().value.safeGet(); diff --git a/src/Parsers/QueryParameterVisitor.cpp b/src/Parsers/QueryParameterVisitor.cpp index 1282c12cce6..b8679cc3b96 100644 --- a/src/Parsers/QueryParameterVisitor.cpp +++ b/src/Parsers/QueryParameterVisitor.cpp @@ -31,7 +31,7 @@ private: void visitQueryParameter(const ASTQueryParameter & query_parameter) { - query_parameters[query_parameter.name]= query_parameter.type; + query_parameters[query_parameter.name] = query_parameter.type; } }; diff --git a/tests/clickhouse-test b/tests/clickhouse-test index fc175f2a05a..b428c291ba1 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1197,7 +1197,7 @@ class TestCase: os.environ["CLICKHOUSE_LOG_COMMENT"] = args.testcase_basename params = { - "client": client + " --database=" + database, + "client": client + " --database=" + database + " --param_CLICKHOUSE_DATABASE=" + database, "logs_level": server_logs_level, "options": client_options, "test": self.case_file, diff --git a/tests/queries/0_stateless/00080_show_tables_and_system_tables.reference b/tests/queries/0_stateless/00080_show_tables_and_system_tables.reference index f0b6f0e0c41..e4690f20d3e 100644 --- a/tests/queries/0_stateless/00080_show_tables_and_system_tables.reference +++ b/tests/queries/0_stateless/00080_show_tables_and_system_tables.reference @@ -2,8 +2,8 @@ A B numbers one -A 1 TinyLog CREATE TABLE test_show_tables.A (`A` UInt8) ENGINE = TinyLog -B 1 TinyLog CREATE TABLE test_show_tables.B (`A` UInt8) ENGINE = TinyLog +A 1 TinyLog CREATE TABLE default.A (`A` UInt8) ENGINE = TinyLog +B 1 TinyLog CREATE TABLE default.B (`A` UInt8) ENGINE = TinyLog test_temporary_table -['test_show_tables'] ['test_materialized'] +['default'] ['test_materialized'] 0 diff --git a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql index 137dfb5b6f0..40c26be31c0 100644 --- a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql +++ b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql @@ -1,25 +1,24 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS test_show_tables; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; -CREATE DATABASE test_show_tables; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; -CREATE TABLE test_show_tables.A (A UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_tables.B (A UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.A (A UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.B (A UInt8) ENGINE = TinyLog; -SHOW TABLES from test_show_tables; +SHOW TABLES from {CLICKHOUSE_DATABASE:Identifier}; SHOW TABLES in system where engine like '%System%' and name in ('numbers', 'one'); -SELECT name, toUInt32(metadata_modification_time) > 0, engine_full, create_table_query FROM system.tables WHERE database = 'test_show_tables' ORDER BY name FORMAT TSVRaw; +SELECT name, toUInt32(metadata_modification_time) > 0, engine_full, create_table_query FROM system.tables WHERE database = currentDatabase() ORDER BY name FORMAT TSVRaw; CREATE TEMPORARY TABLE test_temporary_table (id UInt64); SELECT name FROM system.tables WHERE is_temporary = 1 AND name = 'test_temporary_table'; -CREATE TABLE test_show_tables.test_log(id UInt64) ENGINE = Log; -CREATE MATERIALIZED VIEW test_show_tables.test_materialized ENGINE = Log AS SELECT * FROM test_show_tables.test_log; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test_log(id UInt64) ENGINE = Log; +CREATE MATERIALIZED VIEW {CLICKHOUSE_DATABASE:Identifier}.test_materialized ENGINE = Log AS SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_log; SELECT dependencies_database, dependencies_table FROM system.tables WHERE name = 'test_log'; -DROP DATABASE test_show_tables; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; -- Check that create_table_query works for system tables and unusual Databases diff --git a/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql b/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql index 23c157db875..1d988b38b05 100644 --- a/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql +++ b/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql @@ -1,11 +1,10 @@ --- Tags: no-parallel -CREATE DATABASE IF NOT EXISTS test2_00158; -DROP TABLE IF EXISTS test2_00158.mt_buffer_00158; -DROP TABLE IF EXISTS test2_00158.mt_00158; -CREATE TABLE test2_00158.mt_buffer_00158 (d Date DEFAULT today(), x UInt64) ENGINE = Buffer(test2_00158, mt_00158, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000); +CREATE DATABASE IF NOT EXISTS {CLICKHOUSE_DATABASE:Identifier}; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt_buffer_00158; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt_00158; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.mt_buffer_00158 (d Date DEFAULT today(), x UInt64) ENGINE = Buffer({CLICKHOUSE_DATABASE:Identifier}, mt_00158, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000); SET send_logs_level = 'fatal'; -- Supress "Destination table test2.mt doesn't exist. Block of data is discarded." -INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 100000; -INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 1000000; -DROP TABLE IF EXISTS test2_00158.mt_buffer_00158; -DROP DATABASE test2_00158; +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 100000; +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 1000000; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt_buffer_00158; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/00508_materialized_view_to.sql b/tests/queries/0_stateless/00508_materialized_view_to.sql index 522ceb40404..0d8fb85eeeb 100644 --- a/tests/queries/0_stateless/00508_materialized_view_to.sql +++ b/tests/queries/0_stateless/00508_materialized_view_to.sql @@ -1,9 +1,3 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS test_00508; -CREATE DATABASE test_00508; - -USE test_00508; CREATE TABLE src (x UInt8) ENGINE = Null; CREATE TABLE dst (x UInt8) ENGINE = Memory; @@ -20,17 +14,17 @@ SELECT * FROM dst ORDER BY x; USE default; -- Reattach MV (shortcut) -ATTACH TABLE test_00508.mv_00508; +ATTACH TABLE {CLICKHOUSE_DATABASE:Identifier}.mv_00508; -INSERT INTO test_00508.src VALUES (3); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.src VALUES (3); -SELECT * FROM test_00508.mv_00508 ORDER BY x; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.mv_00508 ORDER BY x; -- Drop the MV and see if the data is still readable -DROP TABLE test_00508.mv_00508; -SELECT * FROM test_00508.dst ORDER BY x; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.mv_00508; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.dst ORDER BY x; -DROP TABLE test_00508.src; -DROP TABLE test_00508.dst; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.src; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.dst; -DROP DATABASE test_00508; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql b/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql index 46fc0dd586d..87f2e2b5276 100644 --- a/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql +++ b/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql @@ -1,11 +1,6 @@ --- Tags: no-parallel - -CREATE DATABASE test_00571; - -USE test_00571; DROP DATABASE IF EXISTS none; -DROP TABLE IF EXISTS test_00571; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; DROP TABLE IF EXISTS test_materialized_00571; set allow_deprecated_syntax_for_merge_tree=1; @@ -15,18 +10,16 @@ CREATE MATERIALIZED VIEW test_materialized_00571 ENGINE = MergeTree(date, (platf USE none; -INSERT INTO test_00571.test_00571 VALUES('2018-02-16', 'a', 'a'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.test_00571 VALUES('2018-02-16', 'a', 'a'); -SELECT * FROM test_00571.test_00571; -SELECT * FROM test_00571.test_materialized_00571; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_00571; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; -DETACH TABLE test_00571.test_materialized_00571; -ATTACH TABLE test_00571.test_materialized_00571; +DETACH TABLE {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; +ATTACH TABLE {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; -SELECT * FROM test_00571.test_materialized_00571; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; DROP DATABASE IF EXISTS none; -DROP TABLE IF EXISTS test_00571.test_00571; -DROP TABLE IF EXISTS test_00571.test_materialized_00571; - -DROP DATABASE test_00571; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_00571; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; diff --git a/tests/queries/0_stateless/00604_show_create_database.reference b/tests/queries/0_stateless/00604_show_create_database.reference index c05b088280e..52fd2c48df1 100644 --- a/tests/queries/0_stateless/00604_show_create_database.reference +++ b/tests/queries/0_stateless/00604_show_create_database.reference @@ -1 +1 @@ -CREATE DATABASE test_00604\nENGINE = Atomic +CREATE DATABASE default\nENGINE = Atomic diff --git a/tests/queries/0_stateless/00604_show_create_database.sql b/tests/queries/0_stateless/00604_show_create_database.sql index c990e7abed3..d20d2cb3e53 100644 --- a/tests/queries/0_stateless/00604_show_create_database.sql +++ b/tests/queries/0_stateless/00604_show_create_database.sql @@ -1,5 +1,5 @@ --- Tags: no-ordinary-database, no-parallel +-- Tags: no-ordinary-database -create database if not exists test_00604; -show create database test_00604; -drop database test_00604; +create database if not exists {CLICKHOUSE_DATABASE:Identifier}; +show create database {CLICKHOUSE_DATABASE:Identifier}; +drop database {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql index 0c8e4ae237d..fe1876ed6f8 100644 --- a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql +++ b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel -- NOTE: database = currentDatabase() is not mandatory diff --git a/tests/queries/0_stateless/00740_database_in_nested_view.sql b/tests/queries/0_stateless/00740_database_in_nested_view.sql index e2debe2859b..42c26a709b1 100644 --- a/tests/queries/0_stateless/00740_database_in_nested_view.sql +++ b/tests/queries/0_stateless/00740_database_in_nested_view.sql @@ -1,10 +1,5 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS test_00740; -CREATE DATABASE test_00740; -USE test_00740; - -DROP TABLE IF EXISTS test_00740; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; DROP TABLE IF EXISTS test_view_00740; DROP TABLE IF EXISTS test_nested_view_00740; DROP TABLE IF EXISTS test_joined_view_00740; @@ -19,12 +14,10 @@ SELECT * FROM test_nested_view_00740; SELECT * FROM test_joined_view_00740; USE default; -SELECT * FROM test_00740.test_view_00740; -SELECT * FROM test_00740.test_nested_view_00740; -SELECT * FROM test_00740.test_joined_view_00740; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_view_00740; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_nested_view_00740; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_joined_view_00740; -DROP TABLE IF EXISTS test_00740.test_00740; -DROP TABLE IF EXISTS test_00740.test_view_00740; -DROP TABLE IF EXISTS test_00740.test_nested_view_00740; - -DROP DATABASE test_00740; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_00740; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_view_00740; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_nested_view_00740; diff --git a/tests/queries/0_stateless/00815_left_join_on_stepanel.sql b/tests/queries/0_stateless/00815_left_join_on_stepanel.sql index 13172f8c18d..725c9523cd7 100644 --- a/tests/queries/0_stateless/00815_left_join_on_stepanel.sql +++ b/tests/queries/0_stateless/00815_left_join_on_stepanel.sql @@ -1,7 +1,3 @@ --- Tags: no-parallel - -CREATE DATABASE IF NOT EXISTS test_00815; -USE test_00815; DROP TABLE IF EXISTS fact_cpc_clicks; DROP TABLE IF EXISTS dim_model; @@ -16,6 +12,6 @@ select f.model_id from fact_cpc_clicks as f left join dim_model as d on f.model_ USE default; -select f.model_id from test_00815.fact_cpc_clicks as f left join test_00815.dim_model as d on f.model_id=d.model_id limit 10; +select f.model_id from {CLICKHOUSE_DATABASE:Identifier}.fact_cpc_clicks as f left join {CLICKHOUSE_DATABASE:Identifier}.dim_model as d on f.model_id=d.model_id limit 10; -DROP DATABASE test_00815; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01015_attach_part.sql b/tests/queries/0_stateless/01015_attach_part.sql index a2f949d3499..9ff505efd8f 100644 --- a/tests/queries/0_stateless/01015_attach_part.sql +++ b/tests/queries/0_stateless/01015_attach_part.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel DROP TABLE IF EXISTS table_01; diff --git a/tests/queries/0_stateless/01021_only_tuple_columns.sql b/tests/queries/0_stateless/01021_only_tuple_columns.sql index 02db21bc0b2..d8d146f59fd 100644 --- a/tests/queries/0_stateless/01021_only_tuple_columns.sql +++ b/tests/queries/0_stateless/01021_only_tuple_columns.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel CREATE TABLE test ( From b9638c6387d46e85abe55a333049efb0d2c757e7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 22:16:30 +0000 Subject: [PATCH 1855/2047] path to libc --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 4d4d1fab473..03314a9d2ba 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From ea8ae28c7dfeae6c3641c8617152750aa0d94be2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 00:40:18 +0200 Subject: [PATCH 1856/2047] Fix build with clang-17 --- src/Core/examples/coro.cpp | 194 ------------------------------------- 1 file changed, 194 deletions(-) delete mode 100644 src/Core/examples/coro.cpp diff --git a/src/Core/examples/coro.cpp b/src/Core/examples/coro.cpp deleted file mode 100644 index fbccc261e9d..00000000000 --- a/src/Core/examples/coro.cpp +++ /dev/null @@ -1,194 +0,0 @@ -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -#if defined(__clang__) -#include - -namespace std // NOLINT(cert-dcl58-cpp) -{ - using namespace experimental::coroutines_v1; // NOLINT(cert-dcl58-cpp) -} - -#if __has_warning("-Wdeprecated-experimental-coroutine") -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wdeprecated-experimental-coroutine" -#endif - -#else -#include -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wzero-as-null-pointer-constant" -#endif - - -template -struct suspend_value // NOLINT(readability-identifier-naming) -{ - constexpr bool await_ready() const noexcept { return true; } // NOLINT(readability-identifier-naming) - constexpr void await_suspend(std::coroutine_handle<>) const noexcept {} // NOLINT(readability-identifier-naming) - constexpr T await_resume() const noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " ret " << val << std::endl; - return val; - } - - T val; -}; - -template -struct Task -{ - struct promise_type // NOLINT(readability-identifier-naming) - { - using coro_handle = std::coroutine_handle; - auto get_return_object() { return coro_handle::from_promise(*this); } // NOLINT(readability-identifier-naming) - auto initial_suspend() { return std::suspend_never(); } // NOLINT(readability-identifier-naming) - auto final_suspend() noexcept { return suspend_value{*r->value}; } // NOLINT(readability-identifier-naming) - //void return_void() {} - void return_value(T value_) { r->value = value_; } // NOLINT(readability-identifier-naming) - void unhandled_exception() // NOLINT(readability-identifier-naming) - { - DB::tryLogCurrentException("Logger"); - r->exception = std::current_exception(); // NOLINT(bugprone-throw-keyword-missing) - } - - explicit promise_type(std::string tag_) : tag(tag_) {} - ~promise_type() { std::cout << "~promise_type " << tag << std::endl; } - std::string tag; - coro_handle next; - Task * r = nullptr; - }; - - using coro_handle = std::coroutine_handle; - - bool await_ready() const noexcept { return false; } // NOLINT(readability-identifier-naming) - void await_suspend(coro_handle g) noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " await_suspend " << my.promise().tag << std::endl; - std::cout << " g tag " << g.promise().tag << std::endl; - g.promise().next = my; - } - T await_resume() noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " await_res " << my.promise().tag << std::endl; - return *value; - } - - Task(coro_handle handle) : my(handle), tag(handle.promise().tag) // NOLINT(google-explicit-constructor) - { - assert(handle); - my.promise().r = this; - std::cout << " Task " << tag << std::endl; - } - Task(Task &) = delete; - Task(Task &&rhs) noexcept : my(rhs.my), tag(rhs.tag) - { - rhs.my = {}; - std::cout << " Task&& " << tag << std::endl; - } - static bool resumeImpl(Task *r) - { - if (r->value) - return false; - - auto & next = r->my.promise().next; - - if (next) - { - if (resumeImpl(next.promise().r)) - return true; - next = {}; - } - - if (!r->value) - { - r->my.resume(); - if (r->exception) - std::rethrow_exception(r->exception); - } - return !r->value; - } - - bool resume() - { - return resumeImpl(this); - } - - T res() - { - return *value; - } - - ~Task() - { - std::cout << " ~Task " << tag << std::endl; - } - -private: - coro_handle my; - std::string tag; - std::optional value; - std::exception_ptr exception; -}; - -Task boo([[maybe_unused]] std::string tag) -{ - std::cout << "x" << std::endl; - co_await std::suspend_always(); - std::cout << StackTrace().toString(); - std::cout << "y" << std::endl; - co_return 1; -} - -Task bar([[maybe_unused]] std::string tag) -{ - std::cout << "a" << std::endl; - int res1 = co_await boo("boo1"); - std::cout << "b " << res1 << std::endl; - int res2 = co_await boo("boo2"); - if (res2 == 1) - throw DB::Exception(1, "hello"); - std::cout << "c " << res2 << std::endl; - co_return res1 + res2; // 1 + 1 = 2 -} - -Task foo([[maybe_unused]] std::string tag) -{ - std::cout << "Hello" << std::endl; - auto res1 = co_await bar("bar1"); - std::cout << "Coro " << res1 << std::endl; - auto res2 = co_await bar("bar2"); - std::cout << "World " << res2 << std::endl; - co_return res1 * res2; // 2 * 2 = 4 -} - -int main() -{ - Poco::AutoPtr app_channel(new Poco::ConsoleChannel(std::cerr)); - Poco::Logger::root().setChannel(app_channel); - Poco::Logger::root().setLevel("trace"); - - LOG_INFO(&Poco::Logger::get(""), "Starting"); - - try - { - auto t = foo("foo"); - std::cout << ".. started" << std::endl; - while (t.resume()) - std::cout << ".. yielded" << std::endl; - std::cout << ".. done: " << t.res() << std::endl; - } - catch (DB::Exception & e) - { - std::cout << "Got exception " << e.what() << std::endl; - std::cout << e.getStackTraceString() << std::endl; - } -} From ea62629d83ee487e64b1ba3c051211103f258069 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 00:40:26 +0200 Subject: [PATCH 1857/2047] Fix linking of examples --- src/Core/examples/CMakeLists.txt | 3 --- src/Parsers/examples/CMakeLists.txt | 4 ++-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Core/examples/CMakeLists.txt b/src/Core/examples/CMakeLists.txt index 868173e0e31..2326eada96d 100644 --- a/src/Core/examples/CMakeLists.txt +++ b/src/Core/examples/CMakeLists.txt @@ -9,6 +9,3 @@ target_link_libraries (string_ref_hash PRIVATE clickhouse_common_io) clickhouse_add_executable (mysql_protocol mysql_protocol.cpp) target_link_libraries (mysql_protocol PRIVATE dbms) - -clickhouse_add_executable (coro coro.cpp) -target_link_libraries (coro PRIVATE clickhouse_common_io) diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index e411574bd65..261f234081c 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -4,7 +4,7 @@ clickhouse_add_executable(lexer lexer.cpp ${SRCS}) target_link_libraries(lexer PRIVATE clickhouse_parsers) clickhouse_add_executable(select_parser select_parser.cpp ${SRCS} "../../Server/ServerType.cpp") -target_link_libraries(select_parser PRIVATE clickhouse_parsers) +target_link_libraries(select_parser PRIVATE dbms) clickhouse_add_executable(create_parser create_parser.cpp ${SRCS} "../../Server/ServerType.cpp") -target_link_libraries(create_parser PRIVATE clickhouse_parsers) +target_link_libraries(create_parser PRIVATE dbms) From 3193c6d4ccc7ef02857f14825e1a781e8c2e83c2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 00:55:57 +0200 Subject: [PATCH 1858/2047] Fix build --- src/Functions/array/arrayAUC.cpp | 2 +- src/Server/ServerType.cpp | 2 +- src/Server/ServerType.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index b7bd7dcc0ad..499fe4ce7b2 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -139,7 +139,7 @@ private: for (size_t i = 0; i < size; ++i) { auto next_offset = offsets[i]; - result[i] = apply(data1, data2, current_offset, next_offset); + result[i] = apply(scores, labels, current_offset, next_offset); current_offset = next_offset; } } diff --git a/src/Server/ServerType.cpp b/src/Server/ServerType.cpp index 4952cd1bd24..29ba7224c75 100644 --- a/src/Server/ServerType.cpp +++ b/src/Server/ServerType.cpp @@ -2,10 +2,10 @@ #include #include -#include #include + namespace DB { diff --git a/src/Server/ServerType.h b/src/Server/ServerType.h index 1fab492222a..eafe4f941dd 100644 --- a/src/Server/ServerType.h +++ b/src/Server/ServerType.h @@ -1,13 +1,13 @@ #pragma once #include + namespace DB { class ServerType { public: - enum Type { TCP, From 396db7ab3cee77955173de76f355e1d08c1ec755 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 01:01:43 +0200 Subject: [PATCH 1859/2047] Fix test `00002_log_and_exception_messages_formatting` --- .../00002_log_and_exception_messages_formatting.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 86fe01dc0e3..54daeb928a3 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -36,7 +36,9 @@ create temporary table known_short_messages (s String) as select * from (select 'Database {} doesn''t exist', 'Dictionary ({}) not found', 'Unknown table function {}', 'Unknown format {}', 'Unknown explain kind ''{}''', 'Unknown setting {}', 'Unknown input format {}', 'Unknown identifier: ''{}''', 'User name is empty', 'Expected function, got: {}', -'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'API mode: {}' +'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'API mode: {}', +'Processed: {}%', 'Creating {}: {}', 'Table {}.{} doesn''t exist', 'Invalid cache key hex: {}', +'User has been dropped', 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64' ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. From c98e38611fccca5acfaf466a7ce22e9ae2bc0a6a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 23:36:47 +0000 Subject: [PATCH 1860/2047] path to libc --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 03314a9d2ba..945eb9affa4 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 3b9d2c708096e51b106acffbfd3cc2c4ac8606e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 03:06:07 +0200 Subject: [PATCH 1861/2047] Simplification --- .../config.d/create_logs_at_startup.yaml | 1 - programs/server/config.xml | 3 - src/Common/SystemLogBase.h | 2 +- src/Interpreters/Context.cpp | 16 ++++ src/Interpreters/Context.h | 3 + src/Interpreters/InterpreterSystemQuery.cpp | 82 ++++++++----------- src/Interpreters/SystemLog.cpp | 14 +--- src/Interpreters/SystemLog.h | 5 +- .../config.d/create_logs_at_startup.yaml | 34 -------- tests/config/install.sh | 1 - 10 files changed, 57 insertions(+), 104 deletions(-) delete mode 120000 programs/server/config.d/create_logs_at_startup.yaml delete mode 100644 tests/config/config.d/create_logs_at_startup.yaml diff --git a/programs/server/config.d/create_logs_at_startup.yaml b/programs/server/config.d/create_logs_at_startup.yaml deleted file mode 120000 index a09d2783a9e..00000000000 --- a/programs/server/config.d/create_logs_at_startup.yaml +++ /dev/null @@ -1 +0,0 @@ -../../../tests/config/config.d/create_logs_at_startup.yaml \ No newline at end of file diff --git a/programs/server/config.xml b/programs/server/config.xml index 3d1c92b135c..14b8954fc39 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1037,9 +1037,6 @@ - - - false + And so, to avoid extra memory reference switch *_log to Memory engine. + --> ENGINE = Memory From e357702fd05b77ed01e43a9be38e1e6dfff393a8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Aug 2023 01:26:38 +0200 Subject: [PATCH 1954/2047] What will happen if I remove this? --- .../zzz-perf-comparison-tweaks-config.xml | 26 ------------------- 1 file changed, 26 deletions(-) diff --git a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml index 10a5916264a..292665c4f68 100644 --- a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml +++ b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml @@ -19,32 +19,6 @@ - - - ENGINE = Memory - - - - ENGINE = Memory - - - - ENGINE = Memory - - - - ENGINE = Memory - - - - ENGINE = Memory - - - 1000000000 10 From fea74ce17b45ae38336253f06608d6412e98417c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Kriszti=C3=A1n=20Sz=C5=B1cs?= Date: Sun, 13 Aug 2023 17:02:08 +0200 Subject: [PATCH 1955/2047] Documentation: add Ibis project to the integrations section --- docs/en/interfaces/third-party/integrations.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index 3e1b1e84f5d..a9f1af93495 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -83,8 +83,8 @@ ClickHouse, Inc. does **not** maintain the tools and libraries listed below and - Python - [SQLAlchemy](https://www.sqlalchemy.org) - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - - [pandas](https://pandas.pydata.org) - - [pandahouse](https://github.com/kszucs/pandahouse) + - [PyArrow/Pandas](https://pandas.pydata.org) + - [Ibis](https://github.com/ibis-project/ibis) - PHP - [Doctrine](https://www.doctrine-project.org/) - [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse) From 7135b344bfcfaf22bba21b716962de9303f6409a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Kriszti=C3=A1n=20Sz=C5=B1cs?= Date: Mon, 14 Aug 2023 08:40:49 +0200 Subject: [PATCH 1956/2047] Documentation: exclude PyArrow from spell checking --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 812908931ef..2e231120e41 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1892,7 +1892,6 @@ overfitting packetpool packetsize pageviews -pandahouse parallelization parallelize parallelized @@ -2001,6 +2000,7 @@ ptrs pushdown pwrite py +PyArrow qryn quantile quantileBFloat From 385332a5542997a14e71ae8e2b34cd3b4247d553 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 07:10:50 +0000 Subject: [PATCH 1957/2047] Docs: Update anchors in ANN indexes docs --- .../en/engines/table-engines/mergetree-family/annindexes.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 5944048f6c3..6618c6ddc06 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -1,4 +1,4 @@ -# Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex} +# Approximate Nearest Neighbor Search Indexes [experimental] Nearest neighborhood search is the problem of finding the M closest points for a given point in an N-dimensional vector space. The most straightforward approach to solve this problem is a brute force search where the distance between all points in the vector space and the @@ -45,7 +45,7 @@ With brute force search, both queries are expensive (linear in the number of poi `Point` must be computed. To speed this process up, Approximate Nearest Neighbor Search Indexes (ANN indexes) store a compact representation of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer much quicker (in sub-linear time). -# Creating and Using ANN Indexes +# Creating and Using ANN Indexes {#creating_using_ann_indexes} Syntax to create an ANN index over an [Array](../../../sql-reference/data-types/array.md) column: @@ -138,7 +138,7 @@ back to a smaller `GRANULARITY` values only in case of problems like excessive m was specified for ANN indexes, the default value is 100 million. -# Available ANN Indexes +# Available ANN Indexes {#available_ann_indexes} - [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy) From f71ce2641c09cab4d70e24d867f5014b86edecef Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 07:36:27 +0000 Subject: [PATCH 1958/2047] Fix copyright issues in ANN docs --- .../mergetree-family/annindexes.md | 24 +++++++------------ 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 6618c6ddc06..156f64e94d4 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -188,23 +188,17 @@ ENGINE = MergeTree ORDER BY id; ``` -Annoy currently supports `L2Distance` and `cosineDistance` as distance function `Distance`. If no distance function was specified during -index creation, `L2Distance` is used as default. Parameter `NumTrees` is the number of trees which the algorithm creates (default if not -specified: 100). Higher values of `NumTree` mean more accurate search results but slower index creation / query times (approximately -linearly) as well as larger index sizes. +Annoy currently supports two distance functions: +- `L2Distance`, also called Euclidean distance is the length of a line segment between two points in Euclidean space + ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). +- `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors + ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). -`L2Distance` is also called Euclidean distance, the Euclidean distance between two points in Euclidean space is the length of a line segment between the two points. -For example: If we have point P(p1,p2), Q(q1,q2), their distance will be d(p,q) -![L2Distance](https://en.wikipedia.org/wiki/Euclidean_distance#/media/File:Euclidean_distance_2d.svg) +For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no +distance function was specified during index creation, `L2Distance` is used as default. -`cosineDistance` also called cosine similarity is a measure of similarity between two non-zero vectors defined in an inner product space. Cosine similarity is the cosine of the angle between the vectors; that is, it is the dot product of the vectors divided by the product of their lengths. -![cosineDistance](https://www.tyrrell4innovation.ca/wp-content/uploads/2021/06/rsz_jenny_du_miword.png) - -The Euclidean distance corresponds to the L2-norm of a difference between vectors. The cosine similarity is proportional to the dot product of two vectors and inversely proportional to the product of their magnitudes. -![compare](https://www.researchgate.net/publication/320914786/figure/fig2/AS:558221849841664@1510101868614/The-difference-between-Euclidean-distance-and-cosine-similarity.png) -In one sentence: cosine similarity care only about the angle between them, but do not care about the "distance" we normally think. -![L2 distance](https://www.baeldung.com/wp-content/uploads/sites/4/2020/06/4-1.png) -![cosineDistance](https://www.baeldung.com/wp-content/uploads/sites/4/2020/06/5.png) +Parameter `NumTrees` is the number of trees which the algorithm creates (default if not specified: 100). Higher values of `NumTree` mean +more accurate search results but slower index creation / query times (approximately linearly) as well as larger index sizes. :::note Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use From 1c3f4d3719d9171f4bbe1aee1a7c7109ddb1ce59 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 07:46:15 +0000 Subject: [PATCH 1959/2047] + , --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 156f64e94d4..9c9067099c9 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -189,7 +189,7 @@ ORDER BY id; ``` Annoy currently supports two distance functions: -- `L2Distance`, also called Euclidean distance is the length of a line segment between two points in Euclidean space +- `L2Distance`, also called Euclidean distance, is the length of a line segment between two points in Euclidean space ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). - `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). From 2d3bf55d454880104044804b1142ce1feeeb43ac Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 08:50:20 +0000 Subject: [PATCH 1960/2047] Docs: Update table name in ANN docs --- .../mergetree-family/annindexes.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 9c9067099c9..81c69215472 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -17,7 +17,7 @@ In terms of SQL, the nearest neighborhood problem can be expressed as follows: ``` sql SELECT * -FROM table +FROM table_with_ann_index ORDER BY Distance(vectors, Point) LIMIT N ``` @@ -32,7 +32,7 @@ An alternative formulation of the nearest neighborhood search problem looks as f ``` sql SELECT * -FROM table +FROM table_with_ann_index WHERE Distance(vectors, Point) < MaxDistance LIMIT N ``` @@ -50,7 +50,7 @@ of the search space (using clustering, search trees, etc.) which allows to compu Syntax to create an ANN index over an [Array](../../../sql-reference/data-types/array.md) column: ```sql -CREATE TABLE table +CREATE TABLE table_with_ann_index ( `id` Int64, `vectors` Array(Float32), @@ -63,7 +63,7 @@ ORDER BY id; Syntax to create an ANN index over a [Tuple](../../../sql-reference/data-types/tuple.md) column: ```sql -CREATE TABLE table +CREATE TABLE table_with_ann_index ( `id` Int64, `vectors` Tuple(Float32[, Float32[, ...]]), @@ -83,7 +83,7 @@ ANN indexes support two types of queries: ``` sql SELECT * - FROM table + FROM table_with_ann_index [WHERE ...] ORDER BY Distance(vectors, Point) LIMIT N @@ -93,7 +93,7 @@ ANN indexes support two types of queries: ``` sql SELECT * - FROM table + FROM table_with_ann_index WHERE Distance(vectors, Point) < MaxDistance LIMIT N ``` @@ -103,7 +103,7 @@ To avoid writing out large vectors, you can use [query parameters](/docs/en/interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. ```bash -clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(vectors, {vec: Array(Float32)}) < 1.0" +clickhouse-client --param_vec='hello' --query="SELECT * FROM table_with_ann_index WHERE L2Distance(vectors, {vec: Array(Float32)}) < 1.0" ``` ::: @@ -165,7 +165,7 @@ space in random linear surfaces (lines in 2D, planes in 3D etc.). Syntax to create an Annoy index over an [Array](../../../sql-reference/data-types/array.md) column: ```sql -CREATE TABLE table +CREATE TABLE table_with_annoy_index ( id Int64, vectors Array(Float32), @@ -178,7 +178,7 @@ ORDER BY id; Syntax to create an ANN index over a [Tuple](../../../sql-reference/data-types/tuple.md) column: ```sql -CREATE TABLE table +CREATE TABLE table_with_annoy_index ( id Int64, vectors Tuple(Float32[, Float32[, ...]]), From 9d5fcbf52466d4268134f0e35fbba212e9d08c87 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 09:16:08 +0000 Subject: [PATCH 1961/2047] Follow-up to #52695: Move tests to a more appropriate place --- .../test_from_system_tables.py | 12 -------- .../01161_information_schema.reference | 2 ++ .../0_stateless/01161_information_schema.sql | 29 +++++++++++++------ 3 files changed, 22 insertions(+), 21 deletions(-) diff --git a/tests/integration/test_select_access_rights/test_from_system_tables.py b/tests/integration/test_select_access_rights/test_from_system_tables.py index a4dfa662601..3cd6b90109c 100644 --- a/tests/integration/test_select_access_rights/test_from_system_tables.py +++ b/tests/integration/test_select_access_rights/test_from_system_tables.py @@ -190,15 +190,3 @@ def test_information_schema(): ) == "1\n" ) - assert ( - node.query( - "SELECT count() FROM information_schema.TABLES WHERE table_name='TABLES'" - ) - == "2\n" - ) - assert ( - node.query( - "SELECT count() FROM INFORMATION_SCHEMA.tables WHERE table_name='tables'" - ) - == "3\n" - ) diff --git a/tests/queries/0_stateless/01161_information_schema.reference b/tests/queries/0_stateless/01161_information_schema.reference index 4ec33a70be0..32ad3f16abc 100644 --- a/tests/queries/0_stateless/01161_information_schema.reference +++ b/tests/queries/0_stateless/01161_information_schema.reference @@ -33,3 +33,5 @@ default default v default v f 2 0 Float64 \N \N \N \N \N \N \N \N \N \N \N \N \ tmp tmp d 1 0 Date \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N Date tmp tmp dt 2 0 DateTime \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N DateTime tmp tmp dtms 3 0 DateTime64(3) \N \N \N \N \N 3 \N \N \N \N \N \N \N \N \N DateTime64(3) +1 +1 diff --git a/tests/queries/0_stateless/01161_information_schema.sql b/tests/queries/0_stateless/01161_information_schema.sql index ed77ef1c1c2..68a3b011ced 100644 --- a/tests/queries/0_stateless/01161_information_schema.sql +++ b/tests/queries/0_stateless/01161_information_schema.sql @@ -1,20 +1,31 @@ -show tables from information_schema; +SHOW TABLES FROM information_schema; SHOW TABLES FROM INFORMATION_SCHEMA; -create table t (n UInt64, f Float32, s String, fs FixedString(42), d Decimal(9, 6)) engine=Memory; -create view v (n Nullable(Int32), f Float64) as select n, f from t; -create materialized view mv engine=Null as select * from system.one; -create temporary table tmp (d Date, dt DateTime, dtms DateTime64(3)); +DROP TABLE IF EXISTS t; +DROP VIEW IF EXISTS v; +DROP VIEW IF EXISTS mv; +DROP TABLE IF EXISTS tmp; + +CREATE TABLE t (n UInt64, f Float32, s String, fs FixedString(42), d Decimal(9, 6)) ENGINE=Memory; +CREATE VIEW v (n Nullable(Int32), f Float64) AS SELECT n, f FROM t; +CREATE MATERIALIZED VIEW mv ENGINE=Null AS SELECT * FROM system.one; +CREATE TEMPORARY TABLE tmp (d Date, dt DateTime, dtms DateTime64(3)); -- FIXME #28687 -select * from information_schema.schemata where schema_name ilike 'information_schema'; +SELECT * FROM information_schema.schemata WHERE schema_name ilike 'information_schema'; -- SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (TABLE_SCHEMA=currentDatabase() OR TABLE_SCHEMA='') AND TABLE_NAME NOT LIKE '%inner%'; SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (table_schema=currentDatabase() OR table_schema='') AND table_name NOT LIKE '%inner%'; -select * from information_schema.views where table_schema=currentDatabase(); +SELECT * FROM information_schema.views WHERE table_schema=currentDatabase(); -- SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE (TABLE_SCHEMA=currentDatabase() OR TABLE_SCHEMA='') AND TABLE_NAME NOT LIKE '%inner%'; SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE (table_schema=currentDatabase() OR table_schema='') AND table_name NOT LIKE '%inner%'; -drop table t; -drop view v; +-- mixed upper/lowercase schema and table name: +SELECT count() FROM information_schema.TABLES WHERE table_schema=currentDatabase() AND table_name = 't'; +SELECT count() FROM INFORMATION_SCHEMA.tables WHERE table_schema=currentDatabase() AND table_name = 't'; +SELECT count() FROM INFORMATION_schema.tables WHERE table_schema=currentDatabase() AND table_name = 't'; -- { serverError UNKNOWN_DATABASE } +SELECT count() FROM information_schema.taBLES WHERE table_schema=currentDatabase() AND table_name = 't'; -- { serverError UNKNOWN_TABLE } + drop view mv; +drop view v; +drop table t; From 310ac6feaf0c16ee2f962187ba721054f9929d3a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Aug 2023 14:19:08 +0200 Subject: [PATCH 1962/2047] Tune PRInfo.has_changes_in_documentation --- tests/ci/pr_info.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 86d4985c6b2..dee71b726df 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -279,7 +279,7 @@ class PRInfo: "user_orgs": self.user_orgs, } - def has_changes_in_documentation(self): + def has_changes_in_documentation(self) -> bool: # If the list wasn't built yet the best we can do is to # assume that there were changes. if self.changed_files is None or not self.changed_files: @@ -287,10 +287,9 @@ class PRInfo: for f in self.changed_files: _, ext = os.path.splitext(f) - path_in_docs = "docs" in f - path_in_website = "website" in f + path_in_docs = f.startswith("docs/") if ( - ext in DIFF_IN_DOCUMENTATION_EXT and (path_in_docs or path_in_website) + ext in DIFF_IN_DOCUMENTATION_EXT and path_in_docs ) or "docker/docs" in f: return True return False From 900e38a6768febec05a90d6d79d7cd98e2989b12 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Aug 2023 14:20:40 +0200 Subject: [PATCH 1963/2047] Fail early on missed documentation for new features --- tests/ci/run_check.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 4f022b6c0a5..9e0644d6c6e 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -137,17 +137,19 @@ def main(): if pr_labels_to_remove: remove_labels(gh, pr_info, pr_labels_to_remove) - if FEATURE_LABEL in pr_info.labels: - print(f"The '{FEATURE_LABEL}' in the labels, expect the 'Docs Check' status") + if FEATURE_LABEL in pr_info.labels and not pr_info.has_changes_in_documentation(): + print( + f"The '{FEATURE_LABEL}' in the labels, " + "but there's no changed documentation" + ) post_commit_status( # do not pass pr_info here intentionally commit, - "pending", + "failure", NotSet, f"expect adding docs for {FEATURE_LABEL}", - DOCS_NAME, + CI_STATUS_NAME, ) - elif not description_error: - set_mergeable_check(commit, "skipped") + sys.exit(1) if description_error: print( @@ -173,6 +175,7 @@ def main(): ) sys.exit(1) + set_mergeable_check(commit, "skipped") ci_report_url = create_ci_report(pr_info, []) if not can_run: print("::notice ::Cannot run") From 56a8818cf25b4335c3707ad02f6585c21705bf2b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Aug 2023 14:31:26 +0200 Subject: [PATCH 1964/2047] Fix logic of Mergeable Check --- tests/ci/run_check.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 9e0644d6c6e..db98a2c1ab5 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -147,7 +147,8 @@ def main(): "failure", NotSet, f"expect adding docs for {FEATURE_LABEL}", - CI_STATUS_NAME, + DOCS_NAME, + pr_info, ) sys.exit(1) From f8b1d7474dffa024ff692bec35578c5172aeea8a Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 14 Aug 2023 12:46:23 +0000 Subject: [PATCH 1965/2047] Update test_distributed_inter_server_secret to pass with analyzer --- tests/analyzer_integration_broken_tests.txt | 18 ----- .../test.py | 68 +++++++------------ 2 files changed, 25 insertions(+), 61 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 68822fbf311..3cc4869aa62 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -5,24 +5,6 @@ test_distributed_ddl/test.py::test_default_database[configs_secure] test_distributed_ddl/test.py::test_on_server_fail[configs] test_distributed_ddl/test.py::test_on_server_fail[configs_secure] test_distributed_insert_backward_compatibility/test.py::test_distributed_in_tuple -test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[default-] -test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[nopass-] -test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[pass-foo] -test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[default-] -test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[nopass-] -test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[pass-foo] -test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[default-] -test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[nopass-] -test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[pass-foo] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster[default-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster[nopass-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster[pass-foo] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_from_backward[default-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_from_backward[nopass-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_from_backward[pass-foo] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_with_backward[default-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_with_backward[nopass-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_with_backward[pass-foo] test_distributed_load_balancing/test.py::test_distributed_replica_max_ignored_errors test_distributed_load_balancing/test.py::test_load_balancing_default test_distributed_load_balancing/test.py::test_load_balancing_priority_round_robin[dist_priority] diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 36ac07a550a..1aeaddcf3c5 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -110,10 +110,6 @@ def start_cluster(): cluster.shutdown() -def query_with_id(node, id_, query, **kwargs): - return node.query("WITH '{}' AS __id {}".format(id_, query), **kwargs) - - # @return -- [user, initial_user] def get_query_user_info(node, query_pattern): node.query("SYSTEM FLUSH LOGS") @@ -334,7 +330,7 @@ def test_secure_disagree_insert(): @users def test_user_insecure_cluster(user, password): id_ = "query-dist_insecure-" + user - query_with_id(n1, id_, "SELECT * FROM dist_insecure", user=user, password=password) + n1.query(f"SELECT *, '{id_}' FROM dist_insecure", user=user, password=password) assert get_query_user_info(n1, id_) == [ user, user, @@ -345,7 +341,7 @@ def test_user_insecure_cluster(user, password): @users def test_user_secure_cluster(user, password): id_ = "query-dist_secure-" + user - query_with_id(n1, id_, "SELECT * FROM dist_secure", user=user, password=password) + n1.query(f"SELECT *, '{id_}' FROM dist_secure", user=user, password=password) assert get_query_user_info(n1, id_) == [user, user] assert get_query_user_info(n2, id_) == [user, user] @@ -353,16 +349,14 @@ def test_user_secure_cluster(user, password): @users def test_per_user_inline_settings_insecure_cluster(user, password): id_ = "query-ddl-settings-dist_insecure-" + user - query_with_id( - n1, - id_, - """ - SELECT * FROM dist_insecure - SETTINGS - prefer_localhost_replica=0, - max_memory_usage_for_user=1e9, - max_untracked_memory=0 - """, + n1.query( + f""" + SELECT *, '{id_}' FROM dist_insecure + SETTINGS + prefer_localhost_replica=0, + max_memory_usage_for_user=1e9, + max_untracked_memory=0 + """, user=user, password=password, ) @@ -372,16 +366,14 @@ def test_per_user_inline_settings_insecure_cluster(user, password): @users def test_per_user_inline_settings_secure_cluster(user, password): id_ = "query-ddl-settings-dist_secure-" + user - query_with_id( - n1, - id_, - """ - SELECT * FROM dist_secure - SETTINGS - prefer_localhost_replica=0, - max_memory_usage_for_user=1e9, - max_untracked_memory=0 - """, + n1.query( + f""" + SELECT *, '{id_}' FROM dist_secure + SETTINGS + prefer_localhost_replica=0, + max_memory_usage_for_user=1e9, + max_untracked_memory=0 + """, user=user, password=password, ) @@ -393,10 +385,8 @@ def test_per_user_inline_settings_secure_cluster(user, password): @users def test_per_user_protocol_settings_insecure_cluster(user, password): id_ = "query-protocol-settings-dist_insecure-" + user - query_with_id( - n1, - id_, - "SELECT * FROM dist_insecure", + n1.query( + f"SELECT *, '{id_}' FROM dist_insecure", user=user, password=password, settings={ @@ -411,10 +401,8 @@ def test_per_user_protocol_settings_insecure_cluster(user, password): @users def test_per_user_protocol_settings_secure_cluster(user, password): id_ = "query-protocol-settings-dist_secure-" + user - query_with_id( - n1, - id_, - "SELECT * FROM dist_secure", + n1.query( + f"SELECT *, '{id_}' FROM dist_secure", user=user, password=password, settings={ @@ -431,8 +419,8 @@ def test_per_user_protocol_settings_secure_cluster(user, password): @users def test_user_secure_cluster_with_backward(user, password): id_ = "with-backward-query-dist_secure-" + user - query_with_id( - n1, id_, "SELECT * FROM dist_secure_backward", user=user, password=password + n1.query( + f"SELECT *, '{id_}' FROM dist_secure_backward", user=user, password=password ) assert get_query_user_info(n1, id_) == [user, user] assert get_query_user_info(backward, id_) == [user, user] @@ -441,13 +429,7 @@ def test_user_secure_cluster_with_backward(user, password): @users def test_user_secure_cluster_from_backward(user, password): id_ = "from-backward-query-dist_secure-" + user - query_with_id( - backward, - id_, - "SELECT * FROM dist_secure_backward", - user=user, - password=password, - ) + backward.query(f"SELECT *, '{id_}' FROM dist_secure", user=user, password=password) assert get_query_user_info(n1, id_) == [user, user] assert get_query_user_info(backward, id_) == [user, user] From 3655df0f406792d65b212807eb88e81966c95b98 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Aug 2023 16:32:52 +0200 Subject: [PATCH 1966/2047] Attempt to address reset ENV in init.d script --- programs/install/Install.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index d7086c95beb..e10a9fea86b 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -997,7 +997,9 @@ namespace { /// sudo respects limits in /etc/security/limits.conf e.g. open files, /// that's why we are using it instead of the 'clickhouse su' tool. - command = fmt::format("sudo -u '{}' {}", user, command); + /// by default, sudo resets all the ENV variables, but we should preserve + /// the values /etc/default/clickhouse in /etc/init.d/clickhouse file + command = fmt::format("sudo --preserve-env -u '{}' {}", user, command); } fmt::print("Will run {}\n", command); From ca2f800fa5d739b84d9817263678ba16ae9a8cc4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 14 Aug 2023 14:35:01 +0000 Subject: [PATCH 1967/2047] Remove unnecessary code --- .../ClusterProxy/SelectStreamFactory.h | 3 --- src/Interpreters/ClusterProxy/executeQuery.cpp | 16 +--------------- 2 files changed, 1 insertion(+), 18 deletions(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 1cc5a3b1a77..ca07fd5deda 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -60,9 +60,6 @@ public: /// (When there is a local replica with big delay). bool lazy = false; time_t local_delay = 0; - - /// Set only if parallel reading from replicas is used. - std::shared_ptr coordinator; }; using Shards = std::vector; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 2fed626ffb7..bb5c83eca39 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -281,7 +281,6 @@ void executeQueryWithParallelReplicas( auto all_replicas_count = std::min(static_cast(settings.max_parallel_replicas), new_cluster->getShardCount()); auto coordinator = std::make_shared(all_replicas_count); auto remote_plan = std::make_unique(); - auto plans = std::vector(); /// This is a little bit weird, but we construct an "empty" coordinator without /// any specified reading/coordination method (like Default, InOrder, InReverseOrder) @@ -309,20 +308,7 @@ void executeQueryWithParallelReplicas( &Poco::Logger::get("ReadFromParallelRemoteReplicasStep"), query_info.storage_limits); - remote_plan->addStep(std::move(read_from_remote)); - remote_plan->addInterpreterContext(context); - plans.emplace_back(std::move(remote_plan)); - - if (std::all_of(plans.begin(), plans.end(), [](const QueryPlanPtr & plan) { return !plan; })) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No plans were generated for reading from shard. This is a bug"); - - DataStreams input_streams; - input_streams.reserve(plans.size()); - for (const auto & plan : plans) - input_streams.emplace_back(plan->getCurrentDataStream()); - - auto union_step = std::make_unique(std::move(input_streams)); - query_plan.unitePlans(std::move(union_step), std::move(plans)); + query_plan.addStep(std::move(read_from_remote)); } } From 1738afc1965de150342e0d9a7d52b85fe561d24c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 14 Aug 2023 16:37:34 +0200 Subject: [PATCH 1968/2047] Update insert-into.md --- docs/en/sql-reference/statements/insert-into.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index d6e30827f9b..e0cc98c2351 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -11,7 +11,7 @@ Inserts data into a table. **Syntax** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` You can specify a list of columns to insert using the `(c1, c2, c3)`. You can also use an expression with column [matcher](../../sql-reference/statements/select/index.md#asterisk) such as `*` and/or [modifiers](../../sql-reference/statements/select/index.md#select-modifiers) such as [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#except-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier). @@ -107,7 +107,7 @@ If table has [constraints](../../sql-reference/statements/create/table.md#constr **Syntax** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] SELECT ... ``` Columns are mapped according to their position in the SELECT clause. However, their names in the SELECT expression and the table for INSERT may differ. If necessary, type casting is performed. @@ -126,7 +126,7 @@ To insert a default value instead of `NULL` into a column with not nullable data **Syntax** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] FROM INFILE file_name [COMPRESSION type] FORMAT format_name +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] FROM INFILE file_name [COMPRESSION type] FORMAT format_name ``` Use the syntax above to insert data from a file, or files, stored on the **client** side. `file_name` and `type` are string literals. Input file [format](../../interfaces/formats.md) must be set in the `FORMAT` clause. From 3d5c9bfa1ad5f00fb1dc3cd45444ebb23a3219a2 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 14 Aug 2023 16:39:37 +0200 Subject: [PATCH 1969/2047] Update insert-into.md --- docs/ru/sql-reference/statements/insert-into.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/statements/insert-into.md b/docs/ru/sql-reference/statements/insert-into.md index 4fa6ac4ce66..747e36b8809 100644 --- a/docs/ru/sql-reference/statements/insert-into.md +++ b/docs/ru/sql-reference/statements/insert-into.md @@ -11,7 +11,7 @@ sidebar_label: INSERT INTO **Синтаксис** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` Вы можете указать список столбцов для вставки, используя синтаксис `(c1, c2, c3)`. Также можно использовать выражение cо [звездочкой](../../sql-reference/statements/select/index.md#asterisk) и/или модификаторами, такими как [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#except-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier). @@ -100,7 +100,7 @@ INSERT INTO t FORMAT TabSeparated **Синтаксис** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] SELECT ... ``` Соответствие столбцов определяется их позицией в секции SELECT. При этом, их имена в выражении SELECT и в таблице для INSERT, могут отличаться. При необходимости выполняется приведение типов данных, эквивалентное соответствующему оператору CAST. @@ -120,7 +120,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... **Синтаксис** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] FROM INFILE file_name [COMPRESSION type] FORMAT format_name +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] FROM INFILE file_name [COMPRESSION type] FORMAT format_name ``` Используйте этот синтаксис, чтобы вставить данные из файла, который хранится на стороне **клиента**. `file_name` и `type` задаются в виде строковых литералов. [Формат](../../interfaces/formats.md) входного файла должен быть задан в секции `FORMAT`. From 8f3f47a51fc15a2a5fc7acf98299b187bb69eed3 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 14 Aug 2023 16:40:36 +0200 Subject: [PATCH 1970/2047] Update insert-into.md --- docs/zh/sql-reference/statements/insert-into.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/zh/sql-reference/statements/insert-into.md b/docs/zh/sql-reference/statements/insert-into.md index 9acc1655f9a..f80c0a8a8ea 100644 --- a/docs/zh/sql-reference/statements/insert-into.md +++ b/docs/zh/sql-reference/statements/insert-into.md @@ -8,7 +8,7 @@ INSERT INTO 语句主要用于向系统中添加数据. 查询的基本格式: ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` 您可以在查询中指定要插入的列的列表,如:`[(c1, c2, c3)]`。您还可以使用列[匹配器](../../sql-reference/statements/select/index.md#asterisk)的表达式,例如`*`和/或[修饰符](../../sql-reference/statements/select/index.md#select-modifiers),例如 [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#apply-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier)。 @@ -71,7 +71,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set 例如,下面的查询所使用的输入格式就与上面INSERT … VALUES的中使用的输入格式相同: ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... ``` ClickHouse会清除数据前所有的空白字符与一个换行符(如果有换行符的话)。所以在进行查询时,我们建议您将数据放入到输入输出格式名称后的新的一行中去(如果数据是以空白字符开始的,这将非常重要)。 @@ -93,7 +93,7 @@ INSERT INTO t FORMAT TabSeparated ### 使用`SELECT`的结果写入 {#inserting-the-results-of-select} ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] SELECT ... ``` 写入与SELECT的列的对应关系是使用位置来进行对应的,尽管它们在SELECT表达式与INSERT中的名称可能是不同的。如果需要,会对它们执行对应的类型转换。 From 364b43f19ef4b266291642e5431d9c389bd9c151 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 14:49:40 +0000 Subject: [PATCH 1971/2047] Cosmetics: add "cache" section marker --- src/Interpreters/Context.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 07e0cac79e3..b9311acc337 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -915,6 +915,8 @@ public: void setSystemZooKeeperLogAfterInitializationIfNeeded(); + /// --- Caches ------------------------------------------------------------------------------------------ + /// Create a cache of uncompressed blocks of specified size. This can be done only once. void setUncompressedCache(const String & uncompressed_cache_policy, size_t max_size_in_bytes); std::shared_ptr getUncompressedCache() const; @@ -962,6 +964,8 @@ public: */ void dropCaches() const; + /// ----------------------------------------------------------------------------------------------------- + /// Settings for MergeTree background tasks stored in config.xml BackgroundTaskSchedulingSettings getBackgroundProcessingTaskSchedulingSettings() const; BackgroundTaskSchedulingSettings getBackgroundMoveTaskSchedulingSettings() const; From a81c762928c3766be025fbb4043081d37f897c02 Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Mon, 14 Aug 2023 14:52:26 +0000 Subject: [PATCH 1972/2047] Fix style --- src/Interpreters/ClusterProxy/executeQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index bb5c83eca39..f2d7132b174 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -28,7 +28,6 @@ namespace DB namespace ErrorCodes { extern const int TOO_LARGE_DISTRIBUTED_DEPTH; - extern const int LOGICAL_ERROR; extern const int SUPPORT_IS_DISABLED; } From fed995734ec446175bdb009d85bfa883d439e02e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 14:52:37 +0000 Subject: [PATCH 1973/2047] Cosmetics: Move prefetch threadpool stuff out of "caching" section --- src/Interpreters/Context.cpp | 48 ++++++++++++++++-------------------- src/Interpreters/Context.h | 14 +++++------ 2 files changed, 28 insertions(+), 34 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b8a700f5311..a12cc70a507 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2315,32 +2315,6 @@ ThreadPool & Context::getLoadMarksThreadpool() const return *shared->load_marks_threadpool; } -static size_t getPrefetchThreadpoolSizeFromConfig(const Poco::Util::AbstractConfiguration & config) -{ - return config.getUInt(".prefetch_threadpool_pool_size", 100); -} - -size_t Context::getPrefetchThreadpoolSize() const -{ - const auto & config = getConfigRef(); - return getPrefetchThreadpoolSizeFromConfig(config); -} - -ThreadPool & Context::getPrefetchThreadpool() const -{ - const auto & config = getConfigRef(); - - auto lock = getLock(); - if (!shared->prefetch_threadpool) - { - auto pool_size = getPrefetchThreadpoolSize(); - auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); - shared->prefetch_threadpool = std::make_unique( - CurrentMetrics::IOPrefetchThreads, CurrentMetrics::IOPrefetchThreadsActive, pool_size, pool_size, queue_size); - } - return *shared->prefetch_threadpool; -} - void Context::setIndexUncompressedCache(size_t max_size_in_bytes) { auto lock = getLock(); @@ -2351,7 +2325,6 @@ void Context::setIndexUncompressedCache(size_t max_size_in_bytes) shared->index_uncompressed_cache = std::make_shared(max_size_in_bytes); } - UncompressedCachePtr Context::getIndexUncompressedCache() const { auto lock = getLock(); @@ -2468,6 +2441,27 @@ void Context::dropCaches() const shared->mmap_cache->reset(); } +ThreadPool & Context::getPrefetchThreadpool() const +{ + const auto & config = getConfigRef(); + + auto lock = getLock(); + if (!shared->prefetch_threadpool) + { + auto pool_size = getPrefetchThreadpoolSize(); + auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); + shared->prefetch_threadpool = std::make_unique( + CurrentMetrics::IOPrefetchThreads, CurrentMetrics::IOPrefetchThreadsActive, pool_size, pool_size, queue_size); + } + return *shared->prefetch_threadpool; +} + +size_t Context::getPrefetchThreadpoolSize() const +{ + const auto & config = getConfigRef(); + return config.getUInt(".prefetch_threadpool_pool_size", 100); +} + BackgroundSchedulePool & Context::getBufferFlushSchedulePool() const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b9311acc337..a2f06f4fc14 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -928,13 +928,6 @@ public: void dropMarkCache() const; ThreadPool & getLoadMarksThreadpool() const; - ThreadPool & getPrefetchThreadpool() const; - - /// Note: prefetchThreadpool is different from threadpoolReader - /// in the way that its tasks are - wait for marks to be loaded - /// and make a prefetch by putting a read task to threadpoolReader. - size_t getPrefetchThreadpoolSize() const; - /// Create a cache of index uncompressed blocks of specified size. This can be done only once. void setIndexUncompressedCache(size_t max_size_in_bytes); std::shared_ptr getIndexUncompressedCache() const; @@ -966,6 +959,13 @@ public: /// ----------------------------------------------------------------------------------------------------- + ThreadPool & getPrefetchThreadpool() const; + + /// Note: prefetchThreadpool is different from threadpoolReader + /// in the way that its tasks are - wait for marks to be loaded + /// and make a prefetch by putting a read task to threadpoolReader. + size_t getPrefetchThreadpoolSize() const; + /// Settings for MergeTree background tasks stored in config.xml BackgroundTaskSchedulingSettings getBackgroundProcessingTaskSchedulingSettings() const; BackgroundTaskSchedulingSettings getBackgroundMoveTaskSchedulingSettings() const; From def9bb8d92d6a7dad413d8a3a88eec7e74d5296d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 17:11:25 +0000 Subject: [PATCH 1974/2047] Exclude the query cache from dropCaches() --- src/Interpreters/Context.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a12cc70a507..1d2774c4d77 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2434,11 +2434,10 @@ void Context::dropCaches() const if (shared->index_mark_cache) shared->index_mark_cache->reset(); - if (shared->query_cache) - shared->query_cache->reset(); - if (shared->mmap_cache) shared->mmap_cache->reset(); + + /// Intentionally not dropping the query cache which is transactionally inconsistent by design. } ThreadPool & Context::getPrefetchThreadpool() const From 646c51ea40038a3d5e6f2f275732c9fa4507ddbd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 17:22:15 +0000 Subject: [PATCH 1975/2047] Cosmetics: Move methods into a more logical order. --- src/Interpreters/Context.cpp | 46 ++++++++++++++++++------------------ 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1d2774c4d77..5a7ca4a7221 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2363,6 +2363,29 @@ void Context::dropIndexMarkCache() const shared->index_mark_cache->reset(); } +void Context::setMMappedFileCache(size_t cache_size_in_num_entries) +{ + auto lock = getLock(); + + if (shared->mmap_cache) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mapped file cache has been already created."); + + shared->mmap_cache = std::make_shared(cache_size_in_num_entries); +} + +MMappedFileCachePtr Context::getMMappedFileCache() const +{ + auto lock = getLock(); + return shared->mmap_cache; +} + +void Context::dropMMappedFileCache() const +{ + auto lock = getLock(); + if (shared->mmap_cache) + shared->mmap_cache->reset(); +} + void Context::setQueryCache(const Poco::Util::AbstractConfiguration & config) { auto lock = getLock(); @@ -2394,29 +2417,6 @@ void Context::dropQueryCache() const shared->query_cache->reset(); } -void Context::setMMappedFileCache(size_t cache_size_in_num_entries) -{ - auto lock = getLock(); - - if (shared->mmap_cache) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Mapped file cache has been already created."); - - shared->mmap_cache = std::make_shared(cache_size_in_num_entries); -} - -MMappedFileCachePtr Context::getMMappedFileCache() const -{ - auto lock = getLock(); - return shared->mmap_cache; -} - -void Context::dropMMappedFileCache() const -{ - auto lock = getLock(); - if (shared->mmap_cache) - shared->mmap_cache->reset(); -} - void Context::dropCaches() const { From 3472a928d804c7fea0a3ddd6b356abf91bdbf630 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 10:02:41 +0000 Subject: [PATCH 1976/2047] Cosmetics: Rename dropCache() to clearCaches() "dropCache()" is a bad name, it sounds like that the cache can no longer be used after the call. Not true, we only "clear" the cache. --- src/Interpreters/Context.cpp | 15 +++++++-------- src/Interpreters/Context.h | 14 +++++++------- src/Interpreters/InterpreterDropQuery.cpp | 4 ++-- src/Interpreters/InterpreterSystemQuery.cpp | 12 ++++++------ src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++---- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- 8 files changed, 29 insertions(+), 30 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5a7ca4a7221..39284b4c1b2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2269,7 +2269,7 @@ UncompressedCachePtr Context::getUncompressedCache() const } -void Context::dropUncompressedCache() const +void Context::clearUncompressedCache() const { auto lock = getLock(); if (shared->uncompressed_cache) @@ -2293,7 +2293,7 @@ MarkCachePtr Context::getMarkCache() const return shared->mark_cache; } -void Context::dropMarkCache() const +void Context::clearMarkCache() const { auto lock = getLock(); if (shared->mark_cache) @@ -2332,7 +2332,7 @@ UncompressedCachePtr Context::getIndexUncompressedCache() const } -void Context::dropIndexUncompressedCache() const +void Context::clearIndexUncompressedCache() const { auto lock = getLock(); if (shared->index_uncompressed_cache) @@ -2356,7 +2356,7 @@ MarkCachePtr Context::getIndexMarkCache() const return shared->index_mark_cache; } -void Context::dropIndexMarkCache() const +void Context::clearIndexMarkCache() const { auto lock = getLock(); if (shared->index_mark_cache) @@ -2379,7 +2379,7 @@ MMappedFileCachePtr Context::getMMappedFileCache() const return shared->mmap_cache; } -void Context::dropMMappedFileCache() const +void Context::clearMMappedFileCache() const { auto lock = getLock(); if (shared->mmap_cache) @@ -2410,15 +2410,14 @@ QueryCachePtr Context::getQueryCache() const return shared->query_cache; } -void Context::dropQueryCache() const +void Context::clearQueryCache() const { auto lock = getLock(); if (shared->query_cache) shared->query_cache->reset(); } - -void Context::dropCaches() const +void Context::clearCaches() const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index a2f06f4fc14..c725d032bbe 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -920,34 +920,34 @@ public: /// Create a cache of uncompressed blocks of specified size. This can be done only once. void setUncompressedCache(const String & uncompressed_cache_policy, size_t max_size_in_bytes); std::shared_ptr getUncompressedCache() const; - void dropUncompressedCache() const; + void clearUncompressedCache() const; /// Create a cache of marks of specified size. This can be done only once. void setMarkCache(const String & mark_cache_policy, size_t cache_size_in_bytes); std::shared_ptr getMarkCache() const; - void dropMarkCache() const; + void clearMarkCache() const; ThreadPool & getLoadMarksThreadpool() const; /// Create a cache of index uncompressed blocks of specified size. This can be done only once. void setIndexUncompressedCache(size_t max_size_in_bytes); std::shared_ptr getIndexUncompressedCache() const; - void dropIndexUncompressedCache() const; + void clearIndexUncompressedCache() const; /// Create a cache of index marks of specified size. This can be done only once. void setIndexMarkCache(size_t cache_size_in_bytes); std::shared_ptr getIndexMarkCache() const; - void dropIndexMarkCache() const; + void clearIndexMarkCache() const; /// Create a cache of mapped files to avoid frequent open/map/unmap/close and to reuse from several threads. void setMMappedFileCache(size_t cache_size_in_num_entries); std::shared_ptr getMMappedFileCache() const; - void dropMMappedFileCache() const; + void clearMMappedFileCache() const; /// Create a cache of query results for statements which run repeatedly. void setQueryCache(const Poco::Util::AbstractConfiguration & config); void updateQueryCacheConfiguration(const Poco::Util::AbstractConfiguration & config); std::shared_ptr getQueryCache() const; - void dropQueryCache() const; + void clearQueryCache() const; /** Clear the caches of the uncompressed blocks and marks. * This is usually done when renaming tables, changing the type of columns, deleting a table. @@ -955,7 +955,7 @@ public: * (when deleting a table - it is necessary, since in its place another can appear) * const - because the change in the cache is not considered significant. */ - void dropCaches() const; + void clearCaches() const; /// ----------------------------------------------------------------------------------------------------- diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ed927d550a8..ea4dfc4df65 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -247,10 +247,10 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue DatabaseCatalog::instance().removeDependencies(table_id, check_ref_deps, check_loading_deps, is_drop_or_detach_database); database->dropTable(context_, table_id.table_name, query.sync); - /// We have to drop mmapio cache when dropping table from Ordinary database + /// We have to clear mmapio cache when dropping table from Ordinary database /// to avoid reading old data if new table with the same name is created if (database->getUUID() == UUIDHelpers::Nil) - context_->dropMMappedFileCache(); + context_->clearMMappedFileCache(); } db = database; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index c0341330846..9c8bc256fa2 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -319,27 +319,27 @@ BlockIO InterpreterSystemQuery::execute() } case Type::DROP_MARK_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE); - system_context->dropMarkCache(); + system_context->clearMarkCache(); break; case Type::DROP_UNCOMPRESSED_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_UNCOMPRESSED_CACHE); - system_context->dropUncompressedCache(); + system_context->clearUncompressedCache(); break; case Type::DROP_INDEX_MARK_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE); - system_context->dropIndexMarkCache(); + system_context->clearIndexMarkCache(); break; case Type::DROP_INDEX_UNCOMPRESSED_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_UNCOMPRESSED_CACHE); - system_context->dropIndexUncompressedCache(); + system_context->clearIndexUncompressedCache(); break; case Type::DROP_MMAP_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_MMAP_CACHE); - system_context->dropMMappedFileCache(); + system_context->clearMMappedFileCache(); break; case Type::DROP_QUERY_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_QUERY_CACHE); - getContext()->dropQueryCache(); + getContext()->clearQueryCache(); break; #if USE_EMBEDDED_COMPILER case Type::DROP_COMPILED_EXPRESSION_CACHE: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index db0a7b34d7e..7b1fbd0deb0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2328,7 +2328,7 @@ size_t MergeTreeData::clearOldPartsFromFilesystem(bool force) removePartsFinally(parts_to_remove); /// This is needed to close files to avoid they reside on disk after being deleted. /// NOTE: we can drop files from cache more selectively but this is good enough. - getContext()->dropMMappedFileCache(); + getContext()->clearMMappedFileCache(); return parts_to_remove.size(); } @@ -2799,7 +2799,7 @@ void MergeTreeData::rename(const String & new_table_path, const StorageID & new_ } if (!getStorageID().hasUUID()) - getContext()->dropCaches(); + getContext()->clearCaches(); /// TODO: remove const_cast for (const auto & part : data_parts_by_info) @@ -2840,9 +2840,9 @@ void MergeTreeData::dropAllData() } /// Tables in atomic databases have UUID and stored in persistent locations. - /// No need to drop caches (that are keyed by filesystem path) because collision is not possible. + /// No need to clear caches (that are keyed by filesystem path) because collision is not possible. if (!getStorageID().hasUUID()) - getContext()->dropCaches(); + getContext()->clearCaches(); /// Removing of each data part before recursive removal of directory is to speed-up removal, because there will be less number of syscalls. NameSet part_names_failed; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 87aa71f3e8d..90cf55e53b2 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -777,7 +777,7 @@ void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr num_marks_saved = 0; total_rows = 0; total_bytes = 0; - getContext()->dropMMappedFileCache(); + getContext()->clearMMappedFileCache(); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ad9013d9f13..dc23e3ea635 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2001,7 +2001,7 @@ PartitionCommandsResultInfo StorageMergeTree::attachPartition( } /// New parts with other data may appear in place of deleted parts. - local_context->dropCaches(); + local_context->clearCaches(); return results; } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 0bfef5ed5e5..83336cbd22e 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -426,7 +426,7 @@ void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont num_indices_saved = 0; total_rows = 0; total_bytes = 0; - getContext()->dropMMappedFileCache(); + getContext()->clearMMappedFileCache(); } From 54e54aed7489915a97314ac54c16c398a7cd0efc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 10:19:40 +0000 Subject: [PATCH 1977/2047] Cosmetics: Introduce constants for default cache sizes and policies --- programs/local/LocalServer.cpp | 31 ++++++------------- programs/server/Server.cpp | 18 +++-------- src/Common/CacheBase.h | 5 +-- src/Core/Defines.h | 16 ++++++++++ src/Core/ServerSettings.h | 15 ++++----- src/Interpreters/Cache/QueryCache.cpp | 8 ++--- src/Interpreters/Context.cpp | 14 ++++----- .../JIT/CompiledExpressionCache.h | 2 +- 8 files changed, 54 insertions(+), 55 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 587c88a2745..b5c7c790a83 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -655,43 +655,32 @@ void LocalServer::processConfig() /// There is no need for concurrent queries, override max_concurrent_queries. global_context->getProcessList().setMaxSize(0); - /// Size of cache for uncompressed blocks. Zero means disabled. - String uncompressed_cache_policy = config().getString("uncompressed_cache_policy", ""); - size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", 0); + String uncompressed_cache_policy = config().getString("uncompressed_cache_policy", DEFAULT_UNCOMPRESSED_CACHE_POLICY); + size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE); if (uncompressed_cache_size) global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size); - /// Size of cache for marks (index of MergeTree family of tables). - String mark_cache_policy = config().getString("mark_cache_policy", ""); - size_t mark_cache_size = config().getUInt64("mark_cache_size", 5368709120); + String mark_cache_policy = config().getString("mark_cache_policy", DEFAULT_MARK_CACHE_POLICY); + size_t mark_cache_size = config().getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE); if (mark_cache_size) global_context->setMarkCache(mark_cache_policy, mark_cache_size); - /// Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled. - size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", 0); + size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE); if (index_uncompressed_cache_size) global_context->setIndexUncompressedCache(index_uncompressed_cache_size); - /// Size of cache for index marks (index of MergeTree skip indices). - size_t index_mark_cache_size = config().getUInt64("index_mark_cache_size", 0); + size_t index_mark_cache_size = config().getUInt64("index_mark_cache_size", DEFAULT_INDEX_MARK_CACHE_MAX_SIZE); if (index_mark_cache_size) global_context->setIndexMarkCache(index_mark_cache_size); - /// A cache for mmapped files. - size_t mmap_cache_size = config().getUInt64("mmap_cache_size", 1000); /// The choice of default is arbitrary. + size_t mmap_cache_size = config().getUInt64("mmap_cache_size", DEFAULT_MMAP_CACHE_MAX_SIZE); if (mmap_cache_size) global_context->setMMappedFileCache(mmap_cache_size); #if USE_EMBEDDED_COMPILER - /// 128 MB - constexpr size_t compiled_expression_cache_size_default = 1024 * 1024 * 128; - size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", compiled_expression_cache_size_default); - - constexpr size_t compiled_expression_cache_elements_size_default = 10000; - size_t compiled_expression_cache_elements_size - = config().getUInt64("compiled_expression_cache_elements_size", compiled_expression_cache_elements_size_default); - - CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_size, compiled_expression_cache_elements_size); + size_t compiled_expression_cache_max_size_in_bytes = config().getUInt64("compiled_expression_cache_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE); + size_t compiled_expression_cache_max_elements = config().getUInt64("compiled_expression_cache_elements_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES); + CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_max_size_in_bytes, compiled_expression_cache_max_elements); #endif /// NOTE: it is important to apply any overrides before diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6237186e3a6..e5604ca422d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1488,13 +1488,11 @@ try size_t max_cache_size = static_cast(memory_amount * server_settings.cache_size_to_ram_max_ratio); String uncompressed_cache_policy = server_settings.uncompressed_cache_policy; - LOG_INFO(log, "Uncompressed cache policy name {}", uncompressed_cache_policy); size_t uncompressed_cache_size = server_settings.uncompressed_cache_size; if (uncompressed_cache_size > max_cache_size) { uncompressed_cache_size = max_cache_size; - LOG_INFO(log, "Uncompressed cache size was lowered to {} because the system has low amount of memory", - formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + LOG_INFO(log, "Uncompressed cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); } global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size); @@ -1520,8 +1518,7 @@ try if (mark_cache_size > max_cache_size) { mark_cache_size = max_cache_size; - LOG_INFO(log, "Mark cache size was lowered to {} because the system has low amount of memory", - formatReadableSizeWithBinarySuffix(mark_cache_size)); + LOG_INFO(log, "Mark cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(mark_cache_size)); } global_context->setMarkCache(mark_cache_policy, mark_cache_size); @@ -1538,14 +1535,9 @@ try global_context->setQueryCache(config()); #if USE_EMBEDDED_COMPILER - /// 128 MB - constexpr size_t compiled_expression_cache_size_default = 1024 * 1024 * 128; - size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", compiled_expression_cache_size_default); - - constexpr size_t compiled_expression_cache_elements_size_default = 10000; - size_t compiled_expression_cache_elements_size = config().getUInt64("compiled_expression_cache_elements_size", compiled_expression_cache_elements_size_default); - - CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_size, compiled_expression_cache_elements_size); + size_t compiled_expression_cache_max_size_in_bytes = config().getUInt64("compiled_expression_cache_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE); + size_t compiled_expression_cache_max_elements = config().getUInt64("compiled_expression_cache_elements_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES); + CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_max_size_in_bytes, compiled_expression_cache_max_elements); #endif /// Set path for format schema files diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index 84cbd5b5c6f..aa7b3ea10cf 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -51,10 +51,11 @@ public: { auto on_weight_loss_function = [&](size_t weight_loss) { onRemoveOverflowWeightLoss(weight_loss); }; - static constexpr std::string_view default_cache_policy = "SLRU"; - if (cache_policy_name.empty()) + { + static constexpr auto default_cache_policy = "SLRU"; cache_policy_name = default_cache_policy; + } if (cache_policy_name == "LRU") { diff --git a/src/Core/Defines.h b/src/Core/Defines.h index efe14b93a3d..3039f0a67cf 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -1,6 +1,7 @@ #pragma once #include +#include #define DBMS_DEFAULT_PORT 9000 #define DBMS_DEFAULT_SECURE_PORT 9440 @@ -64,6 +65,21 @@ /// Max depth of hierarchical dictionary #define DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH 1000 +/// Default maximum (total and entry) sizes and policies of various caches +static constexpr auto DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE = 0_MiB; +static constexpr auto DEFAULT_UNCOMPRESSED_CACHE_POLICY = "SLRU"; +static constexpr auto DEFAULT_MARK_CACHE_MAX_SIZE = 5368_MiB; +static constexpr auto DEFAULT_MARK_CACHE_POLICY = "SLRU"; +static constexpr auto DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE = 0_MiB; +static constexpr auto DEFAULT_INDEX_MARK_CACHE_MAX_SIZE = 0_MiB; +static constexpr auto DEFAULT_MMAP_CACHE_MAX_SIZE = 1_KiB; /// chosen by rolling dice +static constexpr auto DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE = 128_MiB; +static constexpr auto DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES = 10'000; +static constexpr auto DEFAULT_QUERY_CACHE_MAX_SIZE = 1_GiB; +static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRIES = 1024uz; +static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES = 1_MiB; +static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS = 30'000'000uz; + /// Query profiler cannot work with sanitizers. /// Sanitizers are using quick "frame walking" stack unwinding (this implies -fno-omit-frame-pointer) /// And they do unwinding frequently (on every malloc/free, thread/mutex operations, etc). diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 7678e8c3f24..f759cd150a8 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -2,6 +2,7 @@ #include +#include namespace Poco::Util @@ -56,13 +57,13 @@ namespace DB M(UInt64, max_concurrent_select_queries, 0, "Limit on total number of concurrently select queries. Zero means Unlimited.", 0) \ \ M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro ram max ratio. Allows to lower cache size on low-memory systems.", 0) \ - M(String, uncompressed_cache_policy, "SLRU", "Uncompressed cache policy name.", 0) \ - M(UInt64, uncompressed_cache_size, 0, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \ - M(UInt64, mark_cache_size, 5368709120, "Size of cache for marks (index of MergeTree family of tables).", 0) \ - M(String, mark_cache_policy, "SLRU", "Mark cache policy name.", 0) \ - M(UInt64, index_uncompressed_cache_size, 0, "Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled.", 0) \ - M(UInt64, index_mark_cache_size, 0, "Size of cache for index marks. Zero means disabled.", 0) \ - M(UInt64, mmap_cache_size, 1000, "A cache for mmapped files.", 0) /* The choice of default is arbitrary. */ \ + M(String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0) \ + M(UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \ + M(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \ + M(String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0) \ + M(UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled.", 0) \ + M(UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for index marks. Zero means disabled.", 0) \ + M(UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0) \ \ M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \ M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \ diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 5982a5ade50..182a186d4e1 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -531,14 +531,14 @@ void QueryCache::updateConfiguration(const Poco::Util::AbstractConfiguration & c { std::lock_guard lock(mutex); - size_t max_size_in_bytes = config.getUInt64("query_cache.max_size_in_bytes", 1_GiB); + size_t max_size_in_bytes = config.getUInt64("query_cache.max_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_SIZE); cache.setMaxSize(max_size_in_bytes); - size_t max_entries = config.getUInt64("query_cache.max_entries", 1024); + size_t max_entries = config.getUInt64("query_cache.max_entries", DEFAULT_QUERY_CACHE_MAX_ENTRIES); cache.setMaxCount(max_entries); - max_entry_size_in_bytes = config.getUInt64("query_cache.max_entry_size_in_bytes", 1_MiB); - max_entry_size_in_rows = config.getUInt64("query_cache.max_entry_rows_in_rows", 30'000'000); + max_entry_size_in_bytes = config.getUInt64("query_cache.max_entry_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES); + max_entry_size_in_rows = config.getUInt64("query_cache.max_entry_rows_in_rows", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 39284b4c1b2..b77b4b79cb9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -245,9 +245,9 @@ struct ContextSharedPart : boost::noncopyable std::optional backups_worker; - String default_profile_name; /// Default profile name used for default values. - String system_profile_name; /// Profile used by system processes - String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying + String default_profile_name; /// Default profile name used for default values. + String system_profile_name; /// Profile used by system processes + String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying std::unique_ptr access_control; mutable ResourceManagerPtr resource_manager; mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. @@ -261,11 +261,11 @@ struct ContextSharedPart : boost::noncopyable ProcessList process_list; /// Executing queries at the moment. SessionTracker session_tracker; GlobalOvercommitTracker global_overcommit_tracker; - MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) - MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) + MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) + MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) ReplicatedFetchList replicated_fetch_list; - ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. - InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. + ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. + InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. mutable std::unique_ptr buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables. mutable std::unique_ptr schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) diff --git a/src/Interpreters/JIT/CompiledExpressionCache.h b/src/Interpreters/JIT/CompiledExpressionCache.h index 21f7c67226c..8354b33bae9 100644 --- a/src/Interpreters/JIT/CompiledExpressionCache.h +++ b/src/Interpreters/JIT/CompiledExpressionCache.h @@ -19,7 +19,7 @@ public: size_t getCompiledExpressionSize() const { return compiled_expression_size; } - virtual ~CompiledExpressionCacheEntry() {} + virtual ~CompiledExpressionCacheEntry() = default; private: From 9e4b4541bc34a58bbc564d63fec4e0710eb45e33 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 15:13:54 +0000 Subject: [PATCH 1978/2047] Cosmetics: More aesthetic code --- src/Interpreters/Context.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b77b4b79cb9..0cc3750dc26 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -250,15 +250,15 @@ struct ContextSharedPart : boost::noncopyable String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying std::unique_ptr access_control; mutable ResourceManagerPtr resource_manager; - mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. - mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. - mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. - mutable std::unique_ptr prefetch_threadpool; /// Threadpool for loading marks cache. - mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices. - mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices. - mutable QueryCachePtr query_cache; /// Cache of query results. - mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. - ProcessList process_list; /// Executing queries at the moment. + mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. + mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. + mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. + mutable std::unique_ptr prefetch_threadpool; /// Threadpool for loading marks cache. + mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices. + mutable QueryCachePtr query_cache; /// Cache of query results. + mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices. + mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. + ProcessList process_list; /// Executing queries at the moment. SessionTracker session_tracker; GlobalOvercommitTracker global_overcommit_tracker; MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) From f126e54b0d76e21f93104d42b3620dfa36fb436a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 18:33:48 +0000 Subject: [PATCH 1979/2047] Respect max_cache_size for all caches So far, max_cache_size was only respected in Server.cpp, and in this file only for the mark cache and the uncompressed cache. This commit respects it also for the other cache types in Server.cpp, as well for all caches in LocalServer.cpp. --- programs/local/LocalServer.cpp | 31 +++++++++++++++++++++++++++++ programs/server/Server.cpp | 36 +++++++++++++++++++++++++--------- 2 files changed, 58 insertions(+), 9 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index b5c7c790a83..d97a4760f49 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include #include @@ -655,25 +657,54 @@ void LocalServer::processConfig() /// There is no need for concurrent queries, override max_concurrent_queries. global_context->getProcessList().setMaxSize(0); + const size_t memory_amount = getMemoryAmount(); + const double cache_size_to_ram_max_ratio = config().getDouble("cache_size_to_ram_max_ratio", 0.5); + const size_t max_cache_size = static_cast(memory_amount * cache_size_to_ram_max_ratio); + String uncompressed_cache_policy = config().getString("uncompressed_cache_policy", DEFAULT_UNCOMPRESSED_CACHE_POLICY); size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE); + if (uncompressed_cache_size > max_cache_size) + { + uncompressed_cache_size = max_cache_size; + LOG_INFO(log, "Lowered uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } if (uncompressed_cache_size) global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size); String mark_cache_policy = config().getString("mark_cache_policy", DEFAULT_MARK_CACHE_POLICY); size_t mark_cache_size = config().getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE); + if (mark_cache_size > max_cache_size) + { + mark_cache_size = max_cache_size; + LOG_INFO(log, "Lowered mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mark_cache_size)); + } if (mark_cache_size) global_context->setMarkCache(mark_cache_policy, mark_cache_size); size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE); + if (index_uncompressed_cache_size > max_cache_size) + { + index_uncompressed_cache_size = max_cache_size; + LOG_INFO(log, "Lowered index uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } if (index_uncompressed_cache_size) global_context->setIndexUncompressedCache(index_uncompressed_cache_size); size_t index_mark_cache_size = config().getUInt64("index_mark_cache_size", DEFAULT_INDEX_MARK_CACHE_MAX_SIZE); + if (index_mark_cache_size > max_cache_size) + { + index_mark_cache_size = max_cache_size; + LOG_INFO(log, "Lowered index mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } if (index_mark_cache_size) global_context->setIndexMarkCache(index_mark_cache_size); size_t mmap_cache_size = config().getUInt64("mmap_cache_size", DEFAULT_MMAP_CACHE_MAX_SIZE); + if (mmap_cache_size > max_cache_size) + { + mmap_cache_size = max_cache_size; + LOG_INFO(log, "Lowered mmap file cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } if (mmap_cache_size) global_context->setMMappedFileCache(mmap_cache_size); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index e5604ca422d..d5ae2513fa0 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -658,7 +659,7 @@ try global_context->addWarningMessage("Server was built with sanitizer. It will work slowly."); #endif - const auto memory_amount = getMemoryAmount(); + const size_t memory_amount = getMemoryAmount(); LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.", formatReadableSizeWithBinarySuffix(memory_amount), @@ -1485,14 +1486,14 @@ try /// Set up caches. - size_t max_cache_size = static_cast(memory_amount * server_settings.cache_size_to_ram_max_ratio); + const size_t max_cache_size = static_cast(memory_amount * server_settings.cache_size_to_ram_max_ratio); String uncompressed_cache_policy = server_settings.uncompressed_cache_policy; size_t uncompressed_cache_size = server_settings.uncompressed_cache_size; if (uncompressed_cache_size > max_cache_size) { uncompressed_cache_size = max_cache_size; - LOG_INFO(log, "Uncompressed cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + LOG_INFO(log, "Lowered uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); } global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size); @@ -1511,27 +1512,44 @@ try server_settings.async_insert_queue_flush_on_shutdown)); } - size_t mark_cache_size = server_settings.mark_cache_size; String mark_cache_policy = server_settings.mark_cache_policy; + size_t mark_cache_size = server_settings.mark_cache_size; if (!mark_cache_size) LOG_ERROR(log, "Too low mark cache size will lead to severe performance degradation."); if (mark_cache_size > max_cache_size) { mark_cache_size = max_cache_size; - LOG_INFO(log, "Mark cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(mark_cache_size)); + LOG_INFO(log, "Lowered mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mark_cache_size)); } global_context->setMarkCache(mark_cache_policy, mark_cache_size); - if (server_settings.index_uncompressed_cache_size) + size_t index_uncompressed_cache_size = server_settings.index_uncompressed_cache_size; + if (index_uncompressed_cache_size > max_cache_size) + { + index_uncompressed_cache_size = max_cache_size; + LOG_INFO(log, "Lowered index uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } + if (index_uncompressed_cache_size) global_context->setIndexUncompressedCache(server_settings.index_uncompressed_cache_size); - if (server_settings.index_mark_cache_size) + size_t index_mark_cache_size = server_settings.index_mark_cache_size; + if (index_mark_cache_size > max_cache_size) + { + index_mark_cache_size = max_cache_size; + LOG_INFO(log, "Lowered index mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } + if (index_mark_cache_size) global_context->setIndexMarkCache(server_settings.index_mark_cache_size); - if (server_settings.mmap_cache_size) + size_t mmap_cache_size = server_settings.mmap_cache_size; + if (mmap_cache_size > max_cache_size) + { + mmap_cache_size = max_cache_size; + LOG_INFO(log, "Lowered mmap file cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } + if (mmap_cache_size) global_context->setMMappedFileCache(server_settings.mmap_cache_size); - /// A cache for query results. global_context->setQueryCache(config()); #if USE_EMBEDDED_COMPILER From bb53d635dfead645fc246b8f38bef99239290014 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 18:47:03 +0000 Subject: [PATCH 1980/2047] More consistency: Run same mark cache size check in ch-local Server.cpp had a check that the configured mark cache size is not 0. Run the same check in LocalServer.cpp. --- programs/local/LocalServer.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index d97a4760f49..5e2c8f927bc 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -673,6 +673,8 @@ void LocalServer::processConfig() String mark_cache_policy = config().getString("mark_cache_policy", DEFAULT_MARK_CACHE_POLICY); size_t mark_cache_size = config().getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE); + if (!mark_cache_size) + LOG_ERROR(log, "Too low mark cache size will lead to severe performance degradation."); if (mark_cache_size > max_cache_size) { mark_cache_size = max_cache_size; From ed90463888742f60939650480104b96242f19996 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 18:55:09 +0000 Subject: [PATCH 1981/2047] Add comment --- programs/local/LocalServer.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 5e2c8f927bc..0dea7e8e643 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -710,6 +710,8 @@ void LocalServer::processConfig() if (mmap_cache_size) global_context->setMMappedFileCache(mmap_cache_size); + /// not initializing the query cache in clickhouse-local + #if USE_EMBEDDED_COMPILER size_t compiled_expression_cache_max_size_in_bytes = config().getUInt64("compiled_expression_cache_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE); size_t compiled_expression_cache_max_elements = config().getUInt64("compiled_expression_cache_elements_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES); From 7ed5166c8aca62e7b9397b8b108afa6b26098bde Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 10:56:32 +0000 Subject: [PATCH 1982/2047] Cosmetics: Make re-cfg of query cache similar to re-cfg of other caches This doesn't change semantics, it only changes in which places the configuration is parsed. This makes the logic more consistent across caches. --- programs/server/Server.cpp | 11 ++++++++++- src/Interpreters/Cache/QueryCache.cpp | 17 ++++++----------- src/Interpreters/Cache/QueryCache.h | 13 +++++-------- src/Interpreters/Context.cpp | 13 +++++++++---- src/Interpreters/Context.h | 2 +- 5 files changed, 31 insertions(+), 25 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d5ae2513fa0..b823cbbe367 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1550,7 +1550,16 @@ try if (mmap_cache_size) global_context->setMMappedFileCache(server_settings.mmap_cache_size); - global_context->setQueryCache(config()); + size_t query_cache_max_size_in_bytes = config().getUInt64("query_cache.max_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_SIZE); + size_t query_cache_max_entries = config().getUInt64("query_cache.max_entries", DEFAULT_QUERY_CACHE_MAX_ENTRIES); + size_t query_cache_query_cache_max_entry_size_in_bytes = config().getUInt64("query_cache.max_entry_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES); + size_t query_cache_max_entry_size_in_rows = config().getUInt64("query_cache.max_entry_rows_in_rows", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS); + if (query_cache_max_size_in_bytes > max_cache_size) + { + query_cache_max_size_in_bytes = max_cache_size; + LOG_INFO(log, "Lowered query cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } + global_context->setQueryCache(query_cache_max_size_in_bytes, query_cache_max_entries, query_cache_query_cache_max_entry_size_in_bytes, query_cache_max_entry_size_in_rows); #if USE_EMBEDDED_COMPILER size_t compiled_expression_cache_max_size_in_bytes = config().getUInt64("compiled_expression_cache_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE); diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 182a186d4e1..134aa0956d1 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -493,7 +493,6 @@ void QueryCache::reset() cache.reset(); std::lock_guard lock(mutex); times_executed.clear(); - cache_size_in_bytes = 0; } size_t QueryCache::weight() const @@ -511,7 +510,7 @@ size_t QueryCache::recordQueryRun(const Key & key) std::lock_guard lock(mutex); size_t times = ++times_executed[key]; // Regularly drop times_executed to avoid DOS-by-unlimited-growth. - static constexpr size_t TIMES_EXECUTED_MAX_SIZE = 10'000; + static constexpr auto TIMES_EXECUTED_MAX_SIZE = 10'000uz; if (times_executed.size() > TIMES_EXECUTED_MAX_SIZE) times_executed.clear(); return times; @@ -522,23 +521,19 @@ std::vector QueryCache::dump() const return cache.dump(); } -QueryCache::QueryCache() +QueryCache::QueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_) : cache(std::make_unique>(std::make_unique())) { + updateConfiguration(max_size_in_bytes, max_entries, max_entry_size_in_bytes_, max_entry_size_in_rows_); } -void QueryCache::updateConfiguration(const Poco::Util::AbstractConfiguration & config) +void QueryCache::updateConfiguration(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_) { std::lock_guard lock(mutex); - - size_t max_size_in_bytes = config.getUInt64("query_cache.max_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_SIZE); cache.setMaxSize(max_size_in_bytes); - - size_t max_entries = config.getUInt64("query_cache.max_entries", DEFAULT_QUERY_CACHE_MAX_ENTRIES); cache.setMaxCount(max_entries); - - max_entry_size_in_bytes = config.getUInt64("query_cache.max_entry_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES); - max_entry_size_in_rows = config.getUInt64("query_cache.max_entry_rows_in_rows", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS); + max_entry_size_in_bytes = max_entry_size_in_bytes_; + max_entry_size_in_rows = max_entry_size_in_rows_; } } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index eaa54c503fa..0c0674c6302 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -4,7 +4,6 @@ #include #include #include -#include #include #include @@ -110,9 +109,6 @@ private: /// query --> query result using Cache = CacheBase; - /// query --> query execution count - using TimesExecuted = std::unordered_map; - public: /// Buffers multiple partial query result chunks (buffer()) and eventually stores them as cache entry (finalizeWrite()). /// @@ -177,9 +173,9 @@ public: friend class QueryCache; /// for createReader() }; - QueryCache(); + QueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_); - void updateConfiguration(const Poco::Util::AbstractConfiguration & config); + void updateConfiguration(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_); Reader createReader(const Key & key); Writer createWriter(const Key & key, std::chrono::milliseconds min_query_runtime, bool squash_partial_results, size_t max_block_size, size_t max_query_cache_size_in_bytes_quota, size_t max_query_cache_entries_quota); @@ -199,14 +195,15 @@ private: Cache cache; /// has its own locking --> not protected by mutex mutable std::mutex mutex; + + /// query --> query execution count + using TimesExecuted = std::unordered_map; TimesExecuted times_executed TSA_GUARDED_BY(mutex); /// Cache configuration size_t max_entry_size_in_bytes TSA_GUARDED_BY(mutex) = 0; size_t max_entry_size_in_rows TSA_GUARDED_BY(mutex) = 0; - size_t cache_size_in_bytes TSA_GUARDED_BY(mutex) = 0; /// Updated in each cache insert/delete - friend class StorageSystemQueryCache; }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0cc3750dc26..0886d699734 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2386,22 +2386,27 @@ void Context::clearMMappedFileCache() const shared->mmap_cache->reset(); } -void Context::setQueryCache(const Poco::Util::AbstractConfiguration & config) +void Context::setQueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes, size_t max_entry_size_in_rows) { auto lock = getLock(); if (shared->query_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Query cache has been already created."); - shared->query_cache = std::make_shared(); - shared->query_cache->updateConfiguration(config); + shared->query_cache = std::make_shared(max_size_in_bytes, max_entries, max_entry_size_in_bytes, max_entry_size_in_rows); } void Context::updateQueryCacheConfiguration(const Poco::Util::AbstractConfiguration & config) { auto lock = getLock(); if (shared->query_cache) - shared->query_cache->updateConfiguration(config); + { + size_t max_size_in_bytes = config.getUInt64("query_cache.max_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_SIZE); + size_t max_entries = config.getUInt64("query_cache.max_entries", DEFAULT_QUERY_CACHE_MAX_ENTRIES); + size_t max_entry_size_in_bytes = config.getUInt64("query_cache.max_entry_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES); + size_t max_entry_size_in_rows = config.getUInt64("query_cache.max_entry_rows_in_rows", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS); + shared->query_cache->updateConfiguration(max_size_in_bytes, max_entries, max_entry_size_in_bytes, max_entry_size_in_rows); + } } QueryCachePtr Context::getQueryCache() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c725d032bbe..587fe402d4e 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -944,7 +944,7 @@ public: void clearMMappedFileCache() const; /// Create a cache of query results for statements which run repeatedly. - void setQueryCache(const Poco::Util::AbstractConfiguration & config); + void setQueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes, size_t max_entry_size_in_rows); void updateQueryCacheConfiguration(const Poco::Util::AbstractConfiguration & config); std::shared_ptr getQueryCache() const; void clearQueryCache() const; From c6dc7a8a0bc1aaffeaf3d967f260c8630fb52154 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 14 Aug 2023 16:04:58 +0000 Subject: [PATCH 1983/2047] Update test --- tests/queries/0_stateless/02404_memory_bound_merging.reference | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.reference b/tests/queries/0_stateless/02404_memory_bound_merging.reference index d9fac433189..41a3b6bf8ec 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.reference +++ b/tests/queries/0_stateless/02404_memory_bound_merging.reference @@ -118,8 +118,7 @@ ExpressionTransform MergingAggregatedBucketTransform × 4 Resize 1 → 4 GroupingAggregatedTransform 3 → 1 - (Union) - (ReadFromRemoteParallelReplicas) + (ReadFromRemoteParallelReplicas) select a, count() from pr_t group by a order by a limit 5 offset 500; 500 1000 501 1000 From 9dafc596d06ece75d1c53bfc287159b8ed849033 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Sat, 12 Aug 2023 01:04:08 +0200 Subject: [PATCH 1984/2047] Analyzer: fix quotas for system tables --- .../InterpreterSelectQueryAnalyzer.cpp | 2 +- src/Interpreters/executeQuery.cpp | 6 ++++ src/Planner/Planner.cpp | 6 ++-- src/Planner/Planner.h | 8 ++--- src/Planner/PlannerJoinTree.cpp | 26 ++++++++++++++- src/Planner/PlannerJoinTree.h | 2 +- tests/analyzer_integration_broken_tests.txt | 32 ------------------- 7 files changed, 40 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 8db1d27c073..b8cace5e0ad 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -184,7 +184,7 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( , context(buildContext(context_, select_query_options_)) , select_query_options(select_query_options_) , query_tree(query_tree_) - , planner(query_tree_, select_query_options_) + , planner(query_tree_, select_query_options) { } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 578ca3b41f9..597c5bda245 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -45,6 +45,7 @@ #include #include #include +#include #include #include #include @@ -1033,6 +1034,11 @@ static std::tuple executeQueryImpl( } + // InterpreterSelectQueryAnalyzer does not build QueryPlan in the constructor. + // We need to force to build it here to check if we need to ingore quota. + if (auto * interpreter_with_analyzer = dynamic_cast(interpreter.get())) + interpreter_with_analyzer->getQueryPlan(); + if (!interpreter->ignoreQuota() && !quota_checked) { quota = context->getQuota(); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 9f6c22f90f3..7cce495dfb8 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1047,7 +1047,7 @@ PlannerContextPtr buildPlannerContext(const QueryTreeNodePtr & query_tree_node, } Planner::Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_) + SelectQueryOptions & select_query_options_) : query_tree(query_tree_) , select_query_options(select_query_options_) , planner_context(buildPlannerContext(query_tree, select_query_options, std::make_shared())) @@ -1055,7 +1055,7 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_, } Planner::Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, + SelectQueryOptions & select_query_options_, GlobalPlannerContextPtr global_planner_context_) : query_tree(query_tree_) , select_query_options(select_query_options_) @@ -1064,7 +1064,7 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_, } Planner::Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, + SelectQueryOptions & select_query_options_, PlannerContextPtr planner_context_) : query_tree(query_tree_) , select_query_options(select_query_options_) diff --git a/src/Planner/Planner.h b/src/Planner/Planner.h index 783a07f6e99..f8d151365cf 100644 --- a/src/Planner/Planner.h +++ b/src/Planner/Planner.h @@ -22,16 +22,16 @@ class Planner public: /// Initialize planner with query tree after analysis phase Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_); + SelectQueryOptions & select_query_options_); /// Initialize planner with query tree after query analysis phase and global planner context Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, + SelectQueryOptions & select_query_options_, GlobalPlannerContextPtr global_planner_context_); /// Initialize planner with query tree after query analysis phase and planner context Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, + SelectQueryOptions & select_query_options_, PlannerContextPtr planner_context_); const QueryPlan & getQueryPlan() const @@ -66,7 +66,7 @@ private: void buildPlanForQueryNode(); QueryTreeNodePtr query_tree; - SelectQueryOptions select_query_options; + SelectQueryOptions & select_query_options; PlannerContextPtr planner_context; QueryPlan query_plan; StorageLimitsList storage_limits; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 56a48ce8328..11de6fcfabe 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -113,6 +113,20 @@ void checkAccessRights(const TableNode & table_node, const Names & column_names, query_context->checkAccess(AccessType::SELECT, storage_id, column_names); } +bool shouldIgnoreQuotaAndLimits(const TableNode & table_node) +{ + const auto & storage_id = table_node.getStorageID(); + if (!storage_id.hasDatabase()) + return false; + if (storage_id.database_name == DatabaseCatalog::SYSTEM_DATABASE) + { + static const boost::container::flat_set tables_ignoring_quota{"quotas", "quota_limits", "quota_usage", "quotas_usage", "one"}; + if (tables_ignoring_quota.count(storage_id.table_name)) + return true; + } + return false; +} + NameAndTypePair chooseSmallestColumnToReadFromStorage(const StoragePtr & storage, const StorageSnapshotPtr & storage_snapshot) { /** We need to read at least one column to find the number of rows. @@ -1375,7 +1389,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, const SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, + SelectQueryOptions & select_query_options, const ColumnIdentifierSet & outer_scope_columns, PlannerContextPtr & planner_context) { @@ -1386,6 +1400,16 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, std::vector table_expressions_outer_scope_columns(table_expressions_stack_size); ColumnIdentifierSet current_outer_scope_columns = outer_scope_columns; + if (is_single_table_expression) + { + auto * table_node = table_expressions_stack[0]->as(); + if (table_node && shouldIgnoreQuotaAndLimits(*table_node)) + { + select_query_options.ignore_quota = true; + select_query_options.ignore_limits = true; + } + } + /// For each table, table function, query, union table expressions prepare before query plan build for (size_t i = 0; i < table_expressions_stack_size; ++i) { diff --git a/src/Planner/PlannerJoinTree.h b/src/Planner/PlannerJoinTree.h index acbc96ddae0..9d3b98175d0 100644 --- a/src/Planner/PlannerJoinTree.h +++ b/src/Planner/PlannerJoinTree.h @@ -20,7 +20,7 @@ struct JoinTreeQueryPlan /// Build JOIN TREE query plan for query node JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, const SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, + SelectQueryOptions & select_query_options, const ColumnIdentifierSet & outer_scope_columns, PlannerContextPtr & planner_context); diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 68822fbf311..b485f3f60cc 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -96,22 +96,6 @@ test_executable_table_function/test.py::test_executable_function_input_python test_settings_profile/test.py::test_show_profiles test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster test_postgresql_protocol/test.py::test_python_client -test_quota/test.py::test_add_remove_interval -test_quota/test.py::test_add_remove_quota -test_quota/test.py::test_consumption_of_show_clusters -test_quota/test.py::test_consumption_of_show_databases -test_quota/test.py::test_consumption_of_show_privileges -test_quota/test.py::test_consumption_of_show_processlist -test_quota/test.py::test_consumption_of_show_tables -test_quota/test.py::test_dcl_introspection -test_quota/test.py::test_dcl_management -test_quota/test.py::test_exceed_quota -test_quota/test.py::test_query_inserts -test_quota/test.py::test_quota_from_users_xml -test_quota/test.py::test_reload_users_xml_by_timer -test_quota/test.py::test_simpliest_quota -test_quota/test.py::test_tracking_quota -test_quota/test.py::test_users_xml_is_readonly test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database test_profile_events_s3/test.py::test_profile_events test_user_defined_object_persistence/test.py::test_persistence @@ -121,22 +105,6 @@ test_select_access_rights/test_main.py::test_alias_columns test_select_access_rights/test_main.py::test_select_count test_select_access_rights/test_main.py::test_select_join test_postgresql_protocol/test.py::test_python_client -test_quota/test.py::test_add_remove_interval -test_quota/test.py::test_add_remove_quota -test_quota/test.py::test_consumption_of_show_clusters -test_quota/test.py::test_consumption_of_show_databases -test_quota/test.py::test_consumption_of_show_privileges -test_quota/test.py::test_consumption_of_show_processlist -test_quota/test.py::test_consumption_of_show_tables -test_quota/test.py::test_dcl_introspection -test_quota/test.py::test_dcl_management -test_quota/test.py::test_exceed_quota -test_quota/test.py::test_query_inserts -test_quota/test.py::test_quota_from_users_xml -test_quota/test.py::test_reload_users_xml_by_timer -test_quota/test.py::test_simpliest_quota -test_quota/test.py::test_tracking_quota -test_quota/test.py::test_users_xml_is_readonly test_replicating_constants/test.py::test_different_versions test_merge_tree_s3/test.py::test_heavy_insert_select_check_memory[node] test_wrong_db_or_table_name/test.py::test_wrong_table_name From a366c1c532d6cb176c8c4ba72e8a3ca6f5ca7f2d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Sun, 13 Aug 2023 01:04:33 +0200 Subject: [PATCH 1985/2047] Update src/Interpreters/executeQuery.cpp --- src/Interpreters/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 597c5bda245..a56007375f4 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1035,7 +1035,7 @@ static std::tuple executeQueryImpl( } // InterpreterSelectQueryAnalyzer does not build QueryPlan in the constructor. - // We need to force to build it here to check if we need to ingore quota. + // We need to force to build it here to check if we need to ignore quota. if (auto * interpreter_with_analyzer = dynamic_cast(interpreter.get())) interpreter_with_analyzer->getQueryPlan(); From 12448285555abc54bf14a3a35f38ced6db736b06 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 14 Aug 2023 19:27:05 +0200 Subject: [PATCH 1986/2047] Analyzer: fix virtual columns in StorageDistributed --- src/Storages/StorageDistributed.cpp | 6 +++++- .../0_stateless/02844_distributed_virtual_columns.reference | 0 .../0_stateless/02844_distributed_virtual_columns.sql | 5 +++++ 3 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02844_distributed_virtual_columns.reference create mode 100644 tests/queries/0_stateless/02844_distributed_virtual_columns.sql diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a7aeb11e2d8..f80e498efa8 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -691,7 +691,11 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, if (remote_storage_id.hasDatabase()) resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); - auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns(), distributed_storage_snapshot->object_columns); + auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); + + auto column_names_and_types = distributed_storage_snapshot->getColumns(get_column_options); + + auto storage = std::make_shared(resolved_remote_storage_id, ColumnsDescription{column_names_and_types}); auto table_node = std::make_shared(std::move(storage), query_context); if (table_expression_modifiers) diff --git a/tests/queries/0_stateless/02844_distributed_virtual_columns.reference b/tests/queries/0_stateless/02844_distributed_virtual_columns.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02844_distributed_virtual_columns.sql b/tests/queries/0_stateless/02844_distributed_virtual_columns.sql new file mode 100644 index 00000000000..31a6780f19e --- /dev/null +++ b/tests/queries/0_stateless/02844_distributed_virtual_columns.sql @@ -0,0 +1,5 @@ +drop table if exists data_01072; +drop table if exists dist_01072; +create table data_01072 (key Int) Engine=MergeTree() ORDER BY key; +create table dist_01072 (key Int) Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01072, key); +select * from dist_01072 where key=0 and _part='0'; From 2c1bd7d3bcba800150812b32d7b01a2098e0cb62 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 14 Aug 2023 18:15:31 +0000 Subject: [PATCH 1987/2047] Remove duplicate function specialization --- src/Common/TransformEndianness.hpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 9d34ce6ba77..7c77e918199 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -77,11 +77,4 @@ inline void transformEndianness(CityHash_v1_0_2::uint128 & x) transformEndianness(x.low64); transformEndianness(x.high64); } - -template -inline void transformEndianness(CityHash_v1_0_2::uint128 & x) -{ - transformEndianness(x.low64); - transformEndianness(x.high64); -} } From 368f6d7b1390b98ccac2610eb88a4237abcab439 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Aug 2023 20:46:41 +0200 Subject: [PATCH 1988/2047] fix --- src/Functions/transform.cpp | 4 ++++ tests/queries/0_stateless/02443_detach_attach_partition.sh | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 16326dd5a44..62ab51abd76 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -776,8 +776,12 @@ namespace UInt64 key = 0; auto * dst = reinterpret_cast(&key); const auto ref = cache.from_column->getDataAt(i); + +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wunreachable-code" if constexpr (std::endian::native == std::endian::big) dst += sizeof(key) - ref.size; +#pragma clang diagnostic pop memcpy(dst, ref.data, ref.size); table[key] = i; diff --git a/tests/queries/0_stateless/02443_detach_attach_partition.sh b/tests/queries/0_stateless/02443_detach_attach_partition.sh index 36bc3309924..13ea966dbf5 100755 --- a/tests/queries/0_stateless/02443_detach_attach_partition.sh +++ b/tests/queries/0_stateless/02443_detach_attach_partition.sh @@ -55,7 +55,7 @@ wait $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table0" $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table1" -$CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table0 ATTACH PARTITION ID 'all'" +while ! $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table0 ATTACH PARTITION ID 'all'" 2>/dev/null; do sleep 0.5; done $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table1 ATTACH PARTITION ID 'all'" 2>/dev/null $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table1" $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table1 ATTACH PARTITION ID 'all'" From 1916aec2e24fca860a3ec0fcae172450f9210ed0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Aug 2023 19:06:32 +0200 Subject: [PATCH 1989/2047] add trash for Replicated database --- src/Interpreters/InterpreterCreateQuery.cpp | 30 ++++++++++++++++--- .../configs/config.xml | 1 + .../test_replicated_database/test.py | 2 +- 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f4aee9ad4db..f1c01d1aadc 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -71,7 +72,6 @@ #include #include -#include #include #include @@ -1329,10 +1329,32 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, } data_path = database->getTableDataPath(create); + auto full_data_path = fs::path{getContext()->getPath()} / data_path; - if (!create.attach && !data_path.empty() && fs::exists(fs::path{getContext()->getPath()} / data_path)) - throw Exception(storage_already_exists_error_code, - "Directory for {} data {} already exists", Poco::toLower(storage_name), String(data_path)); + if (!create.attach && !data_path.empty() && fs::exists(full_data_path)) + { + if (getContext()->getZooKeeperMetadataTransaction() && + !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery() && + !DatabaseCatalog::instance().hasUUIDMapping(create.uuid) && + Context::getGlobalContextInstance()->isServerCompletelyStarted() && + Context::getGlobalContextInstance()->getConfigRef().getBool("allow_moving_table_dir_to_trash", false)) + { + /// This is a secondary query from a Replicated database. It cannot be retried with another UUID, we must execute it as is. + /// We don't have a table with this UUID (and all metadata is loaded), + /// so the existing directory probably contains some leftovers from previous unsuccessful attempts to create the table + + fs::path trash_path = fs::path{getContext()->getPath()} / "trash" / data_path / getHexUIntLowercase(thread_local_rng()); + LOG_WARNING(&Poco::Logger::get("InterpreterCreateQuery"), "Directory for {} data {} already exists. Will move it to {}", + Poco::toLower(storage_name), String(data_path), trash_path); + fs::create_directories(trash_path.parent_path()); + renameNoReplace(full_data_path, trash_path); + } + else + { + throw Exception(storage_already_exists_error_code, + "Directory for {} data {} already exists", Poco::toLower(storage_name), String(data_path)); + } + } bool from_path = create.attach_from_path.has_value(); String actual_data_path = data_path; diff --git a/tests/integration/test_replicated_database/configs/config.xml b/tests/integration/test_replicated_database/configs/config.xml index 16cd942e975..7a67d69c031 100644 --- a/tests/integration/test_replicated_database/configs/config.xml +++ b/tests/integration/test_replicated_database/configs/config.xml @@ -1,5 +1,6 @@ 10 + 1 10 diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index d0a04f40b69..f45841124d9 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1262,7 +1262,7 @@ def test_recover_digest_mismatch(started_cluster): "mv /var/lib/clickhouse/metadata/recover_digest_mismatch/t1.sql /var/lib/clickhouse/metadata/recover_digest_mismatch/m1.sql", "sed --follow-symlinks -i 's/Int32/String/' /var/lib/clickhouse/metadata/recover_digest_mismatch/mv1.sql", "rm -f /var/lib/clickhouse/metadata/recover_digest_mismatch/d1.sql", - # f"rm -rf /var/lib/clickhouse/metadata/recover_digest_mismatch/", # Directory already exists + "rm -rf /var/lib/clickhouse/metadata/recover_digest_mismatch/", # Will trigger "Directory already exists" "rm -rf /var/lib/clickhouse/store", ] From 3d8f2c335feb46c7dcd16ef9ea38d1df0a81c2c2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 18:54:15 +0000 Subject: [PATCH 1990/2047] Protect against invalid asynchronous_metrics_update_period_s settings --- src/Interpreters/ServerAsynchronousMetrics.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 68411e80755..1b78ff4d2b6 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -24,6 +24,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INVALID_SETTING_VALUE; +} + namespace { @@ -52,7 +57,11 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) , WithContext(global_context_) , heavy_metric_update_period(heavy_metrics_update_period_seconds) -{} +{ + /// sanity check + if (update_period_seconds == 0 || heavy_metrics_update_period_seconds == 0) + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Setting asynchronous_metrics_update_period_s and asynchronous_heavy_metrics_update_period_s must not be zero"); +} void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) { From 894ec8e0d084848a6bd9b47f191a7a879fdb56ae Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Aug 2023 18:57:40 +0200 Subject: [PATCH 1991/2047] less exceptions with runtime format string --- programs/keeper/Keeper.cpp | 7 ++- programs/server/Server.cpp | 9 ++- src/Backups/BackupCoordinationRemote.cpp | 6 +- src/Backups/BackupCoordinationStageSync.cpp | 6 +- src/Backups/RestoreCoordinationRemote.cpp | 12 ++-- src/Common/DNSResolver.cpp | 8 +-- src/Common/DNSResolver.h | 6 +- src/Common/DateLUTImpl.cpp | 12 +++- src/Common/NamePrompter.h | 5 ++ src/Common/ZooKeeper/IKeeper.cpp | 34 +++++----- src/Common/ZooKeeper/IKeeper.h | 44 +++++++++++-- src/Common/ZooKeeper/TestKeeper.cpp | 8 +-- src/Common/ZooKeeper/ZooKeeper.cpp | 62 +++++++++---------- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 6 +- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 19 +++--- src/Common/ZooKeeper/ZooKeeperCommon.h | 4 +- src/Common/ZooKeeper/ZooKeeperConstants.cpp | 2 +- src/Common/ZooKeeper/ZooKeeperIO.cpp | 4 +- src/Common/ZooKeeper/ZooKeeperIO.h | 6 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 28 ++++----- .../ZooKeeper/ZooKeeperWithFaultInjection.h | 12 ++-- src/Coordination/KeeperDispatcher.cpp | 2 +- src/Core/SettingsFields.h | 10 ++- src/Databases/DatabaseReplicatedWorker.cpp | 2 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 4 +- ...erDefinedSQLObjectsLoaderFromZooKeeper.cpp | 8 +-- src/Interpreters/DDLWorker.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 4 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 2 +- src/Storages/AlterCommands.cpp | 5 +- src/Storages/ColumnsDescription.cpp | 9 +-- src/Storages/ColumnsDescription.h | 5 +- src/Storages/MergeTree/LeaderElection.h | 2 +- .../PartMovesBetweenShardsOrchestrator.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- src/Storages/MergeTree/ZooKeeperRetries.h | 2 +- src/Storages/ProjectionsDescription.cpp | 10 ++- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 2 +- src/Storages/StorageKeeperMap.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 22 +++---- .../System/StorageSystemDDLWorkerQueue.cpp | 4 +- ...nd_exception_messages_formatting.reference | 1 + ..._log_and_exception_messages_formatting.sql | 9 ++- .../0_stateless/00921_datetime64_basic.sql | 6 +- .../01281_parseDateTime64BestEffort.sql | 2 +- .../0_stateless/02244_make_datetime.sql | 2 +- .../0_stateless/02245_make_datetime64.sql | 2 +- ...2505_forbid_paths_in_datetime_timezone.sql | 10 +-- 49 files changed, 243 insertions(+), 194 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 1723c274fdb..77fbc9430ef 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -114,15 +114,16 @@ void Keeper::createServer(const std::string & listen_host, const char * port_nam if (listen_try) { - LOG_WARNING(&logger(), "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " + LOG_WARNING(&logger(), "Listen [{}]:{} failed: {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, " + "then consider to " "specify not disabled IPv4 or IPv6 address to listen in element of configuration " "file. Example for disabled IPv6: 0.0.0.0 ." " Example for disabled IPv4: ::", - message); + listen_host, port, getCurrentExceptionMessage(false)); } else { - throw Exception::createDeprecated(message, ErrorCodes::NETWORK_ERROR); + throw Exception(ErrorCodes::NETWORK_ERROR, "Listen [{}]:{} failed: {}", listen_host, port, getCurrentExceptionMessage(false)); } } } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6237186e3a6..b9ecbe7852f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -325,19 +325,18 @@ void Server::createServer( } catch (const Poco::Exception &) { - std::string message = "Listen [" + listen_host + "]:" + std::to_string(port) + " failed: " + getCurrentExceptionMessage(false); - if (listen_try) { - LOG_WARNING(&logger(), "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " + LOG_WARNING(&logger(), "Listen [{}]:{} failed: {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, " + "then consider to " "specify not disabled IPv4 or IPv6 address to listen in element of configuration " "file. Example for disabled IPv6: 0.0.0.0 ." " Example for disabled IPv4: ::", - message); + listen_host, port, getCurrentExceptionMessage(false)); } else { - throw Exception::createDeprecated(message, ErrorCodes::NETWORK_ERROR); + throw Exception(ErrorCodes::NETWORK_ERROR, "Listen [{}]:{} failed: {}", listen_host, port, getCurrentExceptionMessage(false)); } } } diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 214873fb765..e5fcbf26781 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -187,7 +187,7 @@ BackupCoordinationRemote::BackupCoordinationRemote( if (code == Coordination::Error::ZNODEEXISTS) zk->handleEphemeralNodeExistenceNoFailureInjection(alive_node_path, ""); else if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException(code, alive_node_path); + throw zkutil::KeeperException::fromPath(code, alive_node_path); } }) { @@ -745,7 +745,7 @@ bool BackupCoordinationRemote::startWritingFile(size_t data_file_index) else if (code == Coordination::Error::ZNODEEXISTS) host_is_assigned = (zk->get(full_path) == host_index_str); /// The previous retry could write this ZooKeeper's node and then fail. else - throw zkutil::KeeperException(code, full_path); + throw zkutil::KeeperException::fromPath(code, full_path); }); if (!host_is_assigned) @@ -815,7 +815,7 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic &) break; bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) - throw zkutil::KeeperException(code, backup_stage_path); + throw zkutil::KeeperException::fromPath(code, backup_stage_path); } }); diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index ebddbb8b82f..9b9ddc8515c 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -56,7 +56,7 @@ void BackupCoordinationStageSync::set(const String & current_host, const String { auto code = zookeeper->trySet(zookeeper_path, new_stage); if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException(code, zookeeper_path); + throw zkutil::KeeperException::fromPath(code, zookeeper_path); } else { @@ -64,7 +64,7 @@ void BackupCoordinationStageSync::set(const String & current_host, const String String alive_node_path = zookeeper_path + "/alive|" + current_host; auto code = zookeeper->tryCreate(alive_node_path, "", zkutil::CreateMode::Ephemeral); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) - throw zkutil::KeeperException(code, alive_node_path); + throw zkutil::KeeperException::fromPath(code, alive_node_path); zookeeper->createIfNotExists(zookeeper_path + "/started|" + current_host, ""); zookeeper->createIfNotExists(zookeeper_path + "/current|" + current_host + "|" + new_stage, message); @@ -90,7 +90,7 @@ void BackupCoordinationStageSync::setError(const String & current_host, const Ex /// so the following line tries to preserve the error status. auto code = zookeeper->trySet(zookeeper_path, Stage::ERROR); if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException(code, zookeeper_path); + throw zkutil::KeeperException::fromPath(code, zookeeper_path); }); } diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index f95969b52a1..37abebb26b7 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -46,7 +46,7 @@ RestoreCoordinationRemote::RestoreCoordinationRemote( if (code == Coordination::Error::ZNODEEXISTS) zk->handleEphemeralNodeExistenceNoFailureInjection(alive_node_path, ""); else if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException(code, alive_node_path); + throw zkutil::KeeperException::fromPath(code, alive_node_path); } }) { @@ -129,7 +129,7 @@ bool RestoreCoordinationRemote::acquireCreatingTableInReplicatedDatabase(const S path += "/" + escapeForFileName(table_name); auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZOK) { @@ -155,7 +155,7 @@ bool RestoreCoordinationRemote::acquireInsertingDataIntoReplicatedTable(const St String path = zookeeper_path + "/repl_tables_data_acquired/" + escapeForFileName(table_zk_path); auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZOK) { @@ -181,7 +181,7 @@ bool RestoreCoordinationRemote::acquireReplicatedAccessStorage(const String & ac String path = zookeeper_path + "/repl_access_storages_acquired/" + escapeForFileName(access_storage_zk_path); auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZOK) { @@ -217,7 +217,7 @@ bool RestoreCoordinationRemote::acquireReplicatedSQLObjects(const String & loade auto code = zk->tryCreate(path, "", zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZOK) { @@ -302,7 +302,7 @@ bool RestoreCoordinationRemote::hasConcurrentRestores(const std::atomic break; bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); } }); diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index a8ff347f399..285362e32f1 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -313,8 +313,8 @@ bool DNSResolver::updateCacheImpl( UpdateF && update_func, ElemsT && elems, UInt32 max_consecutive_failures, - const String & notfound_log_msg, - const String & dropped_log_msg) + FormatStringHelper notfound_log_msg, + FormatStringHelper dropped_log_msg) { bool updated = false; String lost_elems; @@ -351,7 +351,7 @@ bool DNSResolver::updateCacheImpl( } if (!lost_elems.empty()) - LOG_INFO(log, fmt::runtime(notfound_log_msg), lost_elems); + LOG_INFO(log, notfound_log_msg.format(std::move(lost_elems))); if (elements_to_drop.size()) { updated = true; @@ -363,7 +363,7 @@ bool DNSResolver::updateCacheImpl( deleted_elements += cacheElemToString(it->first); elems.erase(it); } - LOG_INFO(log, fmt::runtime(dropped_log_msg), deleted_elements); + LOG_INFO(log, dropped_log_msg.format(std::move(deleted_elements))); } return updated; diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index 84715b392a8..1017607a5bd 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -61,13 +62,12 @@ public: private: template - bool updateCacheImpl( UpdateF && update_func, ElemsT && elems, UInt32 max_consecutive_failures, - const String & notfound_log_msg, - const String & dropped_log_msg); + FormatStringHelper notfound_log_msg, + FormatStringHelper dropped_log_msg); DNSResolver(); diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index 4c21d9c9783..bb677b3a62d 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include @@ -12,6 +12,14 @@ #include +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} +} + /// Embedded timezones. std::string_view getTimeZone(const char * name); @@ -66,7 +74,7 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) cctz::time_zone cctz_time_zone; if (!cctz::load_time_zone(time_zone, &cctz_time_zone)) - throw Poco::Exception("Cannot load time zone " + time_zone_); + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot load time zone {}", time_zone_); constexpr cctz::civil_day epoch{1970, 1, 1}; constexpr cctz::civil_day lut_start{DATE_LUT_MIN_YEAR, 1, 1}; diff --git a/src/Common/NamePrompter.h b/src/Common/NamePrompter.h index 79e78529d95..65a2c76a102 100644 --- a/src/Common/NamePrompter.h +++ b/src/Common/NamePrompter.h @@ -117,6 +117,11 @@ public: DB::appendHintsMessage(error_message, hints); } + String getHintsMessage(const String & name) const + { + return getHintsErrorMessageSuffix(getHints(name)); + } + IHints() = default; IHints(const IHints &) = default; diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index f0a07241735..5897d04b8a2 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -21,29 +21,33 @@ namespace ProfileEvents namespace Coordination { -Exception::Exception(const std::string & msg, const Error code_, int) - : DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code_) +void Exception::incrementErrorMetrics(const Error code_) { - if (Coordination::isUserError(code)) + if (Coordination::isUserError(code_)) ProfileEvents::increment(ProfileEvents::ZooKeeperUserExceptions); - else if (Coordination::isHardwareError(code)) + else if (Coordination::isHardwareError(code_)) ProfileEvents::increment(ProfileEvents::ZooKeeperHardwareExceptions); else ProfileEvents::increment(ProfileEvents::ZooKeeperOtherExceptions); } -Exception::Exception(const std::string & msg, const Error code_) - : Exception(msg + " (" + errorMessage(code_) + ")", code_, 0) +Exception::Exception(const std::string & msg, const Error code_, int) + : DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION) + , code(code_) { + incrementErrorMetrics(code); +} + +Exception::Exception(PreformattedMessage && msg, const Error code_) + : DB::Exception(std::move(msg), DB::ErrorCodes::KEEPER_EXCEPTION) + , code(code_) +{ + extendedMessage(errorMessage(code)); + incrementErrorMetrics(code); } Exception::Exception(const Error code_) - : Exception(errorMessage(code_), code_, 0) -{ -} - -Exception::Exception(const Error code_, const std::string & path) - : Exception(std::string{errorMessage(code_)} + ", path: " + path, code_, 0) + : Exception(code_, "Coordination error: {}", errorMessage(code_)) { } @@ -56,10 +60,10 @@ using namespace DB; static void addRootPath(String & path, const String & root_path) { if (path.empty()) - throw Exception("Path cannot be empty", Error::ZBADARGUMENTS); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "Path cannot be empty"); if (path[0] != '/') - throw Exception("Path must begin with /, got path '" + path + "'", Error::ZBADARGUMENTS); + throw Exception(Error::ZBADARGUMENTS, "Path must begin with /, got path '{}'", path); if (root_path.empty()) return; @@ -76,7 +80,7 @@ static void removeRootPath(String & path, const String & root_path) return; if (path.size() <= root_path.size()) - throw Exception("Received path is not longer than root_path", Error::ZDATAINCONSISTENCY); + throw Exception::fromMessage(Error::ZDATAINCONSISTENCY, "Received path is not longer than root_path"); path = path.substr(root_path.size()); } diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 43f0f9c5f6f..e2c9afdd5da 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -17,6 +17,13 @@ * - ZooKeeper emulation layer on top of Etcd, FoundationDB, whatever. */ +namespace DB +{ +namespace ErrorCodes +{ + extern const int KEEPER_EXCEPTION; +} +} namespace Coordination { @@ -450,17 +457,46 @@ class Exception : public DB::Exception private: /// Delegate constructor, used to minimize repetition; last parameter used for overload resolution. Exception(const std::string & msg, const Error code_, int); /// NOLINT + Exception(PreformattedMessage && msg, const Error code_); + + /// Message must be a compile-time constant + template + requires std::is_convertible_v + Exception(T && message, const Error code_) : DB::Exception(DB::ErrorCodes::KEEPER_EXCEPTION, std::forward(message)), code(code_) + { + incrementErrorMetrics(code); + } + + static void incrementErrorMetrics(const Error code_); public: explicit Exception(const Error code_); /// NOLINT - Exception(const std::string & msg, const Error code_); /// NOLINT - Exception(const Error code_, const std::string & path); /// NOLINT Exception(const Exception & exc); template - Exception(const Error code_, fmt::format_string fmt, Args &&... args) - : Exception(fmt::format(fmt, std::forward(args)...), code_) + Exception(const Error code_, FormatStringHelper fmt, Args &&... args) + : DB::Exception(DB::ErrorCodes::KEEPER_EXCEPTION, std::move(fmt), std::forward(args)...) + , code(code_) { + incrementErrorMetrics(code); + } + + inline static Exception createDeprecated(const std::string & msg, const Error code_) + { + return Exception(msg, code_, 0); + } + + inline static Exception fromPath(const Error code_, const std::string & path) + { + return Exception(code_, "Coordination error: {}, path {}", errorMessage(code_), path); + } + + /// Message must be a compile-time constant + template + requires std::is_convertible_v + inline static Exception fromMessage(const Error code_, T && message) + { + return Exception(std::forward(message), code_); } const char * name() const noexcept override { return "Coordination::Exception"; } diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index dabc0ae4eef..9e355093c9d 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -177,7 +177,7 @@ struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest requests.push_back(std::make_shared(*concrete_request_check)); } else - throw Exception("Illegal command as part of multi ZooKeeper request", Error::ZBADARGUMENTS); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "Illegal command as part of multi ZooKeeper request"); } } @@ -389,7 +389,7 @@ std::pair TestKeeperListRequest::process(TestKeeper::Containe { auto path_prefix = path; if (path_prefix.empty()) - throw Exception("Logical error: path cannot be empty", Error::ZSESSIONEXPIRED); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Logical error: path cannot be empty"); if (path_prefix.back() != '/') path_prefix += '/'; @@ -705,10 +705,10 @@ void TestKeeper::pushRequest(RequestInfo && request) std::lock_guard lock(push_request_mutex); if (expired) - throw Exception("Session expired", Error::ZSESSIONEXPIRED); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Session expired"); if (!requests_queue.tryPush(std::move(request), args.operation_timeout_ms)) - throw Exception("Cannot push request to queue within operation timeout", Error::ZOPERATIONTIMEOUT); + throw Exception::fromMessage(Error::ZOPERATIONTIMEOUT, "Cannot push request to queue within operation timeout"); } catch (...) { diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 10331a4e410..b7f85293e94 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -51,7 +51,7 @@ const int CreateMode::EphemeralSequential = 3; static void check(Coordination::Error code, const std::string & path) { if (code != Coordination::Error::ZOK) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); } @@ -64,7 +64,7 @@ void ZooKeeper::init(ZooKeeperArgs args_) if (args.implementation == "zookeeper") { if (args.hosts.empty()) - throw KeeperException("No hosts passed to ZooKeeper constructor.", Coordination::Error::ZBADARGUMENTS); + throw KeeperException::fromMessage(Coordination::Error::ZBADARGUMENTS, "No hosts passed to ZooKeeper constructor."); Coordination::ZooKeeper::Nodes nodes; nodes.reserve(args.hosts.size()); @@ -107,9 +107,9 @@ void ZooKeeper::init(ZooKeeperArgs args_) { /// For DNS errors we throw exception with ZCONNECTIONLOSS code, so it will be considered as hardware error, not user error if (dns_error) - throw KeeperException("Cannot resolve any of provided ZooKeeper hosts due to DNS error", Coordination::Error::ZCONNECTIONLOSS); + throw KeeperException::fromMessage(Coordination::Error::ZCONNECTIONLOSS, "Cannot resolve any of provided ZooKeeper hosts due to DNS error"); else - throw KeeperException("Cannot use any of provided ZooKeeper nodes", Coordination::Error::ZCONNECTIONLOSS); + throw KeeperException::fromMessage(Coordination::Error::ZCONNECTIONLOSS, "Cannot use any of provided ZooKeeper nodes"); } impl = std::make_unique(nodes, args, zk_log, [this](size_t node_idx, const Coordination::ZooKeeper::Node & node) @@ -145,11 +145,11 @@ void ZooKeeper::init(ZooKeeperArgs args_) auto future = asyncExists("/"); auto res = future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)); if (res != std::future_status::ready) - throw KeeperException("Cannot check if zookeeper root exists.", Coordination::Error::ZOPERATIONTIMEOUT); + throw KeeperException::fromMessage(Coordination::Error::ZOPERATIONTIMEOUT, "Cannot check if zookeeper root exists."); auto code = future.get().error; if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException(code, "/"); + throw KeeperException::fromPath(code, "/"); if (code == Coordination::Error::ZNONODE) throw KeeperException(Coordination::Error::ZNONODE, "ZooKeeper root doesn't exist. You should create root node {} before start.", args.chroot); @@ -260,7 +260,7 @@ Coordination::Error ZooKeeper::tryGetChildren( Coordination::Error code = getChildrenImpl(path, res, stat, callbackForEvent(watch), list_request_type); if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code; } @@ -275,7 +275,7 @@ Coordination::Error ZooKeeper::tryGetChildrenWatch( Coordination::Error code = getChildrenImpl(path, res, stat, watch_callback, list_request_type); if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code; } @@ -314,7 +314,7 @@ Coordination::Error ZooKeeper::tryCreate(const std::string & path, const std::st code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS || code == Coordination::Error::ZNOCHILDRENFOREPHEMERALS)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code; } @@ -333,7 +333,7 @@ void ZooKeeper::createIfNotExists(const std::string & path, const std::string & if (code == Coordination::Error::ZOK || code == Coordination::Error::ZNODEEXISTS) return; else - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); } void ZooKeeper::createAncestors(const std::string & path) @@ -355,14 +355,14 @@ void ZooKeeper::createAncestors(const std::string & path) /// The parent node doesn't exist. Save the current node and try with the parent last_pos = current_node.rfind('/'); if (last_pos == std::string::npos || last_pos == 0) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); pending_nodes.emplace_back(std::move(current_node)); current_node = path.substr(0, last_pos); } else if (code == Coordination::Error::ZOK || code == Coordination::Error::ZNODEEXISTS) break; else - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); } for (const std::string & pending : pending_nodes | std::views::reverse) @@ -423,7 +423,7 @@ Coordination::Error ZooKeeper::tryRemove(const std::string & path, int32_t versi code == Coordination::Error::ZNONODE || code == Coordination::Error::ZBADVERSION || code == Coordination::Error::ZNOTEMPTY)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code; } @@ -457,7 +457,7 @@ bool ZooKeeper::existsWatch(const std::string & path, Coordination::Stat * stat, Coordination::Error code = existsImpl(path, stat, watch_callback); if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code != Coordination::Error::ZNONODE; } @@ -524,7 +524,7 @@ bool ZooKeeper::tryGetWatch( Coordination::Error code = getImpl(path, res, stat, watch_callback); if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); if (return_code) *return_code = code; @@ -566,7 +566,7 @@ void ZooKeeper::createOrUpdate(const std::string & path, const std::string & dat create(path, data, mode); } else if (code != Coordination::Error::ZOK) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); } Coordination::Error ZooKeeper::trySet(const std::string & path, const std::string & data, @@ -577,7 +577,7 @@ Coordination::Error ZooKeeper::trySet(const std::string & path, const std::strin if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE || code == Coordination::Error::ZBADVERSION)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code; } @@ -756,7 +756,7 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab continue; } - throw KeeperException(res.error, batch[i]); + throw KeeperException::fromPath(res.error, batch[i]); } } return removed_as_expected; @@ -823,7 +823,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & return true; if (state->code) - throw KeeperException(static_cast(state->code.load(std::memory_order_seq_cst)), path); + throw KeeperException::fromPath(static_cast(state->code.load(std::memory_order_seq_cst)), path); if (state->event_type == Coordination::DELETED) return true; @@ -844,7 +844,7 @@ void ZooKeeper::handleEphemeralNodeExistence(const std::string & path, const std { auto code = tryRemove(path, stat.version); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw Coordination::Exception(code, path); + throw Coordination::Exception::fromPath(code, path); } else { @@ -893,7 +893,7 @@ std::future ZooKeeper::asyncCreate(const std::stri auto callback = [promise, path](const Coordination::CreateResponse & response) mutable { if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -924,7 +924,7 @@ std::future ZooKeeper::asyncGet(const std::string & p auto callback = [promise, path](const Coordination::GetResponse & response) mutable { if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -956,7 +956,7 @@ std::future ZooKeeper::asyncTryGet(const std::string auto callback = [promise, path](const Coordination::GetResponse & response) mutable { if (response.error != Coordination::Error::ZOK && response.error != Coordination::Error::ZNONODE) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -973,7 +973,7 @@ std::future ZooKeeper::asyncExists(const std::stri auto callback = [promise, path](const Coordination::ExistsResponse & response) mutable { if (response.error != Coordination::Error::ZOK && response.error != Coordination::Error::ZNONODE) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -1004,7 +1004,7 @@ std::future ZooKeeper::asyncSet(const std::string & p auto callback = [promise, path](const Coordination::SetResponse & response) mutable { if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -1037,7 +1037,7 @@ std::future ZooKeeper::asyncGetChildren( auto callback = [promise, path](const Coordination::ListResponse & response) mutable { if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -1070,7 +1070,7 @@ ZooKeeper::asyncTryGetChildren(const std::string & path, Coordination::ListReque auto callback = [promise, path](const Coordination::ListResponse & response) mutable { if (response.error != Coordination::Error::ZOK && response.error != Coordination::Error::ZNONODE) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -1087,7 +1087,7 @@ std::future ZooKeeper::asyncRemove(const std::stri auto callback = [promise, path](const Coordination::RemoveResponse & response) mutable { if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -1108,7 +1108,7 @@ std::future ZooKeeper::asyncTryRemove(const std::s && response.error != Coordination::Error::ZBADVERSION && response.error != Coordination::Error::ZNOTEMPTY) { - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); } else promise->set_value(response); @@ -1244,10 +1244,10 @@ size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination:: KeeperMultiException::KeeperMultiException(Coordination::Error exception_code, const Coordination::Requests & requests_, const Coordination::Responses & responses_) - : KeeperException("Transaction failed", exception_code), + : KeeperException(exception_code, "Transaction failed: Op #{}, path", failed_op_index), requests(requests_), responses(responses_), failed_op_index(getFailedOpIndex(exception_code, responses)) { - addMessage("Op #" + std::to_string(failed_op_index) + ", path: " + getPathForFirstFailedOp()); + addMessage(getPathForFirstFailedOp()); } diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 4c73b9ffc6d..5d01294e9b0 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -36,7 +36,7 @@ ZooKeeperArgs::ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, c } if (session_timeout_ms < 0 || operation_timeout_ms < 0 || connection_timeout_ms < 0) - throw KeeperException("Timeout cannot be negative", Coordination::Error::ZBADARGUMENTS); + throw KeeperException::fromMessage(Coordination::Error::ZBADARGUMENTS, "Timeout cannot be negative"); /// init get_priority_load_balancing get_priority_load_balancing.hostname_differences.resize(hosts.size()); @@ -63,7 +63,7 @@ void ZooKeeperArgs::initFromKeeperServerSection(const Poco::Util::AbstractConfig auto tcp_port_secure = config.getString(key); if (tcp_port_secure.empty()) - throw KeeperException("Empty tcp_port_secure in config file", Coordination::Error::ZBADARGUMENTS); + throw KeeperException::fromMessage(Coordination::Error::ZBADARGUMENTS, "Empty tcp_port_secure in config file"); } bool secure{false}; @@ -81,7 +81,7 @@ void ZooKeeperArgs::initFromKeeperServerSection(const Poco::Util::AbstractConfig } if (tcp_port.empty()) - throw KeeperException("No tcp_port or tcp_port_secure in config file", Coordination::Error::ZBADARGUMENTS); + throw KeeperException::fromMessage(Coordination::Error::ZBADARGUMENTS, "No tcp_port or tcp_port_secure in config file"); if (auto coordination_key = std::string{config_name} + ".coordination_settings"; config.has(coordination_key)) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index e88d66e5444..8341199cd1e 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -461,8 +461,7 @@ void ZooKeeperErrorResponse::readImpl(ReadBuffer & in) Coordination::read(read_error, in); if (read_error != error) - throw Exception(fmt::format("Error code in ErrorResponse ({}) doesn't match error code in header ({})", read_error, error), - Error::ZMARSHALLINGERROR); + throw Exception(Error::ZMARSHALLINGERROR, "Error code in ErrorResponse ({}) doesn't match error code in header ({})", read_error, error); } void ZooKeeperErrorResponse::writeImpl(WriteBuffer & out) const @@ -534,7 +533,7 @@ ZooKeeperMultiRequest::ZooKeeperMultiRequest(const Requests & generic_requests, requests.push_back(std::make_shared(*concrete_request_list)); } else - throw Exception("Illegal command as part of multi ZooKeeper request", Error::ZBADARGUMENTS); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "Illegal command as part of multi ZooKeeper request"); } } @@ -577,9 +576,9 @@ void ZooKeeperMultiRequest::readImpl(ReadBuffer & in) if (done) { if (op_num != OpNum::Error) - throw Exception("Unexpected op_num received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Unexpected op_num received at the end of results for multi transaction"); if (error != -1) - throw Exception("Unexpected error value received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Unexpected error value received at the end of results for multi transaction"); break; } @@ -588,7 +587,7 @@ void ZooKeeperMultiRequest::readImpl(ReadBuffer & in) requests.push_back(request); if (in.eof()) - throw Exception("Not enough results received for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Not enough results received for multi transaction"); } } @@ -621,7 +620,7 @@ void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) Coordination::read(op_error, in); if (done) - throw Exception("Not enough results received for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Not enough results received for multi transaction"); /// op_num == -1 is special for multi transaction. /// For unknown reason, error code is duplicated in header and in response body. @@ -657,11 +656,11 @@ void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) Coordination::read(error_read, in); if (!done) - throw Exception("Too many results received for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Too many results received for multi transaction"); if (op_num != OpNum::Error) - throw Exception("Unexpected op_num received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Unexpected op_num received at the end of results for multi transaction"); if (error_read != -1) - throw Exception("Unexpected error value received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Unexpected error value received at the end of results for multi transaction"); } } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index e4b2cc97744..5b662c7f4c1 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -163,7 +163,7 @@ struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse OpNum getOpNum() const override { chassert(false); - throw Exception("OpNum for watch response doesn't exist", Error::ZRUNTIMEINCONSISTENCY); + throw Exception::fromMessage(Error::ZRUNTIMEINCONSISTENCY, "OpNum for watch response doesn't exist"); } void fillLogElements(LogElements & elems, size_t idx) const override; @@ -214,7 +214,7 @@ struct ZooKeeperCloseResponse final : ZooKeeperResponse { void readImpl(ReadBuffer &) override { - throw Exception("Received response for close request", Error::ZRUNTIMEINCONSISTENCY); + throw Exception::fromMessage(Error::ZRUNTIMEINCONSISTENCY, "Received response for close request"); } void writeImpl(WriteBuffer &) const override {} diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.cpp b/src/Common/ZooKeeper/ZooKeeperConstants.cpp index 9bb9c7b0488..7fffea1d08d 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.cpp +++ b/src/Common/ZooKeeper/ZooKeeperConstants.cpp @@ -33,7 +33,7 @@ static const std::unordered_set VALID_OPERATIONS = OpNum getOpNum(int32_t raw_op_num) { if (!VALID_OPERATIONS.contains(raw_op_num)) - throw Exception("Operation " + std::to_string(raw_op_num) + " is unknown", Error::ZUNIMPLEMENTED); + throw Exception(Error::ZUNIMPLEMENTED, "Operation {} is unknown", raw_op_num); return static_cast(raw_op_num); } diff --git a/src/Common/ZooKeeper/ZooKeeperIO.cpp b/src/Common/ZooKeeper/ZooKeeperIO.cpp index 2911d511254..6a51ffb36fa 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.cpp +++ b/src/Common/ZooKeeper/ZooKeeperIO.cpp @@ -62,10 +62,10 @@ void read(std::string & s, ReadBuffer & in) } if (size < 0) - throw Exception("Negative size while reading string from ZooKeeper", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Negative size while reading string from ZooKeeper"); if (size > MAX_STRING_OR_ARRAY_SIZE) - throw Exception("Too large string size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR,"Too large string size while reading from ZooKeeper"); s.resize(size); size_t read_bytes = in.read(s.data(), size); diff --git a/src/Common/ZooKeeper/ZooKeeperIO.h b/src/Common/ZooKeeper/ZooKeeperIO.h index 81b56a02e27..83973c1ae22 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.h +++ b/src/Common/ZooKeeper/ZooKeeperIO.h @@ -62,7 +62,7 @@ void read(std::array & s, ReadBuffer & in) int32_t size = 0; read(size, in); if (size != N) - throw Exception("Unexpected array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Unexpected array size while reading from ZooKeeper"); in.readStrict(s.data(), N); } @@ -72,9 +72,9 @@ void read(std::vector & arr, ReadBuffer & in) int32_t size = 0; read(size, in); if (size < 0) - throw Exception("Negative size while reading array from ZooKeeper", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Negative size while reading array from ZooKeeper"); if (size > MAX_STRING_OR_ARRAY_SIZE) - throw Exception("Too large array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Too large array size while reading from ZooKeeper"); arr.resize(size); for (auto & elem : arr) read(elem, in); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index d84d5fa3a69..4dbdcf51b24 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -289,7 +289,7 @@ static void removeRootPath(String & path, const String & chroot) return; if (path.size() <= chroot.size()) - throw Exception(Error::ZDATAINCONSISTENCY, "Received path is not longer than chroot"); + throw Exception::fromMessage(Error::ZDATAINCONSISTENCY, "Received path is not longer than chroot"); path = path.substr(chroot.size()); } @@ -387,7 +387,7 @@ void ZooKeeper::connect( Poco::Timespan connection_timeout) { if (nodes.empty()) - throw Exception(Error::ZBADARGUMENTS, "No nodes passed to ZooKeeper constructor"); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "No nodes passed to ZooKeeper constructor"); static constexpr size_t num_tries = 3; bool connected = false; @@ -479,8 +479,6 @@ void ZooKeeper::connect( if (!connected) { WriteBufferFromOwnString message; - - message << "All connection tries failed while connecting to ZooKeeper. nodes: "; bool first = true; for (const auto & node : nodes) { @@ -496,7 +494,7 @@ void ZooKeeper::connect( } message << fail_reasons.str() << "\n"; - throw Exception(Error::ZCONNECTIONLOSS, message.str()); + throw Exception(Error::ZCONNECTIONLOSS, "All connection tries failed while connecting to ZooKeeper. nodes: {}", message.str()); } else { @@ -543,7 +541,7 @@ void ZooKeeper::receiveHandshake() /// It's better for faster failover than just connection drop. /// Implemented in clickhouse-keeper. if (protocol_version_read == KEEPER_PROTOCOL_VERSION_CONNECTION_REJECT) - throw Exception(Error::ZCONNECTIONLOSS, + throw Exception::fromMessage(Error::ZCONNECTIONLOSS, "Keeper server rejected the connection during the handshake. " "Possibly it's overloaded, doesn't see leader or stale"); else @@ -800,7 +798,7 @@ void ZooKeeper::receiveEvent() auto it = operations.find(xid); if (it == operations.end()) - throw Exception("Received response for unknown xid " + DB::toString(xid), Error::ZRUNTIMEINCONSISTENCY); + throw Exception(Error::ZRUNTIMEINCONSISTENCY, "Received response for unknown xid {}", xid); /// After this point, we must invoke callback, that we've grabbed from 'operations'. /// Invariant: all callbacks are invoked either in case of success or in case of error. @@ -1088,9 +1086,9 @@ void ZooKeeper::pushRequest(RequestInfo && info) { info.request->xid = next_xid.fetch_add(1); if (info.request->xid == CLOSE_XID) - throw Exception(Error::ZSESSIONEXPIRED, "xid equal to close_xid"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "xid equal to close_xid"); if (info.request->xid < 0) - throw Exception(Error::ZSESSIONEXPIRED, "XID overflow"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "XID overflow"); if (auto * multi_request = dynamic_cast(info.request.get())) { @@ -1104,7 +1102,7 @@ void ZooKeeper::pushRequest(RequestInfo && info) if (!requests_queue.tryPush(std::move(info), args.operation_timeout_ms)) { if (requests_queue.isFinished()) - throw Exception(Error::ZSESSIONEXPIRED, "Session expired"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Session expired"); throw Exception(Error::ZOPERATIONTIMEOUT, "Cannot push request to queue within operation timeout of {} ms", args.operation_timeout_ms); } @@ -1297,7 +1295,7 @@ void ZooKeeper::list( if (!isFeatureEnabled(KeeperFeatureFlag::FILTERED_LIST)) { if (list_request_type != ListRequestType::ALL) - throw Exception(Error::ZBADARGUMENTS, "Filtered list request type cannot be used because it's not supported by the server"); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "Filtered list request type cannot be used because it's not supported by the server"); request = std::make_shared(); } @@ -1380,7 +1378,7 @@ void ZooKeeper::multi( ZooKeeperMultiRequest request(requests, default_acls); if (request.getOpNum() == OpNum::MultiRead && !isFeatureEnabled(KeeperFeatureFlag::MULTI_READ)) - throw Exception(Error::ZBADARGUMENTS, "MultiRead request type cannot be used because it's not supported by the server"); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "MultiRead request type cannot be used because it's not supported by the server"); RequestInfo request_info; request_info.request = std::make_shared(std::move(request)); @@ -1502,7 +1500,7 @@ void ZooKeeper::setupFaultDistributions() void ZooKeeper::checkSessionDeadline() const { if (unlikely(hasReachedDeadline())) - throw Exception(Error::ZSESSIONEXPIRED, "Session expired (force expiry client-side)"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Session expired (force expiry client-side)"); } bool ZooKeeper::hasReachedDeadline() const @@ -1513,13 +1511,13 @@ bool ZooKeeper::hasReachedDeadline() const void ZooKeeper::maybeInjectSendFault() { if (unlikely(inject_setup.test() && send_inject_fault && send_inject_fault.value()(thread_local_rng))) - throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); } void ZooKeeper::maybeInjectRecvFault() { if (unlikely(inject_setup.test() && recv_inject_fault && recv_inject_fault.value()(thread_local_rng))) - throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); } void ZooKeeper::maybeInjectSendSleep() diff --git a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h index 9d02d674010..4887e896e9b 100644 --- a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h +++ b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h @@ -29,7 +29,7 @@ public: if (distribution(rndgen) || must_fail_before_op) { must_fail_before_op = false; - throw zkutil::KeeperException("Fault injection before operation", Coordination::Error::ZSESSIONEXPIRED); + throw zkutil::KeeperException::fromMessage(Coordination::Error::ZSESSIONEXPIRED, "Fault injection before operation"); } } void afterOperation() @@ -37,7 +37,7 @@ public: if (distribution(rndgen) || must_fail_after_op) { must_fail_after_op = false; - throw zkutil::KeeperException("Fault injection after operation", Coordination::Error::ZOPERATIONTIMEOUT); + throw zkutil::KeeperException::fromMessage(Coordination::Error::ZOPERATIONTIMEOUT, "Fault injection after operation"); } } @@ -263,7 +263,7 @@ public: auto code = tryCreate(path, data, mode, path_created); if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); return path_created; } @@ -327,7 +327,7 @@ public: if (code == Coordination::Error::ZOK || code == Coordination::Error::ZNODEEXISTS) return; - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); } Coordination::Responses multi(const Coordination::Requests & requests) @@ -507,8 +507,8 @@ private: ++calls_total; if (!keeper) - throw zkutil::KeeperException( - "Session is considered to be expired due to fault injection", Coordination::Error::ZSESSIONEXPIRED); + throw zkutil::KeeperException::fromMessage(Coordination::Error::ZSESSIONEXPIRED, + "Session is considered to be expired due to fault injection"); if constexpr (inject_failure_before_op) { diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 99c28674273..1fd67aef72c 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -687,7 +687,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) } if (response->error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(zkutil::KeeperException("SessionID request failed with error", response->error))); + promise->set_exception(std::make_exception_ptr(zkutil::KeeperException::fromMessage(response->error, "SessionID request failed with error"))); promise->set_value(session_id_response.session_id); }; diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index db14f91512f..0e1f35d4352 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -428,9 +428,8 @@ constexpr auto getEnumValues() auto it = map.find(value); \ if (it != map.end()) \ return it->second; \ - throw Exception::createDeprecated( \ - "Unexpected value of " #NEW_NAME ":" + std::to_string(std::underlying_type::type(value)), \ - ERROR_CODE_FOR_UNEXPECTED_NAME); \ + throw Exception(ERROR_CODE_FOR_UNEXPECTED_NAME, \ + "Unexpected value of " #NEW_NAME ":{}", std::to_string(std::underlying_type::type(value))); \ } \ \ typename SettingField##NEW_NAME::EnumType SettingField##NEW_NAME##Traits::fromString(std::string_view str) \ @@ -444,7 +443,7 @@ constexpr auto getEnumValues() auto it = map.find(str); \ if (it != map.end()) \ return it->second; \ - String msg = "Unexpected value of " #NEW_NAME ": '" + String{str} + "'. Must be one of ["; \ + String msg; \ bool need_comma = false; \ for (auto & name : map | boost::adaptors::map_keys) \ { \ @@ -452,8 +451,7 @@ constexpr auto getEnumValues() msg += ", "; \ msg += "'" + String{name} + "'"; \ } \ - msg += "]"; \ - throw Exception::createDeprecated(msg, ERROR_CODE_FOR_UNEXPECTED_NAME); \ + throw Exception(ERROR_CODE_FOR_UNEXPECTED_NAME, "Unexpected value of " #NEW_NAME ": '{}'. Must be one of [{}]", String{str}, msg); \ } // Mostly like SettingFieldEnum, but can have multiple enum values (or none) set at once. diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 4976f54e417..367f772d402 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -356,7 +356,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na /// We use tryRemove(...) because multiple hosts (including initiator) may try to do it concurrently. auto code = zookeeper->tryRemove(try_node_path); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw Coordination::Exception(code, try_node_path); + throw Coordination::Exception::fromPath(code, try_node_path); if (!zookeeper->exists(fs::path(entry_path) / "committed")) { diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index c9e10b27caa..3578362b8dd 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -141,7 +141,6 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S { bool first = true; WriteBufferFromOwnString error_message; - error_message << "Illegal MySQL variables, the MaterializedMySQL engine requires "; for (const auto & [variable_name, variable_error_val] : variables_error_message) { error_message << (first ? "" : ", ") << variable_name << "='" << variable_error_val << "'"; @@ -150,7 +149,8 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S first = false; } - throw Exception::createDeprecated(error_message.str(), ErrorCodes::ILLEGAL_MYSQL_VARIABLE); + throw Exception(ErrorCodes::ILLEGAL_MYSQL_VARIABLE, "Illegal MySQL variables, the MaterializedMySQL engine requires {}", + error_message.str()); } } diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp index 76ebcf769d5..29aff666da5 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp @@ -223,7 +223,7 @@ bool UserDefinedSQLObjectsLoaderFromZooKeeper::storeObject( { auto code = zookeeper->tryCreate(path, create_statement, zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZNODEEXISTS) { @@ -234,14 +234,14 @@ bool UserDefinedSQLObjectsLoaderFromZooKeeper::storeObject( code = zookeeper->trySet(path, create_statement); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNONODE)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); } if (code == Coordination::Error::ZOK) break; if (!--num_attempts) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); } LOG_DEBUG(log, "Object {} stored", backQuote(object_name)); @@ -262,7 +262,7 @@ bool UserDefinedSQLObjectsLoaderFromZooKeeper::removeObject( auto code = zookeeper->tryRemove(path); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNONODE)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZNONODE) { diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 92e6bcb326c..8be334d6223 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -574,7 +574,7 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper) if (create_active_res != Coordination::Error::ZNONODE && create_active_res != Coordination::Error::ZNODEEXISTS) { chassert(Coordination::isHardwareError(create_active_res)); - throw Coordination::Exception(create_active_res, active_node_path); + throw Coordination::Exception::fromPath(create_active_res, active_node_path); } /// Status dirs were not created in enqueueQuery(...) or someone is removing entry diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 631e7f5c746..66b23f09ba0 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -409,7 +409,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn, bool { std::bernoulli_distribution fault(fault_probability_before_commit); if (fault(thread_local_rng)) - throw Coordination::Exception("Fault injected (before commit)", Coordination::Error::ZCONNECTIONLOSS); + throw Coordination::Exception::fromMessage(Coordination::Error::ZCONNECTIONLOSS, "Fault injected (before commit)"); } /// Commit point @@ -419,7 +419,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn, bool { std::bernoulli_distribution fault(fault_probability_after_commit); if (fault(thread_local_rng)) - throw Coordination::Exception("Fault injected (after commit)", Coordination::Error::ZCONNECTIONLOSS); + throw Coordination::Exception::fromMessage(Coordination::Error::ZCONNECTIONLOSS, "Fault injected (after commit)"); } } catch (const Coordination::Exception & e) diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index b34108644fb..750affdfe71 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -559,7 +559,7 @@ Strings DDLQueryStatusSource::getChildrenAllowNoNode(const std::shared_ptrtryGetChildren(node_path, res); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw Coordination::Exception(code, node_path); + throw Coordination::Exception::fromPath(code, node_path); return res; } diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index a9247f9b898..96dcd74f947 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1077,9 +1077,8 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const { if (!command.if_exists) { - String exception_message = fmt::format("Wrong column. Cannot find column {} to modify", backQuote(column_name)); - all_columns.appendHintsMessage(exception_message, column_name); - throw Exception::createDeprecated(exception_message, ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); + throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Wrong column. Cannot find column {} to modify{}", + backQuote(column_name), all_columns.getHintsMessage(column_name)); } else continue; diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 045afd7e6e6..0c918bda5fd 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -232,9 +232,7 @@ void ColumnsDescription::remove(const String & column_name) auto range = getNameRange(columns, column_name); if (range.first == range.second) { - String exception_message = fmt::format("There is no column {} in table", column_name); - appendHintsMessage(exception_message, column_name); - throw Exception::createDeprecated(exception_message, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table{}", column_name, getHintsMessage(column_name)); } for (auto list_it = range.first; list_it != range.second;) @@ -249,9 +247,8 @@ void ColumnsDescription::rename(const String & column_from, const String & colum auto it = columns.get<1>().find(column_from); if (it == columns.get<1>().end()) { - String exception_message = fmt::format("Cannot find column {} in ColumnsDescription", column_from); - appendHintsMessage(exception_message, column_from); - throw Exception::createDeprecated(exception_message, ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find column {} in ColumnsDescription{}", + column_from, getHintsMessage(column_from)); } columns.get<1>().modify_key(it, [&column_to] (String & old_name) diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 365a999673e..fb1eeed3127 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -160,9 +160,8 @@ public: auto it = columns.get<1>().find(column_name); if (it == columns.get<1>().end()) { - String exception_message = fmt::format("Cannot find column {} in ColumnsDescription", column_name); - appendHintsMessage(exception_message, column_name); - throw Exception::createDeprecated(exception_message, ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find column {} in ColumnsDescription{}", + column_name, getHintsMessage(column_name)); } removeSubcolumns(it->name); diff --git a/src/Storages/MergeTree/LeaderElection.h b/src/Storages/MergeTree/LeaderElection.h index f694ecab8e3..2e48892563b 100644 --- a/src/Storages/MergeTree/LeaderElection.h +++ b/src/Storages/MergeTree/LeaderElection.h @@ -36,7 +36,7 @@ void checkNoOldLeaders(Poco::Logger * log, ZooKeeper & zookeeper, const String p if (code == Coordination::Error::ZNONODE) return; else if (code != Coordination::Error::ZOK) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); Coordination::Requests ops; diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index 3b382b7b32d..76b8080f64c 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -684,7 +684,7 @@ CancellationCode PartMovesBetweenShardsOrchestrator::killPartMoveToShard(const U continue; } else - throw Coordination::Exception(code, entry.znode_path); + throw Coordination::Exception::fromPath(code, entry.znode_path); } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 21d5597e614..84307a3ca7a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -2169,7 +2169,7 @@ CommittingBlocks BaseMergePredicate::getCommitti { auto & response = locks_children[i]; if (response.error != Coordination::Error::ZOK && !partition_ids_hint) - throw Coordination::Exception(response.error, paths[i]); + throw Coordination::Exception::fromPath(response.error, paths[i]); if (response.error != Coordination::Error::ZOK) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 0db3464a637..2ad629c1493 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -351,7 +351,7 @@ size_t ReplicatedMergeTreeSinkImpl::checkQuorumPrecondition(const if (active_replicas < quorum_size) { if (Coordination::isHardwareError(keeper_error)) - throw Coordination::Exception("Failed to check number of alive replicas", keeper_error); + throw Coordination::Exception::fromMessage(keeper_error, "Failed to check number of alive replicas"); throw Exception(ErrorCodes::TOO_FEW_LIVE_REPLICAS, "Number of alive replicas ({}) is less than requested quorum ({}/{}).", active_replicas, quorum_size, replicas_number); diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h index 512c0800de7..e46c3f974c7 100644 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -159,7 +159,7 @@ public: void setKeeperError(Coordination::Error code, std::string message) { - setKeeperError(std::make_exception_ptr(zkutil::KeeperException(message, code)), code, message); + setKeeperError(std::make_exception_ptr(zkutil::KeeperException::createDeprecated(message, code)), code, message); } template diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 48825361a16..cddf252a7e1 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -356,9 +356,8 @@ const ProjectionDescription & ProjectionsDescription::get(const String & project auto it = map.find(projection_name); if (it == map.end()) { - String exception_message = fmt::format("There is no projection {} in table", projection_name); - appendHintsMessage(exception_message, projection_name); - throw Exception::createDeprecated(exception_message, ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE); + throw Exception(ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE, "There is no projection {} in table{}", + projection_name, getHintsMessage(projection_name)); } return *(it->second); @@ -401,9 +400,8 @@ void ProjectionsDescription::remove(const String & projection_name, bool if_exis if (if_exists) return; - String exception_message = fmt::format("There is no projection {} in table", projection_name); - appendHintsMessage(exception_message, projection_name); - throw Exception::createDeprecated(exception_message, ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE); + throw Exception(ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE, "There is no projection {} in table{}", + projection_name, getHintsMessage(projection_name)); } projections.erase(it->second); diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index de4e0a806e0..4624566a517 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -339,7 +339,7 @@ std::shared_ptr S3QueueFilesMetadata::acquireLock(z } else if (code != Coordination::Error::ZOK) { - throw Coordination::Exception(code, zookeeper_lock_path); + throw Coordination::Exception::fromPath(code, zookeeper_lock_path); } else { diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index f2b1b907832..f98728c012e 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -419,7 +419,7 @@ StorageKeeperMap::StorageKeeperMap( } else if (code != Coordination::Error::ZOK) { - throw Coordination::Exception(code, dropped_lock_path); + throw Coordination::Exception::fromPath(code, dropped_lock_path); } else { @@ -918,7 +918,7 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca { auto code = client->tryRemove(delete_request->getPath()); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw zkutil::KeeperException(code, delete_request->getPath()); + throw zkutil::KeeperException::fromPath(code, delete_request->getPath()); } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7fce373e26b..39c911d4f20 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -713,7 +713,7 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes() { auto res = future.get(); if (res.error != Coordination::Error::ZOK && res.error != Coordination::Error::ZNODEEXISTS) - throw Coordination::Exception(fmt::format("Failed to create new nodes at {}", zookeeper_path), res.error); + throw Coordination::Exception(res.error, "Failed to create new nodes at {}", zookeeper_path); } } @@ -749,7 +749,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr } else if (code != Coordination::Error::ZOK) { - throw Coordination::Exception(code, drop_lock_path); + throw Coordination::Exception::fromPath(code, drop_lock_path); } else { @@ -4307,7 +4307,7 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name, bool is_ continue; } else - throw Coordination::Exception(code, quorum_status_path); + throw Coordination::Exception::fromPath(code, quorum_status_path); } else { @@ -4331,7 +4331,7 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name, bool is_ continue; } else - throw Coordination::Exception(code, quorum_status_path); + throw Coordination::Exception::fromPath(code, quorum_status_path); } } } @@ -4389,7 +4389,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) continue; } else - throw Coordination::Exception(code, quorum_last_part_path); + throw Coordination::Exception::fromPath(code, quorum_last_part_path); } } @@ -5888,7 +5888,7 @@ void StorageReplicatedMergeTree::alter( } else { - throw Coordination::Exception("Alter cannot be assigned because of Zookeeper error", rc); + throw Coordination::Exception::fromMessage(rc, "Alter cannot be assigned because of Zookeeper error"); } } @@ -7049,7 +7049,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte continue; } else - throw Coordination::Exception("Unable to create a mutation znode", rc); + throw Coordination::Exception::fromMessage(rc, "Unable to create a mutation znode"); } merge_selecting_task->schedule(); @@ -8672,7 +8672,7 @@ void StorageReplicatedMergeTree::createTableSharedID() const } else if (code != Coordination::Error::ZOK) { - throw zkutil::KeeperException(code, zookeeper_table_id_path); + throw zkutil::KeeperException::fromPath(code, zookeeper_table_id_path); } } @@ -9115,7 +9115,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( } else { - throw zkutil::KeeperException(ec, zookeeper_part_replica_node); + throw zkutil::KeeperException::fromPath(ec, zookeeper_part_replica_node); } } @@ -9150,7 +9150,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( } else { - throw zkutil::KeeperException(error_code, zookeeper_part_uniq_node); + throw zkutil::KeeperException::fromPath(error_code, zookeeper_part_uniq_node); } @@ -9182,7 +9182,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( } else { - throw zkutil::KeeperException(error_code, zookeeper_part_uniq_node); + throw zkutil::KeeperException::fromPath(error_code, zookeeper_part_uniq_node); } } else diff --git a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp index 67867b6c577..bae7a266dcd 100644 --- a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp +++ b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp @@ -290,7 +290,7 @@ void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, Context } else { - throw Coordination::Exception(maybe_finished_hosts.error, fs::path(task.entry_path) / "finished"); + throw Coordination::Exception::fromPath(maybe_finished_hosts.error, fs::path(task.entry_path) / "finished"); } /// Process active nodes @@ -322,7 +322,7 @@ void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, Context } else { - throw Coordination::Exception(maybe_active_hosts.error, fs::path(task.entry_path) / "active"); + throw Coordination::Exception::fromPath(maybe_active_hosts.error, fs::path(task.entry_path) / "active"); } /// Process the rest hosts diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference index 11b660b54a3..00a2cd14700 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -1,5 +1,6 @@ runtime messages 0.001 runtime exceptions 0.05 +unknown runtime exceptions 0.01 messages shorter than 10 1 messages shorter than 16 3 exceptions shorter than 30 3 diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index eb8e9826eff..2ae95928b75 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -12,7 +12,14 @@ create view logs as select * from system.text_log where now() - toIntervalMinute select 'runtime messages', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.001) from logs; -- Check the same for exceptions. The value was 0.03 -select 'runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.05) from logs where message like '%DB::Exception%'; +select 'runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.05) from logs + where (message like '%DB::Exception%' or message like '%Coordination::Exception%') + and message not like '%Received from CLICKHOUSE_CI_LOGS_HOST:9440%'; + +select 'unknown runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.01) from logs where + (message like '%DB::Exception%' or message like '%Coordination::Exception%') + and message not like '%Received from CLICKHOUSE_CI_LOGS_HOST:9440%' + and message not like '% Received from %' and message not like '%(SYNTAX_ERROR)%'; -- FIXME some of the following messages are not informative and it has to be fixed create temporary table known_short_messages (s String) as select * from (select diff --git a/tests/queries/0_stateless/00921_datetime64_basic.sql b/tests/queries/0_stateless/00921_datetime64_basic.sql index 28205c72ef0..13abe3e64d0 100644 --- a/tests/queries/0_stateless/00921_datetime64_basic.sql +++ b/tests/queries/0_stateless/00921_datetime64_basic.sql @@ -3,17 +3,17 @@ DROP TABLE IF EXISTS A; SELECT CAST(1 as DateTime64('abc')); -- { serverError 43 } # Invalid scale parameter type SELECT CAST(1 as DateTime64(100)); -- { serverError 69 } # too big scale SELECT CAST(1 as DateTime64(-1)); -- { serverError 43 } # signed scale parameter type -SELECT CAST(1 as DateTime64(3, 'qqq')); -- { serverError 1000 } # invalid timezone +SELECT CAST(1 as DateTime64(3, 'qqq')); -- { serverError BAD_ARGUMENTS } # invalid timezone SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # invalid scale SELECT toDateTime64('2019-09-16 19:20:11.234', 100); -- { serverError 69 } # too big scale SELECT toDateTime64(CAST([['CLb5Ph ']], 'String'), uniqHLL12('2Gs1V', 752)); -- { serverError 44 } # non-const string and non-const scale -SELECT toDateTime64('2019-09-16 19:20:11.234', 3, 'qqq'); -- { serverError 1000 } # invalid timezone +SELECT toDateTime64('2019-09-16 19:20:11.234', 3, 'qqq'); -- { serverError BAD_ARGUMENTS } # invalid timezone SELECT ignore(now64(gccMurmurHash())); -- { serverError 43 } # Illegal argument type SELECT ignore(now64('abcd')); -- { serverError 43 } # Illegal argument type SELECT ignore(now64(number)) FROM system.numbers LIMIT 10; -- { serverError 43 } # Illegal argument type -SELECT ignore(now64(3, 'invalid timezone')); -- { serverError 1000 } +SELECT ignore(now64(3, 'invalid timezone')); -- { serverError BAD_ARGUMENTS } SELECT ignore(now64(3, 1111)); -- { serverError 44 } # invalid timezone parameter type WITH 'UTC' as timezone SELECT timezone, timeZoneOf(now64(3, timezone)) == timezone; diff --git a/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql b/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql index ac1186284be..c1cec6ea212 100644 --- a/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql +++ b/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql @@ -5,7 +5,7 @@ SELECT parseDateTime64BestEffort('foo'); -- {serverError 41} SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', 'bar'); -- {serverError 43} -- invalid scale parameter SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', 3, 4); -- {serverError 43} -- invalid timezone parameter -SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', 3, 'baz'); -- {serverError 1000} -- unknown timezone +SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', 3, 'baz'); -- {serverError BAD_ARGUMENTS} -- unknown timezone SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', materialize(3), 4); -- {serverError 44} -- non-const precision SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', 3, materialize('UTC')); -- {serverError 44} -- non-const timezone diff --git a/tests/queries/0_stateless/02244_make_datetime.sql b/tests/queries/0_stateless/02244_make_datetime.sql index 9b8f561994b..a3d88d89682 100644 --- a/tests/queries/0_stateless/02244_make_datetime.sql +++ b/tests/queries/0_stateless/02244_make_datetime.sql @@ -20,7 +20,7 @@ select makeDateTime(1984, 1, 41, 0, 0, 0, 'UTC'); select makeDateTime(1984, 1, 1, 25, 0, 0, 'UTC'); select makeDateTime(1984, 1, 1, 0, 70, 0, 'UTC'); select makeDateTime(1984, 1, 1, 0, 0, 70, 'UTC'); -select makeDateTime(1984, 1, 1, 0, 0, 0, 'not a timezone'); -- { serverError 1000 } +select makeDateTime(1984, 1, 1, 0, 0, 0, 'not a timezone'); -- { serverError BAD_ARGUMENTS } select makeDateTime(1984, 1, 1, 0, 0, 0, 'UTC'); select makeDateTime(1983, 2, 29, 0, 0, 0, 'UTC'); diff --git a/tests/queries/0_stateless/02245_make_datetime64.sql b/tests/queries/0_stateless/02245_make_datetime64.sql index 054cbd8956d..a7b3a3d23c5 100644 --- a/tests/queries/0_stateless/02245_make_datetime64.sql +++ b/tests/queries/0_stateless/02245_make_datetime64.sql @@ -38,7 +38,7 @@ select makeDateTime64(1984, 1, 41, 0, 0, 0, 0, 9, 'UTC'); select makeDateTime64(1984, 1, 1, 25, 0, 0, 0, 9, 'UTC'); select makeDateTime64(1984, 1, 1, 0, 70, 0, 0, 9, 'UTC'); select makeDateTime64(1984, 1, 1, 0, 0, 70, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'not a timezone'); -- { serverError 1000 } +select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'not a timezone'); -- { serverError BAD_ARGUMENTS } select makeDateTime64(1984, 1, 1, 2, 3, 4, 5, 9, 'UTC'); select makeDateTime64(1984, 2, 29, 2, 3, 4, 5, 9, 'UTC'); diff --git a/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql b/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql index b08abcb8a19..63edad6c9e3 100644 --- a/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql +++ b/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql @@ -1,6 +1,6 @@ -select toDateTime(0, '/abc'); -- { serverError POCO_EXCEPTION } -select toDateTime(0, './abc'); -- { serverError POCO_EXCEPTION } -select toDateTime(0, '../abc'); -- { serverError POCO_EXCEPTION } -select toDateTime(0, '~/abc'); -- { serverError POCO_EXCEPTION } -select toDateTime(0, 'abc/../../cba'); -- { serverError POCO_EXCEPTION } +select toDateTime(0, '/abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, './abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, '../abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, '~/abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, 'abc/../../cba'); -- { serverError BAD_ARGUMENTS } From 7312de59c508932934c3bc8aa03818d74215e343 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Aug 2023 23:33:30 +0200 Subject: [PATCH 1992/2047] empty commit From 1e3f9c8cfeb9a3e6e51069881155fbc9dad53143 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Aug 2023 00:41:24 +0300 Subject: [PATCH 1993/2047] Merging #53142 (#53431) * Added session_log events to text_log * user error severity instead of debug for failure * updated test expectation * added user_id to logout message * empty commit --------- Co-authored-by: Alexey Gerasimchuck --- src/Interpreters/Session.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index f8bd70afdb6..bcfaae40a03 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -299,6 +299,7 @@ Session::~Session() if (notified_session_log_about_login) { + LOG_DEBUG(log, "{} Logout, user_id: {}", toString(auth_id), toString(*user_id)); if (auto session_log = getSessionLog()) { /// TODO: We have to ensure that the same info is added to the session log on a LoginSuccess event and on the corresponding Logout event. @@ -320,6 +321,7 @@ AuthenticationType Session::getAuthenticationTypeOrLogInFailure(const String & u } catch (const Exception & e) { + LOG_ERROR(log, "{} Authentication failed with error: {}", toString(auth_id), e.what()); if (auto session_log = getSessionLog()) session_log->addLoginFailure(auth_id, getClientInfo(), user_name, e); From 84131740fdfb7fd7f4c1240f019b239d71d60f2f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 15 Aug 2023 00:22:05 +0200 Subject: [PATCH 1994/2047] Fix sanitizer error --- src/Planner/PlannerJoinTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 11de6fcfabe..f6ce029a295 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -842,8 +842,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } else { + SelectQueryOptions analyze_query_options = SelectQueryOptions(from_stage).analyze(); Planner planner(select_query_info.query_tree, - SelectQueryOptions(from_stage).analyze(), + analyze_query_options, select_query_info.planner_context); planner.buildQueryPlanIfNeeded(); From 376202f7392032131026aa5f46389f99f66638b8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Aug 2023 23:29:28 +0200 Subject: [PATCH 1995/2047] fix creation of empty parts --- src/Storages/MergeTree/MergeTreeData.cpp | 5 ++-- src/Storages/MergeTree/MergeTreeData.h | 4 ++- src/Storages/StorageMergeTree.cpp | 31 ++++++++------------- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 4 files changed, 18 insertions(+), 24 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index db0a7b34d7e..da0a6328894 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8435,7 +8435,7 @@ void MergeTreeData::incrementMergedPartsProfileEvent(MergeTreeDataPartType type) } } -MergeTreeData::MutableDataPartPtr MergeTreeData::createEmptyPart( +std::pair MergeTreeData::createEmptyPart( MergeTreePartInfo & new_part_info, const MergeTreePartition & partition, const String & new_part_name, const MergeTreeTransactionPtr & txn) { @@ -8454,6 +8454,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createEmptyPart( ReservationPtr reservation = reserveSpacePreferringTTLRules(metadata_snapshot, 0, move_ttl_infos, time(nullptr), 0, true); VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); + auto tmp_dir_holder = getTemporaryPartDirectoryHolder(EMPTY_PART_TMP_PREFIX + new_part_name); auto new_data_part = getDataPartBuilder(new_part_name, data_part_volume, EMPTY_PART_TMP_PREFIX + new_part_name) .withBytesAndRowsOnDisk(0, 0) .withPartInfo(new_part_info) @@ -8513,7 +8514,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createEmptyPart( out.finalizePart(new_data_part, sync_on_insert); new_data_part_storage->precommitTransaction(); - return new_data_part; + return std::make_pair(std::move(new_data_part), std::move(tmp_dir_holder)); } bool MergeTreeData::allowRemoveStaleMovingParts() const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 9ee61134740..e4801cffa36 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -936,7 +936,9 @@ public: WriteAheadLogPtr getWriteAheadLog(); constexpr static auto EMPTY_PART_TMP_PREFIX = "tmp_empty_"; - MergeTreeData::MutableDataPartPtr createEmptyPart(MergeTreePartInfo & new_part_info, const MergeTreePartition & partition, const String & new_part_name, const MergeTreeTransactionPtr & txn); + std::pair createEmptyPart( + MergeTreePartInfo & new_part_info, const MergeTreePartition & partition, + const String & new_part_name, const MergeTreeTransactionPtr & txn); MergeTreeDataFormatVersion format_version; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ad9013d9f13..a22c1355015 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1653,11 +1653,7 @@ struct FutureNewEmptyPart MergeTreePartition partition; std::string part_name; - scope_guard tmp_dir_guard; - StorageMergeTree::MutableDataPartPtr data_part; - - std::string getDirName() const { return StorageMergeTree::EMPTY_PART_TMP_PREFIX + part_name; } }; using FutureNewEmptyParts = std::vector; @@ -1688,19 +1684,19 @@ FutureNewEmptyParts initCoverageWithNewEmptyParts(const DataPartsVector & old_pa return future_parts; } -StorageMergeTree::MutableDataPartsVector createEmptyDataParts(MergeTreeData & data, FutureNewEmptyParts & future_parts, const MergeTreeTransactionPtr & txn) +std::pair> createEmptyDataParts( + MergeTreeData & data, FutureNewEmptyParts & future_parts, const MergeTreeTransactionPtr & txn) { - StorageMergeTree::MutableDataPartsVector data_parts; + std::pair> data_parts; for (auto & part: future_parts) - data_parts.push_back(data.createEmptyPart(part.part_info, part.partition, part.part_name, txn)); + { + auto [new_data_part, tmp_dir_holder] = data.createEmptyPart(part.part_info, part.partition, part.part_name, txn); + data_parts.first.emplace_back(std::move(new_data_part)); + data_parts.second.emplace_back(std::move(tmp_dir_holder)); + } return data_parts; } -void captureTmpDirectoryHolders(MergeTreeData & data, FutureNewEmptyParts & future_parts) -{ - for (auto & part : future_parts) - part.tmp_dir_guard = data.getTemporaryPartDirectoryHolder(part.getDirName()); -} void StorageMergeTree::renameAndCommitEmptyParts(MutableDataPartsVector & new_parts, Transaction & transaction) { @@ -1767,9 +1763,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont fmt::join(getPartsNames(future_parts), ", "), fmt::join(getPartsNames(parts), ", "), transaction.getTID()); - captureTmpDirectoryHolders(*this, future_parts); - - auto new_data_parts = createEmptyDataParts(*this, future_parts, txn); + auto [new_data_parts, tmp_dir_holders] = createEmptyDataParts(*this, future_parts, txn); renameAndCommitEmptyParts(new_data_parts, transaction); PartLog::addNewParts(query_context, PartLog::createPartLogEntries(new_data_parts, watch.elapsed(), profile_events_scope.getSnapshot())); @@ -1828,9 +1822,7 @@ void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPt fmt::join(getPartsNames(future_parts), ", "), fmt::join(getPartsNames({part}), ", "), transaction.getTID()); - captureTmpDirectoryHolders(*this, future_parts); - - auto new_data_parts = createEmptyDataParts(*this, future_parts, txn); + auto [new_data_parts, tmp_dir_holders] = createEmptyDataParts(*this, future_parts, txn); renameAndCommitEmptyParts(new_data_parts, transaction); PartLog::addNewParts(query_context, PartLog::createPartLogEntries(new_data_parts, watch.elapsed(), profile_events_scope.getSnapshot())); @@ -1914,9 +1906,8 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont fmt::join(getPartsNames(future_parts), ", "), fmt::join(getPartsNames(parts), ", "), transaction.getTID()); - captureTmpDirectoryHolders(*this, future_parts); - auto new_data_parts = createEmptyDataParts(*this, future_parts, txn); + auto [new_data_parts, tmp_dir_holders] = createEmptyDataParts(*this, future_parts, txn); renameAndCommitEmptyParts(new_data_parts, transaction); PartLog::addNewParts(query_context, PartLog::createPartLogEntries(new_data_parts, watch.elapsed(), profile_events_scope.getSnapshot())); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7fce373e26b..a1bf04c0ead 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9509,7 +9509,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP } } - MergeTreeData::MutableDataPartPtr new_data_part = createEmptyPart(new_part_info, partition, lost_part_name, NO_TRANSACTION_PTR); + auto [new_data_part, tmp_dir_holder] = createEmptyPart(new_part_info, partition, lost_part_name, NO_TRANSACTION_PTR); new_data_part->setName(lost_part_name); try From 5a1265d16d56b0bfae665bf2474c515de157f050 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 15 Aug 2023 00:23:08 +0000 Subject: [PATCH 1996/2047] add docs --- docs/en/operations/settings/settings.md | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index de3cc00e4c9..29434ef556f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -98,6 +98,18 @@ Default value: 0. ``` +## mutations_execute_nondeterministic_on_initiator {#mutations_execute_nondeterministic_on_initiator} + +If true constant nondeterministic functions (e.g. function `now()`) are executed on initiator and replaced to literals in `UPDATE` and `DELETE` queries. It helps to keep data in sync on replicas while executing mutations with constant nondeterministic functions. Default value: `false`. + +## mutations_execute_subqueries_on_initiator {#mutations_execute_subqueries_on_initiator} + +If true scalar subqueries are executed on initiator and replaced to literals in `UPDATE` and `DELETE` queries. Default value: `false`. + +## mutations_max_literal_size_to_replace {#mutations_max_literal_size_to_replace} + +The maximum size of serialized literal in bytes to replace in `UPDATE` and `DELETE` queries. Takes effect only if at least one the two settings above is enabled. Default value: 16384 (16 KiB). + ## distributed_product_mode {#distributed-product-mode} Changes the behaviour of [distributed subqueries](../../sql-reference/operators/in.md). @@ -4298,7 +4310,7 @@ Use this setting only for backward compatibility if your use cases depend on old ## session_timezone {#session_timezone} Sets the implicit time zone of the current session or query. -The implicit time zone is the time zone applied to values of type DateTime/DateTime64 which have no explicitly specified time zone. +The implicit time zone is the time zone applied to values of type DateTime/DateTime64 which have no explicitly specified time zone. The setting takes precedence over the globally configured (server-level) implicit time zone. A value of '' (empty string) means that the implicit time zone of the current session or query is equal to the [server time zone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). @@ -4333,7 +4345,7 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric ``` :::warning -Not all functions that parse DateTime/DateTime64 respect `session_timezone`. This can lead to subtle errors. +Not all functions that parse DateTime/DateTime64 respect `session_timezone`. This can lead to subtle errors. See the following example and explanation. ::: From 357903973357becef3fed58e86ad32fd4de36f9f Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Tue, 15 Aug 2023 14:02:36 +0800 Subject: [PATCH 1997/2047] Fix an unexpected behavior in #53152 The problematic case happens when an array column of date/datetime is simultaneously specified in an Array Join clause without aliases and in a time converter, such as toYYYYMM. After applying Array Join without aliases, the column's name refers to the flattened array items, however, its data type is recognized as a ColumnArray still, which leads to the unexpected exception throws when building the preimage for the time converters. As a quick fix, we more strictly check the data types of the time converters and quit the preimage optimization pass early. --- ...ptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp | 7 +++++-- .../02843_date_predicate_optimizations_bugs.reference | 2 ++ .../02843_date_predicate_optimizations_bugs.sql | 9 +++++++++ 3 files changed, 16 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.reference create mode 100644 tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.sql diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp index 6a9251cec49..17fad4ba6e4 100644 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp +++ b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp @@ -160,15 +160,18 @@ void OptimizeDateOrDateTimeConverterWithPreimageMatcher::visit(const ASTFunction auto data_type_and_name = data.tables[*pos].columns.tryGetByName(column_id->shortName()); if (!data_type_and_name) return; + const auto column_type = data_type_and_name->type; + if (!column_type || (!isDateOrDate32(*column_type) && !isDateTime(*column_type) && !isDateTime64(*column_type))) return; + const auto & converter = FunctionFactory::instance().tryGet(ast_func->name, data.context); if (!converter) return; ColumnsWithTypeAndName args; - args.emplace_back(data_type_and_name->type, "tmp"); + args.emplace_back(column_type, "tmp"); auto converter_base = converter->build(args); if (!converter_base || !converter_base->hasInformationAboutPreimage()) return; - auto preimage_range = converter_base->getPreimage(*(data_type_and_name->type), literal->value); + auto preimage_range = converter_base->getPreimage(*column_type, literal->value); if (!preimage_range) return; const auto new_ast = generateOptimizedDateFilterAST(comparator, *data_type_and_name, *preimage_range); diff --git a/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.reference b/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.reference new file mode 100644 index 00000000000..d641328e9a5 --- /dev/null +++ b/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.reference @@ -0,0 +1,2 @@ +202308 1 +202308 2 diff --git a/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.sql b/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.sql new file mode 100644 index 00000000000..6e26a5166cb --- /dev/null +++ b/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.sql @@ -0,0 +1,9 @@ +select + toYYYYMM(date) as date_, + n +from (select + [toDate('20230815'), toDate('20230816')] as date, + [1, 2] as n +) as data +array join date, n +where date_ >= 202303; From b9fa29c4c8004f1b1a483cfa2ce21dd350163c05 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Aug 2023 09:46:40 +0300 Subject: [PATCH 1998/2047] Update OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp --- ...OrDateTimeConverterWithPreimageVisitor.cpp | 40 +++++++++++-------- 1 file changed, 24 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp index 17fad4ba6e4..dd205ae6508 100644 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp +++ b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp @@ -118,35 +118,37 @@ void OptimizeDateOrDateTimeConverterWithPreimageMatcher::visit(const ASTFunction {"greaterOrEquals", "lessOrEquals"}, }; - if (!swap_relations.contains(function.name)) return; + if (!swap_relations.contains(function.name)) + return; - if (!function.arguments || function.arguments->children.size() != 2) return; + if (!function.arguments || function.arguments->children.size() != 2) + return; size_t func_id = function.arguments->children.size(); for (size_t i = 0; i < function.arguments->children.size(); i++) - { if (const auto * func = function.arguments->children[i]->as()) - { func_id = i; - } - } - if (func_id == function.arguments->children.size()) return; + if (func_id == function.arguments->children.size()) + return; size_t literal_id = 1 - func_id; const auto * literal = function.arguments->children[literal_id]->as(); - if (!literal || literal->value.getType() != Field::Types::UInt64) return; + if (!literal || literal->value.getType() != Field::Types::UInt64) + return; String comparator = literal_id > func_id ? function.name : swap_relations.at(function.name); const auto * ast_func = function.arguments->children[func_id]->as(); /// Currently we only handle single-argument functions. - if (!ast_func || !ast_func->arguments || ast_func->arguments->children.size() != 1) return; + if (!ast_func || !ast_func->arguments || ast_func->arguments->children.size() != 1) + return; const auto * column_id = ast_func->arguments->children.at(0)->as(); - if (!column_id) return; + if (!column_id) + return; auto pos = IdentifierSemantic::getMembership(*column_id); if (!pos) @@ -158,24 +160,30 @@ void OptimizeDateOrDateTimeConverterWithPreimageMatcher::visit(const ASTFunction return; auto data_type_and_name = data.tables[*pos].columns.tryGetByName(column_id->shortName()); - if (!data_type_and_name) return; + if (!data_type_and_name) + return; const auto column_type = data_type_and_name->type; - if (!column_type || (!isDateOrDate32(*column_type) && !isDateTime(*column_type) && !isDateTime64(*column_type))) return; + if (!column_type || (!isDateOrDate32(*column_type) && !isDateTime(*column_type) && !isDateTime64(*column_type))) + return; const auto & converter = FunctionFactory::instance().tryGet(ast_func->name, data.context); - if (!converter) return; + if (!converter) + return; ColumnsWithTypeAndName args; args.emplace_back(column_type, "tmp"); auto converter_base = converter->build(args); - if (!converter_base || !converter_base->hasInformationAboutPreimage()) return; + if (!converter_base || !converter_base->hasInformationAboutPreimage()) + return; auto preimage_range = converter_base->getPreimage(*column_type, literal->value); - if (!preimage_range) return; + if (!preimage_range) + return; const auto new_ast = generateOptimizedDateFilterAST(comparator, *data_type_and_name, *preimage_range); - if (!new_ast) return; + if (!new_ast) + return; ast = new_ast; } From df02512ebfa8efc455519c5e5edd7492e5ad0c16 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Aug 2023 08:53:08 +0200 Subject: [PATCH 1999/2047] Do not send logs to CI if the credentials are not set --- tests/ci/ast_fuzzer_check.py | 10 ++++++---- tests/ci/functional_test_check.py | 9 +++++---- tests/ci/stress_check.py | 9 +++++---- 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 56b356f5449..82b2732c2b2 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -145,10 +145,12 @@ def main(): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - subprocess.check_call( - f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}' '{main_log_path}'", - shell=True, - ) + + if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + subprocess.check_call( + f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}' '{main_log_path}'", + shell=True, + ) check_name_lower = ( check_name.lower().replace("(", "").replace(")", "").replace(" ", "") diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index d06da94d0f0..2d9ab77c9cf 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -394,10 +394,11 @@ def main(): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - subprocess.check_call( - f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", - shell=True, - ) + if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + subprocess.check_call( + f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", + shell=True, + ) report_url = upload_results( s3_helper, diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 42d372efb5d..b9af5fd5e83 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -209,10 +209,11 @@ def run_stress_test(docker_image_name): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - subprocess.check_call( - f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", - shell=True, - ) + if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + subprocess.check_call( + f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", + shell=True, + ) report_url = upload_results( s3_helper, From a92fe25ff9968a2edd51f918802c4485957f989a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 15 Aug 2023 07:15:58 +0000 Subject: [PATCH 2000/2047] Automatic style fix --- tests/ci/ast_fuzzer_check.py | 2 +- tests/ci/functional_test_check.py | 2 +- tests/ci/stress_check.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 82b2732c2b2..1a75d02bef4 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -146,7 +146,7 @@ def main(): "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}' '{main_log_path}'", shell=True, diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 2d9ab77c9cf..22210390b09 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -394,7 +394,7 @@ def main(): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", shell=True, diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index b9af5fd5e83..9c18bcbfe40 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -209,7 +209,7 @@ def run_stress_test(docker_image_name): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", shell=True, From 8992cc5af4653365351a98f2d00c7e4416c4965f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Jul 2023 10:48:00 +0000 Subject: [PATCH 2001/2047] Factorize constants --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index f77cfe4fed0..1c92645dbfa 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -25,6 +25,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +static constexpr auto DISTANCE_FUNCTION_L2 = "L2Distance"; +static constexpr auto DISTANCE_FUNCTION_COSINE = "cosineDistance"; + +static constexpr auto DEFAULT_TREES = 100uz; +static constexpr auto DEFAULT_DISTANCE_FUNCTION = DISTANCE_FUNCTION_L2; template AnnoyIndexWithSerialization::AnnoyIndexWithSerialization(size_t dimensions) @@ -224,9 +229,9 @@ bool MergeTreeIndexConditionAnnoy::alwaysUnknownOrTrue() const std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const { - if (distance_function == "L2Distance") + if (distance_function == DISTANCE_FUNCTION_L2) return getUsefulRangesImpl(idx_granule); - else if (distance_function == "cosineDistance") + else if (distance_function == DISTANCE_FUNCTION_COSINE) return getUsefulRangesImpl(idx_granule); std::unreachable(); } @@ -289,9 +294,9 @@ MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, UInt64 MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { - if (distance_function == "L2Distance") + if (distance_function == DISTANCE_FUNCTION_L2) return std::make_shared>(index.name, index.sample_block); - else if (distance_function == "cosineDistance") + else if (distance_function == DISTANCE_FUNCTION_COSINE) return std::make_shared>(index.name, index.sample_block); std::unreachable(); } @@ -299,9 +304,9 @@ MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 - if (distance_function == "L2Distance") + if (distance_function == DISTANCE_FUNCTION_L2) return std::make_shared>(index.name, index.sample_block, trees); - else if (distance_function == "cosineDistance") + else if (distance_function == DISTANCE_FUNCTION_COSINE) return std::make_shared>(index.name, index.sample_block, trees); std::unreachable(); } @@ -313,14 +318,11 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const Selec MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) { - static constexpr auto default_trees = 100uz; - static constexpr auto default_distance_function = "L2Distance"; - - String distance_function = default_distance_function; + String distance_function = DEFAULT_DISTANCE_FUNCTION; if (!index.arguments.empty()) distance_function = index.arguments[0].get(); - UInt64 trees = default_trees; + UInt64 trees = DEFAULT_TREES; if (index.arguments.size() > 1) trees = index.arguments[1].get(); @@ -350,8 +352,8 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (!index.arguments.empty()) { String distance_name = index.arguments[0].get(); - if (distance_name != "L2Distance" && distance_name != "cosineDistance") - throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index only supports distance functions 'L2Distance' and 'cosineDistance'"); + if (distance_name != DISTANCE_FUNCTION_L2 && distance_name != DISTANCE_FUNCTION_COSINE) + throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index only supports distance functions '{}' and '{}'", DISTANCE_FUNCTION_L2, DISTANCE_FUNCTION_COSINE); } /// Check data type of indexed column: From 33948a150fefe36ebf82bb8196b52215e577270b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Aug 2023 11:50:11 +0200 Subject: [PATCH 2002/2047] Restart killed PublishedReleaseCI workflows --- tests/ci/workflow_approve_rerun_lambda/app.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 5e2331ece3c..e511d773577 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -64,6 +64,7 @@ NEED_RERUN_WORKFLOWS = { "DocsCheck", "MasterCI", "NightlyBuilds", + "PublishedReleaseCI", "PullRequestCI", "ReleaseBranchCI", } From bf40767f10e16d9fd6c5b29a8af1ae81c93694fc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Aug 2023 14:27:49 +0200 Subject: [PATCH 2003/2047] fix another race --- src/Storages/MergeTree/MergeTreeData.cpp | 25 +++++++++++++------ src/Storages/StorageMergeTree.cpp | 12 ++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++- ..._replace_partition_from_table_zookeeper.sh | 20 --------------- .../00933_ttl_replicated_zookeeper.sh | 16 ------------ ...034_move_partition_from_table_zookeeper.sh | 17 ------------- .../02443_detach_attach_partition.sh | 2 +- .../0_stateless/02482_load_parts_refcounts.sh | 17 ------------- tests/queries/shell_config.sh | 20 +++++++++++++++ 9 files changed, 51 insertions(+), 82 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 561eef28c78..4026be31286 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5832,18 +5832,21 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const { const String source_dir = "detached/"; - std::map name_to_disk; - /// Let's compose a list of parts that should be added. if (attach_part) { const String part_id = partition->as().value.safeGet(); validateDetachedPartName(part_id); - auto disk = getDiskForDetachedPart(part_id); - renamed_parts.addPart(part_id, "attaching_" + part_id, disk); - - if (MergeTreePartInfo::tryParsePartName(part_id, format_version)) - name_to_disk[part_id] = getDiskForDetachedPart(part_id); + if (temporary_parts.contains(String(DETACHED_DIR_NAME) + "/" + part_id)) + { + LOG_WARNING(log, "Will not try to attach part {} because its directory is temporary, " + "probably it's being detached right now", part_id); + } + else + { + auto disk = getDiskForDetachedPart(part_id); + renamed_parts.addPart(part_id, "attaching_" + part_id, disk); + } } else { @@ -5860,6 +5863,12 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const for (const auto & part_info : detached_parts) { + if (temporary_parts.contains(String(DETACHED_DIR_NAME) + "/" + part_info.dir_name)) + { + LOG_WARNING(log, "Will not try to attach part {} because its directory is temporary, " + "probably it's being detached right now", part_info.dir_name); + continue; + } LOG_DEBUG(log, "Found part {}", part_info.dir_name); active_parts.add(part_info.dir_name); } @@ -5870,6 +5879,8 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const for (const auto & part_info : detached_parts) { const String containing_part = active_parts.getContainingPart(part_info.dir_name); + if (containing_part.empty()) + continue; LOG_DEBUG(log, "Found containing part {} for part {}", containing_part, part_info.dir_name); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9506d6f1075..03bb1b554eb 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1816,7 +1816,9 @@ void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPt if (detach) { auto metadata_snapshot = getInMemoryMetadataPtr(); - LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory()); + String part_dir = part->getDataPartStorage().getPartDirectory(); + LOG_INFO(log, "Detaching {}", part_dir); + auto holder = getTemporaryPartDirectoryHolder(String(DETACHED_DIR_NAME) + "/" + part_dir); part->makeCloneInDetached("", metadata_snapshot, /*disk_transaction*/ {}); } @@ -1901,7 +1903,9 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont for (const auto & part : parts) { auto metadata_snapshot = getInMemoryMetadataPtr(); - LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory()); + String part_dir = part->getDataPartStorage().getPartDirectory(); + LOG_INFO(log, "Detaching {}", part_dir); + auto holder = getTemporaryPartDirectoryHolder(String(DETACHED_DIR_NAME) + "/" + part_dir); part->makeCloneInDetached("", metadata_snapshot, /*disk_transaction*/ {}); } } @@ -1943,7 +1947,9 @@ void StorageMergeTree::dropPartsImpl(DataPartsVector && parts_to_remove, bool de /// NOTE: no race with background cleanup until we hold pointers to parts for (const auto & part : parts_to_remove) { - LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory()); + String part_dir = part->getDataPartStorage().getPartDirectory(); + LOG_INFO(log, "Detaching {}", part_dir); + auto holder = getTemporaryPartDirectoryHolder(String(DETACHED_DIR_NAME) + "/" + part_dir); part->makeCloneInDetached("", metadata_snapshot, /*disk_transaction*/ {}); } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bc2cff80c59..6b4ee3334c7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2097,7 +2097,9 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) { if (auto part_to_detach = part.getPartIfItWasActive()) { - LOG_INFO(log, "Detaching {}", part_to_detach->getDataPartStorage().getPartDirectory()); + String part_dir = part_to_detach->getDataPartStorage().getPartDirectory(); + LOG_INFO(log, "Detaching {}", part_dir); + auto holder = getTemporaryPartDirectoryHolder(String(DETACHED_DIR_NAME) + "/" + part_dir); part_to_detach->makeCloneInDetached("", metadata_snapshot, /*disk_transaction*/ {}); } } diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index c32b6d04a42..334025cba28 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -11,26 +11,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function query_with_retry -{ - local query="$1" && shift - - local retry=0 - until [ $retry -ge 5 ] - do - local result - result="$($CLICKHOUSE_CLIENT "$@" --query="$query" 2>&1)" - if [ "$?" == 0 ]; then - echo -n "$result" - return - else - retry=$((retry + 1)) - sleep 3 - fi - done - echo "Query '$query' failed with '$result'" -} - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r1;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r2;" diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sh b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sh index 22d9e0690b3..d06037fb836 100755 --- a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sh @@ -5,22 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function query_with_retry -{ - retry=0 - until [ $retry -ge 5 ] - do - result=$($CLICKHOUSE_CLIENT $2 --query="$1" 2>&1) - if [ "$?" == 0 ]; then - echo -n "$result" - return - else - retry=$(($retry + 1)) - sleep 3 - fi - done - echo "Query '$1' failed with '$result'" -} $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS ttl_repl1" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS ttl_repl2" diff --git a/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh index e0a84323dbd..39c5742e7a7 100755 --- a/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh @@ -7,23 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function query_with_retry -{ - retry=0 - until [ $retry -ge 5 ] - do - result=$($CLICKHOUSE_CLIENT $2 --query="$1" 2>&1) - if [ "$?" == 0 ]; then - echo -n "$result" - return - else - retry=$(($retry + 1)) - sleep 3 - fi - done - echo "Query '$1' failed with '$result'" -} - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst;" diff --git a/tests/queries/0_stateless/02443_detach_attach_partition.sh b/tests/queries/0_stateless/02443_detach_attach_partition.sh index 13ea966dbf5..5a3f1b64065 100755 --- a/tests/queries/0_stateless/02443_detach_attach_partition.sh +++ b/tests/queries/0_stateless/02443_detach_attach_partition.sh @@ -55,7 +55,7 @@ wait $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table0" $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table1" -while ! $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table0 ATTACH PARTITION ID 'all'" 2>/dev/null; do sleep 0.5; done +query_with_retry "ALTER TABLE alter_table0 ATTACH PARTITION ID 'all'" 2>/dev/null; $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table1 ATTACH PARTITION ID 'all'" 2>/dev/null $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table1" $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table1 ATTACH PARTITION ID 'all'" diff --git a/tests/queries/0_stateless/02482_load_parts_refcounts.sh b/tests/queries/0_stateless/02482_load_parts_refcounts.sh index 4d588dabeb9..fe3cee1359e 100755 --- a/tests/queries/0_stateless/02482_load_parts_refcounts.sh +++ b/tests/queries/0_stateless/02482_load_parts_refcounts.sh @@ -5,23 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function query_with_retry -{ - retry=0 - until [ $retry -ge 5 ] - do - result=$($CLICKHOUSE_CLIENT $2 --query="$1" 2>&1) - if [ "$?" == 0 ]; then - echo -n "$result" - return - else - retry=$(($retry + 1)) - sleep 3 - fi - done - echo "Query '$1' failed with '$result'" -} - $CLICKHOUSE_CLIENT -n --query " DROP TABLE IF EXISTS load_parts_refcounts SYNC; diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index ef70c82aefc..12bc0002191 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -155,3 +155,23 @@ function random_str() local n=$1 && shift tr -cd '[:lower:]' < /dev/urandom | head -c"$n" } + +function query_with_retry +{ + local query="$1" && shift + + local retry=0 + until [ $retry -ge 5 ] + do + local result + result="$($CLICKHOUSE_CLIENT "$@" --query="$query" 2>&1)" + if [ "$?" == 0 ]; then + echo -n "$result" + return + else + retry=$((retry + 1)) + sleep 3 + fi + done + echo "Query '$query' failed with '$result'" +} From 2aa211acc2af778728f87a0cf36be8efb68243b3 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 15 Aug 2023 13:26:39 +0000 Subject: [PATCH 2004/2047] Added integration test for session log --- .../test.py | 5 +- tests/integration/test_session_log/.gitignore | 1 + .../integration/test_session_log/__init__.py | 0 .../test_session_log/configs/log.xml | 9 + .../test_session_log/configs/ports.xml | 9 + .../test_session_log/configs/session_log.xml | 9 + .../test_session_log/configs/users.xml | 23 ++ .../protos/clickhouse_grpc.proto | 1 + tests/integration/test_session_log/test.py | 292 ++++++++++++++++++ 9 files changed, 345 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_session_log/.gitignore create mode 100644 tests/integration/test_session_log/__init__.py create mode 100644 tests/integration/test_session_log/configs/log.xml create mode 100644 tests/integration/test_session_log/configs/ports.xml create mode 100644 tests/integration/test_session_log/configs/session_log.xml create mode 100644 tests/integration/test_session_log/configs/users.xml create mode 120000 tests/integration/test_session_log/protos/clickhouse_grpc.proto create mode 100644 tests/integration/test_session_log/test.py diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index c5c33b1cddb..5eaef09bf6d 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -28,10 +28,7 @@ proto_dir = os.path.join(SCRIPT_DIR, "./protos") gen_dir = os.path.join(SCRIPT_DIR, "./_gen") os.makedirs(gen_dir, exist_ok=True) run_and_check( - "python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \ - {proto_dir}/clickhouse_grpc.proto".format( - proto_dir=proto_dir, gen_dir=gen_dir - ), + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", shell=True, ) diff --git a/tests/integration/test_session_log/.gitignore b/tests/integration/test_session_log/.gitignore new file mode 100644 index 00000000000..edf565ec632 --- /dev/null +++ b/tests/integration/test_session_log/.gitignore @@ -0,0 +1 @@ +_gen diff --git a/tests/integration/test_session_log/__init__.py b/tests/integration/test_session_log/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_session_log/configs/log.xml b/tests/integration/test_session_log/configs/log.xml new file mode 100644 index 00000000000..7a079b81e69 --- /dev/null +++ b/tests/integration/test_session_log/configs/log.xml @@ -0,0 +1,9 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/ports.xml b/tests/integration/test_session_log/configs/ports.xml new file mode 100644 index 00000000000..fbaefc16b3a --- /dev/null +++ b/tests/integration/test_session_log/configs/ports.xml @@ -0,0 +1,9 @@ + + 5433 + 9001 + 9100 + + + false + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml new file mode 100644 index 00000000000..a0e4e3e2216 --- /dev/null +++ b/tests/integration/test_session_log/configs/session_log.xml @@ -0,0 +1,9 @@ + + + system + session_log
+ + toYYYYMM(event_date) + 7500 +
+
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml new file mode 100644 index 00000000000..0416dfadc8a --- /dev/null +++ b/tests/integration/test_session_log/configs/users.xml @@ -0,0 +1,23 @@ + + + + 0 + + + + + + + pass + + + pass + + + pass + + + pass + + + \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto new file mode 120000 index 00000000000..25d15f11e3b --- /dev/null +++ b/tests/integration/test_session_log/protos/clickhouse_grpc.proto @@ -0,0 +1 @@ +../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py new file mode 100644 index 00000000000..b860cde1df6 --- /dev/null +++ b/tests/integration/test_session_log/test.py @@ -0,0 +1,292 @@ +import os + +import grpc +import pymysql.connections +import psycopg2 as py_psql +import pytest +import random +import logging +import sys +import threading + +from helpers.cluster import ClickHouseCluster, run_and_check + +POSTGRES_SERVER_PORT = 5433 +MYSQL_SERVER_PORT = 9001 +GRPC_PORT = 9100 +SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_ENCODING = "utf-8" + +# Use grpcio-tools to generate *pb2.py files from *.proto. +proto_dir = os.path.join(SCRIPT_DIR, "./protos") +gen_dir = os.path.join(SCRIPT_DIR, "./_gen") +os.makedirs(gen_dir, exist_ok=True) +run_and_check( + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", + shell=True, +) + +sys.path.append(gen_dir) + +import clickhouse_grpc_pb2 +import clickhouse_grpc_pb2_grpc + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "node", + main_configs=[ + "configs/ports.xml", + "configs/log.xml", + "configs/session_log.xml", + ], + user_configs=["configs/users.xml"], + # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 + env_variables={ + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") + }, + with_postgres=True, +) + + +def grpc_get_url(): + return f"{instance.ip_address}:{GRPC_PORT}" + + +def grpc_create_insecure_channel(): + channel = grpc.insecure_channel(grpc_get_url()) + grpc.channel_ready_future(channel).result(timeout=2) + return channel + + +session_id_counter = 0 + + +def next_session_id(): + global session_id_counter + session_id = session_id_counter + session_id_counter += 1 + return str(session_id) + + +def grpc_query(query, user_, pass_, raise_exception): + try: + query_info = clickhouse_grpc_pb2.QueryInfo( + query=query, + session_id=next_session_id(), + user_name=user_, + password=pass_, + ) + channel = grpc_create_insecure_channel() + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) + result = stub.ExecuteQuery(query_info) + if result and result.HasField("exception"): + raise Exception(result.exception.display_text) + + return result.output.decode(DEFAULT_ENCODING) + except Exception: + assert raise_exception + + +def postgres_query(query, user_, pass_, raise_exception): + try: + connection_string = f"host={instance.hostname} port={POSTGRES_SERVER_PORT} dbname=default user={user_} password={pass_}" + cluster.exec_in_container(cluster.postgres_id, + [ + "/usr/bin/psql", + connection_string, + "--no-align", + "--field-separator=' '", + "-c", + query + ], + shell=True + ) + except Exception: + assert raise_exception + + +def mysql_query(query, user_, pass_, raise_exception): + try: + client = pymysql.connections.Connection( + host=instance.ip_address, + user=user_, + password=pass_, + database="default", + port=MYSQL_SERVER_PORT, + ) + cursor = client.cursor(pymysql.cursors.DictCursor) + if raise_exception: + with pytest.raises(Exception): + cursor.execute(query) + else: + cursor.execute(query) + cursor.fetchall() + except Exception: + assert raise_exception + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_grpc_session(started_cluster): + grpc_query("SELECT 1", "grpc_user", "pass", False) + grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) + grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "grpc_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" + ) + assert logout_records == "grpc_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "grpc_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_mysql_session(started_cluster): + mysql_query("SELECT 1", "mysql_user", "pass", False) + mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) + mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "mysql_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" + ) + assert logout_records == "mysql_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "mysql_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_postgres_session(started_cluster): + postgres_query("SELECT 1", "postgres_user", "pass", False) + postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) + postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "postgres_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" + ) + assert logout_records == "postgres_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "postgres_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_parallel_sessions(started_cluster): + thread_list = [] + for _ in range(10): + # Sleep time does not significantly matter here, + # test should pass even without sleeping. + for function in [postgres_query, grpc_query, mysql_query]: + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "pass", + False, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "wrong_pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "wrong_parallel_user", + "pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + + for thread in thread_list: + thread.join() + + instance.query("SYSTEM FLUSH LOGS") + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" + ) + assert port_0_sessions == "90\n" + + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" + ) + assert port_0_sessions == "0\n" + + address_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" + ) + assert address_0_sessions == "0\n" + + grpc_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" + ) + assert grpc_sessions == "30\n" + + mysql_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" + ) + assert mysql_sessions == "30\n" + + postgres_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" + ) + assert postgres_sessions == "30\n" + + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "30\n" + + logout_failure_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" + ) + assert logout_failure_sessions == "30\n" From 0e1728801eccb11a9cadf181fc3f555a4e39e125 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 15 Aug 2023 13:31:53 +0000 Subject: [PATCH 2005/2047] black run --- tests/integration/test_session_log/test.py | 28 +++++++++++----------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py index b860cde1df6..bb7cafa4ee6 100644 --- a/tests/integration/test_session_log/test.py +++ b/tests/integration/test_session_log/test.py @@ -2,10 +2,8 @@ import os import grpc import pymysql.connections -import psycopg2 as py_psql import pytest import random -import logging import sys import threading @@ -92,19 +90,20 @@ def grpc_query(query, user_, pass_, raise_exception): def postgres_query(query, user_, pass_, raise_exception): try: connection_string = f"host={instance.hostname} port={POSTGRES_SERVER_PORT} dbname=default user={user_} password={pass_}" - cluster.exec_in_container(cluster.postgres_id, - [ - "/usr/bin/psql", - connection_string, - "--no-align", - "--field-separator=' '", - "-c", - query - ], - shell=True - ) + cluster.exec_in_container( + cluster.postgres_id, + [ + "/usr/bin/psql", + connection_string, + "--no-align", + "--field-separator=' '", + "-c", + query, + ], + shell=True, + ) except Exception: - assert raise_exception + assert raise_exception def mysql_query(query, user_, pass_, raise_exception): @@ -126,6 +125,7 @@ def mysql_query(query, user_, pass_, raise_exception): except Exception: assert raise_exception + @pytest.fixture(scope="module") def started_cluster(): try: From 78d8557a566ac900a83e1f9aad18e5f49d4a3b96 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Aug 2023 15:37:09 +0200 Subject: [PATCH 2006/2047] fix --- src/Common/MemoryTracker.cpp | 5 ++++- src/DataTypes/registerDataTypeDateTime.cpp | 10 +++++----- src/Interpreters/Session.cpp | 2 +- src/Server/TCPHandler.cpp | 20 ++++++++++++------- .../0_stateless/02668_ulid_decoding.sql | 2 +- 5 files changed, 24 insertions(+), 15 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 7005a57d175..93bd50a0b49 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -151,8 +151,11 @@ void MemoryTracker::logPeakMemoryUsage() { log_peak_memory_usage_in_destructor = false; const auto * description = description_ptr.load(std::memory_order_relaxed); + auto peak_bytes = peak.load(std::memory_order::relaxed); + if (peak_bytes < 128 * 1024) + return; LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), - "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak)); + "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak_bytes)); } void MemoryTracker::logMemoryUsage(Int64 current) const diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 8080179ad47..2b5c4a0a143 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -22,11 +22,11 @@ enum class ArgumentKind Mandatory }; -String getExceptionMessage( +PreformattedMessage getExceptionMessage( const String & message, size_t argument_index, const char * argument_name, const std::string & context_data_type_name, Field::Types::Which field_type) { - return fmt::format("Parameter #{} '{}' for {}{}, expected {} literal", + return PreformattedMessage::create("Parameter #{} '{}' for {}{}, expected {} literal", argument_index, argument_name, context_data_type_name, message, field_type); } @@ -47,10 +47,10 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume else { if (argument && argument->value.getType() != field_type) - throw Exception::createDeprecated(getExceptionMessage(fmt::format(" has wrong type: {}", argument->value.getTypeName()), + throw Exception(getExceptionMessage(fmt::format(" has wrong type: {}", argument->value.getTypeName()), argument_index, argument_name, context_data_type_name, field_type), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); else - throw Exception::createDeprecated(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type), + throw Exception(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } } @@ -67,7 +67,7 @@ static DataTypePtr create(const ASTPtr & arguments) const auto timezone = getArgument(arguments, scale ? 1 : 0, "timezone", "DateTime"); if (!scale && !timezone) - throw Exception::createDeprecated(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64), + throw Exception(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); /// If scale is defined, the data type is DateTime when scale = 0 otherwise the data type is DateTime64 diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index bcfaae40a03..e0b5db44593 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -622,7 +622,7 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (auto query_context_user = query_context->getAccess()->tryGetUser()) { - LOG_DEBUG(log, "{} Creating query context from {} context, user_id: {}, parent context user: {}", + LOG_TRACE(log, "{} Creating query context from {} context, user_id: {}, parent context user: {}", toString(auth_id), from_session_context ? "session" : "global", toString(*user_id), diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index cd17c2bcb55..ac3928b4abe 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -263,6 +263,17 @@ void TCPHandler::runImpl() std::unique_ptr exception; bool network_error = false; bool query_duration_already_logged = false; + auto log_query_duration = [this, &query_duration_already_logged]() + { + if (query_duration_already_logged) + return; + query_duration_already_logged = true; + auto elapsed_sec = state.watch.elapsedSeconds(); + /// We already logged more detailed info if we read some rows + if (elapsed_sec < 1.0 && state.progress.read_rows) + return; + LOG_DEBUG(log, "Processed in {} sec.", elapsed_sec); + }; try { @@ -492,9 +503,7 @@ void TCPHandler::runImpl() /// Do it before sending end of stream, to have a chance to show log message in client. query_scope->logPeakMemoryUsage(); - - LOG_DEBUG(log, "Processed in {} sec.", state.watch.elapsedSeconds()); - query_duration_already_logged = true; + log_query_duration(); if (state.is_connection_closed) break; @@ -616,10 +625,7 @@ void TCPHandler::runImpl() LOG_WARNING(log, "Can't skip data packets after query failure."); } - if (!query_duration_already_logged) - { - LOG_DEBUG(log, "Processed in {} sec.", state.watch.elapsedSeconds()); - } + log_query_duration(); /// QueryState should be cleared before QueryScope, since otherwise /// the MemoryTracker will be wrong for possible deallocations. diff --git a/tests/queries/0_stateless/02668_ulid_decoding.sql b/tests/queries/0_stateless/02668_ulid_decoding.sql index df94025b7b5..ecab5004df6 100644 --- a/tests/queries/0_stateless/02668_ulid_decoding.sql +++ b/tests/queries/0_stateless/02668_ulid_decoding.sql @@ -4,7 +4,7 @@ SELECT dateDiff('minute', ULIDStringToDateTime(generateULID()), now()) = 0; SELECT toTimezone(ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E'), 'America/Costa_Rica'); SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E', 'America/Costa_Rica'); SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9', 'America/Costa_Rica'); -- { serverError ILLEGAL_COLUMN } -SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E', 'America/Costa_Ric'); -- { serverError POCO_EXCEPTION } +SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E', 'America/Costa_Ric'); -- { serverError BAD_ARGUMENTS } SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E0'); -- { serverError ILLEGAL_COLUMN } SELECT ULIDStringToDateTime(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT ULIDStringToDateTime(1, 2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 0fd28bf3309a65e5c0204c814bef0a5f13dada9d Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 15 Aug 2023 13:39:34 +0000 Subject: [PATCH 2007/2047] added remote session log test --- .../02834_remote_session_log.reference | 13 +++++ .../0_stateless/02834_remote_session_log.sh | 56 +++++++++++++++++++ 2 files changed, 69 insertions(+) create mode 100644 tests/queries/0_stateless/02834_remote_session_log.reference create mode 100755 tests/queries/0_stateless/02834_remote_session_log.sh diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference new file mode 100644 index 00000000000..e2680982ab0 --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.reference @@ -0,0 +1,13 @@ +0 +0 +0 +0 +client_port 0 connections: +0 +client_address '::' connections: +0 +login failures: +0 +TCP Login and logout count is equal +HTTP Login and logout count is equal +MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh new file mode 100755 index 00000000000..3bedfb6c9ee --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ +readonly TEST_USER=$"02834_USER_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "client_port 0 connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" + +echo "client_address '::' connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" + +echo "login failures:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" + +# remote(...) function sometimes reuses old cached sessions for query execution. +# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. + +for interface in 'TCP' 'HTTP' 'MySQL' +do + LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` + CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` + + if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then + echo "${interface} Login and logout count is equal" + else + TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` + echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" + fi +done + +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" From cbf9f88b90f69a08bd51377338d2a679e629cd82 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 15 Aug 2023 13:42:42 +0000 Subject: [PATCH 2008/2047] Added concurrent session session_log tests --- .../02833_concurrrent_sessions.reference | 34 +++++ .../0_stateless/02833_concurrrent_sessions.sh | 138 ++++++++++++++++++ 2 files changed, 172 insertions(+) create mode 100644 tests/queries/0_stateless/02833_concurrrent_sessions.reference create mode 100755 tests/queries/0_stateless/02833_concurrrent_sessions.sh diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference new file mode 100644 index 00000000000..bfe507e8eac --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.reference @@ -0,0 +1,34 @@ +sessions: +150 +port_0_sessions: +0 +address_0_sessions: +0 +tcp_sessions +60 +http_sessions +30 +http_with_session_id_sessions +30 +my_sql_sessions +30 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh new file mode 100755 index 00000000000..26b48462a76 --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -0,0 +1,138 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +# Each user uses a separate thread. +readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users +readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) +readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) +readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") +readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) + +readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" + +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; +done + +# All _session functions execute in separate threads. +# These functions try to create a session with successful login and logout. +# Sleep a small, random amount of time to make concurrency more intense. +# and try to login with an invalid password. +function tcp_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' + done +} + +function http_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" + done +} + +function http_with_session_id_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" + done +} + +function mysql_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" + + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; +export -f mysql_session; + +for user in "${TCP_USERS[@]}"; do + timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_USERS[@]}"; do + timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do + timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & +done + +for user in "${MYSQL_USERS[@]}"; do + timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & +done + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" + +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" + +echo "tcp_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" +echo "http_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "http_with_session_id_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "my_sql_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" + echo "Corresponding LoginSuccess/Logout" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" + echo "LoginFailure" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" + done From 4b5874b512802022e4c5581e17c9ed86c505129e Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 15 Aug 2023 13:45:06 +0000 Subject: [PATCH 2009/2047] added drop user during session test --- .../02835_drop_user_during_session.reference | 8 ++ .../02835_drop_user_during_session.sh | 114 ++++++++++++++++++ 2 files changed, 122 insertions(+) create mode 100644 tests/queries/0_stateless/02835_drop_user_during_session.reference create mode 100755 tests/queries/0_stateless/02835_drop_user_during_session.sh diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference new file mode 100644 index 00000000000..7252faab8c6 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.reference @@ -0,0 +1,8 @@ +port_0_sessions: +0 +address_0_sessions: +0 +Corresponding LoginSuccess/Logout +9 +LoginFailure +0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh new file mode 100755 index 00000000000..347ebd22f96 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -0,0 +1,114 @@ +#!/usr/bin/env bash +# Tags: no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +readonly TEST_USER="02835_USER_${PID}" +readonly TEST_ROLE="02835_ROLE_${PID}" +readonly TEST_PROFILE="02835_PROFILE_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +function tcp_session() +{ + local user=$1 + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" +} + +function http_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" +} + +function http_with_session_id_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" +} + +# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. +function wait_for_queries_start() +{ + local user=$1 + local queries_count=$2 + # 10 seconds waiting + counter=0 retries=100 + while [[ $counter -lt $retries ]]; do + result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") + if [[ $result == "${queries_count}" ]]; then + break; + fi + sleep 0.1 + ((++counter)) + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +# DROP USE CASE +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP ROLE CASE +${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP PROFILE CASE +${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" +echo "Corresponding LoginSuccess/Logout" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" +echo "LoginFailure" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From f95c8599594c3a8e7101a02a4e376f5cb6ca7b8a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Aug 2023 15:47:11 +0200 Subject: [PATCH 2010/2047] rename setting --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- tests/integration/test_replicated_database/configs/config.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f1c01d1aadc..57189012317 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1337,7 +1337,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery() && !DatabaseCatalog::instance().hasUUIDMapping(create.uuid) && Context::getGlobalContextInstance()->isServerCompletelyStarted() && - Context::getGlobalContextInstance()->getConfigRef().getBool("allow_moving_table_dir_to_trash", false)) + Context::getGlobalContextInstance()->getConfigRef().getBool("allow_moving_table_directory_to_trash", false)) { /// This is a secondary query from a Replicated database. It cannot be retried with another UUID, we must execute it as is. /// We don't have a table with this UUID (and all metadata is loaded), diff --git a/tests/integration/test_replicated_database/configs/config.xml b/tests/integration/test_replicated_database/configs/config.xml index 7a67d69c031..e598cc28d5d 100644 --- a/tests/integration/test_replicated_database/configs/config.xml +++ b/tests/integration/test_replicated_database/configs/config.xml @@ -1,6 +1,6 @@ 10 - 1 + 1 10 From 14977e60722b2efb76de402eba775c19843137a8 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Tue, 15 Aug 2023 13:54:28 +0000 Subject: [PATCH 2011/2047] Eliminate duplicate include --- src/Common/TransformEndianness.hpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 7c77e918199..1657305acda 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -6,8 +6,6 @@ #include -#include - #include namespace DB From 8343ebd4ecfc90e4fafb22d660fd0ad04aaa0e74 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 15 Aug 2023 14:44:13 +0000 Subject: [PATCH 2012/2047] Clarify comment --- programs/local/LocalServer.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 0dea7e8e643..b38e17ecade 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -710,7 +710,8 @@ void LocalServer::processConfig() if (mmap_cache_size) global_context->setMMappedFileCache(mmap_cache_size); - /// not initializing the query cache in clickhouse-local + /// In Server.cpp (./clickhouse-server), we would initialize the query cache here. + /// Intentionally not doing this in clickhouse-local as it doesn't make sense. #if USE_EMBEDDED_COMPILER size_t compiled_expression_cache_max_size_in_bytes = config().getUInt64("compiled_expression_cache_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE); From cbe4c8adc2973ee8d6583f178bd44915c55f21f1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 14 Aug 2023 23:24:41 +0000 Subject: [PATCH 2013/2047] Fix more functions with 'Context has expired' error --- src/Functions/FunctionFactory.h | 4 +-- src/Functions/FunctionsExternalDictionaries.h | 9 +++--- src/Functions/FunctionsJSON.h | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 8 ++--- .../QueryPlan/Optimizations/Optimizations.h | 4 +-- .../Optimizations/addPlansForSets.cpp | 6 ++-- .../QueryPlan/Optimizations/optimizeTree.cpp | 4 +-- src/Processors/QueryPlan/QueryPlan.cpp | 6 ++-- src/Processors/QueryPlan/QueryPlan.h | 2 +- .../02843_context_has_expired.reference | 4 +++ .../0_stateless/02843_context_has_expired.sql | 29 ++++++++++++++----- 11 files changed, 48 insertions(+), 30 deletions(-) diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index deea41e6677..588cae64e16 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -20,8 +20,8 @@ using FunctionCreator = std::function; using FunctionFactoryData = std::pair; /** Creates function by name. - * Function could use for initialization (take ownership of shared_ptr, for example) - * some dictionaries from Context. + * The provided Context is guaranteed to outlive the created function. Functions may use it for + * things like settings, current database, permission checks, etc. */ class FunctionFactory : private boost::noncopyable, public IFactoryWithAliases { diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 1b2e2eb3bd6..db6529da73c 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -62,13 +62,14 @@ namespace ErrorCodes */ -class FunctionDictHelper +class FunctionDictHelper : WithContext { public: - explicit FunctionDictHelper(ContextPtr context_) : current_context(context_) {} + explicit FunctionDictHelper(ContextPtr context_) : WithContext(context_) {} std::shared_ptr getDictionary(const String & dictionary_name) { + auto current_context = getContext(); auto dict = current_context->getExternalDictionariesLoader().getDictionary(dictionary_name, current_context); if (!access_checked) @@ -131,12 +132,10 @@ public: DictionaryStructure getDictionaryStructure(const String & dictionary_name) const { - return current_context->getExternalDictionariesLoader().getDictionaryStructure(dictionary_name, current_context); + return getContext()->getExternalDictionariesLoader().getDictionaryStructure(dictionary_name, getContext()); } private: - ContextPtr current_context; - /// Access cannot be not granted, since in this case checkAccess() will throw and access_checked will not be updated. std::atomic access_checked = false; diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index ca797eed856..094de0c27c2 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -336,7 +336,7 @@ private: template typename Impl> -class ExecutableFunctionJSON : public IExecutableFunction, WithContext +class ExecutableFunctionJSON : public IExecutableFunction { public: diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 3a381cd8dab..39cc4df5c2d 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -541,13 +541,13 @@ QueryPipeline InterpreterExplainQuery::executeImpl() InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); interpreter.buildQueryPlan(plan); context = interpreter.getContext(); - // collect the selected marks, rows, parts during build query pipeline. - plan.buildQueryPipeline( + // Collect the selected marks, rows, parts during build query pipeline. + // Hold on to the returned QueryPipelineBuilderPtr because `plan` may have pointers into + // it (through QueryPlanResourceHolder). + auto builder = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - if (settings.optimize) - plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); plan.explainEstimate(res_columns); insert_buf = false; break; diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 6ecec1359c5..2230e50425c 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -16,7 +16,7 @@ void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, Query void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); /// Third pass is used to apply filters such as key conditions and skip indexes to the storages that support them. /// After that it add CreateSetsStep for the subqueries that has not be used in the filters. -void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes); +void optimizeTreeThirdPass(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::Nodes & nodes); /// Optimization (first pass) is a function applied to QueryPlan::Node. /// It can read and update subtree of specified node. @@ -113,7 +113,7 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections); bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); -bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes); +bool addPlansForSets(QueryPlan & plan, QueryPlan::Node & node, QueryPlan::Nodes & nodes); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp index e9100ae9d02..47df05301c9 100644 --- a/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp +++ b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp @@ -6,7 +6,7 @@ namespace DB::QueryPlanOptimizations { -bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +bool addPlansForSets(QueryPlan & root_plan, QueryPlan::Node & node, QueryPlan::Nodes & nodes) { auto * delayed = typeid_cast(node.step.get()); if (!delayed) @@ -23,7 +23,9 @@ bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { input_streams.push_back(plan->getCurrentDataStream()); node.children.push_back(plan->getRootNode()); - nodes.splice(nodes.end(), QueryPlan::detachNodes(std::move(*plan))); + auto [add_nodes, add_resources] = QueryPlan::detachNodesAndResources(std::move(*plan)); + nodes.splice(nodes.end(), std::move(add_nodes)); + root_plan.addResources(std::move(add_resources)); } auto creating_sets = std::make_unique(std::move(input_streams)); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index b13dda9a8f0..0caedff67a5 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -181,7 +181,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s "No projection is used when optimize_use_projections = 1 and force_optimize_projection = 1"); } -void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes) +void optimizeTreeThirdPass(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { Stack stack; stack.push_back({.node = &root}); @@ -205,7 +205,7 @@ void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes) source_step_with_filter->applyFilters(); } - addPlansForSets(*frame.node, nodes); + addPlansForSets(plan, *frame.node, nodes); stack.pop_back(); } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 687260441ff..ceda9f97bab 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -482,7 +482,7 @@ void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_sett QueryPlanOptimizations::optimizeTreeFirstPass(optimization_settings, *root, nodes); QueryPlanOptimizations::optimizeTreeSecondPass(optimization_settings, *root, nodes); - QueryPlanOptimizations::optimizeTreeThirdPass(*root, nodes); + QueryPlanOptimizations::optimizeTreeThirdPass(*this, *root, nodes); updateDataStreams(*root); } @@ -542,9 +542,9 @@ void QueryPlan::explainEstimate(MutableColumns & columns) } } -QueryPlan::Nodes QueryPlan::detachNodes(QueryPlan && plan) +std::pair QueryPlan::detachNodesAndResources(QueryPlan && plan) { - return std::move(plan.nodes); + return {std::move(plan.nodes), std::move(plan.resources)}; } } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index d89bdc534be..f4a6c9097f2 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -108,7 +108,7 @@ public: using Nodes = std::list; Node * getRootNode() const { return root; } - static Nodes detachNodes(QueryPlan && plan); + static std::pair detachNodesAndResources(QueryPlan && plan); private: QueryPlanResourceHolder resources; diff --git a/tests/queries/0_stateless/02843_context_has_expired.reference b/tests/queries/0_stateless/02843_context_has_expired.reference index 573541ac970..229972f2924 100644 --- a/tests/queries/0_stateless/02843_context_has_expired.reference +++ b/tests/queries/0_stateless/02843_context_has_expired.reference @@ -1 +1,5 @@ 0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/02843_context_has_expired.sql b/tests/queries/0_stateless/02843_context_has_expired.sql index ccef3458ed7..8355ce2c18c 100644 --- a/tests/queries/0_stateless/02843_context_has_expired.sql +++ b/tests/queries/0_stateless/02843_context_has_expired.sql @@ -1,23 +1,36 @@ -DROP DICTIONARY IF EXISTS dict; -DROP TABLE IF EXISTS source; +DROP DICTIONARY IF EXISTS 02843_dict; +DROP TABLE IF EXISTS 02843_source; +DROP TABLE IF EXISTS 02843_join; -CREATE TABLE source +CREATE TABLE 02843_source ( id UInt64, value String ) ENGINE=Memory; -CREATE DICTIONARY dict +CREATE DICTIONARY 02843_dict ( id UInt64, value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(TABLE 'source')) +SOURCE(CLICKHOUSE(TABLE '02843_source')) LAYOUT(DIRECT()); -SELECT 1 IN (SELECT dictGet('dict', 'value', materialize('1'))); +SELECT 1 IN (SELECT dictGet('02843_dict', 'value', materialize('1'))); -DROP DICTIONARY dict; -DROP TABLE source; +CREATE TABLE 02843_join (id UInt8, value String) ENGINE Join(ANY, LEFT, id); +SELECT 1 IN (SELECT joinGet(02843_join, 'value', materialize(1))); +SELECT 1 IN (SELECT joinGetOrNull(02843_join, 'value', materialize(1))); + +SELECT 1 IN (SELECT materialize(connectionId())); +SELECT 1000000 IN (SELECT materialize(getSetting('max_threads'))); +SELECT 1 in (SELECT file(materialize('a'))); -- { serverError 107 } + +EXPLAIN ESTIMATE SELECT 1 IN (SELECT dictGet('02843_dict', 'value', materialize('1'))); +EXPLAIN ESTIMATE SELECT 1 IN (SELECT joinGet(`02843_join`, 'value', materialize(1))); + +DROP DICTIONARY 02843_dict; +DROP TABLE 02843_source; +DROP TABLE 02843_join; From 3fc65a68c7cd6aabea5250784ee068014ffaa801 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Aug 2023 20:37:39 +0200 Subject: [PATCH 2014/2047] more parallel tests --- src/Client/ClientBase.cpp | 2 +- src/Interpreters/InterpreterUseQuery.cpp | 2 +- .../ReplaceQueryParameterVisitor.cpp | 6 + src/Parsers/ASTUseQuery.h | 26 +- src/Parsers/ParserCreateQuery.cpp | 24 +- src/Parsers/ParserUseQuery.cpp | 4 +- tests/clickhouse-test | 33 +- .../00080_show_tables_and_system_tables.sql | 12 +- ...s_and_insert_without_explicit_database.sql | 13 +- ...t_database_when_create_materializ_view.sql | 8 +- .../00604_show_create_database.sql | 4 +- ...12_http_max_query_size_for_distributed.sql | 2 - ...3_max_block_size_system_tables_columns.sql | 1 + ...51_default_databasename_for_view.reference | 8 +- .../00751_default_databasename_for_view.sql | 31 +- .../00800_versatile_storage_join.sql | 29 +- .../00857_global_joinsavel_table_alias.sql | 6 +- .../0_stateless/01012_show_tables_limit.sql | 25 +- .../01018_ddl_dictionaries_select.reference | 6 +- .../01018_ddl_dictionaries_select.sql | 99 +- .../01018_ddl_dictionaries_special.sql | 66 +- .../01018_dictionaries_from_dictionaries.sql | 59 +- .../0_stateless/01018_ip_dictionary_long.sql | 1020 ++++++++--------- .../01033_dictionaries_lifetime.sql | 29 +- ...rfluous_dict_reload_on_create_database.sql | 38 +- .../01041_create_dictionary_if_not_exists.sql | 26 +- ...dictionary_attribute_properties_values.sql | 22 +- .../01045_dictionaries_restrictions.sql | 12 +- ...7_window_view_parser_inner_table.reference | 32 +- .../01047_window_view_parser_inner_table.sql | 139 ++- ...0_clickhouse_dict_source_with_subquery.sql | 23 +- .../01053_drop_database_mat_view.sql | 17 +- .../0_stateless/01056_create_table_as.sql | 13 +- ...ndary_index_with_old_format_merge_tree.sql | 1 - .../01073_attach_if_not_exists.sql | 1 - .../01073_show_tables_not_like.sql | 17 +- ..._error_incorrect_size_of_nested_column.sql | 41 +- .../0_stateless/01084_regexp_empty.sql | 9 +- .../0_stateless/01085_window_view_attach.sql | 33 +- 39 files changed, 944 insertions(+), 995 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9ad6a46866f..9092f12b62b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1811,7 +1811,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin } if (const auto * use_query = parsed_query->as()) { - const String & new_database = use_query->database; + const String & new_database = use_query->getDatabase(); /// If the client initiates the reconnection, it takes the settings from the config. config().setString("database", new_database); /// If the connection initiates the reconnection, it uses its variable. diff --git a/src/Interpreters/InterpreterUseQuery.cpp b/src/Interpreters/InterpreterUseQuery.cpp index d8a5ae57470..b71f3a9cc1c 100644 --- a/src/Interpreters/InterpreterUseQuery.cpp +++ b/src/Interpreters/InterpreterUseQuery.cpp @@ -10,7 +10,7 @@ namespace DB BlockIO InterpreterUseQuery::execute() { - const String & new_database = query_ptr->as().database; + const String & new_database = query_ptr->as().getDatabase(); getContext()->checkAccess(AccessType::SHOW_DATABASES, new_database); getContext()->getSessionContext()->setCurrentDatabase(new_database); return {}; diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index e4a052388ac..89a40baaff9 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -131,6 +131,7 @@ void ReplaceQueryParameterVisitor::visitIdentifier(ASTPtr & ast) if (ast_identifier->children.empty()) return; + bool replaced_parameter = false; auto & name_parts = ast_identifier->name_parts; for (size_t i = 0, j = 0, size = name_parts.size(); i < size; ++i) { @@ -138,9 +139,14 @@ void ReplaceQueryParameterVisitor::visitIdentifier(ASTPtr & ast) { const auto & ast_param = ast_identifier->children[j++]->as(); name_parts[i] = getParamValue(ast_param.name); + replaced_parameter = true; } } + /// Do not touch AST if there are no parameters + if (!replaced_parameter) + return; + /// FIXME: what should this mean? if (!ast_identifier->semantic->special && name_parts.size() >= 2) ast_identifier->semantic->table = ast_identifier->name_parts.end()[-2]; diff --git a/src/Parsers/ASTUseQuery.h b/src/Parsers/ASTUseQuery.h index f767a6bbdb7..873a316e653 100644 --- a/src/Parsers/ASTUseQuery.h +++ b/src/Parsers/ASTUseQuery.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -14,19 +15,34 @@ namespace DB class ASTUseQuery : public IAST { public: - String database; + IAST * database; + + String getDatabase() const + { + String name; + tryGetIdentifierNameInto(database, name); + return name; + } /** Get the text that identifies this element. */ - String getID(char delim) const override { return "UseQuery" + (delim + database); } + String getID(char delim) const override { return "UseQuery" + (delim + getDatabase()); } - ASTPtr clone() const override { return std::make_shared(*this); } + ASTPtr clone() const override + { + auto res = std::make_shared(*this); + res->children.clear(); + if (database) + res->set(res->database, database->clone()); + return res; + } QueryKind getQueryKind() const override { return QueryKind::Use; } protected: - void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "USE " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(database); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "USE " << (settings.hilite ? hilite_none : ""); + database->formatImpl(settings, state, frame); } }; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 82674ab1a35..fb6dae248c0 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -502,7 +502,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_table("TABLE"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserCompoundIdentifier table_name_p(true, true); + ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true); ParserKeyword s_from("FROM"); ParserKeyword s_on("ON"); ParserToken s_dot(TokenType::Dot); @@ -740,7 +740,7 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserKeyword s_create("CREATE"); ParserKeyword s_attach("ATTACH"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserCompoundIdentifier table_name_p(true, true); + ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); ParserKeyword s_live("LIVE"); @@ -878,7 +878,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_attach("ATTACH"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserCompoundIdentifier table_name_p(true); + ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); ParserKeyword s_window("WINDOW"); @@ -1015,12 +1015,17 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->if_not_exists = if_not_exists; query->is_window_view = true; - StorageID table_id = table->as()->getTableId(); - query->setDatabase(table_id.database_name); - query->setTable(table_id.table_name); - query->uuid = table_id.uuid; + auto * table_id = table->as(); + query->database = table_id->getDatabase(); + query->table = table_id->getTable(); + query->uuid = table_id->uuid; query->cluster = cluster_str; + if (query->database) + query->children.push_back(query->database); + if (query->table) + query->children.push_back(query->table); + if (to_table) query->to_table_id = to_table->as()->getTableId(); @@ -1265,7 +1270,8 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserKeyword s_create("CREATE"); ParserKeyword s_attach("ATTACH"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserCompoundIdentifier table_name_p(true, true); + ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true); + ParserCompoundIdentifier to_table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ false); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); ParserKeyword s_materialized("MATERIALIZED"); @@ -1487,7 +1493,7 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E ParserKeyword s_dictionary("DICTIONARY"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); ParserKeyword s_on("ON"); - ParserCompoundIdentifier dict_name_p(true, true); + ParserCompoundIdentifier dict_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true); ParserToken s_left_paren(TokenType::OpeningRoundBracket); ParserToken s_right_paren(TokenType::ClosingRoundBracket); ParserToken s_dot(TokenType::Dot); diff --git a/src/Parsers/ParserUseQuery.cpp b/src/Parsers/ParserUseQuery.cpp index 40511d8c142..350e573281c 100644 --- a/src/Parsers/ParserUseQuery.cpp +++ b/src/Parsers/ParserUseQuery.cpp @@ -11,7 +11,7 @@ namespace DB bool ParserUseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_use("USE"); - ParserIdentifier name_p; + ParserIdentifier name_p{/*allow_query_parameter*/ true}; if (!s_use.ignore(pos, expected)) return false; @@ -21,7 +21,7 @@ bool ParserUseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; auto query = std::make_shared(); - tryGetIdentifierNameInto(database, query->database); + query->set(query->database, database); node = query; return true; diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 20c769bd701..7057f3df90f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1196,8 +1196,18 @@ class TestCase: # This is for .sh tests os.environ["CLICKHOUSE_LOG_COMMENT"] = args.testcase_basename + query_params = "" + if "need-query-parameters" in self.tags: + query_params = ( + " --param_CLICKHOUSE_DATABASE=" + + database + + " --param_CLICKHOUSE_DATABASE_1=" + + database + + "_1" + ) + params = { - "client": client + " --database=" + database + " --param_CLICKHOUSE_DATABASE=" + database, + "client": client + " --database=" + database + query_params, "logs_level": server_logs_level, "options": client_options, "test": self.case_file, @@ -1457,13 +1467,13 @@ class TestSuite: else: raise Exception(f"Unknown file_extension: {filename}") - def parse_tags_from_line(line, comment_sign): + def parse_tags_from_line(line, comment_sign) -> set[str]: if not line.startswith(comment_sign): - return None + return set() tags_str = line[len(comment_sign) :].lstrip() # noqa: ignore E203 tags_prefix = "Tags:" if not tags_str.startswith(tags_prefix): - return None + return set() tags_str = tags_str[len(tags_prefix) :] # noqa: ignore E203 tags = tags_str.split(",") tags = {tag.strip() for tag in tags} @@ -1481,12 +1491,23 @@ class TestSuite: def load_tags_from_file(filepath): comment_sign = get_comment_sign(filepath) + need_query_params = False with open(filepath, "r", encoding="utf-8") as file: try: - line = find_tag_line(file) + tag_line = find_tag_line(file) except UnicodeDecodeError: return [] - return parse_tags_from_line(line, comment_sign) + try: + if filepath.endswith(".sql"): + for line in file: + if "{CLICKHOUSE_DATABASE" in line: + need_query_params = True + except UnicodeDecodeError: + pass + parsed_tags = parse_tags_from_line(tag_line, comment_sign) + if need_query_params: + parsed_tags.add("need-query-parameters") + return parsed_tags all_tags = {} start_time = datetime.now() diff --git a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql index 40c26be31c0..a58f9ddb0ac 100644 --- a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql +++ b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql @@ -16,16 +16,12 @@ SELECT name FROM system.tables WHERE is_temporary = 1 AND name = 'test_temporary CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test_log(id UInt64) ENGINE = Log; CREATE MATERIALIZED VIEW {CLICKHOUSE_DATABASE:Identifier}.test_materialized ENGINE = Log AS SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_log; -SELECT dependencies_database, dependencies_table FROM system.tables WHERE name = 'test_log'; +SELECT dependencies_database, dependencies_table FROM system.tables WHERE name = 'test_log' and database=currentDatabase(); DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; - -- Check that create_table_query works for system tables and unusual Databases -DROP DATABASE IF EXISTS test_DatabaseMemory; -CREATE DATABASE test_DatabaseMemory ENGINE = Memory; -CREATE TABLE test_DatabaseMemory.A (A UInt8) ENGINE = Null; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE = Memory; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.A (A UInt8) ENGINE = Null; -SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables WHERE database = 'test_DatabaseMemory'; - -DROP DATABASE test_DatabaseMemory; +SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables WHERE database = '{CLICKHOUSE_DATABASE:String}'; diff --git a/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql b/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql index 95d46032601..7d925bc4ff1 100644 --- a/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql +++ b/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql @@ -1,8 +1,3 @@ --- Tags: no-parallel - -CREATE DATABASE IF NOT EXISTS test_00101_0; - -USE test_00101_0; DROP TABLE IF EXISTS test_table; DROP TABLE IF EXISTS test_view; @@ -25,9 +20,9 @@ DROP TABLE test_view_filtered; -- Check only sophisticated constructors and desctructors: -CREATE DATABASE IF NOT EXISTS test_00101_1; +CREATE DATABASE IF NOT EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; -USE test_00101_1; +USE {CLICKHOUSE_DATABASE_1:Identifier}; DROP TABLE IF EXISTS tmp; DROP TABLE IF EXISTS tmp_mv; @@ -57,5 +52,5 @@ EXISTS TABLE `.inner.tmp_mv4`; DROP TABLE tmp; -DROP DATABASE test_00101_0; -DROP DATABASE test_00101_1; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; +DROP DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; diff --git a/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql b/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql index 87f2e2b5276..d24a57187b4 100644 --- a/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql +++ b/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql @@ -1,14 +1,14 @@ -DROP DATABASE IF EXISTS none; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; DROP TABLE IF EXISTS test_materialized_00571; set allow_deprecated_syntax_for_merge_tree=1; -CREATE DATABASE none; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; CREATE TABLE test_00571 ( date Date, platform Enum8('a' = 0, 'b' = 1, 'c' = 2), app Enum8('a' = 0, 'b' = 1) ) ENGINE = MergeTree(date, (platform, app), 8192); CREATE MATERIALIZED VIEW test_materialized_00571 ENGINE = MergeTree(date, (platform, app), 8192) POPULATE AS SELECT date, platform, app FROM (SELECT * FROM test_00571); -USE none; +USE {CLICKHOUSE_DATABASE_1:Identifier}; INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.test_00571 VALUES('2018-02-16', 'a', 'a'); @@ -20,6 +20,6 @@ ATTACH TABLE {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; -DROP DATABASE IF EXISTS none; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_00571; DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; diff --git a/tests/queries/0_stateless/00604_show_create_database.sql b/tests/queries/0_stateless/00604_show_create_database.sql index d20d2cb3e53..23ebd23c9f1 100644 --- a/tests/queries/0_stateless/00604_show_create_database.sql +++ b/tests/queries/0_stateless/00604_show_create_database.sql @@ -1,5 +1,3 @@ --- Tags: no-ordinary-database +-- Tags: no-ordinary-database, no-replicated-database -create database if not exists {CLICKHOUSE_DATABASE:Identifier}; show create database {CLICKHOUSE_DATABASE:Identifier}; -drop database {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/00612_http_max_query_size_for_distributed.sql b/tests/queries/0_stateless/00612_http_max_query_size_for_distributed.sql index 1802fadc57b..462fd0fef61 100644 --- a/tests/queries/0_stateless/00612_http_max_query_size_for_distributed.sql +++ b/tests/queries/0_stateless/00612_http_max_query_size_for_distributed.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP TABLE IF EXISTS data_00612; DROP TABLE IF EXISTS dist_00612; diff --git a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql index fe1876ed6f8..0c8e4ae237d 100644 --- a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql +++ b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql @@ -1,3 +1,4 @@ +-- Tags: no-parallel -- NOTE: database = currentDatabase() is not mandatory diff --git a/tests/queries/0_stateless/00751_default_databasename_for_view.reference b/tests/queries/0_stateless/00751_default_databasename_for_view.reference index b3f1875ae91..4899e230924 100644 --- a/tests/queries/0_stateless/00751_default_databasename_for_view.reference +++ b/tests/queries/0_stateless/00751_default_databasename_for_view.reference @@ -1,4 +1,4 @@ -CREATE MATERIALIZED VIEW test_00751.t_mv_00751 +CREATE MATERIALIZED VIEW default.t_mv_00751 ( `date` Date, `platform` Enum8('a' = 0, 'b' = 1), @@ -11,14 +11,14 @@ SELECT date, platform, app -FROM test_00751.t_00751 +FROM default.t_00751 WHERE (app = ( SELECT min(app) - FROM test_00751.u_00751 + FROM default.u_00751 )) AND (platform = ( SELECT ( SELECT min(platform) - FROM test_00751.v_00751 + FROM default.v_00751 ) )) 2000-01-01 a a diff --git a/tests/queries/0_stateless/00751_default_databasename_for_view.sql b/tests/queries/0_stateless/00751_default_databasename_for_view.sql index 9292b612d87..599ef5b89ae 100644 --- a/tests/queries/0_stateless/00751_default_databasename_for_view.sql +++ b/tests/queries/0_stateless/00751_default_databasename_for_view.sql @@ -1,7 +1,3 @@ --- Tags: no-parallel - -CREATE DATABASE IF NOT EXISTS test_00751; -USE test_00751; DROP TABLE IF EXISTS t_00751; DROP TABLE IF EXISTS t_mv_00751; @@ -25,25 +21,24 @@ CREATE MATERIALIZED VIEW t_mv_00751 ENGINE = MergeTree ORDER BY date AS SELECT date, platform, app FROM t_00751 WHERE app = (SELECT min(app) from u_00751) AND platform = (SELECT (SELECT min(platform) from v_00751)); -SHOW CREATE TABLE test_00751.t_mv_00751 FORMAT TabSeparatedRaw; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.t_mv_00751 FORMAT TabSeparatedRaw; USE default; -DETACH TABLE test_00751.t_mv_00751; -ATTACH TABLE test_00751.t_mv_00751; +DETACH TABLE {CLICKHOUSE_DATABASE:Identifier}.t_mv_00751; +ATTACH TABLE {CLICKHOUSE_DATABASE:Identifier}.t_mv_00751; -INSERT INTO test_00751.t_00751 VALUES ('2000-01-01', 'a', 'a') ('2000-01-02', 'b', 'b'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.t_00751 VALUES ('2000-01-01', 'a', 'a') ('2000-01-02', 'b', 'b'); -INSERT INTO test_00751.u_00751 VALUES ('a'); -INSERT INTO test_00751.v_00751 VALUES ('a'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.u_00751 VALUES ('a'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.v_00751 VALUES ('a'); -INSERT INTO test_00751.t_00751 VALUES ('2000-01-03', 'a', 'a') ('2000-01-04', 'b', 'b'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.t_00751 VALUES ('2000-01-03', 'a', 'a') ('2000-01-04', 'b', 'b'); -SELECT * FROM test_00751.t_00751 ORDER BY date; -SELECT * FROM test_00751.t_mv_00751 ORDER BY date; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.t_00751 ORDER BY date; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.t_mv_00751 ORDER BY date; -DROP TABLE test_00751.t_00751; -DROP TABLE test_00751.t_mv_00751; -DROP TABLE test_00751.u_00751; -DROP TABLE test_00751.v_00751; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.t_00751; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.t_mv_00751; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.u_00751; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.v_00751; -DROP DATABASE test_00751; diff --git a/tests/queries/0_stateless/00800_versatile_storage_join.sql b/tests/queries/0_stateless/00800_versatile_storage_join.sql index 3690ea6bf89..cccc655e34b 100644 --- a/tests/queries/0_stateless/00800_versatile_storage_join.sql +++ b/tests/queries/0_stateless/00800_versatile_storage_join.sql @@ -1,8 +1,3 @@ --- Tags: no-parallel - -CREATE DATABASE IF NOT EXISTS test_00800; - -USE test_00800; DROP TABLE IF EXISTS join_any_inner; DROP TABLE IF EXISTS join_any_left; @@ -54,24 +49,22 @@ SELECT joinGet('join_string_key', 'x', 'abc'), joinGet('join_string_key', 'k', ' USE default; -DROP TABLE test_00800.join_any_inner; -DROP TABLE test_00800.join_any_left; -DROP TABLE test_00800.join_any_left_null; -DROP TABLE test_00800.join_all_inner; -DROP TABLE test_00800.join_all_left; -DROP TABLE test_00800.join_string_key; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_any_inner; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_any_left; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_any_left_null; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_all_inner; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_all_left; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_string_key; -- test provided by Alexander Zaitsev -DROP TABLE IF EXISTS test_00800.join_test; -CREATE TABLE test_00800.join_test (a UInt8, b UInt8) Engine = Join(ANY, LEFT, a); +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.join_test; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.join_test (a UInt8, b UInt8) Engine = Join(ANY, LEFT, a); -USE test_00800; +USE {CLICKHOUSE_DATABASE:Identifier}; select joinGet('join_test', 'b', 1); USE system; -SELECT joinGet('test_00800.join_test', 'b', 1); +SELECT joinGet({CLICKHOUSE_DATABASE:String} || '.join_test', 'b', 1); USE default; -DROP TABLE test_00800.join_test; - -DROP DATABASE test_00800; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_test; diff --git a/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql b/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql index e7f034131a2..2044a9b8d22 100644 --- a/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql +++ b/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql @@ -1,7 +1,4 @@ --- Tags: global, no-parallel -CREATE DATABASE IF NOT EXISTS test_00857; -USE test_00857; DROP TABLE IF EXISTS local_table; DROP TABLE IF EXISTS other_table; @@ -27,7 +24,7 @@ INSERT INTO other_table VALUES(100, 'One Hundred', now(), 1000); INSERT INTO other_table VALUES(200, 'Two Hundred', now(), 2000); select t2.name from remote('127.0.0.2', currentDatabase(), 'local_table') as t1 -left join test_00857.other_table as t2 -- FIXME: doesn't work properly on remote without explicit database prefix +left join {CLICKHOUSE_DATABASE:Identifier}.other_table as t2 -- FIXME: doesn't work properly on remote without explicit database prefix on t1.oth_id = t2.id order by t2.name; @@ -58,4 +55,3 @@ order by other_table.name; DROP TABLE local_table; DROP TABLE other_table; -DROP DATABASE test_00857; diff --git a/tests/queries/0_stateless/01012_show_tables_limit.sql b/tests/queries/0_stateless/01012_show_tables_limit.sql index 82cde6917d0..18a11f66d08 100644 --- a/tests/queries/0_stateless/01012_show_tables_limit.sql +++ b/tests/queries/0_stateless/01012_show_tables_limit.sql @@ -1,22 +1,15 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS test_show_limit; - -CREATE DATABASE test_show_limit; - -CREATE TABLE test_show_limit.test1 (test UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_limit.test2 (test UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_limit.test3 (test UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_limit.test4 (test UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_limit.test5 (test UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_limit.test6 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test1 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test2 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test3 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test4 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test5 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test6 (test UInt8) ENGINE = TinyLog; SELECT '*** Should show 6: ***'; -SHOW TABLES FROM test_show_limit; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier}; SELECT '*** Should show 2: ***'; -SHOW TABLES FROM test_show_limit LIMIT 2; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} LIMIT 2; SELECT '*** Should show 4: ***'; -SHOW TABLES FROM test_show_limit LIMIT 2 * 2; - -DROP DATABASE test_show_limit; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} LIMIT 2 * 2; diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_select.reference b/tests/queries/0_stateless/01018_ddl_dictionaries_select.reference index f69302fb90f..17a77b91d45 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_select.reference +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_select.reference @@ -15,12 +15,12 @@ dict1 Dictionary dict2 Dictionary table_for_dict MergeTree -database_for_dict dict1 ComplexKeyCache -database_for_dict dict2 Hashed +default dict1 ComplexKeyCache +default dict2 Hashed 6 6 6 6 6 -database_for_dict.dict3 6 +default.dict3 6 6 diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql index 748c733bf9a..523b057d4e1 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql @@ -1,13 +1,9 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-fasttest SET send_logs_level = 'fatal'; SET check_table_dependencies=0; -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - -CREATE TABLE database_for_dict.table_for_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict ( key_column UInt64, second_column UInt8, @@ -17,9 +13,9 @@ CREATE TABLE database_for_dict.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); -CREATE DICTIONARY database_for_dict.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -27,35 +23,35 @@ CREATE DICTIONARY database_for_dict.dict1 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -SELECT second_column FROM database_for_dict.dict1 WHERE key_column = 11; -SELECT dictGetString('database_for_dict.dict1', 'third_column', toUInt64(12)); -SELECT third_column FROM database_for_dict.dict1 WHERE key_column = 12; -SELECT dictGetFloat64('database_for_dict.dict1', 'fourth_column', toUInt64(14)); -SELECT fourth_column FROM database_for_dict.dict1 WHERE key_column = 14; +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(11)); +SELECT second_column FROM {CLICKHOUSE_DATABASE:Identifier}.dict1 WHERE key_column = 11; +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict1', 'third_column', toUInt64(12)); +SELECT third_column FROM {CLICKHOUSE_DATABASE:Identifier}.dict1 WHERE key_column = 12; +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'fourth_column', toUInt64(14)); +SELECT fourth_column FROM {CLICKHOUSE_DATABASE:Identifier}.dict1 WHERE key_column = 14; -SELECT count(distinct(dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(number)))) from numbers(100); +SELECT count(distinct(dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(number)))) from numbers(100); -DETACH DICTIONARY database_for_dict.dict1; +DETACH DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -- {serverError 36} +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(11)); -- {serverError 36} -ATTACH DICTIONARY database_for_dict.dict1; +ATTACH DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(11)); -DROP DICTIONARY database_for_dict.dict1; +DROP DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -- {serverError 36} +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(11)); -- {serverError 36} -- SOURCE(CLICKHOUSE(...)) uses default params if not specified -DROP DICTIONARY IF EXISTS database_for_dict.dict1; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict1; -CREATE DICTIONARY database_for_dict.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -63,17 +59,17 @@ CREATE DICTIONARY database_for_dict.dict1 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(11)); -SELECT count(distinct(dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(number)))) from numbers(100); +SELECT count(distinct(dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(number)))) from numbers(100); -DROP DICTIONARY database_for_dict.dict1; +DROP DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -CREATE DICTIONARY database_for_dict.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -81,69 +77,68 @@ CREATE DICTIONARY database_for_dict.dict1 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column, third_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 1)); -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121')); -SELECT dictGetFloat64('database_for_dict.dict1', 'fourth_column', tuple(toUInt64(14), '196')); +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', tuple(toUInt64(11), '121')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'fourth_column', tuple(toUInt64(14), '196')); -DETACH DICTIONARY database_for_dict.dict1; +DETACH DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121')); -- {serverError 36} +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', tuple(toUInt64(11), '121')); -- {serverError 36} -ATTACH DICTIONARY database_for_dict.dict1; +ATTACH DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121')); +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', tuple(toUInt64(11), '121')); -CREATE DICTIONARY database_for_dict.dict2 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict2 ( key_column UInt64 DEFAULT 0, some_column String EXPRESSION toString(fourth_column), fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT dictGetString('database_for_dict.dict2', 'some_column', toUInt64(12)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict2', 'some_column', toUInt64(12)); -SELECT name, engine FROM system.tables WHERE database = 'database_for_dict' ORDER BY name; +-- NOTE: database = currentDatabase() is not mandatory +SELECT name, engine FROM system.tables WHERE database = {CLICKHOUSE_DATABASE:String} ORDER BY name; -SELECT database, name, type FROM system.dictionaries WHERE database = 'database_for_dict' ORDER BY name; +SELECT database, name, type FROM system.dictionaries WHERE database = {CLICKHOUSE_DATABASE:String} ORDER BY name; -- check dictionary will not update -CREATE DICTIONARY database_for_dict.dict3 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict3 ( key_column UInt64 DEFAULT 0, some_column String EXPRESSION toString(fourth_column), fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(0) LAYOUT(HASHED()); -SELECT dictGetString('database_for_dict.dict3', 'some_column', toUInt64(12)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict3', 'some_column', toUInt64(12)); -- dictGet with table name -USE database_for_dict; +USE {CLICKHOUSE_DATABASE:Identifier}; SELECT dictGetString(dict3, 'some_column', toUInt64(12)); -SELECT dictGetString(database_for_dict.dict3, 'some_column', toUInt64(12)); +SELECT dictGetString({CLICKHOUSE_DATABASE:Identifier}.dict3, 'some_column', toUInt64(12)); SELECT dictGetString(default.dict3, 'some_column', toUInt64(12)); -- {serverError 36} SELECT dictGet(dict3, 'some_column', toUInt64(12)); -SELECT dictGet(database_for_dict.dict3, 'some_column', toUInt64(12)); +SELECT dictGet({CLICKHOUSE_DATABASE:Identifier}.dict3, 'some_column', toUInt64(12)); SELECT dictGet(default.dict3, 'some_column', toUInt64(12)); -- {serverError 36} USE default; -- alias should be handled correctly -SELECT 'database_for_dict.dict3' as n, dictGet(n, 'some_column', toUInt64(12)); +SELECT {CLICKHOUSE_DATABASE:String} || '.dict3' as n, dictGet(n, 'some_column', toUInt64(12)); -DROP TABLE database_for_dict.table_for_dict; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict; SYSTEM RELOAD DICTIONARIES; -- {serverError 60} -SELECT dictGetString('database_for_dict.dict3', 'some_column', toUInt64(12)); - -DROP DATABASE IF EXISTS database_for_dict; +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict3', 'some_column', toUInt64(12)); diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql index 1877c289d3f..51e1eb2e12f 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql @@ -1,14 +1,10 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-fasttest SET send_logs_level = 'fatal'; -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - SELECT '***date dict***'; -CREATE TABLE database_for_dict.date_table +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.date_table ( CountryID UInt64, StartDate Date, @@ -18,11 +14,11 @@ CREATE TABLE database_for_dict.date_table ENGINE = MergeTree() ORDER BY CountryID; -INSERT INTO database_for_dict.date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); -INSERT INTO database_for_dict.date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); -INSERT INTO database_for_dict.date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), 0.46); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), 0.46); -CREATE DICTIONARY database_for_dict.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 ( CountryID UInt64, StartDate Date, @@ -30,19 +26,19 @@ CREATE DICTIONARY database_for_dict.dict1 Tax Float64 ) PRIMARY KEY CountryID -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB currentDatabase())) LIFETIME(MIN 1 MAX 1000) LAYOUT(RANGE_HASHED()) RANGE(MIN StartDate MAX EndDate); -SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(1), toDate('2019-05-15')); -SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(1), toDate('2019-05-29')); -SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(2), toDate('2019-05-29')); -SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(2), toDate('2019-05-31')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'Tax', toUInt64(1), toDate('2019-05-15')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'Tax', toUInt64(1), toDate('2019-05-29')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'Tax', toUInt64(2), toDate('2019-05-29')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'Tax', toUInt64(2), toDate('2019-05-31')); SELECT '***datetime dict***'; -CREATE TABLE database_for_dict.datetime_table +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.datetime_table ( CountryID UInt64, StartDate DateTime, @@ -52,11 +48,11 @@ CREATE TABLE database_for_dict.datetime_table ENGINE = MergeTree() ORDER BY CountryID; -INSERT INTO database_for_dict.datetime_table VALUES(1, toDateTime('2019-05-05 00:00:00'), toDateTime('2019-05-20 00:00:00'), 0.33); -INSERT INTO database_for_dict.datetime_table VALUES(1, toDateTime('2019-05-21 00:00:00'), toDateTime('2019-05-30 00:00:00'), 0.42); -INSERT INTO database_for_dict.datetime_table VALUES(2, toDateTime('2019-05-21 00:00:00'), toDateTime('2019-05-30 00:00:00'), 0.46); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.datetime_table VALUES(1, toDateTime('2019-05-05 00:00:00'), toDateTime('2019-05-20 00:00:00'), 0.33); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.datetime_table VALUES(1, toDateTime('2019-05-21 00:00:00'), toDateTime('2019-05-30 00:00:00'), 0.42); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.datetime_table VALUES(2, toDateTime('2019-05-21 00:00:00'), toDateTime('2019-05-30 00:00:00'), 0.46); -CREATE DICTIONARY database_for_dict.dict2 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict2 ( CountryID UInt64, StartDate DateTime, @@ -64,19 +60,19 @@ CREATE DICTIONARY database_for_dict.dict2 Tax Float64 ) PRIMARY KEY CountryID -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'datetime_table' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'datetime_table' DB currentDatabase())) LIFETIME(MIN 1 MAX 1000) LAYOUT(RANGE_HASHED()) RANGE(MIN StartDate MAX EndDate); -SELECT dictGetFloat64('database_for_dict.dict2', 'Tax', toUInt64(1), toDateTime('2019-05-15 00:00:00')); -SELECT dictGetFloat64('database_for_dict.dict2', 'Tax', toUInt64(1), toDateTime('2019-05-29 00:00:00')); -SELECT dictGetFloat64('database_for_dict.dict2', 'Tax', toUInt64(2), toDateTime('2019-05-29 00:00:00')); -SELECT dictGetFloat64('database_for_dict.dict2', 'Tax', toUInt64(2), toDateTime('2019-05-31 00:00:00')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict2', 'Tax', toUInt64(1), toDateTime('2019-05-15 00:00:00')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict2', 'Tax', toUInt64(1), toDateTime('2019-05-29 00:00:00')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict2', 'Tax', toUInt64(2), toDateTime('2019-05-29 00:00:00')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict2', 'Tax', toUInt64(2), toDateTime('2019-05-31 00:00:00')); SELECT '***hierarchy dict***'; -CREATE TABLE database_for_dict.table_with_hierarchy +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_with_hierarchy ( RegionID UInt64, ParentRegionID UInt64, @@ -85,24 +81,24 @@ CREATE TABLE database_for_dict.table_with_hierarchy ENGINE = MergeTree() ORDER BY RegionID; -INSERT INTO database_for_dict.table_with_hierarchy VALUES (3, 2, 'Hamovniki'), (2, 1, 'Moscow'), (1, 10000, 'Russia') (7, 10000, 'Ulan-Ude'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_with_hierarchy VALUES (3, 2, 'Hamovniki'), (2, 1, 'Moscow'), (1, 10000, 'Russia') (7, 10000, 'Ulan-Ude'); -CREATE DICTIONARY database_for_dict.dictionary_with_hierarchy +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dictionary_with_hierarchy ( RegionID UInt64, ParentRegionID UInt64 HIERARCHICAL, RegionName String ) PRIMARY KEY RegionID -SOURCE(CLICKHOUSE(host 'localhost' port tcpPort() user 'default' db 'database_for_dict' table 'table_with_hierarchy')) +SOURCE(CLICKHOUSE(host 'localhost' port tcpPort() user 'default' db currentDatabase() table 'table_with_hierarchy')) LAYOUT(HASHED()) LIFETIME(MIN 1 MAX 1000); -SELECT dictGetString('database_for_dict.dictionary_with_hierarchy', 'RegionName', toUInt64(2)); -SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', toUInt64(3)); -SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(3), toUInt64(2)); -SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(7), toUInt64(10000)); -SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(1), toUInt64(5)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dictionary_with_hierarchy', 'RegionName', toUInt64(2)); +SELECT dictGetHierarchy({CLICKHOUSE_DATABASE:String} || '.dictionary_with_hierarchy', toUInt64(3)); +SELECT dictIsIn({CLICKHOUSE_DATABASE:String} || '.dictionary_with_hierarchy', toUInt64(3), toUInt64(2)); +SELECT dictIsIn({CLICKHOUSE_DATABASE:String} || '.dictionary_with_hierarchy', toUInt64(7), toUInt64(10000)); +SELECT dictIsIn({CLICKHOUSE_DATABASE:String} || '.dictionary_with_hierarchy', toUInt64(1), toUInt64(5)); -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql index e72e113f859..d958d7d58ef 100644 --- a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql +++ b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql @@ -1,12 +1,7 @@ --- Tags: no-parallel SET send_logs_level = 'fatal'; -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - -CREATE TABLE database_for_dict.table_for_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict ( key_column UInt64, second_column UInt8, @@ -16,9 +11,9 @@ CREATE TABLE database_for_dict.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); -CREATE DICTIONARY database_for_dict.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -26,13 +21,13 @@ CREATE DICTIONARY database_for_dict.dict1 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SELECT count(*) from database_for_dict.dict1; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; -CREATE DICTIONARY database_for_dict.dict2 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict2 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -40,20 +35,20 @@ CREATE DICTIONARY database_for_dict.dict2 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict1' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict1' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM database_for_dict.dict2; +SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict2; -INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100, 100); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100, 100); SYSTEM RELOAD DICTIONARIES; -SELECT count(*) from database_for_dict.dict2; -SELECT count(*) from database_for_dict.dict1; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; -CREATE DICTIONARY database_for_dict.dict3 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict3 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -61,22 +56,22 @@ CREATE DICTIONARY database_for_dict.dict3 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict2' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict2' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM database_for_dict.dict3; +SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict3; -INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(200, 100); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(200, 100); SYSTEM RELOAD DICTIONARIES; -SELECT count(*) from database_for_dict.dict3; -SELECT count(*) from database_for_dict.dict2; -SELECT count(*) from database_for_dict.dict1; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict3; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; -CREATE DICTIONARY database_for_dict.dict4 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict4 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -84,17 +79,17 @@ CREATE DICTIONARY database_for_dict.dict4 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'non_existing_table' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'non_existing_table' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM database_for_dict.dict4; -- {serverError 60} +SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict4; -- {serverError 60} -SELECT name from system.tables WHERE database = 'database_for_dict' ORDER BY name; -SELECT name from system.dictionaries WHERE database = 'database_for_dict' ORDER BY name; +SELECT name from system.tables WHERE database = currentDatabase() ORDER BY name; +SELECT name from system.dictionaries WHERE database = currentDatabase() ORDER BY name; -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; -SELECT count(*) from database_for_dict.dict3; --{serverError 81} -SELECT count(*) from database_for_dict.dict2; --{serverError 81} -SELECT count(*) from database_for_dict.dict1; --{serverError 81} +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict3; --{serverError 81} +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; --{serverError 81} +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; --{serverError 81} diff --git a/tests/queries/0_stateless/01018_ip_dictionary_long.sql b/tests/queries/0_stateless/01018_ip_dictionary_long.sql index 647c36429cc..bb7f120163c 100644 --- a/tests/queries/0_stateless/01018_ip_dictionary_long.sql +++ b/tests/queries/0_stateless/01018_ip_dictionary_long.sql @@ -1,13 +1,9 @@ --- Tags: long, no-parallel +-- Tags: long SET send_logs_level = 'fatal'; -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - SELECT '***ipv4 trie dict***'; -CREATE TABLE database_for_dict.table_ipv4_trie +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie ( prefix String, asn UInt32, @@ -16,648 +12,648 @@ CREATE TABLE database_for_dict.table_ipv4_trie engine = TinyLog; -- numbers reordered to test sorting criteria too -INSERT INTO database_for_dict.table_ipv4_trie +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie SELECT '255.255.255.255/' || toString((number + 1) * 13 % 33) AS prefix, toUInt32((number + 1) * 13 % 33) AS asn, 'NA' as cca2 FROM system.numbers LIMIT 33; -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.2', 1272, 'RU'); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/8', 1270, 'RU'); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.2', 1272, 'RU'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.0/8', 1270, 'RU'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); -- non-unique entries will be squashed into one -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('101.79.55.22', 11212, 'UK'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('101.79.55.22', 11212, 'UK'); -CREATE DICTIONARY database_for_dict.dict_ipv4_trie +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie ( prefix String, asn UInt32, cca2 String ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ipv4_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ipv4_trie')) LAYOUT(IP_TRIE()) LIFETIME(MIN 10 MAX 100); -- fuzzer -SELECT '127.0.0.0/24' = dictGetString('database_for_dict.dict_ipv4_trie', 'prefixprefixprefixprefix', tuple(IPv4StringToNumOrDefault('127.0.0.0127.0.0.0'))); -- { serverError 36 } +SELECT '127.0.0.0/24' = dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefixprefixprefixprefix', tuple(IPv4StringToNumOrDefault('127.0.0.0127.0.0.0'))); -- { serverError 36 } -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('0.0.0.0'))); -SELECT 1 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('128.0.0.0'))); -SELECT 2 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('192.0.0.0'))); -SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('224.0.0.0'))); -SELECT 4 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('240.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('248.0.0.0'))); -SELECT 6 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('252.0.0.0'))); -SELECT 7 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('254.0.0.0'))); -SELECT 8 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.0.0.0'))); -SELECT 9 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.128.0.0'))); -SELECT 10 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.192.0.0'))); -SELECT 11 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.224.0.0'))); -SELECT 12 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.240.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.248.0.0'))); -SELECT 14 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.252.0.0'))); -SELECT 15 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.254.0.0'))); -SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.0.0'))); -SELECT 17 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.128.0'))); -SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.192.0'))); -SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.224.0'))); -SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.240.0'))); -SELECT 21 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.248.0'))); -SELECT 22 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.252.0'))); -SELECT 23 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.254.0'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.0'))); -SELECT 25 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.128'))); -SELECT 26 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.192'))); -SELECT 27 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.224'))); -SELECT 28 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.240'))); -SELECT 29 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.248'))); -SELECT 30 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.252'))); -SELECT 31 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.254'))); -SELECT 32 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.255'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('0.0.0.0'))); +SELECT 1 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('128.0.0.0'))); +SELECT 2 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('192.0.0.0'))); +SELECT 3 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('224.0.0.0'))); +SELECT 4 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('240.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('248.0.0.0'))); +SELECT 6 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('252.0.0.0'))); +SELECT 7 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('254.0.0.0'))); +SELECT 8 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.0.0.0'))); +SELECT 9 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.128.0.0'))); +SELECT 10 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.192.0.0'))); +SELECT 11 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.224.0.0'))); +SELECT 12 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.240.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.248.0.0'))); +SELECT 14 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.252.0.0'))); +SELECT 15 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.254.0.0'))); +SELECT 16 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.0.0'))); +SELECT 17 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.128.0'))); +SELECT 18 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.192.0'))); +SELECT 19 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.224.0'))); +SELECT 20 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.240.0'))); +SELECT 21 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.248.0'))); +SELECT 22 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.252.0'))); +SELECT 23 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.254.0'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.0'))); +SELECT 25 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.128'))); +SELECT 26 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.192'))); +SELECT 27 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.224'))); +SELECT 28 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.240'))); +SELECT 29 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.248'))); +SELECT 30 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.252'))); +SELECT 31 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.254'))); +SELECT 32 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.255'))); -SELECT 'RU' == dictGetString('database_for_dict.dict_ipv4_trie', 'cca2', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 'RU' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'cca2', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.0'))); -SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 1272 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.2'))); -SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.3'))); -SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.255'))); +SELECT 1270 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 1270 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 1272 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.2'))); +SELECT 1270 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.3'))); +SELECT 1270 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.255'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.2'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.3'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.255'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.2'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.3'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.255'))); -SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('101.79.55.22'))); -SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:654f:3716'))); -SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); +SELECT 11212 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('101.79.55.22'))); +SELECT 11212 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:654f:3716'))); +SELECT 11212 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); -SELECT 11211 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('202.79.32.2'))); +SELECT 11211 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('202.79.32.2'))); -- check that dictionary works with aliased types `IPv4` and `IPv6` -SELECT 11211 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(toIPv4('202.79.32.2'))); -SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(toIPv6('::ffff:101.79.55.22'))); +SELECT 11211 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(toIPv4('202.79.32.2'))); +SELECT 11212 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(toIPv6('::ffff:101.79.55.22'))); -CREATE TABLE database_for_dict.table_from_ipv4_trie_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict ( prefix String, asn UInt32, cca2 String -) ENGINE = Dictionary(database_for_dict.dict_ipv4_trie); +) ENGINE = Dictionary({CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie); -SELECT 1272 == asn AND 'RU' == cca2 FROM database_for_dict.table_from_ipv4_trie_dict +SELECT 1272 == asn AND 'RU' == cca2 FROM {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict WHERE prefix == '127.0.0.2/32'; -SELECT 37 == COUNT(*) FROM database_for_dict.table_from_ipv4_trie_dict; -SELECT 37 == COUNT(DISTINCT prefix) FROM database_for_dict.table_from_ipv4_trie_dict; +SELECT 37 == COUNT(*) FROM {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict; +SELECT 37 == COUNT(DISTINCT prefix) FROM {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict; -DROP TABLE IF EXISTS database_for_dict.table_from_ipv4_trie_dict; -DROP DICTIONARY IF EXISTS database_for_dict.dict_ipv4_trie; -DROP TABLE IF EXISTS database_for_dict.table_ipv4_trie; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie; SELECT '***ipv4 trie dict mask***'; -CREATE TABLE database_for_dict.table_ipv4_trie +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie ( prefix String, val UInt32 ) engine = TinyLog; -INSERT INTO database_for_dict.table_ipv4_trie +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie SELECT '255.255.255.255/' || toString(number) AS prefix, toUInt32(number) AS val FROM VALUES ('number UInt32', 5, 13, 24, 30); -CREATE DICTIONARY database_for_dict.dict_ipv4_trie +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie ( prefix String, val UInt32 ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ipv4_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ipv4_trie')) LAYOUT(IP_TRIE()) LIFETIME(MIN 10 MAX 100); -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('0.0.0.0'))); -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('128.0.0.0'))); -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('192.0.0.0'))); -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('224.0.0.0'))); -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('240.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('248.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('252.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('254.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.128.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.192.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.224.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.240.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.248.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.252.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.254.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.128.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.192.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.224.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.240.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.248.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.252.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.254.0'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.0'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.128'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.192'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.224'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.240'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.248'))); -SELECT 30 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.252'))); -SELECT 30 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.254'))); -SELECT 30 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.255'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('0.0.0.0'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('128.0.0.0'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('192.0.0.0'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('224.0.0.0'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('240.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('248.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('252.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('254.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.128.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.192.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.224.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.240.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.248.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.252.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.254.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.128.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.192.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.224.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.240.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.248.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.252.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.254.0'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.0'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.128'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.192'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.224'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.240'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.248'))); +SELECT 30 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.252'))); +SELECT 30 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.254'))); +SELECT 30 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.255'))); -DROP DICTIONARY IF EXISTS database_for_dict.dict_ipv4_trie; -DROP TABLE IF EXISTS database_for_dict.table_from_ipv4_trie_dict; -DROP TABLE IF EXISTS database_for_dict.table_ipv4_trie; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie; SELECT '***ipv4 trie dict pt2***'; -CREATE TABLE database_for_dict.table_ipv4_trie ( prefix String, val UInt32 ) engine = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie ( prefix String, val UInt32 ) engine = TinyLog; -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/8', 1); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/16', 2); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/24', 3); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.1/32', 4); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.127.0/32', 5); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.128.1/32', 6); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.255.0/32', 7); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.255.1/32', 8); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.255.255/32', 9); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.1.0.0/16', 10); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.1.1.0', 11); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.1.255.0/24', 12); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.254.0.0/15', 13); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.254.0.127', 14); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.0.0/16', 15); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.0/24', 16); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.1/32', 17); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.10/32', 18); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.128/25', 19); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.255.128/32', 20); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.255.255/32', 21); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.0/8', 1); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.0/16', 2); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.0/24', 3); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.1/32', 4); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.127.0/32', 5); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.128.1/32', 6); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.255.0/32', 7); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.255.1/32', 8); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.255.255/32', 9); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.1.0.0/16', 10); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.1.1.0', 11); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.1.255.0/24', 12); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.254.0.0/15', 13); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.254.0.127', 14); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.0.0/16', 15); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.128.0/24', 16); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.128.1/32', 17); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.128.10/32', 18); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.128.128/25', 19); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.255.128/32', 20); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.255.255/32', 21); -CREATE DICTIONARY database_for_dict.dict_ipv4_trie ( prefix String, val UInt32 ) +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie ( prefix String, val UInt32 ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ipv4_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ipv4_trie')) LAYOUT(IP_TRIE(ACCESS_TO_KEY_FROM_ATTRIBUTES 1)) LIFETIME(MIN 10 MAX 100); -SELECT '127.0.0.0/24' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.0'))); -SELECT '127.0.0.1/32' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT '127.0.0.0/24' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.127'))); -SELECT '127.0.0.0/16' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.255.127'))); -SELECT '127.255.0.0/16' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.127.127'))); -SELECT '127.255.128.0/24' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.9'))); -SELECT '127.255.128.0/24' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.127'))); -SELECT '127.255.128.10/32' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.10'))); -SELECT '127.255.128.128/25' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.255'))); -SELECT '127.255.255.128/32' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.255.128'))); +SELECT '127.0.0.0/24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT '127.0.0.1/32' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT '127.0.0.0/24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.127'))); +SELECT '127.0.0.0/16' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.255.127'))); +SELECT '127.255.0.0/16' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.127.127'))); +SELECT '127.255.128.0/24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.9'))); +SELECT '127.255.128.0/24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.127'))); +SELECT '127.255.128.10/32' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.10'))); +SELECT '127.255.128.128/25' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.255'))); +SELECT '127.255.255.128/32' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.255.128'))); -SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.0'))); -SELECT 4 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.127'))); -SELECT 2 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.255.127'))); -SELECT 15 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.127.127'))); -SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.9'))); -SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.127'))); -SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.10'))); -SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.255'))); -SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.255.128'))); +SELECT 3 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 4 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 3 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.127'))); +SELECT 2 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.255.127'))); +SELECT 15 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.127.127'))); +SELECT 16 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.9'))); +SELECT 16 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.127'))); +SELECT 18 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.10'))); +SELECT 19 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.255'))); +SELECT 20 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.255.128'))); -SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:0'))); -SELECT 4 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:1'))); -SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:7f'))); -SELECT 2 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:ff7f'))); -SELECT 15 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:7f7f'))); -SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:8009'))); -SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:807f'))); -SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:800a'))); -SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:80ff'))); -SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:ff80'))); +SELECT 3 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:0'))); +SELECT 4 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:1'))); +SELECT 3 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:7f'))); +SELECT 2 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:ff7f'))); +SELECT 15 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:7f7f'))); +SELECT 16 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:8009'))); +SELECT 16 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:807f'))); +SELECT 18 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:800a'))); +SELECT 19 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:80ff'))); +SELECT 20 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:ff80'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.127'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.255.127'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.127.127'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.9'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.127'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.10'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.255'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.255.128'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.127'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.255.127'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.127.127'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.9'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.127'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.10'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.255'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.255.128'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('128.127.127.127'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('128.127.127.0'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('255.127.127.0'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('255.0.0.0'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('0.0.0.0'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('1.1.1.1'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('128.127.127.127'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('128.127.127.0'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('255.127.127.0'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('255.0.0.0'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('0.0.0.0'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('1.1.1.1'))); SELECT '***ipv6 trie dict***'; -CREATE TABLE database_for_dict.table_ip_trie +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie ( prefix String, val String ) engine = TinyLog; -INSERT INTO database_for_dict.table_ip_trie VALUES ('101.79.55.22', 'JA'), ('127.0.0.1', 'RU'), ('2620:0:870::/48', 'US'), ('2a02:6b8:1::/48', 'UK'), ('2001:db8::/32', 'ZZ'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie VALUES ('101.79.55.22', 'JA'), ('127.0.0.1', 'RU'), ('2620:0:870::/48', 'US'), ('2a02:6b8:1::/48', 'UK'), ('2001:db8::/32', 'ZZ'); -INSERT INTO database_for_dict.table_ip_trie +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie SELECT 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/' || toString((number + 1) * 13 % 129) AS prefix, toString((number + 1) * 13 % 129) AS val FROM system.numbers LIMIT 129; -CREATE DICTIONARY database_for_dict.dict_ip_trie +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ip_trie ( prefix String, val String ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ip_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ip_trie')) LAYOUT(IP_TRIE(ACCESS_TO_KEY_FROM_ATTRIBUTES 1)) LIFETIME(MIN 10 MAX 100); -SELECT 'US' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2620:0:870::'))); -SELECT 'UK' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2a02:6b8:1::'))); -SELECT 'ZZ' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2001:db8::'))); -SELECT 'ZZ' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2001:db8:ffff::'))); +SELECT 'US' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('2620:0:870::'))); +SELECT 'UK' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('2a02:6b8:1::'))); +SELECT 'ZZ' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('2001:db8::'))); +SELECT 'ZZ' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('2001:db8:ffff::'))); -SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff::'))); -SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff:ffff::'))); -SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff:1::'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff::'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff:ffff::'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff:1::'))); -SELECT '0' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('654f:3716::'))); +SELECT '0' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('654f:3716::'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:654f:3716'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('101.79.55.22'))); -SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('::ffff:127.0.0.1'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:654f:3716'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('101.79.55.22'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('::ffff:127.0.0.1'))); -SELECT '2620:0:870::/48' == dictGetString('database_for_dict.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2620:0:870::'))); -SELECT '2a02:6b8:1::/48' == dictGetString('database_for_dict.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2a02:6b8:1::1'))); -SELECT '2001:db8::/32' == dictGetString('database_for_dict.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2001:db8::1'))); -SELECT '::ffff:101.79.55.22/128' == dictGetString('database_for_dict.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('::ffff:654f:3716'))); -SELECT '::ffff:101.79.55.22/128' == dictGetString('database_for_dict.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); +SELECT '2620:0:870::/48' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2620:0:870::'))); +SELECT '2a02:6b8:1::/48' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2a02:6b8:1::1'))); +SELECT '2001:db8::/32' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2001:db8::1'))); +SELECT '::ffff:101.79.55.22/128' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('::ffff:654f:3716'))); +SELECT '::ffff:101.79.55.22/128' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); -SELECT '0' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::0'))); -SELECT '1' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('8000::'))); -SELECT '2' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('c000::'))); -SELECT '3' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('e000::'))); -SELECT '4' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('f000::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('f800::'))); -SELECT '6' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fc00::'))); -SELECT '7' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fe00::'))); -SELECT '8' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff00::'))); -SELECT '9' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff80::'))); -SELECT '10' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffc0::'))); -SELECT '11' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffe0::'))); -SELECT '12' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff0::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff8::'))); -SELECT '14' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffc::'))); -SELECT '15' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffe::'))); -SELECT '16' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff::'))); -SELECT '17' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:8000::'))); -SELECT '18' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); -SELECT '19' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); -SELECT '20' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); -SELECT '21' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); -SELECT '22' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); -SELECT '18' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); -SELECT '19' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); -SELECT '20' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); -SELECT '21' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); -SELECT '22' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); -SELECT '23' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fe00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff00::'))); -SELECT '25' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff80::'))); -SELECT '26' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffc0::'))); -SELECT '27' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffe0::'))); -SELECT '28' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff0::'))); -SELECT '29' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff8::'))); -SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffc::'))); -SELECT '31' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffe::'))); -SELECT '32' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff::'))); -SELECT '33' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:8000::'))); -SELECT '34' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:c000::'))); -SELECT '35' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:e000::'))); -SELECT '36' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f000::'))); -SELECT '37' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f800::'))); -SELECT '38' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fc00::'))); -SELECT '39' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fe00::'))); -SELECT '40' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff00::'))); -SELECT '41' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff80::'))); -SELECT '42' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffc0::'))); -SELECT '43' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffe0::'))); -SELECT '44' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff0::'))); -SELECT '45' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff8::'))); -SELECT '46' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffc::'))); -SELECT '47' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffe::'))); -SELECT '48' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:8000::'))); -SELECT '50' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:c000::'))); -SELECT '51' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:e000::'))); -SELECT '52' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f000::'))); -SELECT '53' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f800::'))); -SELECT '54' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fc00::'))); -SELECT '55' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fe00::'))); -SELECT '56' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff00::'))); -SELECT '57' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff80::'))); -SELECT '58' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffc0::'))); -SELECT '59' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffe0::'))); -SELECT '60' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff0::'))); -SELECT '61' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff8::'))); -SELECT '62' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffc::'))); -SELECT '63' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffe::'))); -SELECT '64' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff::'))); -SELECT '65' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:8000::'))); -SELECT '66' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:c000::'))); -SELECT '67' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:e000::'))); -SELECT '68' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f000::'))); -SELECT '69' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f800::'))); -SELECT '70' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fc00::'))); -SELECT '71' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fe00::'))); -SELECT '72' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff00::'))); -SELECT '73' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff80::'))); -SELECT '74' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffc0::'))); -SELECT '75' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffe0::'))); -SELECT '76' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff0::'))); -SELECT '77' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff8::'))); -SELECT '78' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffc::'))); -SELECT '79' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffe::'))); -SELECT '80' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff::'))); -SELECT '81' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:8000::'))); -SELECT '82' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:c000::'))); -SELECT '83' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:e000::'))); -SELECT '84' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f000::'))); -SELECT '85' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f800::'))); -SELECT '86' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fc00::'))); -SELECT '87' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fe00::'))); -SELECT '88' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff00::'))); -SELECT '89' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff80::'))); -SELECT '90' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffc0::'))); -SELECT '91' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffe0::'))); -SELECT '92' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff0::'))); -SELECT '93' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff8::'))); -SELECT '94' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffc::'))); -SELECT '95' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffe::'))); -SELECT '96' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff::'))); -SELECT '97' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:8000:0'))); -SELECT '98' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:c000:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:e000:0'))); -SELECT '100' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f000:0'))); -SELECT '101' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f800:0'))); -SELECT '102' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fc00:0'))); -SELECT '103' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fe00:0'))); -SELECT '104' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff00:0'))); -SELECT '105' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff80:0'))); -SELECT '106' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffc0:0'))); -SELECT '107' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffe0:0'))); -SELECT '108' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff0:0'))); -SELECT '109' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff8:0'))); -SELECT '110' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffc:0'))); -SELECT '111' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffe:0'))); -SELECT '112' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:0'))); -SELECT '113' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:8000'))); -SELECT '114' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:c000'))); -SELECT '115' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:e000'))); -SELECT '116' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f000'))); -SELECT '117' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f800'))); -SELECT '118' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fc00'))); -SELECT '119' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fe00'))); -SELECT '120' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff00'))); -SELECT '121' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff80'))); -SELECT '122' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffc0'))); -SELECT '123' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffe0'))); -SELECT '124' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff0'))); -SELECT '125' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff8'))); -SELECT '126' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffc'))); -SELECT '127' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'))); -SELECT '128' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'))); +SELECT '0' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::0'))); +SELECT '1' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('8000::'))); +SELECT '2' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('c000::'))); +SELECT '3' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('e000::'))); +SELECT '4' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('f000::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('f800::'))); +SELECT '6' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fc00::'))); +SELECT '7' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fe00::'))); +SELECT '8' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff00::'))); +SELECT '9' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff80::'))); +SELECT '10' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffc0::'))); +SELECT '11' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffe0::'))); +SELECT '12' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff0::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff8::'))); +SELECT '14' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffc::'))); +SELECT '15' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffe::'))); +SELECT '16' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff::'))); +SELECT '17' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:8000::'))); +SELECT '18' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); +SELECT '19' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); +SELECT '20' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); +SELECT '21' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); +SELECT '22' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); +SELECT '18' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); +SELECT '19' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); +SELECT '20' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); +SELECT '21' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); +SELECT '22' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); +SELECT '23' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fe00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff00::'))); +SELECT '25' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff80::'))); +SELECT '26' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffc0::'))); +SELECT '27' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffe0::'))); +SELECT '28' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff0::'))); +SELECT '29' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff8::'))); +SELECT '30' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffc::'))); +SELECT '31' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffe::'))); +SELECT '32' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff::'))); +SELECT '33' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:8000::'))); +SELECT '34' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:c000::'))); +SELECT '35' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:e000::'))); +SELECT '36' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f000::'))); +SELECT '37' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f800::'))); +SELECT '38' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fc00::'))); +SELECT '39' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fe00::'))); +SELECT '40' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff00::'))); +SELECT '41' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff80::'))); +SELECT '42' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffc0::'))); +SELECT '43' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffe0::'))); +SELECT '44' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff0::'))); +SELECT '45' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff8::'))); +SELECT '46' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffc::'))); +SELECT '47' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffe::'))); +SELECT '48' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:8000::'))); +SELECT '50' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:c000::'))); +SELECT '51' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:e000::'))); +SELECT '52' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f000::'))); +SELECT '53' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f800::'))); +SELECT '54' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fc00::'))); +SELECT '55' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fe00::'))); +SELECT '56' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff00::'))); +SELECT '57' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff80::'))); +SELECT '58' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffc0::'))); +SELECT '59' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffe0::'))); +SELECT '60' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff0::'))); +SELECT '61' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff8::'))); +SELECT '62' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffc::'))); +SELECT '63' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffe::'))); +SELECT '64' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff::'))); +SELECT '65' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:8000::'))); +SELECT '66' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:c000::'))); +SELECT '67' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:e000::'))); +SELECT '68' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f000::'))); +SELECT '69' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f800::'))); +SELECT '70' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fc00::'))); +SELECT '71' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fe00::'))); +SELECT '72' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff00::'))); +SELECT '73' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff80::'))); +SELECT '74' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffc0::'))); +SELECT '75' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffe0::'))); +SELECT '76' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff0::'))); +SELECT '77' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff8::'))); +SELECT '78' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffc::'))); +SELECT '79' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffe::'))); +SELECT '80' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff::'))); +SELECT '81' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:8000::'))); +SELECT '82' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:c000::'))); +SELECT '83' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:e000::'))); +SELECT '84' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f000::'))); +SELECT '85' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f800::'))); +SELECT '86' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fc00::'))); +SELECT '87' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fe00::'))); +SELECT '88' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff00::'))); +SELECT '89' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff80::'))); +SELECT '90' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffc0::'))); +SELECT '91' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffe0::'))); +SELECT '92' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff0::'))); +SELECT '93' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff8::'))); +SELECT '94' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffc::'))); +SELECT '95' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffe::'))); +SELECT '96' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff::'))); +SELECT '97' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:8000:0'))); +SELECT '98' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:c000:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:e000:0'))); +SELECT '100' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f000:0'))); +SELECT '101' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f800:0'))); +SELECT '102' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fc00:0'))); +SELECT '103' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fe00:0'))); +SELECT '104' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff00:0'))); +SELECT '105' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff80:0'))); +SELECT '106' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffc0:0'))); +SELECT '107' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffe0:0'))); +SELECT '108' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff0:0'))); +SELECT '109' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff8:0'))); +SELECT '110' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffc:0'))); +SELECT '111' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffe:0'))); +SELECT '112' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:0'))); +SELECT '113' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:8000'))); +SELECT '114' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:c000'))); +SELECT '115' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:e000'))); +SELECT '116' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f000'))); +SELECT '117' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f800'))); +SELECT '118' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fc00'))); +SELECT '119' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fe00'))); +SELECT '120' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff00'))); +SELECT '121' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff80'))); +SELECT '122' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffc0'))); +SELECT '123' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffe0'))); +SELECT '124' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff0'))); +SELECT '125' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff8'))); +SELECT '126' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffc'))); +SELECT '127' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'))); +SELECT '128' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'))); -CREATE TABLE database_for_dict.table_from_ip_trie_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_from_ip_trie_dict ( prefix String, val String -) ENGINE = Dictionary(database_for_dict.dict_ip_trie); +) ENGINE = Dictionary({CLICKHOUSE_DATABASE:Identifier}.dict_ip_trie); -SELECT MIN(val == 'US') FROM database_for_dict.table_from_ip_trie_dict +SELECT MIN(val == 'US') FROM {CLICKHOUSE_DATABASE:Identifier}.table_from_ip_trie_dict WHERE prefix == '2620:0:870::/48'; -SELECT 134 == COUNT(*) FROM database_for_dict.table_from_ip_trie_dict; +SELECT 134 == COUNT(*) FROM {CLICKHOUSE_DATABASE:Identifier}.table_from_ip_trie_dict; -DROP TABLE IF EXISTS database_for_dict.table_from_ip_trie_dict; -DROP DICTIONARY IF EXISTS database_for_dict.dict_ip_trie; -DROP TABLE IF EXISTS database_for_dict.table_ip_trie; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_from_ip_trie_dict; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict_ip_trie; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie; SELECT '***ipv6 trie dict mask***'; -CREATE TABLE database_for_dict.table_ip_trie +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie ( prefix String, val String ) engine = TinyLog; -INSERT INTO database_for_dict.table_ip_trie +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie SELECT 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/' || toString(number) AS prefix, toString(number) AS val FROM VALUES ('number UInt32', 5, 13, 24, 48, 49, 99, 127); -INSERT INTO database_for_dict.table_ip_trie VALUES ('101.79.55.22', 'JA'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie VALUES ('101.79.55.22', 'JA'); -INSERT INTO database_for_dict.table_ip_trie +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie SELECT '255.255.255.255/' || toString(number) AS prefix, toString(number) AS val FROM VALUES ('number UInt32', 5, 13, 24, 30); -CREATE DICTIONARY database_for_dict.dict_ip_trie +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ip_trie ( prefix String, val String ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ip_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ip_trie')) LAYOUT(IP_TRIE()) LIFETIME(MIN 10 MAX 100); -SELECT 0 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('::ffff:1:1'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('::ffff:1:1'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('654f:3716::'))); -SELECT 0 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('654f:3716::'))); -SELECT 0 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('654f:3716:ffff::'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('654f:3716::'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('654f:3716::'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('654f:3716:ffff::'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:654f:3716'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('101.79.55.22'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:654f:3716'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('101.79.55.22'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::0'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('8000::'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('c000::'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('e000::'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('f000::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('f800::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fc00::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fe00::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff00::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff80::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffc0::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffe0::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff0::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff8::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffc::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffe::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:8000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fe00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff80::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffc0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffe0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff8::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffc::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffe::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:8000::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:c000::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:e000::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f000::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f800::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fc00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fe00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff80::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffc0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffe0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff8::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffc::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffe::'))); -SELECT '48' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:8000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:c000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:e000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f800::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fc00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fe00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff80::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffc0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffe0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff8::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffc::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffe::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:8000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:c000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:e000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f800::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fc00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fe00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff80::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffc0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffe0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff8::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffc::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffe::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:8000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:c000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:e000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f800::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fc00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fe00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff80::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffc0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffe0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff8::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffc::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffe::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:8000:0'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:c000:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:e000:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f000:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f800:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fc00:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fe00:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff00:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff80:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffc0:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffe0:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff0:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff8:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffc:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffe:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:8000'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:c000'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:e000'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f000'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f800'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fc00'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fe00'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff00'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff80'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffc0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffe0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff8'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffc'))); -SELECT '127' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'))); -SELECT '127' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::0'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('8000::'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('c000::'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('e000::'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('f000::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('f800::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fc00::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fe00::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff00::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff80::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffc0::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffe0::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff0::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff8::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffc::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffe::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:8000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fe00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff80::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffc0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffe0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff8::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffc::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffe::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:8000::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:c000::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:e000::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f000::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f800::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fc00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fe00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff80::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffc0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffe0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff8::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffc::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffe::'))); +SELECT '48' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:8000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:c000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:e000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f800::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fc00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fe00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff80::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffc0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffe0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff8::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffc::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffe::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:8000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:c000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:e000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f800::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fc00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fe00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff80::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffc0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffe0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff8::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffc::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffe::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:8000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:c000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:e000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f800::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fc00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fe00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff80::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffc0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffe0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff8::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffc::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffe::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:8000:0'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:c000:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:e000:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f000:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f800:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fc00:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fe00:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff00:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff80:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffc0:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffe0:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff0:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff8:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffc:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffe:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:8000'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:c000'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:e000'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f000'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f800'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fc00'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fe00'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff00'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff80'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffc0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffe0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff8'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffc'))); +SELECT '127' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'))); +SELECT '127' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('0.0.0.0'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('128.0.0.0'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('240.0.0.0'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('248.0.0.0'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('252.0.0.0'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.240.0.0'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.248.0.0'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.252.0.0'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.254.0'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.0'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.128'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.248'))); -SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.252'))); -SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.254'))); -SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.255'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('0.0.0.0'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('128.0.0.0'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('240.0.0.0'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('248.0.0.0'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('252.0.0.0'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.240.0.0'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.248.0.0'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.252.0.0'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.254.0'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.0'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.128'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.248'))); +SELECT '30' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.252'))); +SELECT '30' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.254'))); +SELECT '30' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.255'))); -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01033_dictionaries_lifetime.sql b/tests/queries/0_stateless/01033_dictionaries_lifetime.sql index 67e1adf5574..e74ac8bde30 100644 --- a/tests/queries/0_stateless/01033_dictionaries_lifetime.sql +++ b/tests/queries/0_stateless/01033_dictionaries_lifetime.sql @@ -1,12 +1,7 @@ --- Tags: no-parallel SET send_logs_level = 'fatal'; -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - -CREATE TABLE database_for_dict.table_for_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict ( key_column UInt64, second_column UInt8, @@ -15,34 +10,34 @@ CREATE TABLE database_for_dict.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO database_for_dict.table_for_dict VALUES (1, 100, 'Hello world'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict VALUES (1, 100, 'Hello world'); -DROP DATABASE IF EXISTS ordinary_db; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; -CREATE DATABASE ordinary_db; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; -CREATE DICTIONARY ordinary_db.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE_1:Identifier}.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, third_column String DEFAULT 'qqq' ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); SELECT 'INITIALIZING DICTIONARY'; -SELECT dictGetUInt8('ordinary_db.dict1', 'second_column', toUInt64(100500)); +SELECT dictGetUInt8({CLICKHOUSE_DATABASE_1:String}||'.dict1', 'second_column', toUInt64(100500)); -SELECT lifetime_min, lifetime_max FROM system.dictionaries WHERE database='ordinary_db' AND name = 'dict1'; +SELECT lifetime_min, lifetime_max FROM system.dictionaries WHERE database={CLICKHOUSE_DATABASE_1:String} AND name = 'dict1'; -DROP DICTIONARY IF EXISTS ordinary_db.dict1; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}.dict1; -DROP DATABASE IF EXISTS ordinary_db; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; -DROP TABLE IF EXISTS database_for_dict.table_for_dict; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_for_dict; -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql b/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql index 9040d7b3231..bbd848f81d2 100644 --- a/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql +++ b/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql @@ -1,35 +1,31 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS dict_db_01036; -CREATE DATABASE dict_db_01036; - -CREATE TABLE dict_db_01036.dict_data (key UInt64, val UInt64) Engine=Memory(); -CREATE DICTIONARY dict_db_01036.dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.dict_data (key UInt64, val UInt64) Engine=Memory(); +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict ( key UInt64 DEFAULT 0, val UInt64 DEFAULT 10 ) PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01036')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB currentDatabase())) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; -SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SELECT dictGetUInt64({CLICKHOUSE_DATABASE:String}||'.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; SELECT 'SYSTEM RELOAD DICTIONARY'; -SYSTEM RELOAD DICTIONARY dict_db_01036.dict; -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; -SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SYSTEM RELOAD DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict; +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SELECT dictGetUInt64({CLICKHOUSE_DATABASE:String}||'.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; SELECT 'CREATE DATABASE'; -DROP DATABASE IF EXISTS empty_db_01036; -CREATE DATABASE empty_db_01036; -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; -DROP DICTIONARY dict_db_01036.dict; -DROP TABLE dict_db_01036.dict_data; -DROP DATABASE dict_db_01036; -DROP DATABASE empty_db_01036; +DROP DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.dict_data; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; +DROP DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; diff --git a/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql b/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql index ad364237544..ab0b5a243ba 100644 --- a/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql +++ b/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql @@ -1,10 +1,5 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS dictdb; - -CREATE DATABASE dictdb; - -CREATE TABLE dictdb.table_for_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict ( key_column UInt64, value Float64 @@ -12,33 +7,32 @@ CREATE TABLE dictdb.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO dictdb.table_for_dict VALUES (1, 1.1); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict VALUES (1, 1.1); -CREATE DICTIONARY IF NOT EXISTS dictdb.dict_exists +CREATE DICTIONARY IF NOT EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict_exists ( key_column UInt64, value Float64 DEFAULT 77.77 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'dictdb')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(1) LAYOUT(FLAT()); -SELECT dictGetFloat64('dictdb.dict_exists', 'value', toUInt64(1)); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict_exists', 'value', toUInt64(1)); -CREATE DICTIONARY IF NOT EXISTS dictdb.dict_exists +CREATE DICTIONARY IF NOT EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict_exists ( key_column UInt64, value Float64 DEFAULT 77.77 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'dictdb')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(1) LAYOUT(FLAT()); -SELECT dictGetFloat64('dictdb.dict_exists', 'value', toUInt64(1)); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict_exists', 'value', toUInt64(1)); -DROP DICTIONARY dictdb.dict_exists; -DROP TABLE dictdb.table_for_dict; -DROP DATABASE dictdb; +DROP DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_exists; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict; diff --git a/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql b/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql index 5d629d4e8db..4f078499972 100644 --- a/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql +++ b/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql @@ -1,13 +1,9 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS dictdb_01043; -CREATE DATABASE dictdb_01043; - -CREATE TABLE dictdb_01043.dicttbl(key Int64, value_default String, value_expression String) ENGINE = MergeTree ORDER BY tuple(); -INSERT INTO dictdb_01043.dicttbl VALUES (12, 'hello', '55:66:77'); +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.dicttbl(key Int64, value_default String, value_expression String) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.dicttbl VALUES (12, 'hello', '55:66:77'); -CREATE DICTIONARY dictdb_01043.dict +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict ( key Int64 DEFAULT -1, value_default String DEFAULT 'world', @@ -15,15 +11,13 @@ CREATE DICTIONARY dictdb_01043.dict ) PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dicttbl' DB 'dictdb_01043')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dicttbl' DB currentDatabase())) LAYOUT(FLAT()) LIFETIME(1); -SELECT dictGetString('dictdb_01043.dict', 'value_default', toUInt64(12)); -SELECT dictGetString('dictdb_01043.dict', 'value_default', toUInt64(14)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict', 'value_default', toUInt64(12)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict', 'value_default', toUInt64(14)); -SELECT dictGetString('dictdb_01043.dict', 'value_expression', toUInt64(12)); -SELECT dictGetString('dictdb_01043.dict', 'value_expression', toUInt64(14)); - -DROP DATABASE IF EXISTS dictdb_01043; +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict', 'value_expression', toUInt64(12)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict', 'value_expression', toUInt64(14)); diff --git a/tests/queries/0_stateless/01045_dictionaries_restrictions.sql b/tests/queries/0_stateless/01045_dictionaries_restrictions.sql index d41be7482f8..b4dbd741767 100644 --- a/tests/queries/0_stateless/01045_dictionaries_restrictions.sql +++ b/tests/queries/0_stateless/01045_dictionaries_restrictions.sql @@ -1,10 +1,5 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS dictdb_01045; - -CREATE DATABASE dictdb_01045; - -CREATE DICTIONARY dictdb_01045.restricted_dict ( +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.restricted_dict ( key UInt64, value String ) @@ -14,10 +9,9 @@ LIFETIME(MIN 0 MAX 1) LAYOUT(CACHE(SIZE_IN_CELLS 10)); -- because of lazy load we can check only in dictGet query -select dictGetString('dictdb_01045.restricted_dict', 'value', toUInt64(1)); -- {serverError 482} +select dictGetString({CLICKHOUSE_DATABASE:String} || '.restricted_dict', 'value', toUInt64(1)); -- {serverError 482} select 'Ok.'; -DROP DICTIONARY IF EXISTS dictdb_01045.restricted_dict; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.restricted_dict; -DROP DATABASE IF EXISTS dictdb_01045; diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference index 53df91c5523..bb4fb9ddb47 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference @@ -1,32 +1,32 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---DATA COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY b\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY b\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `count(a)` AggregateFunction(count, Int32),\n `windowID(____timestamp, toIntervalSecond(\'1\'))` UInt32\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `count(a)` AggregateFunction(count, Int32),\n `windowID(____timestamp, toIntervalSecond(\'1\'))` UInt32\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---JOIN--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---DATA COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY b\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY b\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `count(a)` AggregateFunction(count, Int32),\n `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `count(a)` AggregateFunction(count, Int32),\n `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---JOIN--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql index bf1ac254783..e292447512c 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql @@ -1,110 +1,109 @@ --- Tags: no-parallel SET allow_experimental_analyzer = 0; SET allow_experimental_window_view = 1; -DROP DATABASE IF EXISTS test_01047; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; set allow_deprecated_database_ordinary=1; -CREATE DATABASE test_01047 ENGINE=Ordinary; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE=Ordinary; -DROP TABLE IF EXISTS test_01047.mt; -DROP TABLE IF EXISTS test_01047.mt_2; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt_2; -CREATE TABLE test_01047.mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE TABLE test_01047.mt_2(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.mt_2(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---TUMBLE---'; SELECT '||---WINDOW COLUMN NAME---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a), tumbleEnd(wid) AS count FROM test_01047.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND) as wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a), tumbleEnd(wid) AS count FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND) as wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND) AS wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---DATA COLUMN ALIAS---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY id ENGINE Memory AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, tumble(timestamp, INTERVAL '1' SECOND); -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY id ENGINE Memory AS SELECT count(a) AS count, b as id FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY id, tumble(timestamp, INTERVAL '1' SECOND); +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---IDENTIFIER---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, tumble(timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY b, tumble(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---FUNCTION---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, tumble(timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY plus(a, b) as _type, tumble(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---PARTITION---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid ENGINE Memory AS SELECT count(a) AS count, tumble(now(), INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid ENGINE Memory AS SELECT count(a) AS count, tumble(now(), INTERVAL '1' SECOND) AS wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---JOIN---'; -DROP TABLE IF EXISTS test_01047.wv; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY tumble(test_01047.mt.timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY tumble(test_01047.mt.timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY tumble({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count({CLICKHOUSE_DATABASE:Identifier}.mt.a), count({CLICKHOUSE_DATABASE:Identifier}.mt_2.b), wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt JOIN {CLICKHOUSE_DATABASE:Identifier}.mt_2 ON {CLICKHOUSE_DATABASE:Identifier}.mt.timestamp = {CLICKHOUSE_DATABASE:Identifier}.mt_2.timestamp GROUP BY tumble({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; -DROP TABLE IF EXISTS test_01047.wv; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY tumble(test_01047.mt.timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count({CLICKHOUSE_DATABASE:Identifier}.mt.a), count({CLICKHOUSE_DATABASE:Identifier}.mt_2.b), wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt JOIN {CLICKHOUSE_DATABASE:Identifier}.mt_2 ON {CLICKHOUSE_DATABASE:Identifier}.mt.timestamp = {CLICKHOUSE_DATABASE:Identifier}.mt_2.timestamp GROUP BY tumble({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count, hopEnd(wid) FROM test_01047.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count, hopEnd(wid) FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(a) AS count, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01047.mt GROUP BY wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(a) AS count, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---DATA COLUMN ALIAS---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY id ENGINE Memory AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND); -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY id ENGINE Memory AS SELECT count(a) AS count, b as id FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY id, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND); +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---IDENTIFIER---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY (hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY (hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY b, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---FUNCTION---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY (hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), plus(a, b)) PRIMARY KEY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY (hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), plus(a, b)) PRIMARY KEY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY plus(a, b) as _type, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---PARTITION---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid ENGINE Memory AS SELECT count(a) AS count, hopEnd(wid) FROM test_01047.mt GROUP BY hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid ENGINE Memory AS SELECT count(a) AS count, hopEnd(wid) FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---JOIN---'; -DROP TABLE IF EXISTS test_01047.wv; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY hop(test_01047.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY hop(test_01047.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY hop({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count({CLICKHOUSE_DATABASE:Identifier}.mt.a), count({CLICKHOUSE_DATABASE:Identifier}.mt_2.b), wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt JOIN {CLICKHOUSE_DATABASE:Identifier}.mt_2 ON {CLICKHOUSE_DATABASE:Identifier}.mt.timestamp = {CLICKHOUSE_DATABASE:Identifier}.mt_2.timestamp GROUP BY hop({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; -DROP TABLE IF EXISTS test_01047.wv; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY hop(test_01047.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count({CLICKHOUSE_DATABASE:Identifier}.mt.a), count({CLICKHOUSE_DATABASE:Identifier}.mt_2.b), wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt JOIN {CLICKHOUSE_DATABASE:Identifier}.mt_2 ON {CLICKHOUSE_DATABASE:Identifier}.mt.timestamp = {CLICKHOUSE_DATABASE:Identifier}.mt_2.timestamp GROUP BY hop({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; -DROP TABLE test_01047.wv; -DROP TABLE test_01047.mt; -DROP TABLE test_01047.mt_2; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.mt; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.mt_2; diff --git a/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql b/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql index ec440baf8d6..a790f384f7a 100644 --- a/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql +++ b/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql @@ -1,18 +1,17 @@ --- Tags: no-parallel -drop dictionary if exists default.test_dict_01051_d; -drop table if exists default.test_01051_d; -drop table if exists default.test_view_01051_d; +drop dictionary if exists {CLICKHOUSE_DATABASE:Identifier}.test_dict_01051_d; +drop table if exists {CLICKHOUSE_DATABASE:Identifier}.test_01051_d; +drop table if exists {CLICKHOUSE_DATABASE:Identifier}.test_view_01051_d; -create table default.test_01051_d (key UInt64, value String) engine = MergeTree order by key; -create view default.test_view_01051_d (key UInt64, value String) as select k2 + 1 as key, v2 || '_x' as value from (select key + 2 as k2, value || '_y' as v2 from default.test_01051_d); +create table {CLICKHOUSE_DATABASE:Identifier}.test_01051_d (key UInt64, value String) engine = MergeTree order by key; +create view {CLICKHOUSE_DATABASE:Identifier}.test_view_01051_d (key UInt64, value String) as select k2 + 1 as key, v2 || '_x' as value from (select key + 2 as k2, value || '_y' as v2 from test_01051_d); -insert into default.test_01051_d values (1, 'a'); +insert into {CLICKHOUSE_DATABASE:Identifier}.test_01051_d values (1, 'a'); -create dictionary default.test_dict_01051_d (key UInt64, value String) primary key key source(clickhouse(host 'localhost' port '9000' user 'default' password '' db 'default' table 'test_view_01051_d')) layout(flat()) lifetime(100500); +create dictionary {CLICKHOUSE_DATABASE:Identifier}.test_dict_01051_d (key UInt64, value String) primary key key source(clickhouse(host 'localhost' port '9000' user 'default' password '' db currentDatabase() table 'test_view_01051_d')) layout(flat()) lifetime(100500); -select dictGet('default.test_dict_01051_d', 'value', toUInt64(4)); +select dictGet({CLICKHOUSE_DATABASE:String} || '.test_dict_01051_d', 'value', toUInt64(4)); -drop dictionary if exists default.test_dict_01051_d; -drop table if exists default.test_01051_d; -drop table if exists default.test_view_01051_d; +drop dictionary if exists {CLICKHOUSE_DATABASE:Identifier}.test_dict_01051_d; +drop table if exists {CLICKHOUSE_DATABASE:Identifier}.test_01051_d; +drop table if exists {CLICKHOUSE_DATABASE:Identifier}.test_view_01051_d; diff --git a/tests/queries/0_stateless/01053_drop_database_mat_view.sql b/tests/queries/0_stateless/01053_drop_database_mat_view.sql index 7651ac4885c..2642430eb05 100644 --- a/tests/queries/0_stateless/01053_drop_database_mat_view.sql +++ b/tests/queries/0_stateless/01053_drop_database_mat_view.sql @@ -1,14 +1,13 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS some_tests; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; set allow_deprecated_database_ordinary=1; -CREATE DATABASE some_tests ENGINE=Ordinary; -- Different inner table name with Atomic +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE=Ordinary; -- Different inner table name with Atomic set allow_deprecated_syntax_for_merge_tree=1; -create table some_tests.my_table ENGINE = MergeTree(day, (day), 8192) as select today() as day, 'mystring' as str; -show tables from some_tests; -create materialized view some_tests.my_materialized_view ENGINE = MergeTree(day, (day), 8192) as select * from some_tests.my_table; -show tables from some_tests; -select * from some_tests.my_materialized_view; +create table {CLICKHOUSE_DATABASE:Identifier}.my_table ENGINE = MergeTree(day, (day), 8192) as select today() as day, 'mystring' as str; +show tables from {CLICKHOUSE_DATABASE:Identifier}; +create materialized view {CLICKHOUSE_DATABASE:Identifier}.my_materialized_view ENGINE = MergeTree(day, (day), 8192) as select * from {CLICKHOUSE_DATABASE:Identifier}.my_table; +show tables from {CLICKHOUSE_DATABASE:Identifier}; +select * from {CLICKHOUSE_DATABASE:Identifier}.my_materialized_view; -DROP DATABASE some_tests; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01056_create_table_as.sql b/tests/queries/0_stateless/01056_create_table_as.sql index 6df660dba61..aa2dffb6e2d 100644 --- a/tests/queries/0_stateless/01056_create_table_as.sql +++ b/tests/queries/0_stateless/01056_create_table_as.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; @@ -23,9 +22,9 @@ DROP TABLE v; -- dictionary DROP DICTIONARY IF EXISTS dict; -DROP DATABASE if exists test_01056_dict_data; -CREATE DATABASE test_01056_dict_data; -CREATE TABLE test_01056_dict_data.dict_data (key Int, value UInt16) Engine=Memory(); +DROP DATABASE if exists {CLICKHOUSE_DATABASE_1:Identifier}; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; +CREATE TABLE {CLICKHOUSE_DATABASE_1:Identifier}.dict_data (key Int, value UInt16) Engine=Memory(); CREATE DICTIONARY dict ( `key` UInt64, @@ -34,7 +33,7 @@ CREATE DICTIONARY dict PRIMARY KEY key SOURCE(CLICKHOUSE( HOST '127.0.0.1' PORT tcpPort() - TABLE 'dict_data' DB 'test_01056_dict_data' USER 'default' PASSWORD '')) + TABLE 'dict_data' DB concat(currentDatabase(), '_1') USER 'default' PASSWORD '')) LIFETIME(MIN 0 MAX 0) LAYOUT(SPARSE_HASHED()); CREATE TABLE t3 AS dict; -- { serverError 80 } @@ -42,9 +41,9 @@ CREATE TABLE t3 AS dict; -- { serverError 80 } DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t3; DROP DICTIONARY dict; -DROP TABLE test_01056_dict_data.dict_data; +DROP TABLE {CLICKHOUSE_DATABASE_1:Identifier}.dict_data; -DROP DATABASE test_01056_dict_data; +DROP DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; CREATE TABLE t1 (x String) ENGINE = Memory AS SELECT 1; SELECT x, toTypeName(x) FROM t1; diff --git a/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql b/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql index 992973c97e8..f92b6779587 100644 --- a/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql +++ b/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE old_syntax_01071_test (date Date, id UInt8) ENGINE = MergeTree(date, id, 8192); diff --git a/tests/queries/0_stateless/01073_attach_if_not_exists.sql b/tests/queries/0_stateless/01073_attach_if_not_exists.sql index 8bd356b466c..a99d5fb5041 100644 --- a/tests/queries/0_stateless/01073_attach_if_not_exists.sql +++ b/tests/queries/0_stateless/01073_attach_if_not_exists.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel CREATE TABLE aine (a Int) ENGINE = Log; ATTACH TABLE aine; -- { serverError 57 } diff --git a/tests/queries/0_stateless/01073_show_tables_not_like.sql b/tests/queries/0_stateless/01073_show_tables_not_like.sql index 405a6cb199b..9ff2afe7f27 100644 --- a/tests/queries/0_stateless/01073_show_tables_not_like.sql +++ b/tests/queries/0_stateless/01073_show_tables_not_like.sql @@ -1,11 +1,6 @@ --- Tags: no-parallel SHOW TABLES NOT LIKE '%'; -DROP DATABASE IF EXISTS test_01073; -CREATE DATABASE test_01073; -USE test_01073; - SHOW TABLES; SELECT '---'; CREATE TABLE test1 (x UInt8) ENGINE = Memory; @@ -22,14 +17,14 @@ SELECT '--'; SHOW TABLES NOT LIKE 'tes%2'; SELECT '---'; -SHOW TABLES FROM test_01073; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier}; SELECT '--'; -SHOW TABLES FROM test_01073 LIKE 'tes%'; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} LIKE 'tes%'; SELECT '--'; -SHOW TABLES FROM test_01073 NOT LIKE 'tes%'; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} NOT LIKE 'tes%'; SELECT '--'; -SHOW TABLES FROM test_01073 LIKE 'tes%1'; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} LIKE 'tes%1'; SELECT '--'; -SHOW TABLES FROM test_01073 NOT LIKE 'tes%2'; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} NOT LIKE 'tes%2'; -DROP DATABASE test_01073; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01080_check_for_error_incorrect_size_of_nested_column.sql b/tests/queries/0_stateless/01080_check_for_error_incorrect_size_of_nested_column.sql index 1eb6e849851..d7b05bb7d78 100644 --- a/tests/queries/0_stateless/01080_check_for_error_incorrect_size_of_nested_column.sql +++ b/tests/queries/0_stateless/01080_check_for_error_incorrect_size_of_nested_column.sql @@ -1,40 +1,33 @@ --- Tags: no-parallel --- TODO: can't just remove default prefix, it breaks the test! +drop table if exists {CLICKHOUSE_DATABASE:Identifier}.test_table_01080; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test_table_01080 (dim_key Int64, dim_id String) ENGINE = MergeTree Order by (dim_key); +insert into {CLICKHOUSE_DATABASE:Identifier}.test_table_01080 values(1,'test1'); -drop database if exists db_01080; -create database db_01080; +drop DICTIONARY if exists {CLICKHOUSE_DATABASE:Identifier}.test_dict_01080; -drop table if exists db_01080.test_table_01080; -CREATE TABLE db_01080.test_table_01080 (dim_key Int64, dim_id String) ENGINE = MergeTree Order by (dim_key); -insert into db_01080.test_table_01080 values(1,'test1'); - -drop DICTIONARY if exists db_01080.test_dict_01080; - -CREATE DICTIONARY db_01080.test_dict_01080 ( dim_key Int64, dim_id String ) +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.test_dict_01080 ( dim_key Int64, dim_id String ) PRIMARY KEY dim_key -source(clickhouse(host 'localhost' port tcpPort() user 'default' password '' db 'db_01080' table 'test_table_01080')) +source(clickhouse(host 'localhost' port tcpPort() user 'default' password '' db currentDatabase() table 'test_table_01080')) LIFETIME(MIN 0 MAX 0) LAYOUT(complex_key_hashed()); -SELECT dictGetString('db_01080.test_dict_01080', 'dim_id', tuple(toInt64(1))); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', tuple(toInt64(1))); -SELECT dictGetString('db_01080.test_dict_01080', 'dim_id', tuple(toInt64(0))); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', tuple(toInt64(0))); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(0)) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(0)) as x); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(1)) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(1)) as x); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(number)) as x from numbers(5)); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(number)) as x from numbers(5)); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(rand64()*0)) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(rand64()*0)) as x); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(blockSize()=0)) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(blockSize()=0)) as x); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(materialize(0))) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(materialize(0))) as x); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(materialize(1))) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(materialize(1))) as x); -drop DICTIONARY db_01080.test_dict_01080; -drop table db_01080.test_table_01080; -drop database db_01080; +drop DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.test_dict_01080; +drop table {CLICKHOUSE_DATABASE:Identifier}.test_table_01080; diff --git a/tests/queries/0_stateless/01084_regexp_empty.sql b/tests/queries/0_stateless/01084_regexp_empty.sql index 3ccd4af80ab..5dd060ab58c 100644 --- a/tests/queries/0_stateless/01084_regexp_empty.sql +++ b/tests/queries/0_stateless/01084_regexp_empty.sql @@ -1,10 +1,9 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS test_01084; -CREATE DATABASE test_01084; -USE test_01084; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; +USE {CLICKHOUSE_DATABASE_1:Identifier}; CREATE TABLE t (x UInt8) ENGINE = Memory; SELECT * FROM merge('', ''); -DROP DATABASE test_01084; +DROP DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; diff --git a/tests/queries/0_stateless/01085_window_view_attach.sql b/tests/queries/0_stateless/01085_window_view_attach.sql index 051557a6a76..51a88a04f95 100644 --- a/tests/queries/0_stateless/01085_window_view_attach.sql +++ b/tests/queries/0_stateless/01085_window_view_attach.sql @@ -1,30 +1,29 @@ --- Tags: no-parallel SET allow_experimental_analyzer = 0; SET allow_experimental_window_view = 1; -DROP DATABASE IF EXISTS test_01085; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; set allow_deprecated_database_ordinary=1; -CREATE DATABASE test_01085 ENGINE=Ordinary; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE=Ordinary; -DROP TABLE IF EXISTS test_01085.mt; -DROP TABLE IF EXISTS test_01085.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; -CREATE TABLE test_01085.mt(a Int32, market Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test_01085.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, market, tumbleEnd(wid) AS w_end FROM test_01085.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid, market; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.mt(a Int32, market Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, market, tumbleEnd(wid) AS w_end FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid, market; -SHOW tables FROM test_01085; +SHOW tables FROM {CLICKHOUSE_DATABASE:Identifier}; -DROP TABLE test_01085.wv SYNC; -SHOW tables FROM test_01085; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.wv SYNC; +SHOW tables FROM {CLICKHOUSE_DATABASE:Identifier}; -CREATE WINDOW VIEW test_01085.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, market, tumbleEnd(wid) AS w_end FROM test_01085.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid, market; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, market, tumbleEnd(wid) AS w_end FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid, market; -DETACH TABLE test_01085.wv; -SHOW tables FROM test_01085; +DETACH TABLE {CLICKHOUSE_DATABASE:Identifier}.wv; +SHOW tables FROM {CLICKHOUSE_DATABASE:Identifier}; -ATTACH TABLE test_01085.wv; -SHOW tables FROM test_01085; +ATTACH TABLE {CLICKHOUSE_DATABASE:Identifier}.wv; +SHOW tables FROM {CLICKHOUSE_DATABASE:Identifier}; -DROP TABLE test_01085.wv SYNC; -SHOW tables FROM test_01085; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.wv SYNC; +SHOW tables FROM {CLICKHOUSE_DATABASE:Identifier}; From ba44d7260e12ba6e32ff1033cf9538ce9abe0645 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Aug 2023 00:20:28 +0200 Subject: [PATCH 2015/2047] fix --- programs/keeper/Keeper.cpp | 2 -- src/Access/LDAPClient.cpp | 4 ++-- src/Formats/ProtobufReader.cpp | 10 ++++------ src/Processors/Merges/Algorithms/Graphite.cpp | 8 ++------ tests/integration/test_grpc_protocol/test.py | 5 ++++- 5 files changed, 12 insertions(+), 17 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 77fbc9430ef..4791f42312e 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -110,8 +110,6 @@ void Keeper::createServer(const std::string & listen_host, const char * port_nam } catch (const Poco::Exception &) { - std::string message = "Listen [" + listen_host + "]:" + std::to_string(port) + " failed: " + getCurrentExceptionMessage(false); - if (listen_try) { LOG_WARNING(&logger(), "Listen [{}]:{} failed: {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, " diff --git a/src/Access/LDAPClient.cpp b/src/Access/LDAPClient.cpp index ac2f1683f0c..2af779aa9ae 100644 --- a/src/Access/LDAPClient.cpp +++ b/src/Access/LDAPClient.cpp @@ -549,7 +549,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params) if (rc != LDAP_SUCCESS) { - String message = "LDAP search failed"; + String message; const char * raw_err_str = ldap_err2string(rc); if (raw_err_str && *raw_err_str != '\0') @@ -570,7 +570,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params) message += matched_msg; } - throw Exception::createDeprecated(message, ErrorCodes::LDAP_ERROR); + throw Exception(ErrorCodes::LDAP_ERROR, "LDAP search failed{}", message); } break; diff --git a/src/Formats/ProtobufReader.cpp b/src/Formats/ProtobufReader.cpp index 159a1d33c49..577342bf299 100644 --- a/src/Formats/ProtobufReader.cpp +++ b/src/Formats/ProtobufReader.cpp @@ -429,12 +429,10 @@ void ProtobufReader::ignoreGroup() [[noreturn]] void ProtobufReader::throwUnknownFormat() const { - throw Exception::createDeprecated( - std::string("Protobuf messages are corrupted or don't match the provided schema.") - + (root_message_has_length_delimiter - ? " Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint." - : ""), - ErrorCodes::UNKNOWN_PROTOBUF_FORMAT); + throw Exception(ErrorCodes::UNKNOWN_PROTOBUF_FORMAT, "Protobuf messages are corrupted or don't match the provided schema.{}", + root_message_has_length_delimiter + ? " Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint." + : ""); } } diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp index 418bf5e3f13..a3ff7a4ef87 100644 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -163,12 +163,8 @@ static bool compareRetentions(const Retention & a, const Retention & b) { return false; } - String error_msg = "age and precision should only grow up: " - + std::to_string(a.age) + ":" + std::to_string(a.precision) + " vs " - + std::to_string(b.age) + ":" + std::to_string(b.precision); - throw Exception::createDeprecated( - error_msg, - DB::ErrorCodes::BAD_ARGUMENTS); + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Age and precision should only grow up: {}:{} vs {}:{}", + a.age, a.precision, b.age, b.precision); } bool operator==(const Retention & a, const Retention & b) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index dead4d447ec..4a28dae900b 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -352,7 +352,10 @@ def test_authentication(): def test_logs(): - logs = query_and_get_logs("SELECT 1", settings={"send_logs_level": "debug"}) + logs = query_and_get_logs( + "SELECT has(groupArray(number), 42) FROM numbers(1000)", + settings={"send_logs_level": "debug"}, + ) assert "SELECT 1" in logs assert "Read 1 rows" in logs assert "Peak memory usage" in logs From 387ce81895d0d9a6a8e994bf24801b00dc3af049 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Aug 2023 00:46:53 +0200 Subject: [PATCH 2016/2047] Clean all containers properly --- tests/ci/install_check.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index 010b0dab408..700550bf077 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -191,6 +191,9 @@ def test_install(image: DockerImage, tests: Dict[str, str]) -> TestResults: retcode = process.wait() if retcode == 0: status = OK + subprocess.check_call( + f"docker kill -s 9 {container_id}", shell=True + ) break status = FAIL @@ -198,8 +201,8 @@ def test_install(image: DockerImage, tests: Dict[str, str]) -> TestResults: archive_path = TEMP_PATH / f"{container_name}-{retry}.tar.gz" compress_fast(LOGS_PATH, archive_path) logs.append(archive_path) + subprocess.check_call(f"docker kill -s 9 {container_id}", shell=True) - subprocess.check_call(f"docker kill -s 9 {container_id}", shell=True) test_results.append(TestResult(name, status, stopwatch.duration_seconds, logs)) return test_results From 790475385acc5b722460e5b9581f637ac6ff9b1e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Aug 2023 00:47:39 +0200 Subject: [PATCH 2017/2047] Improve downloading: skip dbg, do not pull images on --no-download --- tests/ci/install_check.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index 700550bf077..2ca947192da 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -279,7 +279,7 @@ def main(): sys.exit(0) docker_images = { - name: get_image_with_version(REPORTS_PATH, name) + name: get_image_with_version(REPORTS_PATH, name, args.download) for name in (RPM_IMAGE, DEB_IMAGE) } prepare_test_scripts() @@ -296,6 +296,8 @@ def main(): is_match = is_match or path.endswith(".rpm") if args.tgz: is_match = is_match or path.endswith(".tgz") + # We don't need debug packages, so let's filter them out + is_match = is_match and "-dbg" not in path return is_match download_builds_filter( From 3cd9fa395d2d3483e9e71274076cf151ef8ff682 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Aug 2023 00:51:44 +0200 Subject: [PATCH 2018/2047] Add test for systemd + /etc/default/clickhouse --- tests/ci/install_check.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index 2ca947192da..b08e94c52b4 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -50,8 +50,11 @@ def prepare_test_scripts(): server_test = r"""#!/bin/bash set -e trap "bash -ex /packages/preserve_logs.sh" ERR +test_env='TEST_THE_DEFAULT_PARAMETER=15' +echo "$test_env" >> /etc/default/clickhouse systemctl start clickhouse-server -clickhouse-client -q 'SELECT version()'""" +clickhouse-client -q 'SELECT version()' +grep "$test_env" /proc/$(cat /var/run/clickhouse-server/clickhouse-server.pid)/environ""" keeper_test = r"""#!/bin/bash set -e trap "bash -ex /packages/preserve_logs.sh" ERR From 651a45b04d1cc4ec0b8be5b0fbb3068b09813fce Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Aug 2023 00:57:22 +0200 Subject: [PATCH 2019/2047] Add tests for initd start --- docker/test/install/deb/Dockerfile | 1 + tests/ci/install_check.py | 12 ++++++++++++ 2 files changed, 13 insertions(+) diff --git a/docker/test/install/deb/Dockerfile b/docker/test/install/deb/Dockerfile index 9614473c69b..e9c928b1fe7 100644 --- a/docker/test/install/deb/Dockerfile +++ b/docker/test/install/deb/Dockerfile @@ -12,6 +12,7 @@ ENV \ # install systemd packages RUN apt-get update && \ apt-get install -y --no-install-recommends \ + sudo \ systemd \ && \ apt-get clean && \ diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index b08e94c52b4..a5788e2af3f 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -54,6 +54,14 @@ test_env='TEST_THE_DEFAULT_PARAMETER=15' echo "$test_env" >> /etc/default/clickhouse systemctl start clickhouse-server clickhouse-client -q 'SELECT version()' +grep "$test_env" /proc/$(cat /var/run/clickhouse-server/clickhouse-server.pid)/environ""" + initd_test = r"""#!/bin/bash +set -e +trap "bash -ex /packages/preserve_logs.sh" ERR +test_env='TEST_THE_DEFAULT_PARAMETER=15' +echo "$test_env" >> /etc/default/clickhouse +/etc/init.d/clickhouse-server start +clickhouse-client -q 'SELECT version()' grep "$test_env" /proc/$(cat /var/run/clickhouse-server/clickhouse-server.pid)/environ""" keeper_test = r"""#!/bin/bash set -e @@ -105,6 +113,7 @@ chmod a+rw -R /tests_logs exit 1 """ (TEMP_PATH / "server_test.sh").write_text(server_test, encoding="utf-8") + (TEMP_PATH / "initd_test.sh").write_text(initd_test, encoding="utf-8") (TEMP_PATH / "keeper_test.sh").write_text(keeper_test, encoding="utf-8") (TEMP_PATH / "binary_test.sh").write_text(binary_test, encoding="utf-8") (TEMP_PATH / "preserve_logs.sh").write_text(preserve_logs, encoding="utf-8") @@ -115,6 +124,9 @@ def test_install_deb(image: DockerImage) -> TestResults: "Install server deb": r"""#!/bin/bash -ex apt-get install /packages/clickhouse-{server,client,common}*deb bash -ex /packages/server_test.sh""", + "Run server init.d": r"""#!/bin/bash -ex +apt-get install /packages/clickhouse-{server,client,common}*deb +bash -ex /packages/initd_test.sh""", "Install keeper deb": r"""#!/bin/bash -ex apt-get install /packages/clickhouse-keeper*deb bash -ex /packages/keeper_test.sh""", From 23353f5aff80ad704ba6c7b1d6c83e77056405eb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Aug 2023 12:06:36 +0200 Subject: [PATCH 2020/2047] fix --- tests/integration/test_graphite_merge_tree/test.py | 2 +- tests/integration/test_grpc_protocol/test.py | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_graphite_merge_tree/test.py b/tests/integration/test_graphite_merge_tree/test.py index c4364a03fd9..7c19888df6b 100644 --- a/tests/integration/test_graphite_merge_tree/test.py +++ b/tests/integration/test_graphite_merge_tree/test.py @@ -509,6 +509,6 @@ CREATE TABLE test.graphite_not_created ) # The order of retentions is not guaranteed - assert "age and precision should only grow up: " in str(exc.value) + assert "Age and precision should only grow up: " in str(exc.value) assert "36000:600" in str(exc.value) assert "72000:300" in str(exc.value) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 4a28dae900b..5b56c3da8dc 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -352,11 +352,12 @@ def test_authentication(): def test_logs(): + query = "SELECT has(groupArray(number), 42) FROM numbers(1000)" logs = query_and_get_logs( - "SELECT has(groupArray(number), 42) FROM numbers(1000)", + query, settings={"send_logs_level": "debug"}, ) - assert "SELECT 1" in logs + assert query in logs assert "Read 1 rows" in logs assert "Peak memory usage" in logs From 428a05a560dd9561f1729c38b963250b980c2f19 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 16 Aug 2023 14:04:14 +0300 Subject: [PATCH 2021/2047] Follow-up: Do not send logs to CI if the credentials are not set (#53456) * Follow-up * Automatic style fix * Update tests/ci/ast_fuzzer_check.py * Update tests/ci/functional_test_check.py * Update tests/ci/stress_check.py * Automatic style fix --------- Co-authored-by: robot-clickhouse Co-authored-by: Alexander Tokmakov --- tests/ci/ast_fuzzer_check.py | 2 +- tests/ci/functional_test_check.py | 2 +- tests/ci/stress_check.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 1a75d02bef4..fecf207589e 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -146,7 +146,7 @@ def main(): "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": + if ci_logs_host not in ("CLICKHOUSE_CI_LOGS_HOST", ""): subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}' '{main_log_path}'", shell=True, diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 22210390b09..2bab330bd66 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -394,7 +394,7 @@ def main(): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": + if ci_logs_host not in ("CLICKHOUSE_CI_LOGS_HOST", ""): subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", shell=True, diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 9c18bcbfe40..21c3178faab 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -209,7 +209,7 @@ def run_stress_test(docker_image_name): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": + if ci_logs_host not in ("CLICKHOUSE_CI_LOGS_HOST", ""): subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", shell=True, From 56888912368c2e4620fbe95322cd53e443e79626 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 16 Aug 2023 19:00:27 +0000 Subject: [PATCH 2022/2047] curl: update to latest master (fixes CVE-2023-32001) --- contrib/curl | 2 +- contrib/curl-cmake/CMakeLists.txt | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/contrib/curl b/contrib/curl index b0edf0b7dae..eb3b049df52 160000 --- a/contrib/curl +++ b/contrib/curl @@ -1 +1 @@ -Subproject commit b0edf0b7dae44d9e66f270a257cf654b35d5263d +Subproject commit eb3b049df526bf125eda23218e680ce7fa9ec46c diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index 70d9c2816dc..ca4edd77b3a 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -40,7 +40,6 @@ set (SRCS "${LIBRARY_DIR}/lib/strcase.c" "${LIBRARY_DIR}/lib/easy.c" "${LIBRARY_DIR}/lib/curl_fnmatch.c" - "${LIBRARY_DIR}/lib/curl_log.c" "${LIBRARY_DIR}/lib/fileinfo.c" "${LIBRARY_DIR}/lib/krb5.c" "${LIBRARY_DIR}/lib/memdebug.c" @@ -113,6 +112,7 @@ set (SRCS "${LIBRARY_DIR}/lib/psl.c" "${LIBRARY_DIR}/lib/doh.c" "${LIBRARY_DIR}/lib/urlapi.c" + "${LIBRARY_DIR}/lib/curl_trc.c" "${LIBRARY_DIR}/lib/curl_get_line.c" "${LIBRARY_DIR}/lib/altsvc.c" "${LIBRARY_DIR}/lib/socketpair.c" @@ -142,12 +142,10 @@ set (SRCS "${LIBRARY_DIR}/lib/vtls/openssl.c" "${LIBRARY_DIR}/lib/vtls/gtls.c" "${LIBRARY_DIR}/lib/vtls/vtls.c" - "${LIBRARY_DIR}/lib/vtls/nss.c" "${LIBRARY_DIR}/lib/vtls/wolfssl.c" "${LIBRARY_DIR}/lib/vtls/schannel.c" "${LIBRARY_DIR}/lib/vtls/schannel_verify.c" "${LIBRARY_DIR}/lib/vtls/sectransp.c" - "${LIBRARY_DIR}/lib/vtls/gskit.c" "${LIBRARY_DIR}/lib/vtls/mbedtls.c" "${LIBRARY_DIR}/lib/vtls/bearssl.c" "${LIBRARY_DIR}/lib/vtls/keylog.c" From ea8a99c103342c28f20d0b07ed19318e9f2d5b9f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 16 Aug 2023 19:04:28 +0000 Subject: [PATCH 2023/2047] Sort stuff --- contrib/curl-cmake/CMakeLists.txt | 224 +++++++++++++++--------------- 1 file changed, 112 insertions(+), 112 deletions(-) diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index ca4edd77b3a..733865d5101 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -8,125 +8,122 @@ endif() set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/curl") set (SRCS - "${LIBRARY_DIR}/lib/fopen.c" - "${LIBRARY_DIR}/lib/noproxy.c" - "${LIBRARY_DIR}/lib/idn.c" - "${LIBRARY_DIR}/lib/cfilters.c" - "${LIBRARY_DIR}/lib/cf-socket.c" + "${LIBRARY_DIR}/lib/altsvc.c" + "${LIBRARY_DIR}/lib/amigaos.c" + "${LIBRARY_DIR}/lib/asyn-thread.c" + "${LIBRARY_DIR}/lib/base64.c" + "${LIBRARY_DIR}/lib/bufq.c" + "${LIBRARY_DIR}/lib/bufref.c" + "${LIBRARY_DIR}/lib/cf-h1-proxy.c" "${LIBRARY_DIR}/lib/cf-haproxy.c" "${LIBRARY_DIR}/lib/cf-https-connect.c" - "${LIBRARY_DIR}/lib/file.c" - "${LIBRARY_DIR}/lib/timeval.c" - "${LIBRARY_DIR}/lib/base64.c" - "${LIBRARY_DIR}/lib/hostip.c" - "${LIBRARY_DIR}/lib/progress.c" - "${LIBRARY_DIR}/lib/formdata.c" - "${LIBRARY_DIR}/lib/cookie.c" - "${LIBRARY_DIR}/lib/http.c" - "${LIBRARY_DIR}/lib/sendf.c" - "${LIBRARY_DIR}/lib/url.c" - "${LIBRARY_DIR}/lib/dict.c" - "${LIBRARY_DIR}/lib/if2ip.c" - "${LIBRARY_DIR}/lib/speedcheck.c" - "${LIBRARY_DIR}/lib/ldap.c" - "${LIBRARY_DIR}/lib/version.c" - "${LIBRARY_DIR}/lib/getenv.c" - "${LIBRARY_DIR}/lib/escape.c" - "${LIBRARY_DIR}/lib/mprintf.c" - "${LIBRARY_DIR}/lib/telnet.c" - "${LIBRARY_DIR}/lib/netrc.c" - "${LIBRARY_DIR}/lib/getinfo.c" - "${LIBRARY_DIR}/lib/transfer.c" - "${LIBRARY_DIR}/lib/strcase.c" - "${LIBRARY_DIR}/lib/easy.c" - "${LIBRARY_DIR}/lib/curl_fnmatch.c" - "${LIBRARY_DIR}/lib/fileinfo.c" - "${LIBRARY_DIR}/lib/krb5.c" - "${LIBRARY_DIR}/lib/memdebug.c" - "${LIBRARY_DIR}/lib/http_chunks.c" - "${LIBRARY_DIR}/lib/strtok.c" + "${LIBRARY_DIR}/lib/cf-socket.c" + "${LIBRARY_DIR}/lib/cfilters.c" + "${LIBRARY_DIR}/lib/conncache.c" "${LIBRARY_DIR}/lib/connect.c" - "${LIBRARY_DIR}/lib/llist.c" - "${LIBRARY_DIR}/lib/hash.c" - "${LIBRARY_DIR}/lib/multi.c" "${LIBRARY_DIR}/lib/content_encoding.c" - "${LIBRARY_DIR}/lib/share.c" - "${LIBRARY_DIR}/lib/http_digest.c" - "${LIBRARY_DIR}/lib/md4.c" - "${LIBRARY_DIR}/lib/md5.c" - "${LIBRARY_DIR}/lib/http_negotiate.c" - "${LIBRARY_DIR}/lib/inet_pton.c" - "${LIBRARY_DIR}/lib/strtoofft.c" - "${LIBRARY_DIR}/lib/strerror.c" - "${LIBRARY_DIR}/lib/amigaos.c" + "${LIBRARY_DIR}/lib/cookie.c" + "${LIBRARY_DIR}/lib/curl_addrinfo.c" + "${LIBRARY_DIR}/lib/curl_des.c" + "${LIBRARY_DIR}/lib/curl_endian.c" + "${LIBRARY_DIR}/lib/curl_fnmatch.c" + "${LIBRARY_DIR}/lib/curl_get_line.c" + "${LIBRARY_DIR}/lib/curl_gethostname.c" + "${LIBRARY_DIR}/lib/curl_gssapi.c" + "${LIBRARY_DIR}/lib/curl_memrchr.c" + "${LIBRARY_DIR}/lib/curl_multibyte.c" + "${LIBRARY_DIR}/lib/curl_ntlm_core.c" + "${LIBRARY_DIR}/lib/curl_ntlm_wb.c" + "${LIBRARY_DIR}/lib/curl_path.c" + "${LIBRARY_DIR}/lib/curl_range.c" + "${LIBRARY_DIR}/lib/curl_rtmp.c" + "${LIBRARY_DIR}/lib/curl_sasl.c" + "${LIBRARY_DIR}/lib/curl_sspi.c" + "${LIBRARY_DIR}/lib/curl_threads.c" + "${LIBRARY_DIR}/lib/curl_trc.c" + "${LIBRARY_DIR}/lib/dict.c" + "${LIBRARY_DIR}/lib/doh.c" + "${LIBRARY_DIR}/lib/dynbuf.c" + "${LIBRARY_DIR}/lib/dynhds.c" + "${LIBRARY_DIR}/lib/easy.c" + "${LIBRARY_DIR}/lib/escape.c" + "${LIBRARY_DIR}/lib/file.c" + "${LIBRARY_DIR}/lib/fileinfo.c" + "${LIBRARY_DIR}/lib/fopen.c" + "${LIBRARY_DIR}/lib/formdata.c" + "${LIBRARY_DIR}/lib/getenv.c" + "${LIBRARY_DIR}/lib/getinfo.c" + "${LIBRARY_DIR}/lib/gopher.c" + "${LIBRARY_DIR}/lib/hash.c" + "${LIBRARY_DIR}/lib/headers.c" + "${LIBRARY_DIR}/lib/hmac.c" "${LIBRARY_DIR}/lib/hostasyn.c" + "${LIBRARY_DIR}/lib/hostip.c" "${LIBRARY_DIR}/lib/hostip4.c" "${LIBRARY_DIR}/lib/hostip6.c" "${LIBRARY_DIR}/lib/hostsyn.c" + "${LIBRARY_DIR}/lib/hsts.c" + "${LIBRARY_DIR}/lib/http.c" + "${LIBRARY_DIR}/lib/http2.c" + "${LIBRARY_DIR}/lib/http_aws_sigv4.c" + "${LIBRARY_DIR}/lib/http_chunks.c" + "${LIBRARY_DIR}/lib/http_digest.c" + "${LIBRARY_DIR}/lib/http_negotiate.c" + "${LIBRARY_DIR}/lib/http_ntlm.c" + "${LIBRARY_DIR}/lib/http_proxy.c" + "${LIBRARY_DIR}/lib/idn.c" + "${LIBRARY_DIR}/lib/if2ip.c" + "${LIBRARY_DIR}/lib/imap.c" "${LIBRARY_DIR}/lib/inet_ntop.c" + "${LIBRARY_DIR}/lib/inet_pton.c" + "${LIBRARY_DIR}/lib/krb5.c" + "${LIBRARY_DIR}/lib/ldap.c" + "${LIBRARY_DIR}/lib/llist.c" + "${LIBRARY_DIR}/lib/md4.c" + "${LIBRARY_DIR}/lib/md5.c" + "${LIBRARY_DIR}/lib/memdebug.c" + "${LIBRARY_DIR}/lib/mime.c" + "${LIBRARY_DIR}/lib/mprintf.c" + "${LIBRARY_DIR}/lib/mqtt.c" + "${LIBRARY_DIR}/lib/multi.c" + "${LIBRARY_DIR}/lib/netrc.c" + "${LIBRARY_DIR}/lib/nonblock.c" + "${LIBRARY_DIR}/lib/noproxy.c" + "${LIBRARY_DIR}/lib/openldap.c" "${LIBRARY_DIR}/lib/parsedate.c" + "${LIBRARY_DIR}/lib/pingpong.c" + "${LIBRARY_DIR}/lib/pop3.c" + "${LIBRARY_DIR}/lib/progress.c" + "${LIBRARY_DIR}/lib/psl.c" + "${LIBRARY_DIR}/lib/rand.c" + "${LIBRARY_DIR}/lib/rename.c" + "${LIBRARY_DIR}/lib/rtsp.c" "${LIBRARY_DIR}/lib/select.c" - "${LIBRARY_DIR}/lib/splay.c" - "${LIBRARY_DIR}/lib/strdup.c" + "${LIBRARY_DIR}/lib/sendf.c" + "${LIBRARY_DIR}/lib/setopt.c" + "${LIBRARY_DIR}/lib/sha256.c" + "${LIBRARY_DIR}/lib/share.c" + "${LIBRARY_DIR}/lib/slist.c" + "${LIBRARY_DIR}/lib/smb.c" + "${LIBRARY_DIR}/lib/smtp.c" + "${LIBRARY_DIR}/lib/socketpair.c" "${LIBRARY_DIR}/lib/socks.c" - "${LIBRARY_DIR}/lib/curl_addrinfo.c" "${LIBRARY_DIR}/lib/socks_gssapi.c" "${LIBRARY_DIR}/lib/socks_sspi.c" - "${LIBRARY_DIR}/lib/curl_sspi.c" - "${LIBRARY_DIR}/lib/slist.c" - "${LIBRARY_DIR}/lib/nonblock.c" - "${LIBRARY_DIR}/lib/curl_memrchr.c" - "${LIBRARY_DIR}/lib/imap.c" - "${LIBRARY_DIR}/lib/pop3.c" - "${LIBRARY_DIR}/lib/smtp.c" - "${LIBRARY_DIR}/lib/pingpong.c" - "${LIBRARY_DIR}/lib/rtsp.c" - "${LIBRARY_DIR}/lib/curl_threads.c" - "${LIBRARY_DIR}/lib/warnless.c" - "${LIBRARY_DIR}/lib/hmac.c" - "${LIBRARY_DIR}/lib/curl_rtmp.c" - "${LIBRARY_DIR}/lib/openldap.c" - "${LIBRARY_DIR}/lib/curl_gethostname.c" - "${LIBRARY_DIR}/lib/gopher.c" - "${LIBRARY_DIR}/lib/http_proxy.c" - "${LIBRARY_DIR}/lib/asyn-thread.c" - "${LIBRARY_DIR}/lib/curl_gssapi.c" - "${LIBRARY_DIR}/lib/http_ntlm.c" - "${LIBRARY_DIR}/lib/curl_ntlm_wb.c" - "${LIBRARY_DIR}/lib/curl_ntlm_core.c" - "${LIBRARY_DIR}/lib/curl_sasl.c" - "${LIBRARY_DIR}/lib/rand.c" - "${LIBRARY_DIR}/lib/curl_multibyte.c" - "${LIBRARY_DIR}/lib/conncache.c" - "${LIBRARY_DIR}/lib/cf-h1-proxy.c" - "${LIBRARY_DIR}/lib/http2.c" - "${LIBRARY_DIR}/lib/smb.c" - "${LIBRARY_DIR}/lib/curl_endian.c" - "${LIBRARY_DIR}/lib/curl_des.c" + "${LIBRARY_DIR}/lib/speedcheck.c" + "${LIBRARY_DIR}/lib/splay.c" + "${LIBRARY_DIR}/lib/strcase.c" + "${LIBRARY_DIR}/lib/strdup.c" + "${LIBRARY_DIR}/lib/strerror.c" + "${LIBRARY_DIR}/lib/strtok.c" + "${LIBRARY_DIR}/lib/strtoofft.c" "${LIBRARY_DIR}/lib/system_win32.c" - "${LIBRARY_DIR}/lib/mime.c" - "${LIBRARY_DIR}/lib/sha256.c" - "${LIBRARY_DIR}/lib/setopt.c" - "${LIBRARY_DIR}/lib/curl_path.c" - "${LIBRARY_DIR}/lib/curl_range.c" - "${LIBRARY_DIR}/lib/psl.c" - "${LIBRARY_DIR}/lib/doh.c" - "${LIBRARY_DIR}/lib/urlapi.c" - "${LIBRARY_DIR}/lib/curl_trc.c" - "${LIBRARY_DIR}/lib/curl_get_line.c" - "${LIBRARY_DIR}/lib/altsvc.c" - "${LIBRARY_DIR}/lib/socketpair.c" - "${LIBRARY_DIR}/lib/bufref.c" - "${LIBRARY_DIR}/lib/bufq.c" - "${LIBRARY_DIR}/lib/dynbuf.c" - "${LIBRARY_DIR}/lib/dynhds.c" - "${LIBRARY_DIR}/lib/hsts.c" - "${LIBRARY_DIR}/lib/http_aws_sigv4.c" - "${LIBRARY_DIR}/lib/mqtt.c" - "${LIBRARY_DIR}/lib/rename.c" - "${LIBRARY_DIR}/lib/headers.c" + "${LIBRARY_DIR}/lib/telnet.c" "${LIBRARY_DIR}/lib/timediff.c" - "${LIBRARY_DIR}/lib/vauth/vauth.c" + "${LIBRARY_DIR}/lib/timeval.c" + "${LIBRARY_DIR}/lib/transfer.c" + "${LIBRARY_DIR}/lib/url.c" + "${LIBRARY_DIR}/lib/urlapi.c" "${LIBRARY_DIR}/lib/vauth/cleartext.c" "${LIBRARY_DIR}/lib/vauth/cram.c" "${LIBRARY_DIR}/lib/vauth/digest.c" @@ -138,21 +135,24 @@ set (SRCS "${LIBRARY_DIR}/lib/vauth/oauth2.c" "${LIBRARY_DIR}/lib/vauth/spnego_gssapi.c" "${LIBRARY_DIR}/lib/vauth/spnego_sspi.c" + "${LIBRARY_DIR}/lib/vauth/vauth.c" + "${LIBRARY_DIR}/lib/version.c" "${LIBRARY_DIR}/lib/vquic/vquic.c" - "${LIBRARY_DIR}/lib/vtls/openssl.c" + "${LIBRARY_DIR}/lib/vssh/libssh.c" + "${LIBRARY_DIR}/lib/vssh/libssh2.c" + "${LIBRARY_DIR}/lib/vtls/bearssl.c" "${LIBRARY_DIR}/lib/vtls/gtls.c" - "${LIBRARY_DIR}/lib/vtls/vtls.c" - "${LIBRARY_DIR}/lib/vtls/wolfssl.c" + "${LIBRARY_DIR}/lib/vtls/hostcheck.c" + "${LIBRARY_DIR}/lib/vtls/keylog.c" + "${LIBRARY_DIR}/lib/vtls/mbedtls.c" + "${LIBRARY_DIR}/lib/vtls/openssl.c" "${LIBRARY_DIR}/lib/vtls/schannel.c" "${LIBRARY_DIR}/lib/vtls/schannel_verify.c" "${LIBRARY_DIR}/lib/vtls/sectransp.c" - "${LIBRARY_DIR}/lib/vtls/mbedtls.c" - "${LIBRARY_DIR}/lib/vtls/bearssl.c" - "${LIBRARY_DIR}/lib/vtls/keylog.c" + "${LIBRARY_DIR}/lib/vtls/vtls.c" + "${LIBRARY_DIR}/lib/vtls/wolfssl.c" "${LIBRARY_DIR}/lib/vtls/x509asn1.c" - "${LIBRARY_DIR}/lib/vtls/hostcheck.c" - "${LIBRARY_DIR}/lib/vssh/libssh2.c" - "${LIBRARY_DIR}/lib/vssh/libssh.c" + "${LIBRARY_DIR}/lib/warnless.c" ) add_library (_curl ${SRCS}) From d5ed014ec4e4a2a0c49ac95a193aa0c15a511f4c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Aug 2023 22:56:32 +0300 Subject: [PATCH 2024/2047] Fix flaky test `02443_detach_attach_partition` (#53478) * fix flaky test * empty commit --- .../02443_detach_attach_partition.reference | 4 ++-- .../02443_detach_attach_partition.sh | 18 ++++++++++++++---- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02443_detach_attach_partition.reference b/tests/queries/0_stateless/02443_detach_attach_partition.reference index 70930ea6d9a..77cfb77479d 100644 --- a/tests/queries/0_stateless/02443_detach_attach_partition.reference +++ b/tests/queries/0_stateless/02443_detach_attach_partition.reference @@ -1,4 +1,4 @@ default begin inserts default end inserts -20 210 -20 210 +30 465 +30 465 diff --git a/tests/queries/0_stateless/02443_detach_attach_partition.sh b/tests/queries/0_stateless/02443_detach_attach_partition.sh index 5a3f1b64065..ae104b833e3 100755 --- a/tests/queries/0_stateless/02443_detach_attach_partition.sh +++ b/tests/queries/0_stateless/02443_detach_attach_partition.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, zookeeper, no-parallel +# Tags: race, zookeeper, long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -29,9 +29,19 @@ function thread_attach() done } +insert_type=$(($RANDOM % 3)) +$CLICKHOUSE_CLIENT -q "SELECT '$CLICKHOUSE_DATABASE', 'insert_type $insert_type' FORMAT Null" + function insert() { - $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $i" 2>/dev/null + # Fault injection may lead to duplicates + if [[ "$insert_type" -eq 0 ]]; then + $CLICKHOUSE_CLIENT --insert_deduplication_token=$1 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null + elif [[ "$insert_type" -eq 1 ]]; then + $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $1, $1" 2>/dev/null + else + $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null + fi } thread_detach & PID_1=$! @@ -41,8 +51,8 @@ thread_attach & PID_4=$! function do_inserts() { - for i in {1..20}; do - while ! insert; do $CLICKHOUSE_CLIENT -q "SELECT '$CLICKHOUSE_DATABASE', 'retrying insert $i' FORMAT Null"; done + for i in {1..30}; do + while ! insert $i; do $CLICKHOUSE_CLIENT -q "SELECT '$CLICKHOUSE_DATABASE', 'retrying insert $i' FORMAT Null"; done done } From 30ccfa1638356dafaa7b84c8bdb11077874aa99a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Aug 2023 20:23:10 +0000 Subject: [PATCH 2025/2047] Fix wrong columns order for queries with parallel FINAL. --- src/Processors/QueryPlan/PartsSplitter.cpp | 27 +++++++++++++++++++ ...rallel_final_wrong_columns_order.reference | 1 + ...841_parallel_final_wrong_columns_order.sql | 5 ++++ 3 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.reference create mode 100644 tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index 533fbde1e13..61c6422de5a 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -254,6 +254,32 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +static void reorderColumns(ActionsDAG & dag, const Block & header, const std::string & filter_column) +{ + std::unordered_map inputs_map; + for (const auto * input : dag.getInputs()) + inputs_map[input->result_name] = input; + + for (const auto & col : header) + { + auto & input = inputs_map[col.name]; + if (!input) + input = &dag.addInput(col); + } + + ActionsDAG::NodeRawConstPtrs new_outputs; + new_outputs.reserve(header.columns() + 1); + + new_outputs.push_back(&dag.findInOutputs(filter_column)); + for (const auto & col : header) + { + auto & input = inputs_map[col.name]; + new_outputs.push_back(input); + } + + dag.getOutputs() = std::move(new_outputs); +} + Pipes buildPipesForReadingByPKRanges( const KeyDescription & primary_key, ExpressionActionsPtr sorting_expr, @@ -279,6 +305,7 @@ Pipes buildPipesForReadingByPKRanges( continue; auto syntax_result = TreeRewriter(context).analyze(filter_function, primary_key.expression->getRequiredColumnsWithTypes()); auto actions = ExpressionAnalyzer(filter_function, syntax_result, context).getActionsDAG(false); + reorderColumns(*actions, pipes[i].getHeader(), filter_function->getColumnName()); ExpressionActionsPtr expression_actions = std::make_shared(std::move(actions)); auto description = fmt::format( "filter values in [{}, {})", i ? ::toString(borders[i - 1]) : "-inf", i < borders.size() ? ::toString(borders[i]) : "+inf"); diff --git a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.reference b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.reference new file mode 100644 index 00000000000..749fce669df --- /dev/null +++ b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.reference @@ -0,0 +1 @@ +1000000 diff --git a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql new file mode 100644 index 00000000000..12ff8972761 --- /dev/null +++ b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql @@ -0,0 +1,5 @@ +drop table if exists tab2; +create table tab2 (id String, version Int64, l String, accountCode String, z Int32) engine = ReplacingMergeTree(z) PRIMARY KEY (accountCode, id) ORDER BY (accountCode, id, version, l); +insert into tab2 select toString(number), number, toString(number), toString(number), 0 from numbers(1e6); +set max_threads=2; +select count() from tab2 final; From 91c241532fefcab27f0277c8ebc0538645aeaa69 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Aug 2023 20:26:23 +0000 Subject: [PATCH 2026/2047] Add a check for header structure. --- src/Processors/Transforms/FilterSortedStreamByRange.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/Transforms/FilterSortedStreamByRange.h b/src/Processors/Transforms/FilterSortedStreamByRange.h index e1141ebd299..7f38c10d691 100644 --- a/src/Processors/Transforms/FilterSortedStreamByRange.h +++ b/src/Processors/Transforms/FilterSortedStreamByRange.h @@ -27,6 +27,9 @@ public: true) , filter_transform(header_, expression_, filter_column_name_, remove_filter_column_, on_totals_) { + assertBlocksHaveEqualStructure( + header_, getOutputPort().getHeader(), + "Expression for FilterSortedStreamByRange shoud not change header"); } String getName() const override { return "FilterSortedStreamByRange"; } From 08e4e2bb0b1f2ce790f0f1b6cf8dbad45db330f2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Aug 2023 22:35:45 +0200 Subject: [PATCH 2027/2047] fix --- src/Client/ClientBase.cpp | 8 ++- .../ReplaceQueryParameterVisitor.cpp | 7 ++- .../ReplaceQueryParameterVisitor.h | 4 ++ src/Interpreters/executeQuery.cpp | 5 +- .../01018_dictionaries_from_dictionaries.sql | 59 ++++++++++--------- ...rfluous_dict_reload_on_create_database.sql | 38 ++++++------ 6 files changed, 71 insertions(+), 50 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9092f12b62b..495e3bdfd4e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -847,7 +847,9 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa visitor.visit(parsed_query); /// Get new query after substitutions. - query = serializeAST(*parsed_query); + if (visitor.getNumberOfReplacedParameters()) + query = serializeAST(*parsed_query); + chassert(!query.empty()); } if (allow_merge_tree_settings && parsed_query->as()) @@ -1332,7 +1334,9 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars visitor.visit(parsed_query); /// Get new query after substitutions. - query = serializeAST(*parsed_query); + if (visitor.getNumberOfReplacedParameters()) + query = serializeAST(*parsed_query); + chassert(!query.empty()); } /// Process the query that requires transferring data blocks to the server. diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 89a40baaff9..46dcc6129bc 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -66,10 +66,11 @@ void ReplaceQueryParameterVisitor::visitChildren(ASTPtr & ast) const String & ReplaceQueryParameterVisitor::getParamValue(const String & name) { auto search = query_parameters.find(name); - if (search != query_parameters.end()) - return search->second; - else + if (search == query_parameters.end()) throw Exception(ErrorCodes::UNKNOWN_QUERY_PARAMETER, "Substitution {} is not set", backQuote(name)); + + ++num_replaced_parameters; + return search->second; } void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.h b/src/Interpreters/ReplaceQueryParameterVisitor.h index dd785cd768e..7d5da7ea85b 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.h +++ b/src/Interpreters/ReplaceQueryParameterVisitor.h @@ -20,8 +20,12 @@ public: void visit(ASTPtr & ast); + size_t getNumberOfReplacedParameters() const { return num_replaced_parameters; } + private: const NameToNameMap & query_parameters; + size_t num_replaced_parameters = 0; + const String & getParamValue(const String & name); void visitIdentifier(ASTPtr & ast); void visitQueryParameter(ASTPtr & ast); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 1e6843f65e6..bac6807b682 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -735,7 +735,10 @@ static std::tuple executeQueryImpl( { ReplaceQueryParameterVisitor visitor(context->getQueryParameters()); visitor.visit(ast); - query = serializeAST(*ast); + if (visitor.getNumberOfReplacedParameters()) + query = serializeAST(*ast); + else + query.assign(begin, query_end); } else { diff --git a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql index d958d7d58ef..e72e113f859 100644 --- a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql +++ b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql @@ -1,7 +1,12 @@ +-- Tags: no-parallel SET send_logs_level = 'fatal'; -CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict +DROP DATABASE IF EXISTS database_for_dict; + +CREATE DATABASE database_for_dict; + +CREATE TABLE database_for_dict.table_for_dict ( key_column UInt64, second_column UInt8, @@ -11,9 +16,9 @@ CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); +INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); -CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 +CREATE DICTIONARY database_for_dict.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -21,13 +26,13 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; +SELECT count(*) from database_for_dict.dict1; -CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict2 +CREATE DICTIONARY database_for_dict.dict2 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -35,20 +40,20 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict2 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict1' DB currentDatabase())) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict1' DB 'database_for_dict')) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict2; +SELECT count(*) FROM database_for_dict.dict2; -INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100, 100); +INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100, 100); SYSTEM RELOAD DICTIONARIES; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; +SELECT count(*) from database_for_dict.dict2; +SELECT count(*) from database_for_dict.dict1; -CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict3 +CREATE DICTIONARY database_for_dict.dict3 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -56,22 +61,22 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict3 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict2' DB currentDatabase())) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict2' DB 'database_for_dict')) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict3; +SELECT count(*) FROM database_for_dict.dict3; -INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(200, 100); +INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(200, 100); SYSTEM RELOAD DICTIONARIES; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict3; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; +SELECT count(*) from database_for_dict.dict3; +SELECT count(*) from database_for_dict.dict2; +SELECT count(*) from database_for_dict.dict1; -CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict4 +CREATE DICTIONARY database_for_dict.dict4 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -79,17 +84,17 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict4 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'non_existing_table' DB currentDatabase())) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'non_existing_table' DB 'database_for_dict')) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict4; -- {serverError 60} +SELECT count(*) FROM database_for_dict.dict4; -- {serverError 60} -SELECT name from system.tables WHERE database = currentDatabase() ORDER BY name; -SELECT name from system.dictionaries WHERE database = currentDatabase() ORDER BY name; +SELECT name from system.tables WHERE database = 'database_for_dict' ORDER BY name; +SELECT name from system.dictionaries WHERE database = 'database_for_dict' ORDER BY name; -DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; +DROP DATABASE IF EXISTS database_for_dict; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict3; --{serverError 81} -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; --{serverError 81} -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; --{serverError 81} +SELECT count(*) from database_for_dict.dict3; --{serverError 81} +SELECT count(*) from database_for_dict.dict2; --{serverError 81} +SELECT count(*) from database_for_dict.dict1; --{serverError 81} diff --git a/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql b/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql index bbd848f81d2..9040d7b3231 100644 --- a/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql +++ b/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql @@ -1,31 +1,35 @@ +-- Tags: no-parallel -CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.dict_data (key UInt64, val UInt64) Engine=Memory(); -CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict +DROP DATABASE IF EXISTS dict_db_01036; +CREATE DATABASE dict_db_01036; + +CREATE TABLE dict_db_01036.dict_data (key UInt64, val UInt64) Engine=Memory(); +CREATE DICTIONARY dict_db_01036.dict ( key UInt64 DEFAULT 0, val UInt64 DEFAULT 10 ) PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB currentDatabase())) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01036')) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); -SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; -SELECT dictGetUInt64({CLICKHOUSE_DATABASE:String}||'.dict', 'val', toUInt64(0)); -SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; SELECT 'SYSTEM RELOAD DICTIONARY'; -SYSTEM RELOAD DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict; -SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; -SELECT dictGetUInt64({CLICKHOUSE_DATABASE:String}||'.dict', 'val', toUInt64(0)); -SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SYSTEM RELOAD DICTIONARY dict_db_01036.dict; +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; SELECT 'CREATE DATABASE'; -DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; -CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; -SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +DROP DATABASE IF EXISTS empty_db_01036; +CREATE DATABASE empty_db_01036; +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; -DROP DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict; -DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.dict_data; -DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; -DROP DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; +DROP DICTIONARY dict_db_01036.dict; +DROP TABLE dict_db_01036.dict_data; +DROP DATABASE dict_db_01036; +DROP DATABASE empty_db_01036; From 3a6a3d491efaca938ef6ba709ae9d5136b22f057 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Aug 2023 20:40:28 +0000 Subject: [PATCH 2028/2047] Fix typo --- src/Processors/Transforms/FilterSortedStreamByRange.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FilterSortedStreamByRange.h b/src/Processors/Transforms/FilterSortedStreamByRange.h index 7f38c10d691..e3d3f6f10ef 100644 --- a/src/Processors/Transforms/FilterSortedStreamByRange.h +++ b/src/Processors/Transforms/FilterSortedStreamByRange.h @@ -29,7 +29,7 @@ public: { assertBlocksHaveEqualStructure( header_, getOutputPort().getHeader(), - "Expression for FilterSortedStreamByRange shoud not change header"); + "Expression for FilterSortedStreamByRange should not change header"); } String getName() const override { return "FilterSortedStreamByRange"; } From 970eb24067ac54ed709a6ca4823b37e8d891a578 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Aug 2023 22:57:49 +0200 Subject: [PATCH 2029/2047] fix --- tests/integration/test_grpc_protocol/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 5b56c3da8dc..c149e1a4096 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -352,13 +352,13 @@ def test_authentication(): def test_logs(): - query = "SELECT has(groupArray(number), 42) FROM numbers(1000)" + query = "SELECT has(groupArray(number), 42) FROM numbers(100000)" logs = query_and_get_logs( query, settings={"send_logs_level": "debug"}, ) assert query in logs - assert "Read 1 rows" in logs + assert "Read 100000 rows" in logs assert "Peak memory usage" in logs From 3c36af111d559a883d1c85662a9eacd02309531f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Aug 2023 23:47:46 +0200 Subject: [PATCH 2030/2047] Update style checker --- docker/test/style/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index bd1c26855b7..a4feae27c67 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -1,5 +1,5 @@ # docker build -t clickhouse/style-test . -FROM ubuntu:20.04 +FROM ubuntu:22.04 ARG ACT_VERSION=0.2.33 ARG ACTIONLINT_VERSION=1.6.22 From a0a6ec18095db249b124414e6e7d524018de6205 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Aug 2023 01:15:40 +0300 Subject: [PATCH 2031/2047] Update 02841_parallel_final_wrong_columns_order.sql --- .../0_stateless/02841_parallel_final_wrong_columns_order.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql index 12ff8972761..3f4658e6abe 100644 --- a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql +++ b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql @@ -3,3 +3,4 @@ create table tab2 (id String, version Int64, l String, accountCode String, z Int insert into tab2 select toString(number), number, toString(number), toString(number), 0 from numbers(1e6); set max_threads=2; select count() from tab2 final; +DROP TABLE tab2; From 53fdbac7b039fba7aa6f68fb88c73e350d00a0fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Aug 2023 01:16:58 +0300 Subject: [PATCH 2032/2047] Update 02841_parallel_final_wrong_columns_order.sql --- .../0_stateless/02841_parallel_final_wrong_columns_order.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql index 3f4658e6abe..8a28388cf15 100644 --- a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql +++ b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql @@ -1,3 +1,6 @@ +-- Tags: no-random-settings +-- Because we insert one million rows, it shouldn't choose too low index granularity. + drop table if exists tab2; create table tab2 (id String, version Int64, l String, accountCode String, z Int32) engine = ReplacingMergeTree(z) PRIMARY KEY (accountCode, id) ORDER BY (accountCode, id, version, l); insert into tab2 select toString(number), number, toString(number), toString(number), 0 from numbers(1e6); From 368191af2fc55e528691765a4a564fa46fc14e76 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Aug 2023 01:18:00 +0300 Subject: [PATCH 2033/2047] Update 02841_parallel_final_wrong_columns_order.sql --- .../0_stateless/02841_parallel_final_wrong_columns_order.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql index 8a28388cf15..db15abb28cb 100644 --- a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql +++ b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql @@ -1,4 +1,4 @@ --- Tags: no-random-settings +-- Tags: no-random-merge-tree-settings -- Because we insert one million rows, it shouldn't choose too low index granularity. drop table if exists tab2; From d7272410bdc8d9f19830a6acb01e165471cdd2ff Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 17 Aug 2023 00:42:51 +0200 Subject: [PATCH 2034/2047] the worst fix for a deadlock ever --- src/Interpreters/InterpreterDropQuery.cpp | 3 ++- src/Interpreters/InterpreterDropQuery.h | 2 +- src/Storages/StorageMaterializedView.cpp | 7 +++++-- tests/clickhouse-test | 8 ++++---- 4 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ed927d550a8..55256e8f9f7 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -432,8 +432,9 @@ AccessRightsElements InterpreterDropQuery::getRequiredAccessForDDLOnCluster() co } void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, - const StorageID & target_table_id, bool sync, bool ignore_sync_setting) + const StorageID & target_table_id, bool sync, bool ignore_sync_setting, bool need_ddl_guard) { + auto ddl_guard = (need_ddl_guard ? DatabaseCatalog::instance().getDDLGuard(target_table_id.database_name, target_table_id.table_name) : nullptr); if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context)) { /// We create and execute `drop` query for internal table. diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index af7a4ddef25..7ae544a7356 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -25,7 +25,7 @@ public: BlockIO execute() override; static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, - const StorageID & target_table_id, bool sync, bool ignore_sync_setting = false); + const StorageID & target_table_id, bool sync, bool ignore_sync_setting = false, bool need_ddl_guard = false); bool supportsTransactions() const override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 16d724d54d8..7354dd56552 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -228,10 +228,13 @@ void StorageMaterializedView::dropInnerTableIfAny(bool sync, ContextPtr local_co { /// We will use `sync` argument wneh this function is called from a DROP query /// and will ignore database_atomic_wait_for_drop_and_detach_synchronously when it's called from drop task. - /// See the comment in StorageMaterializedView::drop + /// See the comment in StorageMaterializedView::drop. + /// DDL queries with StorageMaterializedView are fundamentally broken. + /// Best-effort to make them work: the inner table name is almost always less than the MV name (so it's safe to lock DDLGuard) + bool may_lock_ddl_guard = getStorageID().getQualifiedName() < target_table_id.getQualifiedName(); if (has_inner_table && tryGetTargetTable()) InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, - sync, /* ignore_sync_setting */ true); + sync, /* ignore_sync_setting */ true, may_lock_ddl_guard); } void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 1ce5ad981ad..49a56476a41 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -383,19 +383,19 @@ def get_stacktraces_from_clickhouse(args): ) replicated_msg = ( f"{args.client} {settings_str} --query " - '"SELECT materialize((hostName(), tcpPort())) as host, thread_id, ' + '"SELECT materialize((hostName(), tcpPort())) as host, thread_name, thread_id, query_id, trace, ' "arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), " "arrayMap(x -> addressToLine(x), trace), " - "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace " + "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace_str " "FROM clusterAllReplicas('test_cluster_database_replicated', 'system.stack_trace') " 'ORDER BY host, thread_id FORMAT Vertical"' ) msg = ( f"{args.client} {settings_str} --query " - "\"SELECT arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), " + "\"SELECT thread_name, thread_id, query_id, trace, arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), " "arrayMap(x -> addressToLine(x), trace), " - "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace " + "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace_str " 'FROM system.stack_trace FORMAT Vertical"' ) From 87685a7c0c824ba9cdcd64115b491302f1ead5a4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 17 Aug 2023 00:49:00 +0200 Subject: [PATCH 2035/2047] Address shellcheck issues --- tests/queries/0_stateless/00301_csv.sh | 10 +++++----- .../0_stateless/00900_long_parquet_load.sh | 6 +++--- .../0_stateless/01747_system_session_log_long.sh | 16 ++++++++++------ .../0_stateless/02833_concurrrent_sessions.sh | 15 ++++++++++----- tests/queries/0_stateless/replication.lib | 4 +++- 5 files changed, 31 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/00301_csv.sh b/tests/queries/0_stateless/00301_csv.sh index 80053c99a17..2e2e1384534 100755 --- a/tests/queries/0_stateless/00301_csv.sh +++ b/tests/queries/0_stateless/00301_csv.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo === Test input_format_csv_empty_as_default +echo '=== Test input_format_csv_empty_as_default' $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS csv"; $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64 DEFAULT 1, d Date DEFAULT '2019-06-19') ENGINE = Memory"; @@ -19,7 +19,7 @@ Hello "world", 789 ,2016-01-03 $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY d, s"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; -echo === Test datetime +echo '=== Test datetime' $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (t DateTime('Asia/Istanbul'), s String) ENGINE = Memory"; echo '"2016-01-01 01:02:03","1" @@ -30,7 +30,7 @@ echo '"2016-01-01 01:02:03","1" $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; -echo === Test nullable datetime +echo '=== Test nullable datetime' $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (t Nullable(DateTime('Asia/Istanbul')), s Nullable(String)) ENGINE = Memory"; echo 'NULL, NULL @@ -41,7 +41,7 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s NULLS LAST"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; -echo === Test ignore extra columns +echo '=== Test ignore extra columns' $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64 DEFAULT 3, d String DEFAULT 'String4') ENGINE = Memory"; echo '"Hello", 1, "String1" @@ -55,7 +55,7 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s, n"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; -echo === Test missing as default +echo '=== Test missing as default' $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (f1 String, f2 UInt64, f3 UInt256, f4 UInt64 Default 33, f5 Nullable(UInt64), f6 Nullable(UInt64) Default 55, f7 String DEFAULT 'Default') ENGINE = Memory"; echo ' diff --git a/tests/queries/0_stateless/00900_long_parquet_load.sh b/tests/queries/0_stateless/00900_long_parquet_load.sh index 60317e1671c..2f19cce9107 100755 --- a/tests/queries/0_stateless/00900_long_parquet_load.sh +++ b/tests/queries/0_stateless/00900_long_parquet_load.sh @@ -43,9 +43,9 @@ for NAME in $(find "$DATA_DIR"/*.parquet -print0 | xargs -0 -n 1 basename | LC_A JSON=$DATA_DIR/$NAME.json COLUMNS_FILE=$DATA_DIR/$NAME.columns - ([ -z "$PARQUET_READER" ] || [ ! -s "$PARQUET_READER" ]) && [ ! -s "$COLUMNS_FILE" ] && continue + { [ -z "$PARQUET_READER" ] || [ ! -s "$PARQUET_READER" ]; } && [ ! -s "$COLUMNS_FILE" ] && continue - echo === Try load data from "$NAME" + echo "=== Try load data from $NAME" # If you want change or add .parquet file - rm data_parquet/*.json data_parquet/*.columns [ -n "$PARQUET_READER" ] && [ ! -s "$COLUMNS_FILE" ] && [ ! -s "$JSON" ] && "$PARQUET_READER" --json "$DATA_DIR"/"$NAME" > "$JSON" @@ -64,7 +64,7 @@ CREATE TABLE parquet_load ($COLUMNS) ENGINE = Memory; EOF # Some files contain unsupported data structures, exception is ok. - cat "$DATA_DIR"/"$NAME" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_load FORMAT Parquet" 2>&1 | sed 's/Exception/Ex---tion/' + ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_load FORMAT Parquet" < "$DATA_DIR"/"$NAME" 2>&1 | sed 's/Exception/Ex---tion/' ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load ORDER BY tuple(*) LIMIT 100" ${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_load" diff --git a/tests/queries/0_stateless/01747_system_session_log_long.sh b/tests/queries/0_stateless/01747_system_session_log_long.sh index c6e93f4abd7..6bd58facb51 100755 --- a/tests/queries/0_stateless/01747_system_session_log_long.sh +++ b/tests/queries/0_stateless/01747_system_session_log_long.sh @@ -33,8 +33,10 @@ set -eu # Since there is no way to cleanup system.session_log table, # make sure that we can identify log entries from this test by a random user name. -readonly BASE_USERNAME="session_log_test_user_$(cat /dev/urandom | tr -cd 'a-f0-9' | head -c 32)" -readonly TMP_QUERY_FILE=$(mktemp /tmp/tmp_query.log.XXXXXX) +BASE_USERNAME="session_log_test_user_$(tr -cd 'a-f0-9' < /dev/urandom | head -c 32)" +readonly BASE_USERNAME +TMP_QUERY_FILE=$(mktemp /tmp/tmp_query.log.XXXXXX) +readonly TMP_QUERY_FILE declare -a ALL_USERNAMES ALL_USERNAMES+=("${BASE_USERNAME}") @@ -80,7 +82,7 @@ trap "cleanup" EXIT function executeQueryExpectError() { cat - > "${TMP_QUERY_FILE}" - ! ${CLICKHOUSE_CLIENT} --multiquery --queries-file "${TMP_QUERY_FILE}" "${@}" 2>&1 | tee -a ${TMP_QUERY_FILE} + ! ${CLICKHOUSE_CLIENT} --multiquery --queries-file "${TMP_QUERY_FILE}" "${@}" 2>&1 | tee -a "${TMP_QUERY_FILE}" } function createUser() @@ -95,7 +97,8 @@ function createUser() elif [[ "${auth_type}" == "plaintext_password" ]] then - password="${password}" + # password="${password}" + : elif [[ "${auth_type}" == "sha256_password" ]] then @@ -198,7 +201,7 @@ function testHTTPNamedSession() { echo "HTTP endpoint with named session" local HTTP_SESSION_ID - HTTP_SESSION_ID="session_id_$(cat /dev/urandom | tr -cd 'a-f0-9' | head -c 32)" + HTTP_SESSION_ID="session_id_$(tr -cd 'a-f0-9' < /dev/urandom | head -c 32)" if [ -v CLICKHOUSE_URL_PARAMS ] then CLICKHOUSE_URL_WITH_SESSION_ID="${CLICKHOUSE_URL}&session_id=${HTTP_SESSION_ID}" @@ -343,7 +346,8 @@ SET DEFAULT ROLE session_log_test_role, session_log_test_role2 TO ${username}; } # to cut off previous runs -readonly start_time="$(executeQuery <<< 'SELECT now64(6);')" +start_time="$(executeQuery <<< 'SELECT now64(6);')" +readonly start_time # Special case: user and profile are both defined in XML runEndpointTests "User with profile from XML" "no_password" "session_log_test_xml_user" '' diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh index 26b48462a76..c5b6204529b 100755 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -14,11 +14,16 @@ readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_$ readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) -readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly TCP_USERS_SQL_COLLECTION_STRING +HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_USERS_SQL_COLLECTION_STRING +HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING +MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly MYSQL_USERS_SQL_COLLECTION_STRING +ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly ALL_USERS_SQL_COLLECTION_STRING readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 1805b56f8dc..e0e11990d83 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -37,7 +37,7 @@ function try_sync_replicas() pids[${i}]=$! i=$((i + 1)) done - for pid in ${pids[*]}; do + for pid in "${pids[@]}"; do wait $pid || (echo "Failed to sync some replicas" && exit 1) done echo "Replication did not hang: synced all replicas of $table_name_prefix" @@ -114,3 +114,5 @@ function check_replication_consistency() fi } + +# vi: ft=bash From 857856b8b674c46e4c768780efdc9631a1fdcc87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Aug 2023 03:58:32 +0200 Subject: [PATCH 2036/2047] Leave only simplifications --- docker/test/performance-comparison/compare.sh | 38 ------------------- tests/ci/performance_comparison_check.py | 18 --------- 2 files changed, 56 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index ce8c4903c00..4b1b5c13b9b 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -63,25 +63,6 @@ function left_or_right() function configure { - # Setup a cluster for logs export to ClickHouse Cloud - # Note: these variables are provided to the Docker run command by the Python script in tests/ci - if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] - then - set +x - echo " -remote_servers: - system_logs_export: - shard: - replica: - secure: 1 - user: ci - host: '${CLICKHOUSE_CI_LOGS_HOST}' - port: 9440 - password: '${CLICKHOUSE_CI_LOGS_PASSWORD}' -" > right/config/config.d/system_logs_export.yaml - set -x - fi - # Use the new config for both servers, so that we can change it in a PR. rm right/config/config.d/text_log.xml ||: cp -rv right/config left ||: @@ -111,25 +92,6 @@ remote_servers: wait_for_server $LEFT_SERVER_PORT $left_pid echo "Server for setup started" - # Initialize export of system logs to ClickHouse Cloud - # Note: it is set up for the "left" server, and its database is then cloned to the "right" server. - if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] - then - ( - set +x - export EXTRA_COLUMNS_EXPRESSION="$PR_TO_TEST AS pull_request_number, '$SHA_TO_TEST' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type" - export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}" - - /setup_export_logs.sh "--port $LEFT_SERVER_PORT" - - # Unset variables after use - export CONNECTION_PARAMETERS='' - export CLICKHOUSE_CI_LOGS_HOST='' - export CLICKHOUSE_CI_LOGS_PASSWORD='' - set -x - ) - fi - clickhouse-client --port $LEFT_SERVER_PORT --query "create database test" ||: clickhouse-client --port $LEFT_SERVER_PORT --query "rename table datasets.hits_v1 to test.hits" ||: diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index 70369f9881e..27a67e2ae0e 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -39,8 +39,6 @@ def get_run_command( instance_type = get_instance_type() envs = [ - "-e CLICKHOUSE_CI_LOGS_HOST", - "-e CLICKHOUSE_CI_LOGS_PASSWORD", f"-e CHECK_START_TIME='{check_start_time}'", f"-e CHECK_NAME='{check_name}'", f"-e INSTANCE_TYPE='{instance_type}'", @@ -203,22 +201,6 @@ if __name__ == "__main__": subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) - # Cleanup run log from the credentials of CI logs database. - # Note: a malicious user can still print them by splitting the value into parts. - # But we will be warned when a malicious user modifies CI script. - # Although they can also print them from inside tests. - # Nevertheless, the credentials of the CI logs have limited scope - # and does not provide access to sensitive info. - - ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST", "CLICKHOUSE_CI_LOGS_HOST") - ci_logs_password = os.getenv( - "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" - ) - subprocess.check_call( - f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}' '{compare_log_path}'", - shell=True, - ) - paths = { "compare.log": compare_log_path, "output.7z": os.path.join(result_path, "output.7z"), From c3e6f7e9ae792b54ef713beb8a5513307af119f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Aug 2023 03:59:15 +0200 Subject: [PATCH 2037/2047] Leave only simplifications --- docker/test/base/setup_export_logs.sh | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index ef510552d2f..12fae855b03 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -1,5 +1,4 @@ #!/bin/bash -# shellcheck disable=SC2086 # This script sets up export of system log tables to a remote server. # Remote tables are created if not exist, and augmented with extra columns, @@ -8,7 +7,6 @@ # Pre-configured destination cluster, where to export the data CLUSTER=${CLUSTER:=system_logs_export} -LOCAL_PARAMETERS=$1 EXTRA_COLUMNS=${EXTRA_COLUMNS:="pull_request_number UInt32, commit_sha String, check_start_time DateTime, check_name LowCardinality(String), instance_type LowCardinality(String), "} EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:="0 AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type"} @@ -17,13 +15,13 @@ EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:="check_name, "} CONNECTION_PARAMETERS=${CONNECTION_PARAMETERS:=""} # Create all configured system logs: -clickhouse-client $LOCAL_PARAMETERS --query "SYSTEM FLUSH LOGS" +clickhouse-client --query "SYSTEM FLUSH LOGS" # For each system log table: -clickhouse-client $LOCAL_PARAMETERS --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table +clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table do # Calculate hash of its structure: - hash=$(clickhouse-client $LOCAL_PARAMETERS --query " + hash=$(clickhouse-client --query " SELECT sipHash64(groupArray((name, type))) FROM (SELECT name, type FROM system.columns WHERE database = 'system' AND table = '$table' @@ -31,7 +29,7 @@ do ") # Create the destination table with adapted name and structure: - statement=$(clickhouse-client $LOCAL_PARAMETERS --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e ' + statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e ' s/^\($/('"$EXTRA_COLUMNS"'/; s/ORDER BY \(/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"'/; s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/; @@ -45,7 +43,7 @@ do echo "Creating table system.${table}_sender" >&2 # Create Distributed table and materialized view to watch on the original table: - clickhouse-client $LOCAL_PARAMETERS --query " + clickhouse-client --query " CREATE TABLE system.${table}_sender ENGINE = Distributed(${CLUSTER}, default, ${table}_${hash}) EMPTY AS @@ -55,7 +53,7 @@ do echo "Creating materialized view system.${table}_watcher" >&2 - clickhouse-client $LOCAL_PARAMETERS --query " + clickhouse-client --query " CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS SELECT ${EXTRA_COLUMNS_EXPRESSION}, * FROM system.${table} From 17ca2661a1ce0498883d1fc801c0ddc9528efd66 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 17 Aug 2023 08:47:47 +0200 Subject: [PATCH 2038/2047] Add ability to turn off flush of Distributed on DETACH/DROP/server shutdown Sometimes you can have tons of data there, i.e. few TiBs, and sending them on server shutdown does not looks sane (maybe there is a bug and you need to update/restart to fix flushing). Signed-off-by: Azat Khuzhin --- .../Distributed/DistributedSettings.h | 1 + src/Storages/StorageDistributed.cpp | 6 ++++ ...2860_distributed_flush_on_detach.reference | 27 +++++++++++++++ .../02860_distributed_flush_on_detach.sql | 33 +++++++++++++++++++ 4 files changed, 67 insertions(+) create mode 100644 tests/queries/0_stateless/02860_distributed_flush_on_detach.reference create mode 100644 tests/queries/0_stateless/02860_distributed_flush_on_detach.sql diff --git a/src/Storages/Distributed/DistributedSettings.h b/src/Storages/Distributed/DistributedSettings.h index 86b77707dee..85e64cabfd7 100644 --- a/src/Storages/Distributed/DistributedSettings.h +++ b/src/Storages/Distributed/DistributedSettings.h @@ -26,6 +26,7 @@ class ASTStorage; M(UInt64, monitor_split_batch_on_failure, 0, "Default - distributed_directory_monitor_split_batch_on_failure", 0) \ M(Milliseconds, monitor_sleep_time_ms, 0, "Default - distributed_directory_monitor_sleep_time_ms", 0) \ M(Milliseconds, monitor_max_sleep_time_ms, 0, "Default - distributed_directory_monitor_max_sleep_time_ms", 0) \ + M(Bool, flush_on_detach, true, "Flush data to remote nodes on DETACH/DROP/server shutdown", 0) \ DECLARE_SETTINGS_TRAITS(DistributedSettingsTraits, LIST_OF_DISTRIBUTED_SETTINGS) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f80e498efa8..6f0072c4560 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1438,6 +1438,12 @@ ActionLock StorageDistributed::getActionLock(StorageActionBlockType type) void StorageDistributed::flushAndPrepareForShutdown() { + if (!getDistributedSettingsRef().flush_on_detach) + { + LOG_INFO(log, "Skip flushing data (due to flush_on_detach=0)"); + return; + } + try { flushClusterNodesAllData(getContext()); diff --git a/tests/queries/0_stateless/02860_distributed_flush_on_detach.reference b/tests/queries/0_stateless/02860_distributed_flush_on_detach.reference new file mode 100644 index 00000000000..8e5815188e7 --- /dev/null +++ b/tests/queries/0_stateless/02860_distributed_flush_on_detach.reference @@ -0,0 +1,27 @@ +-- { echoOn } + +create table data (key Int) engine=Memory(); +create table dist (key Int) engine=Distributed(default, currentDatabase(), data); +system stop distributed sends dist; +-- check that FLUSH DISTRIBUTED does flushing anyway +insert into dist values (1); +select * from data; +system flush distributed dist; +select * from data; +1 +truncate table data; +-- check that flush_on_detach=1 by default +insert into dist values (1); +detach table dist; +select * from data; +1 +attach table dist; +truncate table data; +-- check flush_on_detach=0 +drop table dist; +create table dist (key Int) engine=Distributed(default, currentDatabase(), data) settings flush_on_detach=0; +system stop distributed sends dist; +insert into dist values (1); +detach table dist; +select * from data; +attach table dist; diff --git a/tests/queries/0_stateless/02860_distributed_flush_on_detach.sql b/tests/queries/0_stateless/02860_distributed_flush_on_detach.sql new file mode 100644 index 00000000000..5ba45d72c08 --- /dev/null +++ b/tests/queries/0_stateless/02860_distributed_flush_on_detach.sql @@ -0,0 +1,33 @@ +set prefer_localhost_replica=0; + +drop table if exists data; +drop table if exists dist; + +-- { echoOn } + +create table data (key Int) engine=Memory(); +create table dist (key Int) engine=Distributed(default, currentDatabase(), data); +system stop distributed sends dist; + +-- check that FLUSH DISTRIBUTED does flushing anyway +insert into dist values (1); +select * from data; +system flush distributed dist; +select * from data; +truncate table data; + +-- check that flush_on_detach=1 by default +insert into dist values (1); +detach table dist; +select * from data; +attach table dist; +truncate table data; + +-- check flush_on_detach=0 +drop table dist; +create table dist (key Int) engine=Distributed(default, currentDatabase(), data) settings flush_on_detach=0; +system stop distributed sends dist; +insert into dist values (1); +detach table dist; +select * from data; +attach table dist; From fd3e6c93ee58c40d303055ca1bc508e322e8920a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 17 Aug 2023 09:14:47 +0200 Subject: [PATCH 2039/2047] Produce standard report for cases when AST failed --- tests/ci/ast_fuzzer_check.py | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index fecf207589e..a8c7f2c3a9c 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -21,7 +21,6 @@ from commit_status_helper import ( ) from docker_pull_helper import get_image_with_version from env_helper import ( - GITHUB_RUN_URL, REPORTS_PATH, TEMP_PATH, ) @@ -30,6 +29,7 @@ from pr_info import PRInfo from report import TestResult from s3_helper import S3Helper from stopwatch import Stopwatch +from upload_result_helper import upload_results IMAGE_NAME = "clickhouse/fuzzer" @@ -183,10 +183,6 @@ def main(): logging.info("Exception uploading file %s text %s", f, ex) paths[f] = "" - report_url = GITHUB_RUN_URL - if paths["report.html"]: - report_url = paths["report.html"] - # Try to get status message saved by the fuzzer try: with open( @@ -208,6 +204,19 @@ def main(): if "fail" in status: test_result.status = "FAIL" + if paths["report.html"]: + report_url = paths["report.html"] + else: + report_url = upload_results( + s3_helper, + pr_info.number, + pr_info.sha, + [test_result], + [], + check_name, + [url for url in paths.values() if url], + ) + ch_helper = ClickHouseHelper() prepared_events = prepare_tests_results_for_clickhouse( From 5253a7f650fc726098a9515734fe9553cf408c70 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 15 Aug 2023 21:04:40 +0000 Subject: [PATCH 2040/2047] Respect cgroup memory limit when reloading configuration Cgroups allows to change the amount of memory available to a process while it runs. The previous logic calculated the amount of available memory only once at server startup. As a result, memory thresholds set via cgroups were not picked up when the settings changed. We now always incorporate the current limits during re-configuraton. Note 1: getMemoryAmount() opens/reads a file which is potentially expensive. Should be fine though since that happens only when the server configuration changes. Note 2: An better approach would be to treat cgroup limit changes as another trigger for ClickHouse server re-configuration (which currently only happens when the config files change). Shied away from that for now because of the case that when the cgroup limit is lowered, there is no guarantee that ClickHouse can shrink the memory amount accordingly in time (afaik, it does so only lazily by denying new allocations). As a result, the OOM killer would kill the server. The same will happen with this PR but at a lower implementation complexity. --- programs/server/Server.cpp | 21 +++++++++++---------- src/Core/ServerSettings.h | 18 +++++++++--------- 2 files changed, 20 insertions(+), 19 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b823cbbe367..a193b132f72 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -659,10 +659,10 @@ try global_context->addWarningMessage("Server was built with sanitizer. It will work slowly."); #endif - const size_t memory_amount = getMemoryAmount(); + const size_t physical_server_memory = getMemoryAmount(); LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.", - formatReadableSizeWithBinarySuffix(memory_amount), + formatReadableSizeWithBinarySuffix(physical_server_memory), getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores std::thread::hardware_concurrency()); @@ -1137,9 +1137,10 @@ try server_settings_.loadSettingsFromConfig(*config); size_t max_server_memory_usage = server_settings_.max_server_memory_usage; - double max_server_memory_usage_to_ram_ratio = server_settings_.max_server_memory_usage_to_ram_ratio; - size_t default_max_server_memory_usage = static_cast(memory_amount * max_server_memory_usage_to_ram_ratio); + + size_t current_physical_server_memory = getMemoryAmount(); /// With cgroups, the amount of memory available to the server can be changed dynamically. + size_t default_max_server_memory_usage = static_cast(current_physical_server_memory * max_server_memory_usage_to_ram_ratio); if (max_server_memory_usage == 0) { @@ -1147,7 +1148,7 @@ try LOG_INFO(log, "Setting max_server_memory_usage was set to {}" " ({} available * {:.2f} max_server_memory_usage_to_ram_ratio)", formatReadableSizeWithBinarySuffix(max_server_memory_usage), - formatReadableSizeWithBinarySuffix(memory_amount), + formatReadableSizeWithBinarySuffix(current_physical_server_memory), max_server_memory_usage_to_ram_ratio); } else if (max_server_memory_usage > default_max_server_memory_usage) @@ -1158,7 +1159,7 @@ try " calculated as {} available" " * {:.2f} max_server_memory_usage_to_ram_ratio", formatReadableSizeWithBinarySuffix(max_server_memory_usage), - formatReadableSizeWithBinarySuffix(memory_amount), + formatReadableSizeWithBinarySuffix(current_physical_server_memory), max_server_memory_usage_to_ram_ratio); } @@ -1168,14 +1169,14 @@ try size_t merges_mutations_memory_usage_soft_limit = server_settings_.merges_mutations_memory_usage_soft_limit; - size_t default_merges_mutations_server_memory_usage = static_cast(memory_amount * server_settings_.merges_mutations_memory_usage_to_ram_ratio); + size_t default_merges_mutations_server_memory_usage = static_cast(current_physical_server_memory * server_settings_.merges_mutations_memory_usage_to_ram_ratio); if (merges_mutations_memory_usage_soft_limit == 0) { merges_mutations_memory_usage_soft_limit = default_merges_mutations_server_memory_usage; LOG_INFO(log, "Setting merges_mutations_memory_usage_soft_limit was set to {}" " ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)", formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit), - formatReadableSizeWithBinarySuffix(memory_amount), + formatReadableSizeWithBinarySuffix(current_physical_server_memory), server_settings_.merges_mutations_memory_usage_to_ram_ratio); } else if (merges_mutations_memory_usage_soft_limit > default_merges_mutations_server_memory_usage) @@ -1184,7 +1185,7 @@ try LOG_WARNING(log, "Setting merges_mutations_memory_usage_soft_limit was set to {}" " ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)", formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit), - formatReadableSizeWithBinarySuffix(memory_amount), + formatReadableSizeWithBinarySuffix(current_physical_server_memory), server_settings_.merges_mutations_memory_usage_to_ram_ratio); } @@ -1486,7 +1487,7 @@ try /// Set up caches. - const size_t max_cache_size = static_cast(memory_amount * server_settings.cache_size_to_ram_max_ratio); + const size_t max_cache_size = static_cast(physical_server_memory * server_settings.cache_size_to_ram_max_ratio); String uncompressed_cache_policy = server_settings.uncompressed_cache_policy; size_t uncompressed_cache_size = server_settings.uncompressed_cache_size; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index f759cd150a8..3740929b491 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -44,19 +44,19 @@ namespace DB M(String, tmp_policy, "", "Policy for storage with temporary data.", 0) \ M(UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0) \ M(String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0) \ - M(UInt64, max_server_memory_usage, 0, "Limit on total memory usage. Zero means Unlimited.", 0) \ - M(Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to ram ratio. Allows to lower max memory on low-memory systems.", 0) \ - M(UInt64, merges_mutations_memory_usage_soft_limit, 0, "Limit on total memory usage for merges and mutations. Zero means Unlimited.", 0) \ - M(Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to ram ratio. Allows to lower memory limit on low-memory systems.", 0) \ + M(UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0) \ + M(Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0) \ + M(UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0) \ + M(Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0) \ M(Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0) \ M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \ \ - M(UInt64, max_concurrent_queries, 0, "Limit on total number of concurrently executed queries. Zero means Unlimited.", 0) \ - M(UInt64, max_concurrent_insert_queries, 0, "Limit on total number of concurrently insert queries. Zero means Unlimited.", 0) \ - M(UInt64, max_concurrent_select_queries, 0, "Limit on total number of concurrently select queries. Zero means Unlimited.", 0) \ + M(UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0) \ + M(UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0) \ + M(UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0) \ \ - M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro ram max ratio. Allows to lower cache size on low-memory systems.", 0) \ + M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0) \ M(String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0) \ M(UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \ M(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \ @@ -71,7 +71,7 @@ namespace DB \ M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \ M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ - M(UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means Unlimited.", 0) \ + M(UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0) \ M(UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \ \ M(UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0) \ From 9e69120bee9e0b256b1e4124eec2d31c3bd232fd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 17 Aug 2023 12:03:38 +0200 Subject: [PATCH 2041/2047] fix --- tests/integration/test_grpc_protocol/test.py | 2 +- .../00002_log_and_exception_messages_formatting.sql | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index c149e1a4096..42d6d2bfda3 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -352,7 +352,7 @@ def test_authentication(): def test_logs(): - query = "SELECT has(groupArray(number), 42) FROM numbers(100000)" + query = "SELECT has(groupArray(number), 42) FROM numbers(100000) SETTINGS max_block_size=10000" logs = query_and_get_logs( query, settings={"send_logs_level": "debug"}, diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index c4a94174125..e2de14a4a8b 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -14,11 +14,10 @@ select 'runtime messages', greatest(coalesce(sum(length(message_format_string) = -- Check the same for exceptions. The value was 0.03 select 'runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.05) from logs where (message like '%DB::Exception%' or message like '%Coordination::Exception%') - and message not like '%Received from CLICKHOUSE_CI_LOGS_HOST:9440%'; + and message not like '% Received from %clickhouse-staging.com:9440%'; select 'unknown runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.01) from logs where (message like '%DB::Exception%' or message like '%Coordination::Exception%') - and message not like '%Received from CLICKHOUSE_CI_LOGS_HOST:9440%' and message not like '% Received from %' and message not like '%(SYNTAX_ERROR)%'; -- FIXME some of the following messages are not informative and it has to be fixed From 76702a03bb75e9d94cb4313d0a67da97a3497d27 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 17 Aug 2023 13:08:27 +0300 Subject: [PATCH 2042/2047] Update materialized_with_ddl.py (#53494) --- .../materialized_with_ddl.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 36be2aa1672..e64884707c1 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -1249,7 +1249,7 @@ def err_sync_user_privs_with_materialized_mysql_database( ) assert "priv_err_db" in clickhouse_node.query("SHOW DATABASES") assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM priv_err_db") - clickhouse_node.query_with_retry("DETACH DATABASE priv_err_db") + clickhouse_node.query_with_retry("DETACH DATABASE priv_err_db SYNC") mysql_node.query("REVOKE SELECT ON priv_err_db.* FROM 'test'@'%'") time.sleep(3) @@ -1442,7 +1442,7 @@ def mysql_kill_sync_thread_restore_test(clickhouse_node, mysql_node, service_nam time.sleep(sleep_time) clickhouse_node.query("SELECT * FROM test_database.test_table") - clickhouse_node.query_with_retry("DETACH DATABASE test_database") + clickhouse_node.query_with_retry("DETACH DATABASE test_database SYNC") clickhouse_node.query("ATTACH DATABASE test_database") check_query( clickhouse_node, @@ -1506,7 +1506,7 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): mysql_node.alloc_connection() - clickhouse_node.query_with_retry("DETACH DATABASE kill_mysql_while_insert") + clickhouse_node.query_with_retry("DETACH DATABASE kill_mysql_while_insert SYNC") clickhouse_node.query("ATTACH DATABASE kill_mysql_while_insert") result = mysql_node.query_and_get_data( @@ -2593,7 +2593,7 @@ def named_collections(clickhouse_node, mysql_node, service_name): "1\ta\t1\n2\tb\t2\n", ) clickhouse_node.query(f"ALTER NAMED COLLECTION {db} SET port=9999") - clickhouse_node.query(f"DETACH DATABASE {db}") + clickhouse_node.query_with_retry(f"DETACH DATABASE {db} SYNC") mysql_node.query(f"INSERT INTO {db}.t1 VALUES (3, 'c', 3)") assert "ConnectionFailed:" in clickhouse_node.query_and_get_error( f"ATTACH DATABASE {db}" From 7115110dad245be1c3507b2d3b61045e1f0afc37 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Thu, 17 Aug 2023 15:59:22 +0200 Subject: [PATCH 2043/2047] Update merge-tree-settings.md parts_to_throw_insert=3000 --- docs/en/operations/settings/merge-tree-settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 4122b4af40f..8ea599b9861 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -56,11 +56,11 @@ Possible values: - Any positive integer. -Default value: 300. +Default value: 3000. To achieve maximum performance of `SELECT` queries, it is necessary to minimize the number of parts processed, see [Merge Tree](../../development/architecture.md#merge-tree). -You can set a larger value to 600 (1200), this will reduce the probability of the `Too many parts` error, but at the same time `SELECT` performance might degrade. Also in case of a merge issue (for example, due to insufficient disk space) you will notice it later than it could be with the original 300. +Prior to 23.6 this setting was set to 300. You can set a higher different value, it will reduce the probability of the `Too many parts` error, but at the same time `SELECT` performance might degrade. Also in case of a merge issue (for example, due to insufficient disk space) you will notice it later than it could be with the original 300. ## parts_to_delay_insert {#parts-to-delay-insert} From fdca288a73aefe8cb2d591ffe709613b070c630a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 17 Aug 2023 16:49:50 +0200 Subject: [PATCH 2044/2047] make sending logs less fragile --- docker/test/base/setup_export_logs.sh | 6 +++++- .../0_stateless/02443_detach_attach_partition.sh | 11 +++++++++-- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 12fae855b03..d68fd392d62 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -17,6 +17,9 @@ CONNECTION_PARAMETERS=${CONNECTION_PARAMETERS:=""} # Create all configured system logs: clickhouse-client --query "SYSTEM FLUSH LOGS" +# It's doesn't make sense to try creating tables if SYNC fails +echo "SYSTEM SYNC DATABASE REPLICA default" clickhouse-client --receive_timeout 180 $CONNECTION_PARAMETERS || exit 0 + # For each system log table: clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table do @@ -38,7 +41,7 @@ do echo "Creating destination table ${table}_${hash}" >&2 - echo "$statement" | clickhouse-client $CONNECTION_PARAMETERS + echo "$statement" | clickhouse-client --distributed_ddl_task_timeout=10 $CONNECTION_PARAMETERS || continue echo "Creating table system.${table}_sender" >&2 @@ -46,6 +49,7 @@ do clickhouse-client --query " CREATE TABLE system.${table}_sender ENGINE = Distributed(${CLUSTER}, default, ${table}_${hash}) + SETTINGS flush_on_detach=0 EMPTY AS SELECT ${EXTRA_COLUMNS_EXPRESSION}, * FROM system.${table} diff --git a/tests/queries/0_stateless/02443_detach_attach_partition.sh b/tests/queries/0_stateless/02443_detach_attach_partition.sh index ae104b833e3..2bb7361e9da 100755 --- a/tests/queries/0_stateless/02443_detach_attach_partition.sh +++ b/tests/queries/0_stateless/02443_detach_attach_partition.sh @@ -30,17 +30,24 @@ function thread_attach() } insert_type=$(($RANDOM % 3)) +if [[ "$engine" == "ReplicatedMergeTree" ]]; then + insert_type=$(($RANDOM % 2)) +fi $CLICKHOUSE_CLIENT -q "SELECT '$CLICKHOUSE_DATABASE', 'insert_type $insert_type' FORMAT Null" function insert() { # Fault injection may lead to duplicates if [[ "$insert_type" -eq 0 ]]; then - $CLICKHOUSE_CLIENT --insert_deduplication_token=$1 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null + $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null elif [[ "$insert_type" -eq 1 ]]; then $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $1, $1" 2>/dev/null else - $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null + # It may reproduce something interesting: if the insert status is unknown (due to fault injection in retries) + # and the part was committed locally but not in zk, then it will be active and DETACH may detach it. + # And we will ATTACH it later. But the next INSERT attempt will not be deduplicated because the first one failed. + # So we will get duplicates. + $CLICKHOUSE_CLIENT --insert_deduplication_token=$1 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null fi } From f20dd27ba6385b7432556360d8397f700811e832 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 17 Aug 2023 18:45:26 +0000 Subject: [PATCH 2045/2047] Clean header mess up --- docs/en/sql-reference/dictionaries/index.md | 128 ++++++++++---------- 1 file changed, 64 insertions(+), 64 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index c95ff5758f4..dd8031461e0 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -1092,7 +1092,7 @@ Types of sources (`source_type`): - [Local file](#local_file) - [Executable File](#executable) - [Executable Pool](#executable_pool) -- [HTTP(s)](#http) +- [HTTP(S)](#http) - DBMS - [ODBC](#odbc) - [MySQL](#mysql) @@ -1102,7 +1102,7 @@ Types of sources (`source_type`): - [Cassandra](#cassandra) - [PostgreSQL](#postgresql) -## Local File {#local_file} +### Local File {#local_file} Example of settings: @@ -1132,7 +1132,7 @@ When a dictionary with source `FILE` is created via DDL command (`CREATE DICTION - [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) -## Executable File {#executable} +### Executable File {#executable} Working with executable files depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts the executable file and treats its output as dictionary data. @@ -1161,7 +1161,7 @@ Setting fields: That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled; otherwise, the DB user would be able to execute arbitrary binaries on the ClickHouse node. -## Executable Pool {#executable_pool} +### Executable Pool {#executable_pool} Executable pool allows loading data from pool of processes. This source does not work with dictionary layouts that need to load all data from source. Executable pool works if the dictionary [is stored](#ways-to-store-dictionaries-in-memory) using `cache`, `complex_key_cache`, `ssd_cache`, `complex_key_ssd_cache`, `direct`, or `complex_key_direct` layouts. @@ -1196,9 +1196,9 @@ Setting fields: That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. -## Http(s) {#https} +### HTTP(S) {#https} -Working with an HTTP(s) server depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. +Working with an HTTP(S) server depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. Example of settings: @@ -1248,7 +1248,55 @@ Setting fields: When creating a dictionary using the DDL command (`CREATE DICTIONARY ...`) remote hosts for HTTP dictionaries are checked against the contents of `remote_url_allow_hosts` section from config to prevent database users to access arbitrary HTTP server. -### Known Vulnerability of the ODBC Dictionary Functionality +### DBMS + +#### ODBC + +You can use this method to connect any database that has an ODBC driver. + +Example of settings: + +``` xml + + + DatabaseName + ShemaName.TableName
+ DSN=some_parameters + SQL_QUERY + SELECT id, value_1, value_2 FROM ShemaName.TableName +
+ +``` + +or + +``` sql +SOURCE(ODBC( + db 'DatabaseName' + table 'SchemaName.TableName' + connection_string 'DSN=some_parameters' + invalidate_query 'SQL_QUERY' + query 'SELECT id, value_1, value_2 FROM db_name.table_name' +)) +``` + +Setting fields: + +- `db` – Name of the database. Omit it if the database name is set in the `` parameters. +- `table` – Name of the table and schema if exists. +- `connection_string` – Connection string. +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `query` – The custom query. Optional parameter. + +:::note +The `table` and `query` fields cannot be used together. And either one of the `table` or `query` fields must be declared. +::: + +ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it’s necessary to set table name accordingly to table name case in database. + +If you have a problems with encodings when using Oracle, see the corresponding [FAQ](/knowledgebase/oracle-odbc) item. + +##### Known Vulnerability of the ODBC Dictionary Functionality :::note When connecting to the database through the ODBC driver connection parameter `Servername` can be substituted. In this case values of `USERNAME` and `PASSWORD` from `odbc.ini` are sent to the remote server and can be compromised. @@ -1277,7 +1325,7 @@ SELECT * FROM odbc('DSN=gregtest;Servername=some-server.com', 'test_db'); ODBC driver will send values of `USERNAME` and `PASSWORD` from `odbc.ini` to `some-server.com`. -### Example of Connecting Postgresql +##### Example of Connecting Postgresql Ubuntu OS. @@ -1358,7 +1406,7 @@ LIFETIME(MIN 300 MAX 360) You may need to edit `odbc.ini` to specify the full path to the library with the driver `DRIVER=/usr/local/lib/psqlodbcw.so`. -### Example of Connecting MS SQL Server +##### Example of Connecting MS SQL Server Ubuntu OS. @@ -1462,55 +1510,7 @@ LAYOUT(FLAT()) LIFETIME(MIN 300 MAX 360) ``` -## DBMS - -### ODBC - -You can use this method to connect any database that has an ODBC driver. - -Example of settings: - -``` xml - - - DatabaseName - ShemaName.TableName
- DSN=some_parameters - SQL_QUERY - SELECT id, value_1, value_2 FROM ShemaName.TableName -
- -``` - -or - -``` sql -SOURCE(ODBC( - db 'DatabaseName' - table 'SchemaName.TableName' - connection_string 'DSN=some_parameters' - invalidate_query 'SQL_QUERY' - query 'SELECT id, value_1, value_2 FROM db_name.table_name' -)) -``` - -Setting fields: - -- `db` – Name of the database. Omit it if the database name is set in the `` parameters. -- `table` – Name of the table and schema if exists. -- `connection_string` – Connection string. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). -- `query` – The custom query. Optional parameter. - -:::note -The `table` and `query` fields cannot be used together. And either one of the `table` or `query` fields must be declared. -::: - -ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it’s necessary to set table name accordingly to table name case in database. - -If you have a problems with encodings when using Oracle, see the corresponding [FAQ](/knowledgebase/oracle-odbc) item. - -### Mysql +#### Mysql Example of settings: @@ -1627,7 +1627,7 @@ SOURCE(MYSQL( )) ``` -### ClickHouse +#### ClickHouse Example of settings: @@ -1680,7 +1680,7 @@ Setting fields: The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. ::: -### Mongodb +#### Mongodb Example of settings: @@ -1723,7 +1723,7 @@ Setting fields: - `options` - MongoDB connection string options (optional parameter). -### Redis +#### Redis Example of settings: @@ -1756,7 +1756,7 @@ Setting fields: - `storage_type` – The structure of internal Redis storage using for work with keys. `simple` is for simple sources and for hashed single key sources, `hash_map` is for hashed sources with two keys. Ranged sources and cache sources with complex key are unsupported. May be omitted, default value is `simple`. - `db_index` – The specific numeric index of Redis logical database. May be omitted, default value is 0. -### Cassandra +#### Cassandra Example of settings: @@ -1798,7 +1798,7 @@ Setting fields: The `column_family` or `where` fields cannot be used together with the `query` field. And either one of the `column_family` or `query` fields must be declared. ::: -### PostgreSQL +#### PostgreSQL Example of settings: @@ -1855,7 +1855,7 @@ Setting fields: The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. ::: -## Null +### Null A special source that can be used to create dummy (empty) dictionaries. Such dictionaries can useful for tests or with setups with separated data and query nodes at nodes with Distributed tables. From 557bfea4d812310363eb3785517d62af143523c5 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 17 Aug 2023 19:36:40 +0000 Subject: [PATCH 2046/2047] Use long timeout for S3 copy requests --- src/Backups/BackupIO_S3.cpp | 4 ++- .../ObjectStorages/S3/S3ObjectStorage.cpp | 12 ++++----- src/IO/S3/copyS3File.cpp | 26 ++++++++++++------- src/IO/S3/copyS3File.h | 7 +++++ 4 files changed, 33 insertions(+), 16 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index d32047efae4..7926d0b2564 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -159,6 +159,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s blob_path.size(), mode); copyS3File( + client, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, @@ -218,6 +219,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src { LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName()); copyS3File( + client, client, /* src_bucket */ blob_path[1], /* src_key= */ blob_path[0], @@ -238,7 +240,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { - copyDataToS3File(create_read_buffer, start_pos, length, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, {}, + copyDataToS3File(create_read_buffer, start_pos, length, client, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, {}, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 4da7b3e892f..bbfc6609079 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -431,11 +431,11 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT /// Shortcut for S3 if (auto * dest_s3 = dynamic_cast(&object_storage_to); dest_s3 != nullptr) { - auto client_ptr = clients.get()->client; + auto clients_ = clients.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, + copyS3File(clients_->client, clients_->client_with_long_timeout, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } else @@ -447,11 +447,11 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT void S3ObjectStorage::copyObject( // NOLINT const StoredObject & object_from, const StoredObject & object_to, std::optional object_to_attributes) { - auto client_ptr = clients.get()->client; + auto clients_ = clients.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, + copyS3File(clients_->client, clients_->client_with_long_timeout, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index dda107840cb..002b8dde566 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -53,6 +53,7 @@ namespace public: UploadHelper( const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_with_long_timeout_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, @@ -61,6 +62,7 @@ namespace bool for_disk_s3_, const Poco::Logger * log_) : client_ptr(client_ptr_) + , client_with_long_timeout_ptr(client_with_long_timeout_ptr_) , dest_bucket(dest_bucket_) , dest_key(dest_key_) , request_settings(request_settings_) @@ -76,6 +78,7 @@ namespace protected: std::shared_ptr client_ptr; + std::shared_ptr client_with_long_timeout_ptr; const String & dest_bucket; const String & dest_key; const S3Settings::RequestSettings & request_settings; @@ -176,7 +179,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); - auto outcome = client_ptr->CompleteMultipartUpload(request); + auto outcome = client_with_long_timeout_ptr->CompleteMultipartUpload(request); if (outcome.IsSuccess()) { @@ -430,13 +433,14 @@ namespace size_t offset_, size_t size_, const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_with_long_timeout_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_s3_) - : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File")) + : UploadHelper(client_ptr_, client_with_long_timeout_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File")) , create_read_buffer(create_read_buffer_) , offset(offset_) , size(size_) @@ -598,6 +602,7 @@ namespace public: CopyFileHelper( const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_with_long_timeout_ptr_, const String & src_bucket_, const String & src_key_, size_t src_offset_, @@ -608,7 +613,7 @@ namespace const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_s3_) - : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File")) + : UploadHelper(client_ptr_, client_with_long_timeout_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File")) , src_bucket(src_bucket_) , src_key(src_key_) , offset(src_offset_) @@ -669,7 +674,7 @@ namespace /// If we don't do it, AWS SDK can mistakenly set it to application/xml, see https://github.com/aws/aws-sdk-cpp/issues/1840 request.SetContentType("binary/octet-stream"); - client_ptr->setKMSHeaders(request); + client_with_long_timeout_ptr->setKMSHeaders(request); } void processCopyRequest(const S3::CopyObjectRequest & request) @@ -681,7 +686,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3CopyObject); - auto outcome = client_ptr->CopyObject(request); + auto outcome = client_with_long_timeout_ptr->CopyObject(request); if (outcome.IsSuccess()) { LOG_TRACE( @@ -706,6 +711,7 @@ namespace offset, size, client_ptr, + client_with_long_timeout_ptr, dest_bucket, dest_key, request_settings, @@ -779,7 +785,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3UploadPartCopy); - auto outcome = client_ptr->UploadPartCopy(req); + auto outcome = client_with_long_timeout_ptr->UploadPartCopy(req); if (!outcome.IsSuccess()) { abortMultipartUpload(); @@ -797,6 +803,7 @@ void copyDataToS3File( size_t offset, size_t size, const std::shared_ptr & dest_s3_client, + const std::shared_ptr & dest_s3_client_with_long_timeout, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, @@ -804,13 +811,14 @@ void copyDataToS3File( ThreadPoolCallbackRunner schedule, bool for_disk_s3) { - CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; + CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } void copyS3File( const std::shared_ptr & s3_client, + const std::shared_ptr & s3_client_with_long_timeout, const String & src_bucket, const String & src_key, size_t src_offset, @@ -824,7 +832,7 @@ void copyS3File( { if (settings.allow_native_copy) { - CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; + CopyFileHelper helper{s3_client, s3_client_with_long_timeout, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } else @@ -833,7 +841,7 @@ void copyS3File( { return std::make_unique(s3_client, src_bucket, src_key, "", settings, Context::getGlobalContextInstance()->getReadSettings()); }; - copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); } } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 5d35e5ebe2d..3477f5a20ab 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -26,8 +26,14 @@ using CreateReadBuffer = std::function()>; /// has been disabled (with settings.allow_native_copy) or request failed /// because it is a known issue, it is fallbacks to read-write copy /// (copyDataToS3File()). +/// +/// s3_client_with_long_timeout (may be equal to s3_client) is used for native copy and +/// CompleteMultipartUpload requests. These requests need longer timeout because S3 servers often +/// block on them for multiple seconds without sending or receiving data from us (maybe the servers +/// are copying data internally, or maybe throttling, idk). void copyS3File( const std::shared_ptr & s3_client, + const std::shared_ptr & s3_client_with_long_timeout, const String & src_bucket, const String & src_key, size_t src_offset, @@ -49,6 +55,7 @@ void copyDataToS3File( size_t offset, size_t size, const std::shared_ptr & dest_s3_client, + const std::shared_ptr & dest_s3_client_with_long_timeout, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, From abcf01b809708e92437973bd583bd43d3b080054 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 17 Aug 2023 23:51:58 +0200 Subject: [PATCH 2047/2047] impl --- tests/integration/helpers/client.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index fdeedb9a80d..c5f0584f828 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -182,7 +182,8 @@ class CommandRequest: # we suppress stderror on client becase sometimes thread sanitizer # can print some debug information there env = {} - env["TSAN_OPTIONS"] = "verbosity=0" + env["ASAN_OPTIONS"] = "use_sigaltstack=0" + env["TSAN_OPTIONS"] = "use_sigaltstack=0 verbosity=0" self.process = sp.Popen( command, stdin=stdin_file,